You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by ji...@apache.org on 2015/06/18 06:23:11 UTC

[01/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Repository: incubator-asterixdb
Updated Branches:
  refs/heads/master e05df7be2 -> 65100727f


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
index eb9adcc..8e78bb3 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
@@ -15,17 +15,16 @@
 package edu.uci.ics.asterix.external.indexing.dataflow;
 
 import java.io.InputStream;
-import java.nio.ByteBuffer;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 
 /*
@@ -35,12 +34,10 @@ public class HDFSObjectTupleParser implements ITupleParser{
 
     private ArrayTupleBuilder tb;
     private final FrameTupleAppender appender;
-    private final ByteBuffer frame;
     private IAsterixHDFSRecordParser deserializer;
 
     public HDFSObjectTupleParser(IHyracksTaskContext ctx, ARecordType recType, IAsterixHDFSRecordParser deserializer) throws HyracksDataException {
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        frame = ctx.allocateFrame();
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         this.deserializer = deserializer;
         tb = new ArrayTupleBuilder(1);
     }
@@ -48,7 +45,6 @@ public class HDFSObjectTupleParser implements ITupleParser{
     @Override
     public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
         AbstractHDFSReader reader = (AbstractHDFSReader) in;
-        appender.reset(frame, true);
         Object object;
         try {
             reader.initialize();
@@ -60,9 +56,7 @@ public class HDFSObjectTupleParser implements ITupleParser{
                 addTupleToFrame(writer);
                 object = reader.readNext();
             }
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (AsterixException ae) {
             throw new HyracksDataException(ae);
         } catch (Exception e) {
@@ -72,8 +66,7 @@ public class HDFSObjectTupleParser implements ITupleParser{
 
     protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException();
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
index d845ce3..bfd787b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileControlledTupleParser.java
@@ -25,6 +25,7 @@ import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AInt64;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
@@ -34,7 +35,6 @@ import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 
 public class RCFileControlledTupleParser implements IControlledTupleParser {
@@ -42,7 +42,6 @@ public class RCFileControlledTupleParser implements IControlledTupleParser {
     private ArrayTupleBuilder tb;
     private transient DataOutput dos;
     private final FrameTupleAppender appender;
-    private final ByteBuffer frame;
     private boolean propagateInput;
     private int[] propagatedFields;
     private FrameTupleReference frameTuple;
@@ -60,8 +59,7 @@ public class RCFileControlledTupleParser implements IControlledTupleParser {
     public RCFileControlledTupleParser(IHyracksTaskContext ctx, IAsterixHDFSRecordParser parser,
             RCFileLookupReader reader, boolean propagateInput, int[] propagatedFields, RecordDescriptor inRecDesc,
             int[] ridFields, boolean retainNull, INullWriterFactory iNullWriterFactory) throws HyracksDataException {
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        frame = ctx.allocateFrame();
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         this.parser = parser;
         this.reader = reader;
         this.propagateInput = propagateInput;
@@ -69,7 +67,7 @@ public class RCFileControlledTupleParser implements IControlledTupleParser {
         this.retainNull = retainNull;
         this.inRecDesc = inRecDesc;
         this.ridFields = ridFields;
-        this.tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecDesc);
+        this.tupleAccessor = new FrameTupleAccessor(inRecDesc);
         if (propagateInput) {
             tb = new ArrayTupleBuilder(propagatedFields.length + 1);
         } else {
@@ -77,10 +75,8 @@ public class RCFileControlledTupleParser implements IControlledTupleParser {
         }
         frameTuple = new FrameTupleReference();
         dos = tb.getDataOutput();
-        appender.reset(frame, true);
         bbis = new ByteBufferInputStream();
         dis = new DataInputStream(bbis);
-        appender.reset(frame, true);
         nullByte = ATypeTag.NULL.serialize();
         if (retainNull) {
             INullWriter nullWriter = iNullWriterFactory.createNullWriter();
@@ -100,9 +96,7 @@ public class RCFileControlledTupleParser implements IControlledTupleParser {
     public void close(IFrameWriter writer) throws Exception {
         try {
             reader.close();
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (IOException ioe) {
             throw new HyracksDataException(ioe);
         }
@@ -128,22 +122,24 @@ public class RCFileControlledTupleParser implements IControlledTupleParser {
                 } else {
                     // Get file number
                     bbis.setByteBuffer(frameBuffer, tupleStartOffset + fileNumberStartOffset);
-                    int fileNumber = ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]]
+                    int fileNumber = ((AInt32) inRecDesc
+                            .getFields()[ridFields[IndexingConstants.FILE_NUMBER_FIELD_INDEX]]
                             .deserialize(dis)).getIntegerValue();
                     // Get record group offset
                     bbis.setByteBuffer(
                             frameBuffer,
                             tupleStartOffset
                                     + tupleAccessor.getFieldStartOffset(tupleIndex,
-                                            ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
-                    long recordOffset = ((AInt64) inRecDesc.getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]]
+                                    ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]));
+                    long recordOffset = ((AInt64) inRecDesc
+                            .getFields()[ridFields[IndexingConstants.RECORD_OFFSET_FIELD_INDEX]]
                             .deserialize(dis)).getLongValue();
                     // Get row number
                     bbis.setByteBuffer(
                             frameBuffer,
                             tupleStartOffset
                                     + tupleAccessor.getFieldStartOffset(tupleIndex,
-                                            ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]));
+                                    ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]));
                     int rowNumber = ((AInt32) inRecDesc.getFields()[ridFields[IndexingConstants.ROW_NUMBER_FIELD_INDEX]]
                             .deserialize(dis)).getIntegerValue();
 
@@ -193,8 +189,7 @@ public class RCFileControlledTupleParser implements IControlledTupleParser {
 
     protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException();
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
index 1359d2d..4aaec80 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/SeqOrTxtControlledTupleParser.java
@@ -25,6 +25,7 @@ import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AInt64;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
@@ -34,7 +35,6 @@ import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 
 public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
@@ -42,7 +42,6 @@ public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
     private ArrayTupleBuilder tb;
     private transient DataOutput dos;
     private final FrameTupleAppender appender;
-    private final ByteBuffer frame;
     private boolean propagateInput;
     private int[] propagatedFields;
     private FrameTupleReference frameTuple;
@@ -60,8 +59,7 @@ public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
     public SeqOrTxtControlledTupleParser(IHyracksTaskContext ctx, IAsterixHDFSRecordParser parser,
             ILookupReader reader, boolean propagateInput, int[] propagatedFields, RecordDescriptor inRecDesc,
             int[] ridFields, boolean retainNull, INullWriterFactory iNullWriterFactory) throws HyracksDataException {
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        frame = ctx.allocateFrame();
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         this.parser = parser;
         this.reader = reader;
         this.propagateInput = propagateInput;
@@ -75,8 +73,7 @@ public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
             tb = new ArrayTupleBuilder(1);
         }
         dos = tb.getDataOutput();
-        appender.reset(frame, true);
-        this.tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecDesc);
+        this.tupleAccessor = new FrameTupleAccessor(inRecDesc);
         bbis = new ByteBufferInputStream();
         dis = new DataInputStream(bbis);
         nullByte = ATypeTag.NULL.serialize();
@@ -98,9 +95,7 @@ public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
     public void close(IFrameWriter writer) throws Exception {
         try {
             reader.close();
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (IOException ioe) {
             throw new HyracksDataException(ioe);
         }
@@ -182,8 +177,7 @@ public class SeqOrTxtControlledTupleParser implements IControlledTupleParser {
 
     private void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException();
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
index 1ff2e56..4a70112 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/operators/ExternalIndexBulkModifyOperatorNodePushable.java
@@ -51,7 +51,7 @@ public class ExternalIndexBulkModifyOperatorNodePushable extends IndexBulkLoadOp
     @Override
     public void open() throws HyracksDataException {
         RecordDescriptor recDesc = recDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(recDesc);
         indexHelper.open();
         index = indexHelper.getIndexInstance();
         try {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
----------------------------------------------------------------------
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
index 28abd75..b71be24 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
@@ -30,7 +30,6 @@ import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
@@ -122,7 +121,6 @@ class ConditionalPushTupleParser extends AbstractTupleParser {
     @Override
     public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
         flushTask = new TimeBasedFlushTask(writer, lock);
-        appender.reset(frame, true);
         IDataParser parser = getDataParser();
         try {
             parser.initialize(in, recType, true);
@@ -140,10 +138,10 @@ class ConditionalPushTupleParser extends AbstractTupleParser {
             if (appender.getTupleCount() > 0) {
                 if (activeTimer) {
                     synchronized (lock) {
-                        FrameUtils.flushFrame(frame, writer);
+                        appender.flush(writer, true);
                     }
                 } else {
-                    FrameUtils.flushFrame(frame, writer);
+                    appender.flush(writer, true);
                 }
             }
         } catch (AsterixException ae) {
@@ -169,8 +167,7 @@ class ConditionalPushTupleParser extends AbstractTupleParser {
 
     protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (tuplesInFrame == batchSize || !appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException();
             }
@@ -202,8 +199,7 @@ class ConditionalPushTupleParser extends AbstractTupleParser {
                         if (LOGGER.isLoggable(Level.INFO)) {
                             LOGGER.info("TTL expired flushing frame (" + tuplesInFrame + ")");
                         }
-                        FrameUtils.flushFrame(frame, writer);
-                        appender.reset(frame, true);
+                        appender.flush(writer, true);
                         tuplesInFrame = 0;
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
index 8f9adeb..a6b97eb 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorNodePushable.java
@@ -63,7 +63,7 @@ public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOpe
         this.feedPolicy = feedPolicy;
         policyEnforcer = new FeedPolicyEnforcer(feedId, feedPolicy);
         nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
-        fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+        fta = new FrameTupleAccessor(recordDesc);
         IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
                 .getApplicationContext().getApplicationObject();
         this.feedManager = runtimeCtx.getFeedManager();
@@ -80,7 +80,8 @@ public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOpe
                 adapterRuntimeMgr = new AdapterRuntimeManager(feedId, adapter, feedFrameWriter, partition, inbox,
                         feedManager);
 
-                if (adapter.getDataExchangeMode().equals(DataExchangeMode.PULL) && adapter instanceof IPullBasedFeedAdapter) {
+                if (adapter.getDataExchangeMode().equals(DataExchangeMode.PULL)
+                        && adapter instanceof IPullBasedFeedAdapter) {
                     ((IPullBasedFeedAdapter) adapter).setFeedPolicyEnforcer(policyEnforcer);
                 }
                 if (LOGGER.isLoggable(Level.INFO)) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
index 6ab3659..d636e04 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaOperatorDescriptor.java
@@ -147,7 +147,7 @@ public class FeedMetaOperatorDescriptor extends AbstractSingleActivityOperatorDe
             this.runtimeType = runtimeType;
             this.feedId = feedConnectionId;
             this.nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
-            fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+            fta = new FrameTupleAccessor(recordDesc);
             IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
                     .getApplicationContext().getApplicationObject();
             this.feedManager = runtimeCtx.getFeedManager();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
index e89a27d..d594971 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
@@ -17,6 +17,7 @@ package edu.uci.ics.asterix.runtime.external;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -42,7 +43,7 @@ public class ExternalBTreeSearchOperatorNodePushable extends BTreeSearchOperator
     @Override
     public void open() throws HyracksDataException {
         ExternalBTreeWithBuddyDataflowHelper dataFlowHelper = (ExternalBTreeWithBuddyDataflowHelper) indexHelper;
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+        accessor = new FrameTupleAccessor(inputRecDesc);
         writer.open();
         dataFlowHelper.open();
         index = indexHelper.getIndexInstance();
@@ -66,11 +67,9 @@ public class ExternalBTreeSearchOperatorNodePushable extends BTreeSearchOperator
         ExternalBTreeWithBuddy externalIndex = (ExternalBTreeWithBuddy) index;
         try {
             searchPred = createSearchPredicate();
-            writeBuffer = ctx.allocateFrame();
             tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(ctx.getFrameSize());
-            appender.reset(writeBuffer, true);
+            appender = new FrameTupleAppender(new VSizeFrame(ctx));
             ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
                     .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
             // The next line is the reason we override this method

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
index fbd490e..c2dd0da 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
@@ -17,6 +17,7 @@ package edu.uci.ics.asterix.runtime.external;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -40,7 +41,7 @@ public class ExternalRTreeSearchOperatorNodePushable extends RTreeSearchOperator
     // We override this method to specify the searched version of the index
     @Override
     public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+        accessor = new FrameTupleAccessor(inputRecDesc);
         writer.open();
         indexHelper.open();
         ExternalRTreeDataflowHelper rTreeDataflowHelper = (ExternalRTreeDataflowHelper) indexHelper;
@@ -65,11 +66,9 @@ public class ExternalRTreeSearchOperatorNodePushable extends RTreeSearchOperator
         ExternalRTree rTreeIndex = (ExternalRTree) index;
         try {
             searchPred = createSearchPredicate();
-            writeBuffer = ctx.allocateFrame();
             tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(ctx.getFrameSize());
-            appender.reset(writeBuffer, true);
+            appender = new FrameTupleAppender(new VSizeFrame(ctx));
             ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
                     .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
             // The next line is the reason we override this method

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
index 79f30cb..d9103ef 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
@@ -17,18 +17,17 @@ package edu.uci.ics.asterix.runtime.operators.file;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.InputStream;
-import java.nio.ByteBuffer;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 
 /**
@@ -43,14 +42,12 @@ public abstract class AbstractTupleParser implements ITupleParser {
     protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
     protected DataOutput dos = tb.getDataOutput();
     protected final FrameTupleAppender appender;
-    protected final ByteBuffer frame;
     protected final ARecordType recType;
     protected final IHyracksTaskContext ctx;
     protected String filename;
 
     public AbstractTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        frame = ctx.allocateFrame();
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         this.recType = recType;
         this.ctx = ctx;
     }
@@ -63,7 +60,6 @@ public abstract class AbstractTupleParser implements ITupleParser {
 
     @Override
     public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
-        appender.reset(frame, true);
         IDataParser parser = getDataParser();
         try {
             parser.initialize(in, recType, true);
@@ -75,9 +71,7 @@ public abstract class AbstractTupleParser implements ITupleParser {
                 tb.addFieldEndOffset();
                 addTupleToFrame(writer);
             }
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (AsterixException ae) {
             throw new HyracksDataException(ae);
         } catch (IOException ioe) {
@@ -87,9 +81,7 @@ public abstract class AbstractTupleParser implements ITupleParser {
 
     protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException("Tuple size(" + tb.getSize() + ") is greater than frame size("
                         + AsterixAppContextInfo.getInstance().getCompilerProperties().getFrameSize() + ")");

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-test-framework/src/main/java/edu/uci/ics/asterix/testframework/context/TestCaseContext.java
----------------------------------------------------------------------
diff --git a/asterix-test-framework/src/main/java/edu/uci/ics/asterix/testframework/context/TestCaseContext.java b/asterix-test-framework/src/main/java/edu/uci/ics/asterix/testframework/context/TestCaseContext.java
index 6488c95..9d941c6 100644
--- a/asterix-test-framework/src/main/java/edu/uci/ics/asterix/testframework/context/TestCaseContext.java
+++ b/asterix-test-framework/src/main/java/edu/uci/ics/asterix/testframework/context/TestCaseContext.java
@@ -76,6 +76,7 @@ public class TestCaseContext {
     };
 
     public static final String DEFAULT_TESTSUITE_XML_NAME = "testsuite.xml";
+    public static final String ONLY_TESTSUITE_XML_NAME = "only.xml";
 
     private File tsRoot;
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
----------------------------------------------------------------------
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
index f479501..d3f13f1 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
@@ -39,7 +39,6 @@ import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
@@ -255,7 +254,6 @@ class RateControlledTupleParser extends AbstractTupleParser {
     @Override
     public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
 
-        appender.reset(frame, true);
         IDataParser parser = getDataParser();
         try {
             parser.initialize(in, recType, true);
@@ -270,9 +268,7 @@ class RateControlledTupleParser extends AbstractTupleParser {
                 }
                 addTupleToFrame(writer);
             }
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (AsterixException ae) {
             throw new HyracksDataException(ae);
         } catch (IOException ioe) {


[09/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/data/big-object/lineitem.tbl.big
----------------------------------------------------------------------
diff --git a/asterix-app/data/big-object/lineitem.tbl.big b/asterix-app/data/big-object/lineitem.tbl.big
new file mode 100644
index 0000000..9c67098
--- /dev/null
+++ b/asterix-app/data/big-object/lineitem.tbl.big
@@ -0,0 +1,6005 @@
+1|156|4|1|17|17954.55|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular 
 courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts a
 bove theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above the
 egular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular c
 ourts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts ab
 ove theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above thee
 gular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular co
 urts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts abo
 ve theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theeg
 ular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular cou
 rts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts abov
 e theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegu
 lar courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular cour
 ts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above
  theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegul
 ar courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular court
 s above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above 
 theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegula
 r courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts
  above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above t
 heegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular
  courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts 
 above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above th
 eegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular 
 courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts a
 bove theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above the
 egular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular c
 ourts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts ab
 ove theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above thee
 gular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular co
 urts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts abo
 ve theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theeg
 ular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts ab|
+1|68|9|2|36|34850.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |
+1|64|5|3|8|7712.48|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep|
+1|3|6|4|28|25284.00|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de|
+1|25|8|5|24|22200.48|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re|
+1|16|3|6|32|29312.32|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex|
+2|107|2|1|38|38269.80|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|
+3|5|2|1|45|40725.00|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|
+3|20|10|2|49|45080.98|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|
+3|129|8|3|27|27786.24|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. |
+3|30|5|4|2|1860.06|0.01|0.06|A|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y. fluffily pending d|
+3|184|5|5|28|30357.04|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE BACK RETURN|FOB|ages nag slyly pending|
+3|63|8|6|26|25039.56|0.10|0.02|A|F|1993-10-29|1993-12-18|1993-11-04|TAKE BACK RETURN|RAIL|ges sleep after the caref|
+4|89|10|1|30|29672.40|0.03|0.08|N|O|1996-01-10|1995-12-14|1996-01-18|DELIVER IN PERSON|REG AIR|- quickly regular packages sleep. idly|
+5|109|10|1|15|15136.50|0.02|0.04|R|F|1994-10-31|1994-08-31|1994-11-20|NONE|AIR|ts wake furiously |
+5|124|5|2|26|26627.12|0.07|0.08|R|F|1994-10-16|1994-09-25|1994-10-19|NONE|FOB|sts use slyly quickly special instruc|
+5|38|4|3|50|46901.50|0.08|0.03|A|F|1994-08-08|1994-10-13|1994-08-26|DELIVER IN PERSON|AIR|eodolites. fluffily unusual|
+6|140|6|1|37|38485.18|0.08|0.03|A|F|1992-04-27|1992-05-15|1992-05-02|TAKE BACK RETURN|TRUCK|p furiously special foxes|
+7|183|4|1|12|12998.16|0.07|0.03|N|O|1996-05-07|1996-03-13|1996-06-03|TAKE BACK RETURN|FOB|ss pinto beans wake against th|
+7|146|3|2|9|9415.26|0.08|0.08|N|O|1996-02-01|1996-03-02|1996-02-19|TAKE BACK RETURN|SHIP|es. instructions|
+7|95|8|3|46|45774.14|0.10|0.07|N|O|1996-01-15|1996-03-27|1996-02-03|COLLECT COD|MAIL| unusual reques|
+7|164|5|4|28|29796.48|0.03|0.04|N|O|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|. slyly special requests haggl|
+7|152|4|5|38|39981.70|0.08|0.01|N|O|1996-02-11|1996-02-24|1996-02-18|DELIVER IN PERSON|TRUCK|ns haggle carefully ironic deposits. bl|
+7|80|10|6|35|34302.80|0.06|0.03|N|O|1996-01-16|1996-02-23|1996-01-22|TAKE BACK RETURN|FOB|jole. excuses wake carefully alongside of |
+7|158|3|7|5|5290.75|0.04|0.02|N|O|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely regula|
+32|83|4|1|28|27526.24|0.05|0.08|N|O|1995-10-23|1995-08-27|1995-10-26|TAKE BACK RETURN|TRUCK|sleep quickly. req|
+32|198|10|2|32|35142.08|0.02|0.00|N|O|1995-08-14|1995-10-07|1995-08-27|COLLECT COD|AIR|lithely regular deposits. fluffily |
+32|45|2|3|2|1890.08|0.09|0.02|N|O|1995-08-07|1995-10-07|1995-08-23|DELIVER IN PERSON|AIR| express accounts wake according to the|
+32|3|8|4|4|3612.00|0.09|0.03|N|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG AIR|e slyly final pac|
+32|86|7|5|44|43387.52|0.05|0.06|N|O|1995-08-28|1995-08-20|1995-09-14|DELIVER IN PERSON|AIR|symptotes nag according to the ironic depo|
+32|12|6|6|6|5472.06|0.04|0.03|N|O|1995-07-21|1995-09-23|1995-07-25|COLLECT COD|RAIL| gifts cajole carefully.|
+33|62|7|1|31|29823.86|0.09|0.04|A|F|1993-10-29|1993-12-19|1993-11-08|COLLECT COD|TRUCK|ng to the furiously ironic package|
+33|61|8|2|32|30753.92|0.02|0.05|A|F|1993-12-09|1994-01-04|1993-12-28|COLLECT COD|MAIL|gular theodolites|
+33|138|4|3|5|5190.65|0.05|0.03|A|F|1993-12-09|1993-12-25|1993-12-23|TAKE BACK RETURN|AIR|. stealthily bold exc|
+33|34|5|4|41|38295.23|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE BACK RETURN|MAIL|unusual packages doubt caref|
+34|89|10|1|13|12858.04|0.00|0.07|N|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG AIR|nic accounts. deposits are alon|
+34|90|1|2|22|21781.98|0.08|0.06|N|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely slyly p|
+34|170|7|3|6|6421.02|0.02|0.06|N|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar foxes sleep |
+35|1|4|1|24|21624.00|0.02|0.00|N|O|1996-02-21|1996-01-03|1996-03-18|TAKE BACK RETURN|FOB|, regular tithe|
+35|162|1|2|34|36113.44|0.06|0.08|N|O|1996-01-22|1996-01-06|1996-01-27|DELIVER IN PERSON|RAIL|s are carefully against the f|
+35|121|4|3|7|7147.84|0.06|0.04|N|O|1996-01-19|1995-12-22|1996-01-29|NONE|MAIL| the carefully regular |
+35|86|7|4|25|24652.00|0.06|0.05|N|O|1995-11-26|1995-12-25|1995-12-21|DELIVER IN PERSON|SHIP| quickly unti|
+35|120|7|5|34|34684.08|0.08|0.06|N|O|1995-11-08|1996-01-15|1995-11-26|COLLECT COD|MAIL|. silent, unusual deposits boost|
+35|31|7|6|28|26068.84|0.03|0.02|N|O|1996-02-01|1995-12-24|1996-02-28|COLLECT COD|RAIL|ly alongside of |
+36|120|1|1|42|42845.04|0.09|0.00|N|O|1996-02-03|1996-01-21|1996-02-23|COLLECT COD|SHIP| careful courts. special |
+37|23|8|1|40|36920.80|0.09|0.03|A|F|1992-07-21|1992-08-01|1992-08-15|NONE|REG AIR|luffily regular requests. slyly final acco|
+37|127|6|2|39|40057.68|0.05|0.02|A|F|1992-07-02|1992-08-18|1992-07-28|TAKE BACK RETURN|RAIL|the final requests. ca|
+37|13|7|3|43|39259.43|0.05|0.08|A|F|1992-07-10|1992-07-06|1992-08-02|DELIVER IN PERSON|TRUCK|iously ste|
+38|176|5|1|44|47351.48|0.04|0.02|N|O|1996-09-29|1996-11-17|1996-09-30|COLLECT COD|MAIL|s. blithely unusual theodolites am|
+39|3|10|1|44|39732.00|0.09|0.06|N|O|1996-11-14|1996-12-15|1996-12-12|COLLECT COD|RAIL|eodolites. careful|
+39|187|8|2|26|28266.68|0.08|0.04|N|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages across the slyly silent|
+39|68|3|3|46|44530.76|0.06|0.08|N|O|1996-09-26|1996-12-19|1996-10-26|DELIVER IN PERSON|AIR|he carefully e|
+39|21|6|4|32|29472.64|0.07|0.05|N|O|1996-10-02|1996-12-19|1996-10-14|COLLECT COD|MAIL|heodolites sleep silently pending foxes. ac|
+39|55|10|5|43|41067.15|0.01|0.01|N|O|1996-10-17|1996-11-14|1996-10-26|COLLECT COD|MAIL|yly regular i|
+39|95|7|6|40|39803.60|0.06|0.05|N|O|1996-12-08|1996-10-22|1997-01-01|COLLECT COD|AIR|quickly ironic fox|
+64|86|7|1|21|20707.68|0.05|0.02|R|F|1994-09-30|1994-09-18|1994-10-26|DELIVER IN PERSON|REG AIR|ch slyly final, thin platelets.|
+65|60|5|1|26|24961.56|0.03|0.03|A|F|1995-04-20|1995-04-25|1995-05-13|NONE|TRUCK|pending deposits nag even packages. ca|
+65|74|3|2|22|21429.54|0.00|0.05|N|O|1995-07-17|1995-06-04|1995-07-19|COLLECT COD|FOB| ideas. special, r|
+65|2|5|3|21|18942.00|0.09|0.07|N|O|1995-07-06|1995-05-14|1995-07-31|DELIVER IN PERSON|RAIL|bove the even packages. accounts nag carefu|
+66|116|10|1|31|31499.41|0.00|0.08|R|F|1994-02-19|1994-03-11|1994-02-20|TAKE BACK RETURN|RAIL|ut the unusual accounts sleep at the bo|
+66|174|5|2|41|44040.97|0.04|0.07|A|F|1994-02-21|1994-03-01|1994-03-18|COLLECT COD|AIR| regular de|
+67|22|5|1|4|3688.08|0.09|0.04|N|O|1997-04-17|1997-01-31|1997-04-20|NONE|SHIP| cajole thinly expres|
+67|21|10|2|12|11052.24|0.09|0.05|N|O|1997-01-27|1997-02-21|1997-02-22|NONE|REG AIR| even packages cajole|
+67|174|4|3|5|5370.85|0.03|0.07|N|O|1997-02-20|1997-02-12|1997-02-21|DELIVER IN PERSON|TRUCK|y unusual packages thrash pinto |
+67|88|9|4|44|43475.52|0.08|0.06|N|O|1997-03-18|1997-01-29|1997-04-13|DELIVER IN PERSON|RAIL|se quickly above the even, express reques|
+67|41|10|5|23|21643.92|0.05|0.07|N|O|1997-04-19|1997-02-14|1997-05-06|DELIVER IN PERSON|REG AIR|ly regular deposit|
+67|179|9|6|29|31295.93|0.02|0.05|N|O|1997-01-25|1997-01-27|1997-01-27|DELIVER IN PERSON|FOB|ultipliers |
+68|8|1|1|3|2724.00|0.05|0.02|N|O|1998-07-04|1998-06-05|1998-07-21|NONE|RAIL|fully special instructions cajole. furious|
+68|176|4|2|46|49503.82|0.02|0.05|N|O|1998-06-26|1998-06-07|1998-07-05|NONE|MAIL| requests are unusual, regular pinto |
+68|35|1|3|46|43011.38|0.04|0.05|N|O|1998-08-13|1998-07-08|1998-08-29|NONE|RAIL|egular dependencies affix ironically along |
+68|95|9|4|20|19901.80|0.07|0.01|N|O|1998-06-27|1998-05-23|1998-07-02|NONE|REG AIR| excuses integrate fluffily |
+68|83|4|5|27|26543.16|0.03|0.06|N|O|1998-06-19|1998-06-25|1998-06-29|DELIVER IN PERSON|SHIP|ccounts. deposits use. furiously|
+68|103|6|6|30|30093.00|0.05|0.06|N|O|1998-08-11|1998-07-11|1998-08-14|NONE|RAIL|oxes are slyly blithely fin|
+68|140|6|7|41|42645.74|0.09|0.08|N|O|1998-06-24|1998-06-27|1998-07-06|NONE|SHIP|eposits nag special ideas. furiousl|
+69|116|10|1|48|48773.28|0.01|0.07|A|F|1994-08-17|1994-08-11|1994-09-08|NONE|TRUCK|regular epitaphs. carefully even ideas hag|
+69|105|10|2|32|32163.20|0.08|0.06|A|F|1994-08-24|1994-08-17|1994-08-31|NONE|REG AIR|s sleep carefully bold, |
+69|138|4|3|17|17648.21|0.09|0.00|A|F|1994-07-02|1994-07-07|1994-07-03|TAKE BACK RETURN|AIR|final, pending instr|
+69|38|9|4|3|2814.09|0.09|0.04|R|F|1994-06-06|1994-07-27|1994-06-15|NONE|MAIL| blithely final d|
+69|93|6|5|42|41709.78|0.07|0.04|R|F|1994-07-31|1994-07-26|1994-08-28|DELIVER IN PERSON|REG AIR|tect regular, speci|
+69|19|3|6|23|21137.23|0.05|0.00|A|F|1994-10-03|1994-08-06|1994-10-24|NONE|SHIP|nding accounts ca|
+70|65|2|1|8|7720.48|0.03|0.08|R|F|1994-01-12|1994-02-27|1994-01-14|TAKE BACK RETURN|FOB|ggle. carefully pending dependenc|
+70|197|10|2|13|14263.47|0.06|0.06|A|F|1994-03-03|1994-02-13|1994-03-26|COLLECT COD|AIR|lyly special packag|
+70|180|8|3|1|1080.18|0.03|0.05|R|F|1994-01-26|1994-03-05|1994-01-28|TAKE BACK RETURN|RAIL|quickly. fluffily unusual theodolites c|
+70|46|9|4|11|10406.44|0.01|0.05|A|F|1994-03-17|1994-03-17|1994-03-27|NONE|MAIL|alongside of the deposits. fur|
+70|38|9|5|37|34707.11|0.09|0.04|R|F|1994-02-13|1994-03-16|1994-02-21|COLLECT COD|MAIL|n accounts are. q|
+70|56|8|6|19|18164.95|0.06|0.03|A|F|1994-01-26|1994-02-17|1994-02-06|TAKE BACK RETURN|SHIP| packages wake pending accounts.|
+71|62|3|1|25|24051.50|0.09|0.07|N|O|1998-04-10|1998-04-22|1998-04-11|COLLECT COD|FOB|ckly. slyly|
+71|66|1|2|3|2898.18|0.09|0.07|N|O|1998-05-23|1998-04-03|1998-06-02|COLLECT COD|SHIP|y. pinto beans haggle after the|
+71|35|1|3|45|42076.35|0.00|0.07|N|O|1998-02-23|1998-03-20|1998-03-24|DELIVER IN PERSON|SHIP| ironic packages believe blithely a|
+71|97|9|4|33|32903.97|0.00|0.01|N|O|1998-04-12|1998-03-20|1998-04-15|NONE|FOB| serve quickly fluffily bold deposi|
+71|104|7|5|39|39159.90|0.08|0.06|N|O|1998-01-29|1998-04-07|1998-02-18|DELIVER IN PERSON|RAIL|l accounts sleep across the pack|
+71|196|9|6|34|37270.46|0.04|0.01|N|O|1998-03-05|1998-04-22|1998-03-30|DELIVER IN PERSON|TRUCK|s cajole. |
+96|124|7|1|23|23554.76|0.10|0.06|A|F|1994-07-19|1994-06-29|1994-07-25|DELIVER IN PERSON|TRUCK|ep-- carefully reg|
+96|136|7|2|30|31083.90|0.01|0.06|R|F|1994-06-03|1994-05-29|1994-06-22|DELIVER IN PERSON|TRUCK|e quickly even ideas. furiou|
+97|120|4|1|13|13261.56|0.00|0.02|R|F|1993-04-01|1993-04-04|1993-04-08|NONE|TRUCK|ayers cajole against the furiously|
+97|50|7|2|37|35151.85|0.02|0.06|A|F|1993-04-13|1993-03-30|1993-04-14|DELIVER IN PERSON|SHIP|ic requests boost carefully quic|
+97|78|6|3|19|18583.33|0.06|0.08|R|F|1993-05-14|1993-03-05|1993-05-25|TAKE BACK RETURN|RAIL|gifts. furiously ironic packages cajole. |
+98|41|2|1|28|26349.12|0.06|0.07|A|F|1994-12-24|1994-10-25|1995-01-16|COLLECT COD|REG AIR| pending, regular accounts s|
+98|110|7|2|1|1010.11|0.00|0.00|A|F|1994-12-01|1994-12-12|1994-12-15|DELIVER IN PERSON|TRUCK|. unusual instructions against|
+98|45|6|3|14|13230.56|0.05|0.02|A|F|1994-12-30|1994-11-22|1995-01-27|COLLECT COD|AIR| cajole furiously. blithely ironic ideas |
+98|168|9|4|10|10681.60|0.03|0.03|A|F|1994-10-23|1994-11-08|1994-11-09|COLLECT COD|RAIL| carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. qu
 ickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly
  ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly iron
 ic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic id
 eas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas c
 arefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefu
 lly. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. 
 quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quick
 ly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ir
 onic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic 
 ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas
  carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas care
 fully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully
 . quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. qui
 ckly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly 
 ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironi
 c ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ide
 as carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas ca
 refully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas careful
 ly. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. q
 uickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickl
 y ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly iro
 nic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic i
 deas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas 
 carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas caref
 ully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully.
  quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quic
 kly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly i
 ronic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic
  ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic idea
 s carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas car
 efully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefull
 y. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas|
+99|88|9|1|10|9880.80|0.02|0.01|A|F|1994-05-18|1994-06-03|1994-05-23|COLLECT COD|RAIL|kages. requ|
+99|124|5|2|5|5120.60|0.02|0.07|R|F|1994-05-06|1994-05-28|1994-05-20|TAKE BACK RETURN|RAIL|ests cajole fluffily waters. blithe|
+99|135|1|3|42|43475.46|0.02|0.02|A|F|1994-04-19|1994-05-18|1994-04-20|NONE|RAIL|kages are fluffily furiously ir|
+99|109|2|4|36|36327.60|0.09|0.02|A|F|1994-07-04|1994-04-17|1994-07-30|DELIVER IN PERSON|AIR|slyly. slyly e|
+100|63|4|1|28|26965.68|0.04|0.05|N|O|1998-05-08|1998-05-13|1998-06-07|COLLECT COD|TRUCK|sts haggle. slowl|
+100|116|10|2|22|22354.42|0.00|0.07|N|O|1998-06-24|1998-04-12|1998-06-29|DELIVER IN PERSON|SHIP|nto beans alongside of the fi|
+100|47|4|3|46|43563.84|0.03|0.04|N|O|1998-05-02|1998-04-10|1998-05-22|TAKE BACK RETURN|SHIP|ular accounts. even|
+100|39|10|4|14|13146.42|0.06|0.03|N|O|1998-05-22|1998-05-01|1998-06-03|COLLECT COD|MAIL|y. furiously ironic ideas gr|
+100|54|6|5|37|35299.85|0.05|0.00|N|O|1998-03-06|1998-04-16|1998-03-31|TAKE BACK RETURN|TRUCK|nd the quickly s|
+101|119|9|1|49|49936.39|0.10|0.00|N|O|1996-06-21|1996-05-27|1996-06-29|DELIVER IN PERSON|REG AIR|ts-- final packages sleep furiousl|
+101|164|9|2|36|38309.76|0.00|0.01|N|O|1996-05-19|1996-05-01|1996-06-04|DELIVER IN PERSON|AIR|tes. blithely pending dolphins x-ray f|
+101|139|5|3|12|12469.56|0.06|0.02|N|O|1996-03-29|1996-04-20|1996-04-12|COLLECT COD|MAIL|. quickly regular|
+102|89|10|1|37|36595.96|0.06|0.00|N|O|1997-07-24|1997-08-02|1997-08-07|TAKE BACK RETURN|SHIP|ully across the ideas. final deposit|
+102|170|5|2|34|36385.78|0.03|0.08|N|O|1997-08-09|1997-07-28|1997-08-26|TAKE BACK RETURN|SHIP|eposits cajole across|
+102|183|4|3|25|27079.50|0.01|0.01|N|O|1997-07-31|1997-07-24|1997-08-17|NONE|RAIL|bits. ironic accoun|
+102|62|7|4|15|14430.90|0.07|0.07|N|O|1997-06-02|1997-07-13|1997-06-04|DELIVER IN PERSON|SHIP|final packages. carefully even excu|
+103|195|9|1|6|6571.14|0.03|0.05|N|O|1996-10-11|1996-07-25|1996-10-28|NONE|FOB|cajole. carefully ex|
+103|11|5|2|37|33707.37|0.02|0.07|N|O|1996-09-17|1996-07-27|1996-09-20|TAKE BACK RETURN|MAIL|ies. quickly ironic requests use blithely|
+103|29|10|3|23|21367.46|0.01|0.04|N|O|1996-09-11|1996-09-18|1996-09-26|NONE|FOB|ironic accou|
+103|30|9|4|32|29760.96|0.01|0.07|N|O|1996-07-30|1996-08-06|1996-08-04|NONE|RAIL|kages doze. special, regular deposit|
+128|107|10|1|38|38269.80|0.06|0.01|A|F|1992-09-01|1992-08-27|1992-10-01|TAKE BACK RETURN|FOB| cajole careful|
+129|3|6|1|46|41538.00|0.08|0.02|R|F|1993-02-15|1993-01-24|1993-03-05|COLLECT COD|TRUCK|uietly bold theodolites. fluffil|
+129|186|7|2|36|39102.48|0.01|0.02|A|F|1992-11-25|1992-12-25|1992-12-09|TAKE BACK RETURN|REG AIR|packages are care|
+129|40|6|3|33|31021.32|0.04|0.06|A|F|1993-01-08|1993-02-14|1993-01-29|COLLECT COD|SHIP|sts nag bravely. fluffily|
+129|136|7|4|34|35228.42|0.00|0.01|R|F|1993-01-29|1993-02-14|1993-02-10|COLLECT COD|MAIL|quests. express ideas|
+129|32|8|5|24|22368.72|0.06|0.00|A|F|1992-12-07|1993-01-02|1992-12-11|TAKE BACK RETURN|FOB|uests. foxes cajole slyly after the ca|
+129|78|6|6|22|21517.54|0.06|0.01|R|F|1993-02-15|1993-01-31|1993-02-24|COLLECT COD|SHIP|e. fluffily regular |
+129|169|6|7|1|1069.16|0.05|0.04|R|F|1993-01-26|1993-01-08|1993-02-24|DELIVER IN PERSON|FOB|e carefully blithely bold dolp|
+130|129|10|1|14|14407.68|0.08|0.05|A|F|1992-08-15|1992-07-25|1992-09-13|COLLECT COD|RAIL| requests. final instruction|
+130|2|5|2|48|43296.00|0.03|0.02|R|F|1992-07-01|1992-07-12|1992-07-24|NONE|AIR|lithely alongside of the regu|
+130|12|3|3|18|16416.18|0.04|0.08|A|F|1992-07-04|1992-06-14|1992-07-29|DELIVER IN PERSON|MAIL| slyly ironic decoys abou|
+130|116|6|4|13|13209.43|0.09|0.02|R|F|1992-06-26|1992-07-29|1992-07-05|NONE|FOB| pending dolphins sleep furious|
+130|70|7|5|31|30072.17|0.06|0.05|R|F|1992-09-01|1992-07-18|1992-09-02|TAKE BACK RETURN|RAIL|thily about the ruth|
+131|168|7|1|45|48067.20|0.10|0.02|R|F|1994-09-14|1994-09-02|1994-10-04|NONE|FOB|ironic, bold accounts. careful|
+131|45|8|2|50|47252.00|0.02|0.04|A|F|1994-09-17|1994-08-10|1994-09-21|NONE|SHIP|ending requests. final, ironic pearls slee|
+131|190|1|3|4|4360.76|0.04|0.03|A|F|1994-09-20|1994-08-30|1994-09-23|COLLECT COD|REG AIR| are carefully slyly i|
+132|141|8|1|18|18740.52|0.00|0.08|R|F|1993-07-10|1993-08-05|1993-07-13|NONE|TRUCK|ges. platelets wake furio|
+132|120|1|2|43|43865.16|0.01|0.08|R|F|1993-09-01|1993-08-16|1993-09-22|NONE|TRUCK|y pending theodolites|
+132|115|6|3|32|32483.52|0.04|0.04|A|F|1993-07-12|1993-08-05|1993-08-05|COLLECT COD|TRUCK|d instructions hagg|
+132|29|2|4|23|21367.46|0.10|0.00|A|F|1993-06-16|1993-08-27|1993-06-23|DELIVER IN PERSON|AIR|refully blithely bold acco|
+133|104|7|1|27|27110.70|0.00|0.02|N|O|1997-12-21|1998-02-23|1997-12-27|TAKE BACK RETURN|MAIL|yly even gifts after the sl|
+133|177|5|2|12|12926.04|0.02|0.06|N|O|1997-12-02|1998-01-15|1997-12-29|DELIVER IN PERSON|REG AIR|ts cajole fluffily quickly i|
+133|118|8|3|29|29525.19|0.09|0.08|N|O|1998-02-28|1998-01-30|1998-03-09|DELIVER IN PERSON|RAIL| the carefully regular theodoli|
+133|90|1|4|11|10890.99|0.06|0.01|N|O|1998-03-21|1998-01-15|1998-04-04|DELIVER IN PERSON|REG AIR|e quickly across the dolphins|
+134|1|2|1|21|18921.00|0.00|0.03|A|F|1992-07-17|1992-07-08|1992-07-26|COLLECT COD|SHIP|s. quickly regular|
+134|165|2|2|35|37280.60|0.06|0.07|A|F|1992-08-23|1992-06-01|1992-08-24|NONE|MAIL|ajole furiously. instructio|
+134|189|10|3|26|28318.68|0.09|0.06|A|F|1992-06-20|1992-07-12|1992-07-16|NONE|RAIL| among the pending depos|
+134|145|6|4|47|49121.58|0.05|0.00|A|F|1992-08-16|1992-07-06|1992-08-28|NONE|REG AIR|s! carefully unusual requests boost careful|
+134|36|7|5|12|11232.36|0.05|0.02|A|F|1992-07-03|1992-06-01|1992-07-11|COLLECT COD|TRUCK|nts are quic|
+134|134|10|6|12|12409.56|0.00|0.00|A|F|1992-08-08|1992-07-07|1992-08-20|TAKE BACK RETURN|FOB|lyly regular pac|
+135|109|10|1|47|47427.70|0.06|0.08|N|O|1996-02-18|1996-01-01|1996-02-25|COLLECT COD|RAIL|ctions wake slyly abo|
+135|199|3|2|21|23082.99|0.00|0.07|N|O|1996-02-11|1996-01-12|1996-02-13|DELIVER IN PERSON|SHIP| deposits believe. furiously regular p|
+135|158|10|3|33|34918.95|0.02|0.00|N|O|1996-01-03|1995-11-21|1996-02-01|TAKE BACK RETURN|MAIL|ptotes boost slowly care|
+135|68|7|4|34|32914.04|0.02|0.03|N|O|1996-01-12|1996-01-19|1996-02-05|NONE|TRUCK|counts doze against the blithely ironi|
+135|137|8|5|20|20742.60|0.01|0.04|N|O|1996-01-25|1995-11-20|1996-02-09|NONE|MAIL|theodolites. quickly p|
+135|115|5|6|13|13196.43|0.04|0.02|N|O|1995-11-12|1995-12-22|1995-11-17|NONE|FOB|nal ideas. final instr|
+160|15|2|1|36|32940.36|0.07|0.01|N|O|1997-03-11|1997-03-11|1997-03-20|COLLECT COD|MAIL|old, ironic deposits are quickly abov|
+160|87|8|2|22|21715.76|0.00|0.04|N|O|1997-02-18|1997-03-05|1997-03-05|COLLECT COD|RAIL|ncies about the request|
+160|21|10|3|34|31314.68|0.01|0.05|N|O|1997-01-31|1997-03-13|1997-02-14|NONE|FOB|st sleep even gifts. dependencies along|
+161|103|10|1|19|19058.90|0.01|0.01|A|F|1994-12-13|1994-11-19|1994-12-26|DELIVER IN PERSON|TRUCK|, regular sheaves sleep along|
+162|190|1|1|2|2180.38|0.02|0.01|N|O|1995-09-02|1995-06-17|1995-09-08|COLLECT COD|FOB|es! final somas integrate|
+163|168|3|1|43|45930.88|0.01|0.00|N|O|1997-09-19|1997-11-19|1997-10-03|COLLECT COD|REG AIR|al, bold dependencies wake. iron|
+163|121|2|2|13|13274.56|0.01|0.04|N|O|1997-11-11|1997-10-18|1997-12-07|DELIVER IN PERSON|TRUCK|inal requests. even pinto beans hag|
+163|37|3|3|27|25299.81|0.04|0.08|N|O|1997-12-26|1997-11-28|1998-01-05|COLLECT COD|REG AIR|ously express dependen|
+163|193|5|4|5|5465.95|0.02|0.00|N|O|1997-11-17|1997-10-09|1997-12-05|DELIVER IN PERSON|TRUCK| must belie|
+163|127|2|5|12|12325.44|0.10|0.00|N|O|1997-12-18|1997-10-26|1997-12-22|COLLECT COD|TRUCK|ly blithe accounts cajole |
+163|191|4|6|20|21823.80|0.00|0.07|N|O|1997-09-27|1997-11-15|1997-10-07|TAKE BACK RETURN|FOB|tructions integrate b|
+164|92|4|1|26|25794.34|0.09|0.04|A|F|1993-01-04|1992-11-21|1993-01-07|NONE|RAIL|s. blithely special courts are blithel|
+164|19|6|2|24|22056.24|0.05|0.05|R|F|1992-12-22|1992-11-27|1993-01-06|NONE|AIR|side of the slyly unusual theodolites. f|
+164|126|9|3|38|38992.56|0.03|0.06|R|F|1992-12-04|1992-11-23|1993-01-02|TAKE BACK RETURN|AIR|counts cajole fluffily regular packages. b|
+164|18|2|4|32|29376.32|0.05|0.01|R|F|1992-12-21|1992-12-23|1992-12-28|COLLECT COD|RAIL|ts wake again|
+164|148|1|5|43|45070.02|0.06|0.01|R|F|1992-11-26|1993-01-03|1992-12-08|COLLECT COD|RAIL|y carefully regular dep|
+164|109|10|6|27|27245.70|0.10|0.04|R|F|1992-12-23|1993-01-16|1993-01-10|DELIVER IN PERSON|AIR|ayers wake carefully a|
+164|4|7|7|23|20792.00|0.09|0.04|A|F|1992-11-03|1992-12-02|1992-11-12|NONE|REG AIR|ress packages haggle ideas. blithely spec|
+165|34|5|1|3|2802.09|0.01|0.08|R|F|1993-03-29|1993-03-06|1993-04-12|DELIVER IN PERSON|REG AIR|riously requests. depos|
+165|162|7|2|43|45672.88|0.08|0.05|R|F|1993-02-27|1993-04-19|1993-03-03|DELIVER IN PERSON|TRUCK|jole slyly according |
+165|59|1|3|15|14385.75|0.00|0.05|R|F|1993-04-10|1993-03-29|1993-05-01|COLLECT COD|SHIP| bold packages mainta|
+165|140|1|4|49|50966.86|0.07|0.06|A|F|1993-02-20|1993-04-02|1993-03-10|COLLECT COD|REG AIR|uses sleep slyly ruthlessly regular a|
+165|156|4|5|27|28516.05|0.01|0.04|R|F|1993-04-27|1993-03-04|1993-05-13|NONE|MAIL|around the ironic, even orb|
+166|65|2|1|37|35707.22|0.09|0.03|N|O|1995-11-16|1995-10-17|1995-12-13|NONE|MAIL|lar frays wake blithely a|
+166|167|8|2|13|13873.08|0.09|0.05|N|O|1995-11-09|1995-11-18|1995-11-14|COLLECT COD|SHIP|fully above the blithely fina|
+166|100|2|3|41|41004.10|0.07|0.03|N|O|1995-11-13|1995-11-07|1995-12-08|COLLECT COD|FOB|hily along the blithely pending fo|
+166|46|3|4|8|7568.32|0.05|0.02|N|O|1995-12-30|1995-11-29|1996-01-29|DELIVER IN PERSON|RAIL|e carefully bold |
+167|102|3|1|28|28058.80|0.06|0.01|R|F|1993-02-19|1993-02-16|1993-03-03|DELIVER IN PERSON|TRUCK|sly during the u|
+167|172|2|2|27|28948.59|0.09|0.00|R|F|1993-05-01|1993-03-31|1993-05-31|TAKE BACK RETURN|FOB|eans affix furiously-- packages|
+192|98|1|1|23|22956.07|0.00|0.00|N|O|1998-02-05|1998-02-06|1998-03-07|TAKE BACK RETURN|AIR|ly pending theodolites haggle quickly fluf|
+192|162|7|2|20|21243.20|0.07|0.01|N|O|1998-03-13|1998-02-02|1998-03-31|TAKE BACK RETURN|REG AIR|tes. carefu|
+192|111|8|3|15|15166.65|0.09|0.01|N|O|1998-01-30|1998-02-10|1998-02-23|TAKE BACK RETURN|TRUCK|he ironic requests haggle about|
+192|197|1|4|2|2194.38|0.06|0.02|N|O|1998-03-06|1998-02-03|1998-03-24|COLLECT COD|SHIP|s. dependencies nag furiously alongside|
+192|83|4|5|25|24577.00|0.02|0.03|N|O|1998-02-15|1998-01-11|1998-03-17|COLLECT COD|TRUCK|. carefully regular|
+192|142|9|6|45|46896.30|0.00|0.05|N|O|1998-03-11|1998-01-09|1998-04-03|NONE|MAIL|equests. ideas sleep idea|
+193|93|5|1|9|8937.81|0.06|0.06|A|F|1993-09-17|1993-10-08|1993-09-30|COLLECT COD|TRUCK|against the fluffily regular d|
+193|154|6|2|15|15812.25|0.02|0.07|R|F|1993-11-22|1993-10-09|1993-12-05|TAKE BACK RETURN|SHIP|ffily. regular packages d|
+193|94|6|3|23|22864.07|0.06|0.05|A|F|1993-08-21|1993-10-11|1993-09-02|DELIVER IN PERSON|TRUCK|ly even accounts wake blithely bold|
+194|3|6|1|17|15351.00|0.05|0.04|R|F|1992-05-24|1992-05-22|1992-05-30|COLLECT COD|AIR| regular deposi|
+194|184|5|2|1|1084.18|0.04|0.06|R|F|1992-04-30|1992-05-18|1992-05-23|NONE|REG AIR| regular theodolites. regular, iron|
+194|66|1|3|13|12558.78|0.08|0.08|A|F|1992-05-07|1992-06-18|1992-05-10|NONE|AIR|about the blit|
+194|146|7|4|36|37661.04|0.00|0.05|R|F|1992-05-21|1992-05-18|1992-05-27|TAKE BACK RETURN|RAIL|pecial packages wake after the slyly r|
+194|57|2|5|8|7656.40|0.04|0.00|R|F|1992-07-06|1992-06-25|1992-07-11|COLLECT COD|FOB|uriously unusual excuses|
+194|149|6|6|16|16786.24|0.06|0.03|A|F|1992-05-14|1992-06-14|1992-05-21|TAKE BACK RETURN|TRUCK|y regular requests. furiousy regular requests. furiousy regular requests. furiousy regular

<TRUNCATED>

[08/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/data/big-object/order.tbl.big
----------------------------------------------------------------------
diff --git a/asterix-app/data/big-object/order.tbl.big b/asterix-app/data/big-object/order.tbl.big
new file mode 100644
index 0000000..4bdc986
--- /dev/null
+++ b/asterix-app/data/big-object/order.tbl.big
@@ -0,0 +1,1500 @@
+1|37|O|131251.81|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously
  among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstr
 uctions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sle
 ep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiousl
 y among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nst
 ructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sl
 eep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furious
 ly among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among ns
 tructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions s
 leep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiou
 sly among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among n
 structions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions 
 sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furio
 usly among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among 
 nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions
  sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furi
 ously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among
  nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstruction
 s sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep fur
 iously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously amon
 g nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructio
 ns sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep fu
 riously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously amo
 ng nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructi
 ons sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep f
 uriously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously am
 ong nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstruct
 ions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep 
 furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously a
 mong nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstruc
 tions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep
  furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously 
 among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiously among nstructions sleep furiousl|
+2|79|O|40183.29|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|
+3|124|F|160882.76|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|
+4|137|O|31084.79|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|
+5|46|F|86615.25|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|
+6|56|F|36468.55|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|
+7|40|O|171488.73|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |
+32|131|O|116923.00|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|
+33|67|F|99798.76|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|
+34|62|O|41670.02|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|
+35|128|O|148789.52|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|
+36|116|O|38988.98|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|
+37|88|F|113701.89|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|
+38|125|O|46366.56|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|
+39|82|O|219707.84|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|
+64|34|F|20065.73|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|
+65|17|P|65883.92|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|
+66|130|F|79258.24|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|
+67|58|O|116227.05|1996-12-19|4-NOT SPECIFIED|Clerk#000000547|0|symptotes haggle slyly around the furiously iron|
+68|29|O|215135.72|1998-04-18|3-MEDIUM|Clerk#000000440|0| pinto beans sleep carefully. blithely ironic deposits haggle furiously acro|
+69|85|F|162176.23|1994-06-04|4-NOT SPECIFIED|Clerk#000000330|0| depths atop the slyly thin deposits detect among the furiously silent accou|
+70|65|F|84651.80|1993-12-18|5-LOW|Clerk#000000322|0| carefully ironic request|
+71|4|O|178821.73|1998-01-24|4-NOT SPECIFIED|Clerk#000000271|0| express deposits along the blithely regul|
+96|109|F|55090.67|1994-04-17|2-HIGH|Clerk#000000395|0|oost furiously. pinto|
+97|22|F|68908.31|1993-01-29|3-MEDIUM|Clerk#000000547|0|hang blithely along the regular accounts. furiously even ideas after the|
+98|106|F|51004.44|1994-09-25|1-URGENT|Clerk#000000448|0|c asymptotes. quickly regular packages should have to nag re|
+99|89|F|92326.79|1994-03-13|4-NOT SPECIFIED|Clerk#000000973|0|e carefully ironic packages. pending|
+100|148|O|141311.01|1998-02-28|4-NOT SPECIFIED|Clerk#000000577|0|heodolites detect slyly alongside of the ent|
+101|28|O|95591.40|1996-03-17|3-MEDIUM|Clerk#000000419|0|ding accounts above the slyly final asymptote|
+102|1|O|113954.89|1997-05-09|2-HIGH|Clerk#000000596|0| slyly according to the asymptotes. carefully final packages integrate furious|
+103|31|O|95563.95|1996-06-20|4-NOT SPECIFIED|Clerk#000000090|0|ges. carefully unusual instructions haggle quickly regular f|
+128|74|F|36333.34|1992-06-15|1-URGENT|Clerk#000000385|0|ns integrate fluffily. ironic asymptotes after the regular excuses nag around |
+129|73|F|188124.55|1992-11-19|5-LOW|Clerk#000000859|0|ing tithes. carefully pending deposits boost about the silently express |
+130|37|F|115717.37|1992-05-08|2-HIGH|Clerk#000000036|0|le slyly unusual, regular packages? express deposits det|
+131|94|F|96596.81|1994-06-08|3-MEDIUM|Clerk#000000625|0|after the fluffily special foxes integrate s|
+132|28|F|118802.62|1993-06-11|3-MEDIUM|Clerk#000000488|0|sits are daringly accounts. carefully regular foxes sleep slyly about the|
+133|44|O|80437.72|1997-11-29|1-URGENT|Clerk#000000738|0|usly final asymptotes |
+134|7|F|154260.84|1992-05-01|4-NOT SPECIFIED|Clerk#000000711|0|lar theodolites boos|
+135|61|O|174569.88|1995-10-21|4-NOT SPECIFIED|Clerk#000000804|0|l platelets use according t|
+160|83|O|86076.86|1996-12-19|4-NOT SPECIFIED|Clerk#000000342|0|thely special sauternes wake slyly of t|
+161|17|F|19056.99|1994-08-31|2-HIGH|Clerk#000000322|0|carefully! special instructions sin|
+162|16|O|2158.13|1995-05-08|3-MEDIUM|Clerk#000000378|0|nts hinder fluffily ironic instructions. express, express excuses |
+163|88|O|125170.86|1997-09-05|3-MEDIUM|Clerk#000000379|0|y final packages. final foxes since the quickly even|
+164|1|F|202660.52|1992-10-21|5-LOW|Clerk#000000209|0|cajole ironic courts. slyly final ideas are slyly. blithely final Tiresias sub|
+165|28|F|141824.23|1993-01-30|4-NOT SPECIFIED|Clerk#000000292|0|across the blithely regular accounts. bold|
+166|109|O|93335.60|1995-09-12|2-HIGH|Clerk#000000440|0|lets. ironic, bold asymptotes kindle|
+167|121|F|52982.23|1993-01-04|4-NOT SPECIFIED|Clerk#000000731|0|s nag furiously bold excuses. fluffily iron|
+192|83|O|133002.55|1997-11-25|5-LOW|Clerk#000000483|0|y unusual platelets among the final instructions integrate rut|
+193|80|F|48053.18|1993-08-08|1-URGENT|Clerk#000000025|0|the furiously final pin|
+194|62|F|114097.63|1992-04-05|3-MEDIUM|Clerk#000000352|0|egular requests haggle slyly regular, regular pinto beans. asymptote|
+195|136|F|120053.52|1993-12-28|3-MEDIUM|Clerk#000000216|0|old forges are furiously sheaves. slyly fi|
+196|65|F|33248.04|1993-03-17|2-HIGH|Clerk#000000988|0|beans boost at the foxes. silent foxes|
+197|34|P|100290.07|1995-04-07|2-HIGH|Clerk#000000969|0|solve quickly about the even braids. carefully express deposits affix care|
+198|112|O|125792.83|1998-01-02|4-NOT SPECIFIED|Clerk#000000331|0|its. carefully ironic requests sleep. furiously express fox|
+199|53|O|80592.44|1996-03-07|2-HIGH|Clerk#000000489|0|g theodolites. special packag|
+224|4|F|155680.60|1994-06-18|4-NOT SPECIFIED|Clerk#000000642|0|r the quickly thin courts. carefully|
+225|34|P|165890.47|1995-05-25|1-URGENT|Clerk#000000177|0|s. blithely ironic accounts wake quickly fluffily special acc|
+226|128|F|180119.22|1993-03-10|2-HIGH|Clerk#000000756|0|s are carefully at the blithely ironic acc|
+227|10|O|46076.46|1995-11-10|5-LOW|Clerk#000000919|0| express instructions. slyly regul|
+228|46|F|2638.98|1993-02-25|1-URGENT|Clerk#000000562|0|es was slyly among the regular foxes. blithely regular dependenci|
+229|112|F|142290.77|1993-12-29|1-URGENT|Clerk#000000628|0|he fluffily even instructions. furiously i|
+230|103|F|107231.60|1993-10-27|1-URGENT|Clerk#000000520|0|odolites. carefully quick requ|
+231|91|F|141554.06|1994-09-29|2-HIGH|Clerk#000000446|0| packages haggle slyly after the carefully ironic instruct|
+256|125|F|106315.25|1993-10-19|4-NOT SPECIFIED|Clerk#000000834|0|he fluffily final ideas might are final accounts. carefully f|
+257|124|O|7102.74|1998-03-28|3-MEDIUM|Clerk#000000680|0|ts against the sly warhorses cajole slyly accounts|
+258|43|F|186669.10|1993-12-29|1-URGENT|Clerk#000000167|0|dencies. blithely quick packages cajole. ruthlessly final accounts|
+259|44|F|75661.70|1993-09-29|4-NOT SPECIFIED|Clerk#000000601|0|ages doubt blithely against the final foxes. carefully express deposits dazzle|
+260|106|O|179292.14|1996-12-10|3-MEDIUM|Clerk#000000960|0|lently regular pinto beans sleep after the slyly e|
+261|47|F|201003.12|1993-06-29|3-MEDIUM|Clerk#000000310|0|ully fluffily brave instructions. furiousl|
+262|31|O|108443.84|1995-11-25|4-NOT SPECIFIED|Clerk#000000551|0|l packages. blithely final pinto beans use carefu|
+263|118|F|79782.56|1994-05-17|2-HIGH|Clerk#000000088|0| pending instructions. blithely un|
+288|8|O|163794.53|1997-02-21|1-URGENT|Clerk#000000109|0|uriously final requests. even, final ideas det|
+289|104|O|131092.67|1997-02-10|3-MEDIUM|Clerk#000000103|0|sily. slyly special excuse|
+290|118|F|62814.89|1994-01-01|4-NOT SPECIFIED|Clerk#000000735|0|efully dogged deposits. furiou|
+291|142|F|66817.05|1994-03-13|1-URGENT|Clerk#000000923|0|dolites. carefully regular pinto beans cajol|
+292|23|F|30783.05|1992-01-13|2-HIGH|Clerk#000000193|0|g pinto beans will have to sleep f|
+293|31|F|37248.78|1992-10-02|2-HIGH|Clerk#000000629|0|re bold, ironic deposits. platelets c|
+294|52|F|30059.47|1993-07-16|3-MEDIUM|Clerk#000000499|0|kly according to the frays. final dolphins affix quickly |
+295|19|F|89345.99|1994-09-29|2-HIGH|Clerk#000000155|0| unusual pinto beans play. regular ideas haggle|
+320|1|O|39835.54|1997-11-21|2-HIGH|Clerk#000000573|0|ar foxes nag blithely|
+321|124|F|62251.15|1993-03-21|3-MEDIUM|Clerk#000000289|0|equests run. blithely final dependencies after the deposits wake caref|
+322|134|F|127068.89|1992-03-19|1-URGENT|Clerk#000000158|0|fully across the slyly bold packages. packages against the quickly regular i|
+323|40|F|79683.42|1994-03-26|1-URGENT|Clerk#000000959|0|arefully pending foxes sleep blithely. slyly express accoun|
+324|106|F|26868.85|1992-03-20|1-URGENT|Clerk#000000352|0| about the ironic, regular deposits run blithely against the excuses|
+325|41|F|71543.41|1993-10-17|5-LOW|Clerk#000000844|0|ly sometimes pending pa|
+326|76|O|229165.17|1995-06-04|2-HIGH|Clerk#000000466|0| requests. furiously ironic asymptotes mold carefully alongside of the blit|
+327|145|P|24468.16|1995-04-17|5-LOW|Clerk#000000992|0|ng the slyly final courts. slyly even escapades eat |
+352|107|F|16003.86|1994-03-08|2-HIGH|Clerk#000000932|0|ke slyly bold pinto beans. blithely regular accounts against the spe|
+353|2|F|179984.42|1993-12-31|5-LOW|Clerk#000000449|0| quiet ideas sleep. even instructions cajole slyly. silently spe|
+354|139|O|157062.70|1996-03-14|2-HIGH|Clerk#000000511|0|ly regular ideas wake across the slyly silent ideas. final deposits eat b|
+355|71|F|69447.25|1994-06-14|5-LOW|Clerk#000000532|0|s. sometimes regular requests cajole. regular, pending accounts a|
+356|148|F|162786.67|1994-06-30|4-NOT SPECIFIED|Clerk#000000944|0|as wake along the bold accounts. even, |
+357|61|O|98723.11|1996-10-09|2-HIGH|Clerk#000000301|0|e blithely about the express, final accounts. quickl|
+358|4|F|226806.66|1993-09-20|2-HIGH|Clerk#000000392|0|l, silent instructions are slyly. silently even de|
+359|79|F|142891.22|1994-12-19|3-MEDIUM|Clerk#000000934|0|n dolphins. special courts above the carefully ironic requests use|
+384|115|F|122785.82|1992-03-03|5-LOW|Clerk#000000206|0|, even accounts use furiously packages. slyly ironic pla|
+385|34|O|50724.06|1996-03-22|5-LOW|Clerk#000000600|0|hless accounts unwind bold pain|
+386|61|F|90380.40|1995-01-25|2-HIGH|Clerk#000000648|0| haggle quickly. stealthily bold asymptotes haggle among the furiously even re|
+387|4|O|130647.18|1997-01-26|4-NOT SPECIFIED|Clerk#000000768|0| are carefully among the quickly even deposits. furiously silent req|
+388|46|F|120533.46|1992-12-16|4-NOT SPECIFIED|Clerk#000000356|0|ar foxes above the furiously ironic deposits nag slyly final reque|
+389|127|F|1984.14|1994-02-17|2-HIGH|Clerk#000000062|0|ing to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes abo
 ut the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regul
 ar asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes
  about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the r
 egular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending f
 oxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to t
 he regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pendi
 ng foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling 
 to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, p
 ending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely sil
 ing to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. fina
 l, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely
  siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. 
 final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blit
 hely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptot
 es. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the 
 blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asym
 ptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about 
 the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular 
 asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes ab
 out the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regu
 lar asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxe
 s about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the 
 regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending 
 foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to 
 the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pend
 ing foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling
  to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, 
 pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely si
 ling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. fin
 al, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithel
 y siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes.
  final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about the blithely siling to the regular asymptotes. final, pending foxes about th|
+390|103|O|168562.27|1998-04-07|5-LOW|Clerk#000000404|0|xpress asymptotes use among the regular, final pinto b|
+391|112|F|13282.23|1994-11-17|2-HIGH|Clerk#000000256|0|orges thrash fluffil|
+416|41|F|71362.50|1993-09-27|5-LOW|Clerk#000000294|0| the accounts. fluffily bold depo|
+417|55|F|91982.29|1994-02-06|3-MEDIUM|Clerk#000000468|0|ironic, even packages. thinly unusual accounts sleep along the slyly unusual |
+418|95|P|33124.96|1995-04-13|4-NOT SPECIFIED|Clerk#000000643|0|. furiously ironic instruc|
+419|118|O|111597.96|1996-10-01|3-MEDIUM|Clerk#000000376|0|osits. blithely pending theodolites boost carefully|
+420|91|O|198039.23|1995-10-31|4-NOT SPECIFIED|Clerk#000000756|0|leep carefully final excuses. fluffily pending requests unwind carefully above|
+421|40|F|1084.38|1992-02-22|5-LOW|Clerk#000000405|0|egular, even packages according to the final, un|
+422|74|O|106045.89|1997-05-31|4-NOT SPECIFIED|Clerk#000000049|0|aggle carefully across the accounts. regular accounts eat fluffi|
+423|104|O|26981.31|1996-06-01|1-URGENT|Clerk#000000674|0|quests. deposits cajole quickly. furiously bold accounts haggle q|
+448|149|O|114978.03|1995-08-21|3-MEDIUM|Clerk#000000597|0| regular, express foxes use blithely. quic|
+449|97|O|41605.63|1995-07-20|2-HIGH|Clerk#000000841|0|. furiously regular theodolites affix blithely |
+450|49|P|153386.61|1995-03-05|4-NOT SPECIFIED|Clerk#000000293|0|d theodolites. boldly bold foxes since the pack|
+451|100|O|104664.40|1998-05-25|5-LOW|Clerk#000000048|0|nic pinto beans. theodolites poach carefully; |
+452|61|O|2007.48|1997-10-14|1-URGENT|Clerk#000000498|0|t, unusual instructions above the blithely bold pint|
+453|46|O|216826.73|1997-05-26|5-LOW|Clerk#000000504|0|ss foxes. furiously regular ideas sleep according to t|
+454|49|O|23198.24|1995-12-27|5-LOW|Clerk#000000890|0|dolites sleep carefully blithely regular deposits. quickly regul|
+455|13|O|138010.76|1996-12-04|1-URGENT|Clerk#000000796|0| about the final platelets. dependen|
+480|73|F|20530.97|1993-05-08|5-LOW|Clerk#000000004|0|ealthy pinto beans. fluffily regular requests along the special sheaves wake |
+481|31|F|117827.18|1992-10-08|2-HIGH|Clerk#000000230|0|ly final ideas. packages haggle fluffily|
+482|127|O|136634.34|1996-03-26|1-URGENT|Clerk#000000295|0|ts. deposits wake: final acco|
+483|35|O|39793.05|1995-07-11|2-HIGH|Clerk#000000025|0|cross the carefully final e|
+484|55|O|219920.62|1997-01-03|3-MEDIUM|Clerk#000000545|0|grouches use. furiously bold accounts maintain. bold, regular deposits|
+485|101|O|110432.76|1997-03-26|2-HIGH|Clerk#000000105|0| regular ideas nag thinly furiously s|
+486|52|O|185968.15|1996-03-11|4-NOT SPECIFIED|Clerk#000000803|0|riously dolphins. fluffily ironic requ|
+487|109|F|48502.79|1992-08-18|1-URGENT|Clerk#000000086|0|ithely unusual courts eat accordi|
+512|64|P|124661.48|1995-05-20|5-LOW|Clerk#000000814|0|ding requests. carefully express theodolites was quickly. furious|
+513|61|O|63703.92|1995-05-01|2-HIGH|Clerk#000000522|0|regular packages. pinto beans cajole carefully against the even|
+514|76|O|104585.77|1996-04-04|2-HIGH|Clerk#000000094|0| cajole furiously. slyly final excuses cajole. slyly special instructions |
+515|142|F|153720.22|1993-08-29|4-NOT SPECIFIED|Clerk#000000700|0|eposits are furiously furiously silent pinto beans. pending pack|
+516|44|O|10677.86|1998-04-21|2-HIGH|Clerk#000000305|0|lar, unusual platelets are carefully. even courts sleep bold, final pinto bea|
+517|10|O|82197.79|1997-04-07|5-LOW|Clerk#000000359|0|slyly pending deposits cajole quickly packages. furiou|
+518|145|O|223537.09|1998-02-08|2-HIGH|Clerk#000000768|0| the carefully bold accounts. quickly regular excuses are|
+519|64|O|95731.50|1997-10-31|1-URGENT|Clerk#000000985|0|ains doze furiously against the f|
+544|94|F|47627.89|1993-02-17|2-HIGH|Clerk#000000145|0|the special, final accounts. dogged dolphins|
+545|64|O|23476.12|1995-11-07|2-HIGH|Clerk#000000537|0|as. blithely final hockey players about th|
+546|145|O|14790.37|1996-11-01|2-HIGH|Clerk#000000041|0|osits sleep. slyly special dolphins about the q|
+547|100|O|96855.29|1996-06-22|3-MEDIUM|Clerk#000000976|0|ing accounts eat. carefully regular packa|
+548|124|F|99088.75|1994-09-21|1-URGENT|Clerk#000000435|0|arefully express instru|
+549|110|F|141679.41|1992-07-13|1-URGENT|Clerk#000000196|0|ideas alongside of |
+550|25|O|33123.28|1995-08-02|1-URGENT|Clerk#000000204|0|t requests. blithely |
+551|91|O|46355.83|1995-05-30|1-URGENT|Clerk#000000179|0|xpress accounts boost quic|
+576|31|O|18307.45|1997-05-13|3-MEDIUM|Clerk#000000955|0|l requests affix regular requests. final account|
+577|56|F|34768.68|1994-12-19|5-LOW|Clerk#000000154|0| deposits engage stealthil|
+578|94|O|70392.02|1997-01-10|5-LOW|Clerk#000000281|0|e blithely even packages. slyly pending platelets bes|
+579|68|O|120828.12|1998-03-11|2-HIGH|Clerk#000000862|0| regular instructions. blithely even p|
+580|61|O|88219.12|1997-07-05|2-HIGH|Clerk#000000314|0|tegrate fluffily regular accou|
+581|70|O|126066.00|1997-02-23|4-NOT SPECIFIED|Clerk#000000239|0| requests. even requests use slyly. blithely ironic |
+582|50|O|129004.81|1997-10-21|1-URGENT|Clerk#000000378|0|n pinto beans print a|
+583|49|O|127817.38|1997-03-19|3-MEDIUM|Clerk#000000792|0|efully express requests. a|
+608|26|O|62567.99|1996-02-28|3-MEDIUM|Clerk#000000995|0|nic waters wake slyly slyly expre|
+609|127|F|21088.59|1994-06-01|3-MEDIUM|Clerk#000000348|0|- ironic gifts believe furiously ca|
+610|52|O|175142.28|1995-08-02|1-URGENT|Clerk#000000610|0|totes. ironic, unusual packag|
+611|106|F|73907.63|1993-01-27|1-URGENT|Clerk#000000401|0|ounts detect furiously ac|
+612|82|F|145695.42|1992-10-21|3-MEDIUM|Clerk#000000759|0|boost quickly quickly final excuses. final foxes use bravely afte|
+613|139|O|33396.35|1995-06-18|2-HIGH|Clerk#000000172|0|ts hinder among the deposits. fluffily ironic depos|
+614|134|F|218116.21|1992-12-01|2-HIGH|Clerk#000000388|0| deposits! even, daring theodol|
+615|67|F|32890.89|1992-05-09|5-LOW|Clerk#000000388|0|t to promise asymptotes. packages haggle alongside of the fluffil|
+640|97|F|145495.62|1993-01-23|2-HIGH|Clerk#000000433|0|r, unusual accounts boost carefully final ideas. slyly silent theod|
+641|133|F|120626.49|1993-08-30|5-LOW|Clerk#000000175|0|ents cajole furiously about the quickly silent pac|
+642|40|F|22994.51|1993-12-16|3-MEDIUM|Clerk#000000357|0| among the requests wake slyly alongside of th|
+643|58|P|180396.95|1995-03-25|2-HIGH|Clerk#000000354|0|g dependencies. regular accounts |
+644|8|F|201268.06|1992-05-01|1-URGENT|Clerk#000000550|0| blithely unusual platelets haggle ironic, special excuses. excuses unwi|
+645|115|F|234763.73|1994-12-03|2-HIGH|Clerk#000000090|0|quickly daring theodolites across the regu|
+646|52|F|142070.65|1994-11-22|2-HIGH|Clerk#000000203|0|carefully even foxes. fina|
+647|143|O|56449.23|1997-08-07|1-URGENT|Clerk#000000270|0|egular pearls. carefully express asymptotes are. even account|
+672|109|F|89877.09|1994-04-14|5-LOW|Clerk#000000106|0|egular requests are furiously according to |
+673|80|F|21137.08|1994-03-10|1-URGENT|Clerk#000000448|0| special pinto beans use quickly furiously even depende|
+674|34|F|27204.60|1992-08-29|5-LOW|Clerk#000000448|0|ully special deposits. furiously final warhorses affix carefully. fluffily f|
+675|13|O|125188.72|1997-07-31|2-HIGH|Clerk#000000168|0|ffily between the careful|
+676|38|O|163966.67|1996-12-13|2-HIGH|Clerk#000000248|0|the final deposits. special, pending|
+677|124|F|147915.68|1993-11-24|3-MEDIUM|Clerk#000000824|0|uriously special pinto beans cajole carefully. fi|
+678|131|F|135761.05|1993-02-27|5-LOW|Clerk#000000530|0|. blithely final somas about the|
+679|49|O|8945.03|1995-12-15|2-HIGH|Clerk#000000853|0|tealthy, final pinto beans haggle slyly. pending platelets about the special, |
+704|85|O|56210.26|1996-11-21|3-MEDIUM|Clerk#000000682|0|blithely pending platelets wake alongside of the final, iron|
+705|43|O|83773.49|1997-02-13|4-NOT SPECIFIED|Clerk#000000294|0|ithely regular dependencies. express, even packages sleep slyly pending t|
+706|148|O|23973.60|1995-09-09|1-URGENT|Clerk#000000448|0|g the packages. deposits caj|
+707|118|F|58218.35|1994-11-20|3-MEDIUM|Clerk#000000199|0| ideas about the silent, bold deposits nag dolphins|
+708|32|O|100445.59|1998-07-03|3-MEDIUM|Clerk#000000101|0|lphins cajole about t|
+709|37|O|72055.87|1998-04-21|1-URGENT|Clerk#000000461|0|ons alongside of the carefully bold pinto bea|
+710|133|F|208974.42|1993-01-02|5-LOW|Clerk#000000026|0| regular, regular requests boost. fluffily re|
+711|64|F|92484.70|1993-09-23|4-NOT SPECIFIED|Clerk#000000856|0|its. fluffily regular gifts are furi|
+736|47|O|130204.17|1998-06-21|5-LOW|Clerk#000000881|0|refully of the final pi|
+737|121|F|12984.85|1992-04-26|5-LOW|Clerk#000000233|0|ake blithely express, ironic theodolites. blithely special accounts wa|
+738|22|F|114145.18|1993-03-02|4-NOT SPECIFIED|Clerk#000000669|0|ly even foxes. furiously regular accounts cajole ca|
+739|1|O|159171.69|1998-05-31|5-LOW|Clerk#000000900|0| against the slyly ironic packages nag slyly ironic|
+740|44|O|83490.99|1995-07-16|3-MEDIUM|Clerk#000000583|0|courts haggle furiously across the final, regul|
+741|106|O|47985.98|1998-07-07|2-HIGH|Clerk#000000295|0|ic instructions. slyly express instructions solv|
+742|103|F|207632.55|1994-12-23|5-LOW|Clerk#000000543|0|equests? slyly ironic dolphins boost carefully above the blithely|
+743|79|O|23614.89|1996-10-04|4-NOT SPECIFIED|Clerk#000000933|0|eans. furiously ironic deposits sleep carefully carefully qui|
+768|98|O|220636.82|1996-08-20|3-MEDIUM|Clerk#000000411|0|jole slyly ironic packages. slyly even idea|
+769|80|F|43092.76|1993-06-02|3-MEDIUM|Clerk#000000172|0|ggle furiously. ironic packages haggle slyly. bold platelets affix s|
+770|32|O|64271.75|1998-05-23|5-LOW|Clerk#000000572|0|heodolites. furiously special pinto beans cajole pac|
+771|46|O|105302.05|1995-06-17|1-URGENT|Clerk#000000105|0|s. furiously final instructions across the deposit|
+772|97|F|128234.96|1993-04-17|2-HIGH|Clerk#000000430|0|s boost blithely fluffily idle ideas? fluffily even pin|
+773|133|F|146862.27|1993-09-26|3-MEDIUM|Clerk#000000307|0|tions are quickly accounts. accounts use bold, even pinto beans. gifts ag|
+774|80|O|145857.60|1995-12-04|1-URGENT|Clerk#000000883|0|tealthily even depths|
+775|134|F|59455.61|1995-03-18|1-URGENT|Clerk#000000191|0|kly express requests. fluffily silent accounts poach furiously|
+800|56|O|87892.38|1998-07-14|2-HIGH|Clerk#000000213|0|y alongside of the pending packages? final platelets nag fluffily carefu|
+801|118|F|127717.72|1992-02-18|1-URGENT|Clerk#000000186|0|iously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing re
 quesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiously from the furiously enticing requesiou

<TRUNCATED>

[03/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/results/big-object/big_object_sort/sort.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/big-object/big_object_sort/sort.1.adm b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_sort/sort.1.adm
new file mode 100644
index 0000000..3303425
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_sort/sort.1.adm
@@ -0,0 +1,1501 @@
+[ { "custkey": 1i32, "orderkey": 102i32, "len-comment": 78, "comment": " slyly according to the asymptotes. carefully final packages integrate furious" }
+, { "custkey": 1i32, "orderkey": 164i32, "len-comment": 78, "comment": "cajole ironic courts. slyly final ideas are slyly. blithely final Tiresias sub" }
+, { "custkey": 1i32, "orderkey": 320i32, "len-comment": 21, "comment": "ar foxes nag blithely" }
+, { "custkey": 1i32, "orderkey": 739i32, "len-comment": 51, "comment": " against the slyly ironic packages nag slyly ironic" }
+, { "custkey": 1i32, "orderkey": 1602i32, "len-comment": 58, "comment": "deposits. busily silent instructions haggle furiously. fin" }
+, { "custkey": 2i32, "orderkey": 353i32, "len-comment": 64, "comment": " quiet ideas sleep. even instructions cajole slyly. silently spe" }
+, { "custkey": 2i32, "orderkey": 896i32, "len-comment": 73, "comment": "inal packages eat blithely according to the warhorses. furiously quiet de" }
+, { "custkey": 2i32, "orderkey": 994i32, "len-comment": 33, "comment": "ole. slyly bold excuses nag caref" }
+, { "custkey": 2i32, "orderkey": 1504i32, "len-comment": 25, "comment": ", brave deposits. bold de" }
+, { "custkey": 2i32, "orderkey": 1603i32, "len-comment": 28, "comment": "s. slyly silent deposits boo" }
+, { "custkey": 2i32, "orderkey": 1669i32, "len-comment": 47, "comment": "er ironic requests detect furiously blithely sp" }
+, { "custkey": 2i32, "orderkey": 4704i32, "len-comment": 50, "comment": "lithely final requests about the fluffily regular " }
+, { "custkey": 2i32, "orderkey": 5507i32, "len-comment": 49, "comment": "the carefully ironic instructions are quickly iro" }
+, { "custkey": 2i32, "orderkey": 5893i32, "len-comment": 58, "comment": "final sentiments. instructions boost above the never speci" }
+, { "custkey": 4i32, "orderkey": 71i32, "len-comment": 42, "comment": " express deposits along the blithely regul" }
+, { "custkey": 4i32, "orderkey": 224i32, "len-comment": 36, "comment": "r the quickly thin courts. carefully" }
+, { "custkey": 4i32, "orderkey": 358i32, "len-comment": 50, "comment": "l, silent instructions are slyly. silently even de" }
+, { "custkey": 4i32, "orderkey": 387i32, "len-comment": 68, "comment": " are carefully among the quickly even deposits. furiously silent req" }
+, { "custkey": 4i32, "orderkey": 865i32, "len-comment": 76, "comment": ". special packages wake after the carefully final accounts. express pinto be" }
+, { "custkey": 4i32, "orderkey": 1024i32, "len-comment": 56, "comment": " blithely. even, express theodolites cajole slyly across" }
+, { "custkey": 4i32, "orderkey": 1031i32, "len-comment": 44, "comment": "s; ironic theodolites along the carefully ex" }
+, { "custkey": 4i32, "orderkey": 1635i32, "len-comment": 37, "comment": "s. slyly ironic requests affix slyly " }
+, { "custkey": 4i32, "orderkey": 1696i32, "len-comment": 44, "comment": "bravely bold accounts above the quickly bold" }
+, { "custkey": 4i32, "orderkey": 2374i32, "len-comment": 43, "comment": " blithely regular packages. blithely unusua" }
+, { "custkey": 4i32, "orderkey": 2980i32, "len-comment": 34, "comment": "y quick pinto beans wake. slyly re" }
+, { "custkey": 4i32, "orderkey": 3266i32, "len-comment": 43, "comment": "refully ironic instructions. slyly final pi" }
+, { "custkey": 4i32, "orderkey": 3329i32, "len-comment": 46, "comment": "old deposits. special accounts haggle furiousl" }
+, { "custkey": 4i32, "orderkey": 3427i32, "len-comment": 49, "comment": "y final pinto beans snooze fluffily bold asymptot" }
+, { "custkey": 4i32, "orderkey": 3623i32, "len-comment": 40, "comment": "- ironic excuses boost quickly in place " }
+, { "custkey": 4i32, "orderkey": 4100i32, "len-comment": 68, "comment": "posits. carefully unusual packages use pending deposits. regular she" }
+, { "custkey": 4i32, "orderkey": 4165i32, "len-comment": 64, "comment": "special foxes affix never blithely ironic pinto beans; blithely " }
+, { "custkey": 4i32, "orderkey": 4193i32, "len-comment": 61, "comment": "ng accounts haggle quickly. packages use fluffily ironic excu" }
+, { "custkey": 4i32, "orderkey": 4263i32, "len-comment": 58, "comment": "sly ruthless deposits. final packages are instructions. fu" }
+, { "custkey": 4i32, "orderkey": 4355i32, "len-comment": 42, "comment": "ndencies use furiously across the regular " }
+, { "custkey": 4i32, "orderkey": 4451i32, "len-comment": 51, "comment": ". carefully final foxes along the quickly express T" }
+, { "custkey": 4i32, "orderkey": 4928i32, "len-comment": 73, "comment": "slyly brave instructions after the ironic excuses haggle ruthlessly about" }
+, { "custkey": 5i32, "orderkey": 1475i32, "len-comment": 49, "comment": "cally final packages boost. blithely ironic packa" }
+, { "custkey": 5i32, "orderkey": 1639i32, "len-comment": 47, "comment": "haggle furiously. final requests detect furious" }
+, { "custkey": 5i32, "orderkey": 1667i32, "len-comment": 45, "comment": "e accounts. slyly express accounts must are a" }
+, { "custkey": 5i32, "orderkey": 2375i32, "len-comment": 46, "comment": "unusual, pending theodolites cajole carefully " }
+, { "custkey": 5i32, "orderkey": 3170i32, "len-comment": 24, "comment": "requests. furiously bold" }
+, { "custkey": 5i32, "orderkey": 4199i32, "len-comment": 69, "comment": "e blithely. special deposits haggle slyly final foxes. carefully even" }
+, { "custkey": 5i32, "orderkey": 4295i32, "len-comment": 26, "comment": "e boldly bold dependencies" }
+, { "custkey": 5i32, "orderkey": 4738i32, "len-comment": 19, "comment": "deposits. thin acco" }
+, { "custkey": 5i32, "orderkey": 5859i32, "len-comment": 75, "comment": "requests boost. asymptotes across the deposits solve slyly furiously pendin" }
+, { "custkey": 7i32, "orderkey": 134i32, "len-comment": 20, "comment": "lar theodolites boos" }
+, { "custkey": 7i32, "orderkey": 1217i32, "len-comment": 56, "comment": " foxes nag quickly. ironic excuses nod. blithely pending" }
+, { "custkey": 7i32, "orderkey": 1734i32, "len-comment": 68, "comment": " final ideas haggle. blithely quick foxes sleep busily bold ideas. i" }
+, { "custkey": 7i32, "orderkey": 1895i32, "len-comment": 72, "comment": "ress accounts. bold accounts cajole. slyly final pinto beans poach regul" }
+, { "custkey": 7i32, "orderkey": 1985i32, "len-comment": 33, "comment": "slyly slyly even pains. slyly reg" }
+, { "custkey": 7i32, "orderkey": 2406i32, "len-comment": 27, "comment": "blithely regular accounts u" }
+, { "custkey": 7i32, "orderkey": 2503i32, "len-comment": 76, "comment": "ly even packages was. ironic, regular deposits unwind furiously across the p" }
+, { "custkey": 7i32, "orderkey": 2691i32, "len-comment": 66, "comment": "es at the regular deposits sleep slyly by the fluffy requests. eve" }
+, { "custkey": 7i32, "orderkey": 2726i32, "len-comment": 31, "comment": " blithely even dinos sleep care" }
+, { "custkey": 7i32, "orderkey": 2885i32, "len-comment": 49, "comment": "ly sometimes special excuses. final requests are " }
+, { "custkey": 7i32, "orderkey": 3328i32, "len-comment": 71, "comment": "ake among the express accounts? carefully ironic packages cajole never." }
+, { "custkey": 7i32, "orderkey": 3330i32, "len-comment": 61, "comment": "kages use. carefully regular deposits cajole carefully about " }
+, { "custkey": 7i32, "orderkey": 3521i32, "len-comment": 50, "comment": "y even instructions cajole carefully above the bli" }
+, { "custkey": 7i32, "orderkey": 3654i32, "len-comment": 65, "comment": "s cajole slyly carefully special theodolites. even deposits haggl" }
+, { "custkey": 7i32, "orderkey": 4390i32, "len-comment": 21, "comment": "inal pinto beans. exp" }
+, { "custkey": 7i32, "orderkey": 4806i32, "len-comment": 34, "comment": "ave accounts. furiously pending wa" }
+, { "custkey": 7i32, "orderkey": 5446i32, "len-comment": 20, "comment": " furiously final pac" }
+, { "custkey": 7i32, "orderkey": 5670i32, "len-comment": 68, "comment": "he carefully final packages. deposits are slyly among the requests. " }
+, { "custkey": 7i32, "orderkey": 5953i32, "len-comment": 40, "comment": "ages are furiously. slowly bold requests" }
+, { "custkey": 8i32, "orderkey": 288i32, "len-comment": 46, "comment": "uriously final requests. even, final ideas det" }
+, { "custkey": 8i32, "orderkey": 644i32, "len-comment": 72, "comment": " blithely unusual platelets haggle ironic, special excuses. excuses unwi" }
+, { "custkey": 8i32, "orderkey": 2016i32, "len-comment": 48, "comment": "the carefully ironic foxes. requests nag bold, r" }
+, { "custkey": 8i32, "orderkey": 2178i32, "len-comment": 44, "comment": "thely according to the instructions. furious" }
+, { "custkey": 8i32, "orderkey": 2880i32, "len-comment": 26, "comment": "ves maintain doggedly spec" }
+, { "custkey": 8i32, "orderkey": 2916i32, "len-comment": 39, "comment": "ithely blithe deposits sleep beyond the" }
+, { "custkey": 8i32, "orderkey": 3010i32, "len-comment": 51, "comment": " blithely final requests. special deposits are slyl" }
+, { "custkey": 8i32, "orderkey": 3142i32, "len-comment": 77, "comment": "usual accounts about the carefully special requests sleep slyly quickly regul" }
+, { "custkey": 8i32, "orderkey": 4007i32, "len-comment": 49, "comment": "ecial packages. slyly regular accounts integrate " }
+, { "custkey": 8i32, "orderkey": 5154i32, "len-comment": 70, "comment": "inal requests. slyly regular deposits nag. even deposits haggle agains" }
+, { "custkey": 8i32, "orderkey": 5572i32, "len-comment": 50, "comment": "e fluffily express deposits cajole slyly across th" }
+, { "custkey": 8i32, "orderkey": 5731i32, "len-comment": 48, "comment": " silent excuses among the express accounts wake " }
+, { "custkey": 8i32, "orderkey": 5763i32, "len-comment": 69, "comment": "according to the furiously regular pinto beans. even accounts wake fu" }
+, { "custkey": 8i32, "orderkey": 5794i32, "len-comment": 37, "comment": "t accounts kindle about the gifts. as" }
+, { "custkey": 10i32, "orderkey": 227i32, "len-comment": 34, "comment": " express instructions. slyly regul" }
+, { "custkey": 10i32, "orderkey": 517i32, "len-comment": 54, "comment": "slyly pending deposits cajole quickly packages. furiou" }
+, { "custkey": 10i32, "orderkey": 902i32, "len-comment": 44, "comment": "yly final requests over the furiously regula" }
+, { "custkey": 10i32, "orderkey": 1218i32, "len-comment": 49, "comment": "s cajole. special, silent deposits about the theo" }
+, { "custkey": 10i32, "orderkey": 1222i32, "len-comment": 62, "comment": "theodolites use quickly even accounts. carefully final asympto" }
+, { "custkey": 10i32, "orderkey": 1223i32, "len-comment": 22, "comment": "posits was blithely fr" }
+, { "custkey": 10i32, "orderkey": 1860i32, "len-comment": 42, "comment": "osits. quickly bold deposits according to " }
+, { "custkey": 10i32, "orderkey": 1890i32, "len-comment": 43, "comment": "romise final, regular deposits. regular fox" }
+, { "custkey": 10i32, "orderkey": 2562i32, "len-comment": 41, "comment": "elets. pending dolphins promise slyly. bo" }
+, { "custkey": 10i32, "orderkey": 3204i32, "len-comment": 47, "comment": "ess somas cajole slyly. pending accounts cajole" }
+, { "custkey": 10i32, "orderkey": 3428i32, "len-comment": 41, "comment": "lar excuses. slyly pending ideas detect p" }
+, { "custkey": 10i32, "orderkey": 3618i32, "len-comment": 37, "comment": ". ideas run carefully. thin, pending " }
+, { "custkey": 10i32, "orderkey": 3751i32, "len-comment": 27, "comment": "sheaves. express, unusual t" }
+, { "custkey": 10i32, "orderkey": 3843i32, "len-comment": 45, "comment": "eodolites; slyly unusual accounts nag boldly " }
+, { "custkey": 10i32, "orderkey": 3911i32, "len-comment": 63, "comment": "he fluffily final forges haggle slyly according to the blithely" }
+, { "custkey": 10i32, "orderkey": 4032i32, "len-comment": 50, "comment": "iresias sleep slyly regular ideas. quickly unusual" }
+, { "custkey": 10i32, "orderkey": 4097i32, "len-comment": 72, "comment": "ickly under the even accounts. even packages after the furiously express" }
+, { "custkey": 10i32, "orderkey": 4388i32, "len-comment": 49, "comment": "ts wake against the carefully final accounts. sly" }
+, { "custkey": 10i32, "orderkey": 4421i32, "len-comment": 41, "comment": "t the pending warhorses. express waters a" }
+, { "custkey": 10i32, "orderkey": 4449i32, "len-comment": 43, "comment": "ourts are carefully even deposits. pending " }
+, { "custkey": 10i32, "orderkey": 4867i32, "len-comment": 75, "comment": "ss the slyly regular dependencies. fluffily regular deposits within the car" }
+, { "custkey": 10i32, "orderkey": 5123i32, "len-comment": 74, "comment": "ic requests. furiously ironic packages grow above the express, ironic inst" }
+, { "custkey": 10i32, "orderkey": 5220i32, "len-comment": 42, "comment": " final packages. ideas detect slyly around" }
+, { "custkey": 11i32, "orderkey": 903i32, "len-comment": 26, "comment": "e slyly about the final pl" }
+, { "custkey": 11i32, "orderkey": 1285i32, "len-comment": 49, "comment": "cial deposits cajole after the ironic requests. p" }
+, { "custkey": 11i32, "orderkey": 1572i32, "len-comment": 77, "comment": "fluffily ironic accounts haggle blithely final platelets! slyly regular foxes" }
+, { "custkey": 11i32, "orderkey": 5029i32, "len-comment": 38, "comment": ". regular accounts haggle slyly. regul" }
+, { "custkey": 11i32, "orderkey": 5601i32, "len-comment": 32768, "comment": "gular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? b
 lithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets?
  blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelet
 s? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platel
 ets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake plat
 elets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake pl
 atelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake 
 platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wak
 e platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits w
 ake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits
  wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposi
 ts wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular depo
 sits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular de
 posits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular 
 deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegula
 r deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegu
 lar deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithe
 gular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blit
 hegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? bl
 ithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? 
 blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets
 ? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platele
 ts? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake plate
 lets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake pla
 telets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake p
 latelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake
  platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wa
 ke platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits 
 wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposit
 s wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular depos
 its wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular dep
 osits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular d
 eposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake platelets? blithegular deposits wake pla" }
+, { "custkey": 11i32, "orderkey": 5730i32, "len-comment": 53, "comment": "l platelets. ironic pinto beans wake slyly. quickly b" }
+, { "custkey": 13i32, "orderkey": 455i32, "len-comment": 36, "comment": " about the final platelets. dependen" }
+, { "custkey": 13i32, "orderkey": 675i32, "len-comment": 25, "comment": "ffily between the careful" }
+, { "custkey": 13i32, "orderkey": 901i32, "len-comment": 73, "comment": "lyly even foxes are furious, silent requests. requests about the quickly " }
+, { "custkey": 13i32, "orderkey": 1190i32, "len-comment": 54, "comment": "ccounts above the foxes integrate carefully after the " }
+, { "custkey": 13i32, "orderkey": 1955i32, "len-comment": 34, "comment": "ly special ideas. sometimes final " }
+, { "custkey": 13i32, "orderkey": 2368i32, "len-comment": 43, "comment": "t the bold instructions. carefully unusual " }
+, { "custkey": 13i32, "orderkey": 2438i32, "len-comment": 40, "comment": "the final, regular warhorses. regularly " }
+, { "custkey": 13i32, "orderkey": 3200i32, "len-comment": 60, "comment": " regular dependencies impress evenly even excuses. blithely " }
+, { "custkey": 13i32, "orderkey": 3584i32, "len-comment": 75, "comment": "fully bold packages. fluffily final braids haggle final, ironic dolphins. b" }
+, { "custkey": 13i32, "orderkey": 4452i32, "len-comment": 77, "comment": "oxes are slyly. express, ironic pinto beans wake after the quickly pending re" }
+, { "custkey": 13i32, "orderkey": 4993i32, "len-comment": 34, "comment": "ing instructions nag furiously. un" }
+, { "custkey": 13i32, "orderkey": 5028i32, "len-comment": 34, "comment": "ickly blithely express deposits. b" }
+, { "custkey": 13i32, "orderkey": 5221i32, "len-comment": 25, "comment": "lar accounts above the sl" }
+, { "custkey": 13i32, "orderkey": 5313i32, "len-comment": 51, "comment": "le. final courts haggle furiously according to the " }
+, { "custkey": 13i32, "orderkey": 5409i32, "len-comment": 32, "comment": "eans. regular accounts are regul" }
+, { "custkey": 13i32, "orderkey": 5447i32, "len-comment": 54, "comment": "uternes around the furiously bold accounts wake after " }
+, { "custkey": 13i32, "orderkey": 5860i32, "len-comment": 46, "comment": " beans. bold, special foxes sleep about the ir" }
+, { "custkey": 14i32, "orderkey": 966i32, "len-comment": 25, "comment": "special deposits. furious" }
+, { "custkey": 14i32, "orderkey": 1221i32, "len-comment": 59, "comment": " detect against the silent, even deposits. carefully ironic" }
+, { "custkey": 14i32, "orderkey": 2658i32, "len-comment": 41, "comment": "bout the slyly regular accounts. ironic, " }
+, { "custkey": 14i32, "orderkey": 2944i32, "len-comment": 51, "comment": "deas. permanently special foxes haggle carefully ab" }
+, { "custkey": 14i32, "orderkey": 3234i32, "len-comment": 71, "comment": "ents according to the dependencies will sleep after the blithely even p" }
+, { "custkey": 14i32, "orderkey": 4229i32, "len-comment": 31, "comment": "p furiously: final excuses hagg" }
+, { "custkey": 14i32, "orderkey": 4352i32, "len-comment": 48, "comment": "ly final platelets integrate carefully even requ" }
+, { "custkey": 14i32, "orderkey": 4676i32, "len-comment": 46, "comment": "s. slyly bold accounts sleep furiously special" }
+, { "custkey": 14i32, "orderkey": 4898i32, "len-comment": 74, "comment": "final patterns. special theodolites haggle ruthlessly at the blithely spec" }
+, { "custkey": 14i32, "orderkey": 5666i32, "len-comment": 40, "comment": "mptotes. quickly final instructions are " }
+, { "custkey": 16i32, "orderkey": 162i32, "len-comment": 66, "comment": "nts hinder fluffily ironic instructions. express, express excuses " }
+, { "custkey": 16i32, "orderkey": 803i32, "len-comment": 76, "comment": "ic instructions. even deposits haggle furiously at the deposits-- regular de" }
+, { "custkey": 16i32, "orderkey": 871i32, "len-comment": 27, "comment": "oss the ironic theodolites." }
+, { "custkey": 16i32, "orderkey": 1316i32, "len-comment": 40, "comment": "ully bold theodolites? pending, bold pin" }
+, { "custkey": 16i32, "orderkey": 1479i32, "len-comment": 45, "comment": "he furiously even foxes. thinly bold deposits" }
+, { "custkey": 16i32, "orderkey": 1540i32, "len-comment": 21, "comment": "r ideas hinder blithe" }
+, { "custkey": 16i32, "orderkey": 1633i32, "len-comment": 20, "comment": "y silent accounts sl" }
+, { "custkey": 16i32, "orderkey": 2753i32, "len-comment": 37, "comment": "ending instructions. unusual deposits" }
+, { "custkey": 16i32, "orderkey": 3556i32, "len-comment": 37, "comment": "e. dependencies need to haggle alongs" }
+, { "custkey": 16i32, "orderkey": 3685i32, "len-comment": 27, "comment": " sleep fluffily special ide" }
+, { "custkey": 16i32, "orderkey": 4067i32, "len-comment": 73, "comment": "tes boost furiously quick asymptotes. final deposits of the dolphins solv" }
+, { "custkey": 16i32, "orderkey": 4321i32, "len-comment": 53, "comment": "ending deposits are carefully carefully regular packa" }
+, { "custkey": 16i32, "orderkey": 4359i32, "len-comment": 55, "comment": "sts. special, unusual deposits across the ironic theodo" }
+, { "custkey": 16i32, "orderkey": 4805i32, "len-comment": 58, "comment": "even accounts wake furiously slyly final accounts; blithel" }
+, { "custkey": 16i32, "orderkey": 5120i32, "len-comment": 44, "comment": "against the slyly express requests. furiousl" }
+, { "custkey": 16i32, "orderkey": 5761i32, "len-comment": 37, "comment": "s asymptotes cajole boldly. regular, " }
+, { "custkey": 17i32, "orderkey": 65i32, "len-comment": 77, "comment": "ular requests are blithely pending orbits-- even requests against the deposit" }
+, { "custkey": 17i32, "orderkey": 161i32, "len-comment": 35, "comment": "carefully! special instructions sin" }
+, { "custkey": 17i32, "orderkey": 838i32, "len-comment": 28, "comment": " slyly around the slyly even" }
+, { "custkey": 17i32, "orderkey": 1344i32, "len-comment": 64, "comment": "omise close, silent requests. pending theodolites boost pending " }
+, { "custkey": 17i32, "orderkey": 1568i32, "len-comment": 22, "comment": "d notornis. carefully " }
+, { "custkey": 17i32, "orderkey": 1925i32, "len-comment": 36, "comment": "e slyly regular deposits. furiously " }
+, { "custkey": 17i32, "orderkey": 2048i32, "len-comment": 45, "comment": "s cajole after the blithely final accounts. f" }
+, { "custkey": 17i32, "orderkey": 3139i32, "len-comment": 45, "comment": "ounts against the ruthlessly unusual dolphins" }
+, { "custkey": 17i32, "orderkey": 3269i32, "len-comment": 73, "comment": "ts. accounts wake carefully. carefully dogged accounts wake slyly slyly i" }
+, { "custkey": 17i32, "orderkey": 3877i32, "len-comment": 25, "comment": "foxes. thinly bold reques" }
+, { "custkey": 17i32, "orderkey": 4099i32, "len-comment": 64, "comment": "r platelets. slyly regular requests cajole carefully against the" }
+, { "custkey": 17i32, "orderkey": 4257i32, "len-comment": 48, "comment": "r ideas cajole along the blithely regular gifts." }
+, { "custkey": 19i32, "orderkey": 295i32, "len-comment": 47, "comment": " unusual pinto beans play. regular ideas haggle" }
+, { "custkey": 19i32, "orderkey": 1090i32, "len-comment": 67, "comment": " furiously regular platelets haggle along the slyly unusual foxes! " }
+, { "custkey": 19i32, "orderkey": 1287i32, "len-comment": 67, "comment": "ly ironic dolphins integrate furiously among the final packages. st" }
+, { "custkey": 19i32, "orderkey": 1348i32, "len-comment": 24, "comment": "tly. quickly even deposi" }
+, { "custkey": 19i32, "orderkey": 1793i32, "len-comment": 75, "comment": "regular packages cajole. blithely special packages according to the final d" }
+, { "custkey": 19i32, "orderkey": 1991i32, "len-comment": 74, "comment": "ing accounts can haggle at the carefully final Tiresias-- pending, regular" }
+, { "custkey": 19i32, "orderkey": 2018i32, "len-comment": 23, "comment": "gular accounts wake fur" }
+, { "custkey": 19i32, "orderkey": 2371i32, "len-comment": 19, "comment": "ckages haggle at th" }
+, { "custkey": 19i32, "orderkey": 2466i32, "len-comment": 77, "comment": "c pinto beans. express deposits wake quickly. even, final courts nag. package" }
+, { "custkey": 19i32, "orderkey": 2693i32, "len-comment": 74, "comment": "ndle never. blithely regular packages nag carefully enticing platelets. ca" }
+, { "custkey": 19i32, "orderkey": 2820i32, "len-comment": 29, "comment": "equests are furiously. carefu" }
+, { "custkey": 19i32, "orderkey": 3169i32, "len-comment": 54, "comment": " even pinto beans are blithely special, special multip" }
+, { "custkey": 19i32, "orderkey": 3237i32, "len-comment": 62, "comment": "inal requests. slyly even foxes detect about the furiously exp" }
+, { "custkey": 19i32, "orderkey": 4132i32, "len-comment": 73, "comment": "ld asymptotes solve alongside of the express, final packages. fluffily fi" }
+, { "custkey": 19i32, "orderkey": 4455i32, "len-comment": 72, "comment": "even requests. bravely regular foxes according to the carefully unusual " }
+, { "custkey": 19i32, "orderkey": 4582i32, "len-comment": 27, "comment": "g the furiously regular pac" }
+, { "custkey": 19i32, "orderkey": 4834i32, "len-comment": 53, "comment": "lar accounts. furiously ironic accounts haggle slyly " }
+, { "custkey": 19i32, "orderkey": 5504i32, "len-comment": 28, "comment": "y pending packages. furiousl" }
+, { "custkey": 20i32, "orderkey": 1188i32, "len-comment": 26, "comment": "ully ironic deposits. slyl" }
+, { "custkey": 20i32, "orderkey": 1377i32, "len-comment": 77, "comment": "lly across the blithely express accounts. ironic excuses promise carefully de" }
+, { "custkey": 20i32, "orderkey": 1378i32, "len-comment": 51, "comment": " furiously even tithes cajole slyly among the quick" }
+, { "custkey": 20i32, "orderkey": 3042i32, "len-comment": 22, "comment": " the slyly ironic depo" }
+, { "custkey": 22i32, "orderkey": 97i32, "len-comment": 72, "comment": "hang blithely along the regular accounts. furiously even ideas after the" }
+, { "custkey": 22i32, "orderkey": 738i32, "len-comment": 51, "comment": "ly even foxes. furiously regular accounts cajole ca" }
+, { "custkey": 22i32, "orderkey": 1315i32, "len-comment": 41, "comment": "final theodolites alongside of the carefu" }
+, { "custkey": 22i32, "orderkey": 1735i32, "len-comment": 26, "comment": "ully idle requests wake qu" }
+, { "custkey": 22i32, "orderkey": 2117i32, "len-comment": 50, "comment": "ely even dependencies. regular foxes use blithely." }
+, { "custkey": 22i32, "orderkey": 3207i32, "len-comment": 29, "comment": "uriously accounts. fluffily i" }
+, { "custkey": 22i32, "orderkey": 3905i32, "len-comment": 57, "comment": "usly even accounts lose quietly above the slyly express p" }
+, { "custkey": 22i32, "orderkey": 3909i32, "len-comment": 52, "comment": "nic, special theodolites sleep furiously! furiously " }
+, { "custkey": 22i32, "orderkey": 4102i32, "len-comment": 41, "comment": "nding dependencies was slyly about the bl" }
+, { "custkey": 22i32, "orderkey": 4162i32, "len-comment": 74, "comment": "r packages are slyly accounts. furiously special foxes detect carefully re" }
+, { "custkey": 22i32, "orderkey": 4583i32, "len-comment": 37, "comment": "equests. slyly even platelets was qui" }
+, { "custkey": 22i32, "orderkey": 5092i32, "len-comment": 26, "comment": "are blithely along the pin" }
+, { "custkey": 22i32, "orderkey": 5410i32, "len-comment": 69, "comment": "final deposits: pending excuses boost. ironic theodolites cajole furi" }
+, { "custkey": 22i32, "orderkey": 5826i32, "len-comment": 23, "comment": "even, regular dependenc" }
+, { "custkey": 22i32, "orderkey": 5889i32, "len-comment": 33, "comment": "ites wake across the slyly ironic" }
+, { "custkey": 22i32, "orderkey": 5956i32, "len-comment": 27, "comment": "le even, express platelets." }
+, { "custkey": 23i32, "orderkey": 292i32, "len-comment": 34, "comment": "g pinto beans will have to sleep f" }
+, { "custkey": 23i32, "orderkey": 2116i32, "len-comment": 41, "comment": "efully after the asymptotes. furiously sp" }
+, { "custkey": 23i32, "orderkey": 2182i32, "len-comment": 52, "comment": "ccounts. quickly bold deposits across the excuses sl" }
+, { "custkey": 23i32, "orderkey": 3072i32, "len-comment": 47, "comment": "ely final deposits cajole carefully. ironic, re" }
+, { "custkey": 23i32, "orderkey": 3136i32, "len-comment": 33, "comment": "tructions sleep slyly. pending di" }
+, { "custkey": 23i32, "orderkey": 3684i32, "len-comment": 51, "comment": "bold accounts affix along the carefully ironic requ" }
+, { "custkey": 23i32, "orderkey": 4098i32, "len-comment": 67, "comment": "otes. quickly final requests after the stealthily ironic pinto bean" }
+, { "custkey": 23i32, "orderkey": 5063i32, "len-comment": 65, "comment": "lyly after the pending foxes. express theodolites breach across t" }
+, { "custkey": 23i32, "orderkey": 5408i32, "len-comment": 32, "comment": "egular requests according to the" }
+, { "custkey": 23i32, "orderkey": 5415i32, "len-comment": 55, "comment": "ly even ideas nag blithely above the final instructions" }
+, { "custkey": 23i32, "orderkey": 5959i32, "len-comment": 75, "comment": "into beans use ironic, unusual foxes. carefully regular excuses boost caref" }
+, { "custkey": 25i32, "orderkey": 550i32, "len-comment": 21, "comment": "t requests. blithely " }
+, { "custkey": 25i32, "orderkey": 1125i32, "len-comment": 24, "comment": "ithely final requests. i" }
+, { "custkey": 25i32, "orderkey": 1670i32, "len-comment": 54, "comment": "unusual dependencies. furiously special platelets main" }
+, { "custkey": 25i32, "orderkey": 1767i32, "len-comment": 78, "comment": "eposits use carefully carefully regular platelets. quickly regular packages al" }
+, { "custkey": 25i32, "orderkey": 1889i32, "len-comment": 47, "comment": "p around the regular notornis. unusual deposits" }
+, { "custkey": 25i32, "orderkey": 1892i32, "len-comment": 55, "comment": "sts. slyly regular dependencies use slyly. ironic, spec" }
+, { "custkey": 25i32, "orderkey": 2308i32, "len-comment": 20, "comment": "ts. slyly final depo" }
+, { "custkey": 25i32, "orderkey": 2434i32, "len-comment": 58, "comment": "s. quickly ironic dolphins impress final deposits. blithel" }
+, { "custkey": 25i32, "orderkey": 2657i32, "len-comment": 63, "comment": " foxes-- slyly final dependencies around the slyly final theodo" }
+, { "custkey": 25i32, "orderkey": 2790i32, "len-comment": 44, "comment": " the carefully express deposits sleep slyly " }
+, { "custkey": 25i32, "orderkey": 3047i32, "len-comment": 56, "comment": "as. slyly express deposits are dogged pearls. silent ide" }
+, { "custkey": 25i32, "orderkey": 3457i32, "len-comment": 45, "comment": "ely thin asymptotes. deposits kindle. pending" }
+, { "custkey": 25i32, "orderkey": 3968i32, "len-comment": 29, "comment": " the slyly special accounts; " }
+, { "custkey": 25i32, "orderkey": 4292i32, "len-comment": 21, "comment": " ruthlessly. slyly bo" }
+, { "custkey": 25i32, "orderkey": 4358i32, "len-comment": 45, "comment": "according to the fluffily special asymptotes " }
+, { "custkey": 25i32, "orderkey": 4384i32, "len-comment": 73, "comment": "onic platelets. furiously regular asymptotes according to the special pac" }
+, { "custkey": 25i32, "orderkey": 4706i32, "len-comment": 67, "comment": " packages above the never regular packages nag packages. deposits c" }
+, { "custkey": 25i32, "orderkey": 4839i32, "len-comment": 63, "comment": " even somas. slyly express ideas lose carefully. blithely unusu" }
+, { "custkey": 25i32, "orderkey": 5124i32, "len-comment": 38, "comment": "kly even courts. bold packages solve. " }
+, { "custkey": 25i32, "orderkey": 5287i32, "len-comment": 67, "comment": "regular packages. bold instructions sleep always. carefully final p" }
+, { "custkey": 25i32, "orderkey": 5760i32, "len-comment": 73, "comment": "s among the blithely regular frays haggle ironically bold theodolites. al" }
+, { "custkey": 26i32, "orderkey": 608i32, "len-comment": 33, "comment": "nic waters wake slyly slyly expre" }
+, { "custkey": 26i32, "orderkey": 867i32, "len-comment": 25, "comment": "pades nag quickly final, " }
+, { "custkey": 26i32, "orderkey": 963i32, "len-comment": 68, "comment": "uses haggle carefully. slyly even dependencies after the packages ha" }
+, { "custkey": 26i32, "orderkey": 3107i32, "len-comment": 23, "comment": "ously even deposits acr" }
+, { "custkey": 26i32, "orderkey": 3141i32, "len-comment": 59, "comment": "es. furiously bold instructions after the carefully final p" }
+, { "custkey": 26i32, "orderkey": 3522i32, "len-comment": 46, "comment": "deposits-- slyly stealthy requests boost caref" }
+, { "custkey": 26i32, "orderkey": 4610i32, "len-comment": 25, "comment": "e carefully express pinto" }
+, { "custkey": 26i32, "orderkey": 4709i32, "len-comment": 65, "comment": "he furiously even deposits! ironic theodolites haggle blithely. r" }
+, { "custkey": 26i32, "orderkey": 5792i32, "len-comment": 63, "comment": "packages. doggedly bold deposits integrate furiously across the" }
+, { "custkey": 26i32, "orderkey": 5799i32, "len-comment": 72, "comment": " unusual deposits sleep blithely along the carefully even requests. care" }
+, { "custkey": 28i32, "orderkey": 101i32, "len-comment": 45, "comment": "ding accounts above the slyly final asymptote" }
+, { "custkey": 28i32, "orderkey": 132i32, "len-comment": 73, "comment": "sits are daringly accounts. carefully regular foxes sleep slyly about the" }
+, { "custkey": 28i32, "orderkey": 165i32, "len-comment": 42, "comment": "across the blithely regular accounts. bold" }
+, { "custkey": 28i32, "orderkey": 839i32, "len-comment": 50, "comment": "the carefully even platelets. furiously unusual fo" }
+, { "custkey": 28i32, "orderkey": 1056i32, "len-comment": 47, "comment": "t, even deposits hang about the slyly special i" }
+, { "custkey": 28i32, "orderkey": 1219i32, "len-comment": 59, "comment": "od carefully. slyly final dependencies across the even fray" }
+, { "custkey": 28i32, "orderkey": 2050i32, "len-comment": 57, "comment": "d accounts against the furiously regular packages use bli" }
+, { "custkey": 28i32, "orderkey": 2306i32, "len-comment": 70, "comment": " wake furiously requests. permanent requests affix. final packages caj" }
+, { "custkey": 28i32, "orderkey": 2373i32, "len-comment": 44, "comment": " even, special courts grow quickly. pending," }
+, { "custkey": 28i32, "orderkey": 3558i32, "len-comment": 42, "comment": "around the furiously even requests. quickl" }
+, { "custkey": 28i32, "orderkey": 3717i32, "len-comment": 33, "comment": "t the carefully even ideas use sp" }
+, { "custkey": 28i32, "orderkey": 3777i32, "len-comment": 47, "comment": " regular, special dolphins cajole enticingly ca" }
+, { "custkey": 28i32, "orderkey": 3842i32, "len-comment": 78, "comment": "silent ideas. final deposits use furiously. blithely express excuses cajole fu" }
+, { "custkey": 28i32, "orderkey": 4167i32, "len-comment": 32, "comment": "kly furiously even deposits. unu" }
+, { "custkey": 28i32, "orderkey": 4647i32, "len-comment": 62, "comment": "out the deposits. slyly final pinto beans haggle idly. slyly s" }
+, { "custkey": 28i32, "orderkey": 4772i32, "len-comment": 50, "comment": "es sleep. regular requests haggle furiously slyly " }
+, { "custkey": 28i32, "orderkey": 5026i32, "len-comment": 19, "comment": "y final requests us" }
+, { "custkey": 28i32, "orderkey": 5125i32, "len-comment": 49, "comment": "ructions. dolphins wake slowly carefully unusual " }
+, { "custkey": 28i32, "orderkey": 5574i32, "len-comment": 30, "comment": "n deposits. special, regular t" }
+, { "custkey": 28i32, "orderkey": 5954i32, "len-comment": 24, "comment": "requests along the blith" }
+, { "custkey": 29i32, "orderkey": 68i32, "len-comment": 76, "comment": " pinto beans sleep carefully. blithely ironic deposits haggle furiously acro" }
+, { "custkey": 29i32, "orderkey": 832i32, "len-comment": 63, "comment": "xes. bravely regular packages sleep up the furiously bold accou" }
+, { "custkey": 29i32, "orderkey": 1121i32, "len-comment": 58, "comment": "r escapades. deposits above the fluffily bold requests hag" }
+, { "custkey": 29i32, "orderkey": 1538i32, "len-comment": 28, "comment": " instructions. regular theod" }
+, { "custkey": 29i32, "orderkey": 1764i32, "len-comment": 47, "comment": ". slyly final packages integrate carefully acro" }
+, { "custkey": 29i32, "orderkey": 2945i32, "len-comment": 62, "comment": "ons are carefully toward the permanent, bold pinto beans. regu" }
+, { "custkey": 29i32, "orderkey": 2976i32, "len-comment": 66, "comment": ". furiously ironic asymptotes haggle ruthlessly silently regular r" }
+, { "custkey": 29i32, "orderkey": 3014i32, "len-comment": 34, "comment": "ep blithely according to the blith" }
+, { "custkey": 29i32, "orderkey": 3876i32, "len-comment": 20, "comment": "into beans. blithely" }
+, { "custkey": 29i32, "orderkey": 4039i32, "len-comment": 33, "comment": "ly ironic deposits. ironic reques" }
+, { "custkey": 29i32, "orderkey": 4070i32, "len-comment": 21, "comment": "xpress ideas poach ab" }
+, { "custkey": 29i32, "orderkey": 4326i32, "len-comment": 35, "comment": "packages. carefully express deposit" }
+, { "custkey": 29i32, "orderkey": 4611i32, "len-comment": 71, "comment": ". furiously regular instructions haggle dolphins. even instructions det" }
+, { "custkey": 29i32, "orderkey": 4615i32, "len-comment": 68, "comment": "jole after the fluffily pending foxes. packages affix carefully acco" }
+, { "custkey": 31i32, "orderkey": 103i32, "len-comment": 60, "comment": "ges. carefully unusual instructions haggle quickly regular f" }
+, { "custkey": 31i32, "orderkey": 262i32, "len-comment": 49, "comment": "l packages. blithely final pinto beans use carefu" }
+, { "custkey": 31i32, "orderkey": 293i32, "len-comment": 37, "comment": "re bold, ironic deposits. platelets c" }
+, { "custkey": 31i32, "orderkey": 481i32, "len-comment": 40, "comment": "ly final ideas. packages haggle fluffily" }
+, { "custkey": 31i32, "orderkey": 576i32, "len-comment": 48, "comment": "l requests affix regular requests. final account" }
+, { "custkey": 31i32, "orderkey": 1957i32, "len-comment": 24, "comment": "nding excuses about the " }
+, { "custkey": 31i32, "orderkey": 2049i32, "len-comment": 74, "comment": "ly regular requests thrash blithely about the fluffily even theodolites. r" }
+, { "custkey": 31i32, "orderkey": 2310i32, "len-comment": 63, "comment": "wake carefully. unusual instructions nag ironic, regular excuse" }
+, { "custkey": 31i32, "orderkey": 2372i32, "len-comment": 49, "comment": "s: deposits haggle along the final ideas. careful" }
+, { "custkey": 31i32, "orderkey": 2433i32, "len-comment": 46, "comment": "ess patterns are slyly. packages haggle carefu" }
+, { "custkey": 31i32, "orderkey": 2720i32, "len-comment": 46, "comment": "quickly. special asymptotes are fluffily ironi" }
+, { "custkey": 31i32, "orderkey": 3495i32, "len-comment": 29, "comment": "nticing excuses are carefully" }
+, { "custkey": 31i32, "orderkey": 3589i32, "len-comment": 32768, "comment": "ithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously
  pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously
 . furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits na
 g furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe 
 deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages s
 leep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending
  packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiou
 sly pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiou
 sly. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits
  nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fit
 he deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending package
 s sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pend
 ing packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. fur
 iously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag fur
 iously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe depos
 its nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep 
 fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending pack
 ages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously p
 ending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. 
 furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe deposits nag furiously. furiously pending packages sleep fithe de

<TRUNCATED>


[04/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/results/big-object/big_object_join/join.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/big-object/big_object_join/join.1.adm b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_join/join.1.adm
new file mode 100644
index 0000000..64bffdb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_join/join.1.adm
@@ -0,0 +1,1501 @@
+[ { "c_custkey": 38i32, "o_orderkey": 676i32, "len_c_comment": 110, "len_o_comment": 36, "c_comment": "lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin" }
+, { "c_custkey": 38i32, "o_orderkey": 1251i32, "len_c_comment": 110, "len_o_comment": 43, "c_comment": "lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin" }
+, { "c_custkey": 38i32, "o_orderkey": 3270i32, "len_c_comment": 110, "len_o_comment": 24, "c_comment": "lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin" }
+, { "c_custkey": 38i32, "o_orderkey": 3749i32, "len_c_comment": 110, "len_o_comment": 37, "c_comment": "lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin" }
+, { "c_custkey": 38i32, "o_orderkey": 4391i32, "len_c_comment": 110, "len_o_comment": 27, "c_comment": "lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin" }
+, { "c_custkey": 47i32, "o_orderkey": 261i32, "len_c_comment": 69, "len_o_comment": 42, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 736i32, "len_c_comment": 69, "len_o_comment": 23, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 1376i32, "len_c_comment": 69, "len_o_comment": 42, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 1796i32, "len_c_comment": 69, "len_o_comment": 28, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 2497i32, "len_c_comment": 69, "len_o_comment": 64, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 3171i32, "len_c_comment": 69, "len_o_comment": 19, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 3431i32, "len_c_comment": 69, "len_o_comment": 29, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 4036i32, "len_c_comment": 69, "len_o_comment": 65, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 4357i32, "len_c_comment": 69, "len_o_comment": 20, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 47i32, "o_orderkey": 4997i32, "len_c_comment": 69, "len_o_comment": 61, "c_comment": "ions. express, ironic instructions sleep furiously ironic ideas. furi" }
+, { "c_custkey": 49i32, "o_orderkey": 450i32, "len_c_comment": 64, "len_o_comment": 47, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 454i32, "len_c_comment": 64, "len_o_comment": 64, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 583i32, "len_c_comment": 64, "len_o_comment": 26, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 679i32, "len_c_comment": 64, "len_o_comment": 78, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 897i32, "len_c_comment": 64, "len_o_comment": 22, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 1089i32, "len_c_comment": 64, "len_o_comment": 52, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 1152i32, "len_c_comment": 64, "len_o_comment": 20, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 1220i32, "len_c_comment": 64, "len_o_comment": 55, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 1248i32, "len_c_comment": 64, "len_o_comment": 34, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 1792i32, "len_c_comment": 64, "len_o_comment": 59, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 1824i32, "len_c_comment": 64, "len_o_comment": 19, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 2082i32, "len_c_comment": 64, "len_o_comment": 72, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 2085i32, "len_c_comment": 64, "len_o_comment": 26, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 2243i32, "len_c_comment": 64, "len_o_comment": 34, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 2818i32, "len_c_comment": 64, "len_o_comment": 37, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 2855i32, "len_c_comment": 64, "len_o_comment": 32, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 2981i32, "len_c_comment": 64, "len_o_comment": 76, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 3078i32, "len_c_comment": 64, "len_o_comment": 32, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 3335i32, "len_c_comment": 64, "len_o_comment": 55, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 3361i32, "len_c_comment": 64, "len_o_comment": 47, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 3494i32, "len_c_comment": 64, "len_o_comment": 72, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 3655i32, "len_c_comment": 64, "len_o_comment": 60, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 3846i32, "len_c_comment": 64, "len_o_comment": 23, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 4294i32, "len_c_comment": 64, "len_o_comment": 48, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 5347i32, "len_c_comment": 64, "len_o_comment": 30, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 5542i32, "len_c_comment": 64, "len_o_comment": 39, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 5762i32, "len_c_comment": 64, "len_o_comment": 70, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 5766i32, "len_c_comment": 64, "len_o_comment": 40, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 49i32, "o_orderkey": 5890i32, "len_c_comment": 64, "len_o_comment": 29, "c_comment": "nusual foxes! fluffily pending packages maintain to the regular " }
+, { "c_custkey": 70i32, "o_orderkey": 581i32, "len_c_comment": 90, "len_o_comment": 52, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 836i32, "len_c_comment": 90, "len_o_comment": 73, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 965i32, "len_c_comment": 90, "len_o_comment": 62, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 1028i32, "len_c_comment": 90, "len_o_comment": 47, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 1159i32, "len_c_comment": 90, "len_o_comment": 39, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 1254i32, "len_c_comment": 90, "len_o_comment": 39, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 1474i32, "len_c_comment": 90, "len_o_comment": 40, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 1634i32, "len_c_comment": 90, "len_o_comment": 71, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 1861i32, "len_c_comment": 90, "len_o_comment": 37, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 2021i32, "len_c_comment": 90, "len_o_comment": 73, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 2502i32, "len_c_comment": 90, "len_o_comment": 41, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 2567i32, "len_c_comment": 90, "len_o_comment": 33, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 2848i32, "len_c_comment": 90, "len_o_comment": 53, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 2947i32, "len_c_comment": 90, "len_o_comment": 40, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 3104i32, "len_c_comment": 90, "len_o_comment": 41, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 3939i32, "len_c_comment": 90, "len_o_comment": 75, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 4000i32, "len_c_comment": 90, "len_o_comment": 72, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 4004i32, "len_c_comment": 90, "len_o_comment": 47, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 4224i32, "len_c_comment": 90, "len_o_comment": 71, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 4422i32, "len_c_comment": 90, "len_o_comment": 66, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 4448i32, "len_c_comment": 90, "len_o_comment": 68, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 4512i32, "len_c_comment": 90, "len_o_comment": 31, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 4966i32, "len_c_comment": 90, "len_o_comment": 63, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 5122i32, "len_c_comment": 90, "len_o_comment": 46, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 5248i32, "len_c_comment": 90, "len_o_comment": 78, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 5285i32, "len_c_comment": 90, "len_o_comment": 39, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 5472i32, "len_c_comment": 90, "len_o_comment": 64, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 5479i32, "len_c_comment": 90, "len_o_comment": 37, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 5635i32, "len_c_comment": 90, "len_o_comment": 57, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 70i32, "o_orderkey": 5984i32, "len_c_comment": 90, "len_o_comment": 61, "c_comment": "fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be" }
+, { "c_custkey": 89i32, "o_orderkey": 99i32, "len_c_comment": 77, "len_o_comment": 36, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 1184i32, "len_c_comment": 77, "len_o_comment": 41, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 2277i32, "len_c_comment": 77, "len_o_comment": 23, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 2471i32, "len_c_comment": 77, "len_o_comment": 30, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 2725i32, "len_c_comment": 77, "len_o_comment": 19, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 3172i32, "len_c_comment": 77, "len_o_comment": 30, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 3463i32, "len_c_comment": 77, "len_o_comment": 37, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 3845i32, "len_c_comment": 77, "len_o_comment": 70, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 4291i32, "len_c_comment": 77, "len_o_comment": 47, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 5090i32, "len_c_comment": 77, "len_o_comment": 66, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 5379i32, "len_c_comment": 77, "len_o_comment": 48, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 89i32, "o_orderkey": 5957i32, "len_c_comment": 77, "len_o_comment": 75, "c_comment": "counts are slyly beyond the slyly final accounts. quickly final ideas wake. r" }
+, { "c_custkey": 92i32, "o_orderkey": 2211i32, "len_c_comment": 43, "len_o_comment": 19, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 2593i32, "len_c_comment": 43, "len_o_comment": 21, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 2884i32, "len_c_comment": 43, "len_o_comment": 41, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 3076i32, "len_c_comment": 43, "len_o_comment": 55, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 3333i32, "len_c_comment": 43, "len_o_comment": 56, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 4197i32, "len_c_comment": 43, "len_o_comment": 22, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 4226i32, "len_c_comment": 43, "len_o_comment": 56, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 4258i32, "len_c_comment": 43, "len_o_comment": 38, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 4903i32, "len_c_comment": 43, "len_o_comment": 30, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 92i32, "o_orderkey": 5607i32, "len_c_comment": 43, "len_o_comment": 58, "c_comment": ". pinto beans hang slyly final deposits. ac" }
+, { "c_custkey": 103i32, "o_orderkey": 230i32, "len_c_comment": 107, "len_o_comment": 30, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 390i32, "len_c_comment": 107, "len_o_comment": 54, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 742i32, "len_c_comment": 107, "len_o_comment": 65, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 931i32, "len_c_comment": 107, "len_o_comment": 63, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 1025i32, "len_c_comment": 107, "len_o_comment": 23, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 1061i32, "len_c_comment": 107, "len_o_comment": 32, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 1508i32, "len_c_comment": 107, "len_o_comment": 65, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 1571i32, "len_c_comment": 107, "len_o_comment": 70, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 2241i32, "len_c_comment": 107, "len_o_comment": 71, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 2432i32, "len_c_comment": 107, "len_o_comment": 32, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 2689i32, "len_c_comment": 107, "len_o_comment": 56, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 2787i32, "len_c_comment": 107, "len_o_comment": 19, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 2819i32, "len_c_comment": 107, "len_o_comment": 45, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 3015i32, "len_c_comment": 107, "len_o_comment": 48, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 3360i32, "len_c_comment": 107, "len_o_comment": 51, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 3424i32, "len_c_comment": 107, "len_o_comment": 69, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 3492i32, "len_c_comment": 107, "len_o_comment": 55, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 3973i32, "len_c_comment": 107, "len_o_comment": 56, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 4293i32, "len_c_comment": 107, "len_o_comment": 48, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 4870i32, "len_c_comment": 107, "len_o_comment": 38, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 5249i32, "len_c_comment": 107, "len_o_comment": 22, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 5571i32, "len_c_comment": 107, "len_o_comment": 49, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 5575i32, "len_c_comment": 107, "len_o_comment": 29, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 103i32, "o_orderkey": 5637i32, "len_c_comment": 107, "len_o_comment": 58, "c_comment": "furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl" }
+, { "c_custkey": 109i32, "o_orderkey": 96i32, "len_c_comment": 113, "len_o_comment": 21, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 166i32, "len_c_comment": 113, "len_o_comment": 36, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 487i32, "len_c_comment": 113, "len_o_comment": 33, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 672i32, "len_c_comment": 113, "len_o_comment": 43, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 899i32, "len_c_comment": 113, "len_o_comment": 39, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 997i32, "len_c_comment": 113, "len_o_comment": 68, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 1286i32, "len_c_comment": 113, "len_o_comment": 54, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 1537i32, "len_c_comment": 113, "len_o_comment": 47, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 1988i32, "len_c_comment": 113, "len_o_comment": 50, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 2339i32, "len_c_comment": 113, "len_o_comment": 21, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 2886i32, "len_c_comment": 113, "len_o_comment": 28, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 2887i32, "len_c_comment": 113, "len_o_comment": 67, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 2914i32, "len_c_comment": 113, "len_o_comment": 47, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 3489i32, "len_c_comment": 113, "len_o_comment": 49, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 3525i32, "len_c_comment": 113, "len_o_comment": 26, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 4420i32, "len_c_comment": 113, "len_o_comment": 78, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 4547i32, "len_c_comment": 113, "len_o_comment": 65, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 4551i32, "len_c_comment": 113, "len_o_comment": 26, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 5344i32, "len_c_comment": 113, "len_o_comment": 35, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 5569i32, "len_c_comment": 113, "len_o_comment": 49, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 5638i32, "len_c_comment": 113, "len_o_comment": 70, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 109i32, "o_orderkey": 5668i32, "len_c_comment": 113, "len_o_comment": 28, "c_comment": "es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou" }
+, { "c_custkey": 88i32, "o_orderkey": 37i32, "len_c_comment": 99, "len_o_comment": 62, "c_comment": "s are quickly above the quickly ironic instructions; even requests about the carefully final deposi" }
+, { "c_custkey": 125i32, "o_orderkey": 38i32, "len_c_comment": 52, "len_o_comment": 77, "c_comment": "x-ray finally after the packages? regular requests c" }
+, { "c_custkey": 58i32, "o_orderkey": 67i32, "len_c_comment": 94, "len_o_comment": 48, "c_comment": "ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e" }
+, { "c_custkey": 88i32, "o_orderkey": 163i32, "len_c_comment": 99, "len_o_comment": 52, "c_comment": "s are quickly above the quickly ironic instructions; even requests about the carefully final deposi" }
+, { "c_custkey": 125i32, "o_orderkey": 256i32, "len_c_comment": 52, "len_o_comment": 61, "c_comment": "x-ray finally after the packages? regular requests c" }
+, { "c_custkey": 118i32, "o_orderkey": 263i32, "len_c_comment": 113, "len_o_comment": 34, "c_comment": "uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep" }
+, { "c_custkey": 118i32, "o_orderkey": 290i32, "len_c_comment": 113, "len_o_comment": 30, "c_comment": "uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep" }
+, { "c_custkey": 23i32, "o_orderkey": 292i32, "len_c_comment": 87, "len_o_comment": 34, "c_comment": "deposits. special deposits cajole slyly. fluffily special deposits about the furiously " }
+, { "c_custkey": 115i32, "o_orderkey": 384i32, "len_c_comment": 49, "len_o_comment": 56, "c_comment": "sits haggle above the carefully ironic theodolite" }
+, { "c_custkey": 118i32, "o_orderkey": 419i32, "len_c_comment": 113, "len_o_comment": 51, "c_comment": "uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep" }
+, { "c_custkey": 26i32, "o_orderkey": 608i32, "len_c_comment": 70, "len_o_comment": 33, "c_comment": "c requests use furiously ironic requests. slyly ironic dependencies us" }
+, { "c_custkey": 58i32, "o_orderkey": 643i32, "len_c_comment": 94, "len_o_comment": 33, "c_comment": "ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e" }
+, { "c_custkey": 115i32, "o_orderkey": 645i32, "len_c_comment": 49, "len_o_comment": 42, "c_comment": "sits haggle above the carefully ironic theodolite" }
+, { "c_custkey": 118i32, "o_orderkey": 707i32, "len_c_comment": 113, "len_o_comment": 51, "c_comment": "uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep" }
+, { "c_custkey": 118i32, "o_orderkey": 801i32, "len_c_comment": 113, "len_o_comment": 32768, "c_comment": "uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep" }
+, { "c_custkey": 26i32, "o_orderkey": 867i32, "len_c_comment": 70, "len_o_comment": 25, "c_comment": "c requests use furiously ironic requests. slyly ironic dependencies us" }
+, { "c_custkey": 11i32, "o_orderkey": 903i32, "len_c_comment": 32768, "len_o_comment": 26, "c_comment": "ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto
  beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pi
 nto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular
  pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regu
 lar pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly r
 egular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyl
 y regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the s
 lyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above th
 e slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above
  the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise ab
 ove the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise
  above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans prom
 ise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans p
 romise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto bean
 s promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto b
 eans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pint
 o beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even p
 into beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly eve
 n pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly 
 even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quick
 ly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. qu
 ickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly.
  quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep sly
 ly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep 
 slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sle
 ep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests 
 sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. reques
 ts sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. req
 uests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. 
 requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckage
 s. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ck
 ages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans.
  ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above" }
+, { "c_custkey": 26i32, "o_orderkey": 963i32, "len_c_comment": 70, "len_o_comment": 68, "c_comment": "c requests use furiously ironic requests. slyly ironic dependencies us" }
+, { "c_custkey": 14i32, "o_orderkey": 966i32, "len_c_comment": 33, "len_o_comment": 25, "c_comment": ", ironic packages across the unus" }
+, { "c_custkey": 58i32, "o_orderkey": 1127i32, "len_c_comment": 94, "len_o_comment": 64, "c_comment": "ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e" }
+, { "c_custkey": 14i32, "o_orderkey": 1221i32, "len_c_comment": 33, "len_o_comment": 59, "c_comment": ", ironic packages across the unus" }
+, { "c_custkey": 115i32, "o_orderkey": 1253i32, "len_c_comment": 49, "len_o_comment": 53, "c_comment": "sits haggle above the carefully ironic theodolite" }
+, { "c_custkey": 118i32, "o_orderkey": 1283i32, "len_c_comment": 113, "len_o_comment": 27, "c_comment": "uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep" }
+, { "c_custkey": 11i32, "o_orderkey": 1285i32, "len_c_comment": 32768, "len_o_comment": 49, "c_comment": "ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pint
 o beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular p
 into beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regula
 r pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly reg
 ular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly 
 regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the sly
 ly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the 
 slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above t
 he slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise abov
 e the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise a
 bove the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promis
 e above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans pro
 mise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans 
 promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto bea
 ns promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto 
 beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pin
 to beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even 
 pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly ev
 en pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly
  even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quic
 kly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. q
 uickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly
 . quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep sl
 yly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep
  slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto bean

<TRUNCATED>


[02/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 694077c..575827e 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -13,10 +13,10 @@
  ! limitations under the License.
  !-->
 <test-suite
-    xmlns="urn:xml.testframework.asterix.ics.uci.edu"
-    ResultOffsetPath="results"
-    QueryOffsetPath="queries"
-    QueryFileExtension=".aql">
+        xmlns="urn:xml.testframework.asterix.ics.uci.edu"
+        ResultOffsetPath="results"
+        QueryOffsetPath="queries"
+        QueryFileExtension=".aql">
     <test-group name="flwor">
         <test-case FilePath="flwor">
             <compilation-unit name="at00">
@@ -250,87 +250,87 @@
                 <output-dir compare="Text">count_null</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="aggregate">
-      <compilation-unit name="droptype">
-        <output-dir compare="Text">droptype</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!-- TODO(madhusudancs): These tests that test for local_<agg>/global_<agg> functions should be removed, but
-    before that we should modify the code to make sure those built-in functions are still defined but not exposed
-    by AQL, so leaving these test cases commented.
-    <test-case FilePath="aggregate">
-      <compilation-unit name="global-avg_01">
-        <output-dir compare="Text">global-avg_01</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="global-avg_null">
-        <output-dir compare="Text">global-avg_null</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_double">
-        <output-dir compare="Text">local-avg_double</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_double_null">
-        <output-dir compare="Text">local-avg_double_null</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_float">
-        <output-dir compare="Text">local-avg_float</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_float_null">
-        <output-dir compare="Text">local-avg_float_null</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int16">
-        <output-dir compare="Text">local-avg_int16</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int16_null">
-        <output-dir compare="Text">local-avg_int16_null</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int32">
-        <output-dir compare="Text">local-avg_int32</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int32_null">
-        <output-dir compare="Text">local-avg_int32_null</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int64">
-        <output-dir compare="Text">local-avg_int64</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int64_null">
-        <output-dir compare="Text">local-avg_int64_null</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int8">
-        <output-dir compare="Text">local-avg_int8</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="aggregate">
-      <compilation-unit name="local-avg_int8_null">
-        <output-dir compare="Text">local-avg_int8_null</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="aggregate">
+          <compilation-unit name="droptype">
+            <output-dir compare="Text">droptype</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!-- TODO(madhusudancs): These tests that test for local_<agg>/global_<agg> functions should be removed, but
+        before that we should modify the code to make sure those built-in functions are still defined but not exposed
+        by AQL, so leaving these test cases commented.
+        <test-case FilePath="aggregate">
+          <compilation-unit name="global-avg_01">
+            <output-dir compare="Text">global-avg_01</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="global-avg_null">
+            <output-dir compare="Text">global-avg_null</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_double">
+            <output-dir compare="Text">local-avg_double</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_double_null">
+            <output-dir compare="Text">local-avg_double_null</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_float">
+            <output-dir compare="Text">local-avg_float</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_float_null">
+            <output-dir compare="Text">local-avg_float_null</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int16">
+            <output-dir compare="Text">local-avg_int16</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int16_null">
+            <output-dir compare="Text">local-avg_int16_null</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int32">
+            <output-dir compare="Text">local-avg_int32</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int32_null">
+            <output-dir compare="Text">local-avg_int32_null</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int64">
+            <output-dir compare="Text">local-avg_int64</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int64_null">
+            <output-dir compare="Text">local-avg_int64_null</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int8">
+            <output-dir compare="Text">local-avg_int8</output-dir>
+          </compilation-unit>
+        </test-case>
+        <test-case FilePath="aggregate">
+          <compilation-unit name="local-avg_int8_null">
+            <output-dir compare="Text">local-avg_int8_null</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="aggregate">
             <compilation-unit name="max_empty_01">
                 <output-dir compare="Text">max_empty_01</output-dir>
@@ -1000,13 +1000,13 @@
                 <output-dir compare="Text">neq_01</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="comparison">
-      <compilation-unit name="numeric-comparison_01">
-        <output-dir compare="Text">numeric-comparison_01</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="comparison">
+          <compilation-unit name="numeric-comparison_01">
+            <output-dir compare="Text">numeric-comparison_01</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="comparison">
             <compilation-unit name="string">
                 <output-dir compare="Text">string</output-dir>
@@ -1235,32 +1235,32 @@
                 <output-dir compare="Text">customer_q_08</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="custord">
-      <compilation-unit name="denorm-cust-order_01">
-        <output-dir compare="Text">denorm-cust-order_01</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="custord">
+          <compilation-unit name="denorm-cust-order_01">
+            <output-dir compare="Text">denorm-cust-order_01</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="custord">
             <compilation-unit name="denorm-cust-order_02">
                 <output-dir compare="Text">denorm-cust-order_02</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="custord">
-      <compilation-unit name="denorm-cust-order_03">
-        <output-dir compare="Text">denorm-cust-order_03</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="custord">
-      <compilation-unit name="freq-clerk">
-        <output-dir compare="Text">freq-clerk</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="custord">
+          <compilation-unit name="denorm-cust-order_03">
+            <output-dir compare="Text">denorm-cust-order_03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="custord">
+          <compilation-unit name="freq-clerk">
+            <output-dir compare="Text">freq-clerk</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="custord">
             <compilation-unit name="join_q_01">
                 <output-dir compare="Text">join_q_01</output-dir>
@@ -1328,13 +1328,13 @@
                 <output-dir compare="Text">q2</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="dapd">
-      <compilation-unit name="q3">
-        <output-dir compare="Text">q3</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="dapd">
+          <compilation-unit name="q3">
+            <output-dir compare="Text">q3</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
     </test-group>
     <test-group name="dml">
         <test-case FilePath="dml">
@@ -1494,13 +1494,13 @@
                 <output-dir compare="Text">insert_less_nc</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="dml">
-      <compilation-unit name="load-from-hdfs">
-        <output-dir compare="Text">load-from-hdfs</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="dml">
+          <compilation-unit name="load-from-hdfs">
+            <output-dir compare="Text">load-from-hdfs</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="dml">
             <compilation-unit name="load-with-autogenerated-pk_txt_01">
                 <output-dir compare="Text">load-with-autogenerated-pk_txt_01</output-dir>
@@ -1748,444 +1748,444 @@
         </test-case>
     </test-group>
     <test-group name="failure">
-    <!--
-    <test-case FilePath="failure">
-      <compilation-unit name="q1_pricing_summary_report_failure">
-        <output-dir compare="Text">q1_pricing_summary_report_failure</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="failure">
+          <compilation-unit name="q1_pricing_summary_report_failure">
+            <output-dir compare="Text">q1_pricing_summary_report_failure</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
     </test-group>
-  <!--
-  <test-group name="flwor">
-    <test-case FilePath="flwor">
-      <compilation-unit name="for01">
-        <output-dir compare="Text">for01</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for02">
-        <output-dir compare="Text">for02</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for03">
-        <output-dir compare="Text">for03</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for04">
-        <output-dir compare="Text">for04</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for05">
-        <output-dir compare="Text">for05</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for06">
-        <output-dir compare="Text">for06</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for07">
-        <output-dir compare="Text">for07</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for08">
-        <output-dir compare="Text">for08</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for09">
-        <output-dir compare="Text">for09</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for10">
-        <output-dir compare="Text">for10</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for11">
-        <output-dir compare="Text">for11</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for12">
-        <output-dir compare="Text">for12</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for13">
-        <output-dir compare="Text">for13</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for14">
-        <output-dir compare="Text">for14</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for15">
-        <output-dir compare="Text">for15</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for16">
-        <output-dir compare="Text">for16</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for17">
-        <output-dir compare="Text">for17</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for18">
-        <output-dir compare="Text">for18</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="for19">
-        <output-dir compare="Text">for19</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="grpby01">
-        <output-dir compare="Text">grpby01</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="grpby02">
-        <output-dir compare="Text">grpby02</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let01">
-        <output-dir compare="Text">let01</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let02">
-        <output-dir compare="Text">let02</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let03">
-        <output-dir compare="Text">let03</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let04">
-        <output-dir compare="Text">let04</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let05">
-        <output-dir compare="Text">let05</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let06">
-        <output-dir compare="Text">let06</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let07">
-        <output-dir compare="Text">let07</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let08">
-        <output-dir compare="Text">let08</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let09">
-        <output-dir compare="Text">let09</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let10">
-        <output-dir compare="Text">let10</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let11">
-        <output-dir compare="Text">let11</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let12">
-        <output-dir compare="Text">let12</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let13">
-        <output-dir compare="Text">let13</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let14">
-        <output-dir compare="Text">let14</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let15">
-        <output-dir compare="Text">let15</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let16">
-        <output-dir compare="Text">let16</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let17">
-        <output-dir compare="Text">let17</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let18">
-        <output-dir compare="Text">let18</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let19">
-        <output-dir compare="Text">let19</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let20">
-        <output-dir compare="Text">let20</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let21">
-        <output-dir compare="Text">let21</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let22">
-        <output-dir compare="Text">let22</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let23">
-        <output-dir compare="Text">let23</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let24">
-        <output-dir compare="Text">let24</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let25">
-        <output-dir compare="Text">let25</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let26">
-        <output-dir compare="Text">let26</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let27">
-        <output-dir compare="Text">let27</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let28">
-        <output-dir compare="Text">let28</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let29">
-        <output-dir compare="Text">let29</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let30">
-        <output-dir compare="Text">let30</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let31">
-        <output-dir compare="Text">let31</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="let32">
-        <output-dir compare="Text">let32</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-01">
-        <output-dir compare="Text">order-by-01</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-02">
-        <output-dir compare="Text">order-by-02</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-03">
-        <output-dir compare="Text">order-by-03</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-04">
-        <output-dir compare="Text">order-by-04</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-05">
-        <output-dir compare="Text">order-by-05</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-06">
-        <output-dir compare="Text">order-by-06</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-07">
-        <output-dir compare="Text">order-by-07</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-08">
-        <output-dir compare="Text">order-by-08</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-09">
-        <output-dir compare="Text">order-by-09</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-10">
-        <output-dir compare="Text">order-by-10</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-11">
-        <output-dir compare="Text">order-by-11</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="order-by-12">
-        <output-dir compare="Text">order-by-12</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-01">
-        <output-dir compare="Text">ret-01</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-02">
-        <output-dir compare="Text">ret-02</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-03">
-        <output-dir compare="Text">ret-03</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-04">
-        <output-dir compare="Text">ret-04</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-05">
-        <output-dir compare="Text">ret-05</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-06">
-        <output-dir compare="Text">ret-06</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-07">
-        <output-dir compare="Text">ret-07</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-08">
-        <output-dir compare="Text">ret-08</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-09">
-        <output-dir compare="Text">ret-09</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-10">
-        <output-dir compare="Text">ret-10</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-11">
-        <output-dir compare="Text">ret-11</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-12">
-        <output-dir compare="Text">ret-12</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-13">
-        <output-dir compare="Text">ret-13</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-14">
-        <output-dir compare="Text">ret-14</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-15">
-        <output-dir compare="Text">ret-15</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
-  <test-group name="writers">
-    <test-case FilePath="writers">
-      <compilation-unit name="print_01">
-        <output-dir compare="Text">print_01</output-dir>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-16">
-        <output-dir compare="Text">ret-16</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-17">
-        <output-dir compare="Text">ret-17</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-18">
-        <output-dir compare="Text">ret-18</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="flwor">
-      <compilation-unit name="ret-19">
-        <output-dir compare="Text">ret-19</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
-  -->
+    <!--
+    <test-group name="flwor">
+      <test-case FilePath="flwor">
+        <compilation-unit name="for01">
+          <output-dir compare="Text">for01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for02">
+          <output-dir compare="Text">for02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for03">
+          <output-dir compare="Text">for03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for04">
+          <output-dir compare="Text">for04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for05">
+          <output-dir compare="Text">for05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for06">
+          <output-dir compare="Text">for06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for07">
+          <output-dir compare="Text">for07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for08">
+          <output-dir compare="Text">for08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for09">
+          <output-dir compare="Text">for09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for10">
+          <output-dir compare="Text">for10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for11">
+          <output-dir compare="Text">for11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for12">
+          <output-dir compare="Text">for12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for13">
+          <output-dir compare="Text">for13</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for14">
+          <output-dir compare="Text">for14</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for15">
+          <output-dir compare="Text">for15</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for16">
+          <output-dir compare="Text">for16</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for17">
+          <output-dir compare="Text">for17</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for18">
+          <output-dir compare="Text">for18</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for19">
+          <output-dir compare="Text">for19</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="grpby01">
+          <output-dir compare="Text">grpby01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="grpby02">
+          <output-dir compare="Text">grpby02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let01">
+          <output-dir compare="Text">let01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let02">
+          <output-dir compare="Text">let02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let03">
+          <output-dir compare="Text">let03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let04">
+          <output-dir compare="Text">let04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let05">
+          <output-dir compare="Text">let05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let06">
+          <output-dir compare="Text">let06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let07">
+          <output-dir compare="Text">let07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let08">
+          <output-dir compare="Text">let08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let09">
+          <output-dir compare="Text">let09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let10">
+          <output-dir compare="Text">let10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let11">
+          <output-dir compare="Text">let11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let12">
+          <output-dir compare="Text">let12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let13">
+          <output-dir compare="Text">let13</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let14">
+          <output-dir compare="Text">let14</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let15">
+          <output-dir compare="Text">let15</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let16">
+          <output-dir compare="Text">let16</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let17">
+          <output-dir compare="Text">let17</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let18">
+          <output-dir compare="Text">let18</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let19">
+          <output-dir compare="Text">let19</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let20">
+          <output-dir compare="Text">let20</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let21">
+          <output-dir compare="Text">let21</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let22">
+          <output-dir compare="Text">let22</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let23">
+          <output-dir compare="Text">let23</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let24">
+          <output-dir compare="Text">let24</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let25">
+          <output-dir compare="Text">let25</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let26">
+          <output-dir compare="Text">let26</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let27">
+          <output-dir compare="Text">let27</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let28">
+          <output-dir compare="Text">let28</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let29">
+          <output-dir compare="Text">let29</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let30">
+          <output-dir compare="Text">let30</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let31">
+          <output-dir compare="Text">let31</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let32">
+          <output-dir compare="Text">let32</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-01">
+          <output-dir compare="Text">order-by-01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-02">
+          <output-dir compare="Text">order-by-02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-03">
+          <output-dir compare="Text">order-by-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-04">
+          <output-dir compare="Text">order-by-04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-05">
+          <output-dir compare="Text">order-by-05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-06">
+          <output-dir compare="Text">order-by-06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-07">
+          <output-dir compare="Text">order-by-07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-08">
+          <output-dir compare="Text">order-by-08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-09">
+          <output-dir compare="Text">order-by-09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-10">
+          <output-dir compare="Text">order-by-10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-11">
+          <output-dir compare="Text">order-by-11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-12">
+          <output-dir compare="Text">order-by-12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-01">
+          <output-dir compare="Text">ret-01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-02">
+          <output-dir compare="Text">ret-02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-03">
+          <output-dir compare="Text">ret-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-04">
+          <output-dir compare="Text">ret-04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-05">
+          <output-dir compare="Text">ret-05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-06">
+          <output-dir compare="Text">ret-06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-07">
+          <output-dir compare="Text">ret-07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-08">
+          <output-dir compare="Text">ret-08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-09">
+          <output-dir compare="Text">ret-09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-10">
+          <output-dir compare="Text">ret-10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-11">
+          <output-dir compare="Text">ret-11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-12">
+          <output-dir compare="Text">ret-12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-13">
+          <output-dir compare="Text">ret-13</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-14">
+          <output-dir compare="Text">ret-14</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-15">
+          <output-dir compare="Text">ret-15</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="writers">
+      <test-case FilePath="writers">
+        <compilation-unit name="print_01">
+          <output-dir compare="Text">print_01</output-dir>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-16">
+          <output-dir compare="Text">ret-16</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-17">
+          <output-dir compare="Text">ret-17</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-18">
+          <output-dir compare="Text">ret-18</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-19">
+          <output-dir compare="Text">ret-19</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    -->
     <test-group name="fuzzyjoin">
         <test-case FilePath="fuzzyjoin">
             <compilation-unit name="dblp-1_1">
@@ -2414,13 +2414,13 @@
                 <output-dir compare="Text">dblp-lookup_1</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="fuzzyjoin">
-      <compilation-unit name="dblp-splits-3_1">
-        <output-dir compare="Text">dblp-splits-3_1</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="fuzzyjoin">
+          <compilation-unit name="dblp-splits-3_1">
+            <output-dir compare="Text">dblp-splits-3_1</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="fuzzyjoin">
             <compilation-unit name="opentype">
                 <output-dir compare="Text">opentype</output-dir>
@@ -2781,17 +2781,17 @@
                 <output-dir compare="Text">scan-collection_01</output-dir>
             </compilation-unit>
         </test-case>
-<!--         <test-case FilePath="list">	 
-            <compilation-unit name="union_01">		
-                <output-dir compare="Text">union_01</output-dir>		
-            </compilation-unit>		
-        </test-case>		
-        <test-case FilePath="list">		
-            <compilation-unit name="union_02">		
-                <output-dir compare="Text">union_02</output-dir>		
-            </compilation-unit>		
-        </test-case>
- -->
+        <!--         <test-case FilePath="list">
+                    <compilation-unit name="union_01">
+                        <output-dir compare="Text">union_01</output-dir>
+                    </compilation-unit>
+                </test-case>
+                <test-case FilePath="list">
+                    <compilation-unit name="union_02">
+                        <output-dir compare="Text">union_02</output-dir>
+                    </compilation-unit>
+                </test-case>
+         -->
         <test-case FilePath="list">
             <compilation-unit name="unordered-list-constructor_01">
                 <output-dir compare="Text">unordered-list-constructor_01</output-dir>
@@ -2860,20 +2860,20 @@
                 <output-dir compare="Text">stable_sort</output-dir>
             </compilation-unit>
         </test-case>
-   <!--
-    <test-case FilePath="misc">
-      <compilation-unit name="range_01">
-        <output-dir compare="Text">range_01</output-dir>
-      </compilation-unit>
-    </test-case>
-   -->
-  <!--
-    <test-case FilePath="misc">
-      <compilation-unit name="tid_01">
-        <output-dir compare="Text">tid_01</output-dir>
-      </compilation-unit>
-    </test-case>
-   -->
+        <!--
+         <test-case FilePath="misc">
+           <compilation-unit name="range_01">
+             <output-dir compare="Text">range_01</output-dir>
+           </compilation-unit>
+         </test-case>
+        -->
+        <!--
+          <test-case FilePath="misc">
+            <compilation-unit name="tid_01">
+              <output-dir compare="Text">tid_01</output-dir>
+            </compilation-unit>
+          </test-case>
+         -->
         <test-case FilePath="misc">
             <compilation-unit name="year_01">
                 <output-dir compare="Text">year_01</output-dir>
@@ -3969,27 +3969,27 @@
         </test-case>
     </test-group>
     <test-group name="open-closed">
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="c2c-w-optional">
-        <output-dir compare="Text">c2c-w-optional</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="c2c-wo-optional">
-        <output-dir compare="Text">c2c-wo-optional</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="c2c">
-        <output-dir compare="Text">c2c</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="c2c-w-optional">
+            <output-dir compare="Text">c2c-w-optional</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="c2c-wo-optional">
+            <output-dir compare="Text">c2c-wo-optional</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="c2c">
+            <output-dir compare="Text">c2c</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="open-closed">
             <compilation-unit name="heterog-list-ordered01">
                 <output-dir compare="Text">heterog-list-ordered01</output-dir>
@@ -4000,20 +4000,20 @@
                 <output-dir compare="Text">heterog-list01</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="heterog-list02">
-        <output-dir compare="Text">heterog-list02</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="heterog-list03">
-        <output-dir compare="Text">heterog-list03</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="heterog-list02">
+            <output-dir compare="Text">heterog-list02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="heterog-list03">
+            <output-dir compare="Text">heterog-list03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="open-closed">
             <compilation-unit name="open-closed-01">
                 <output-dir compare="Text">open-closed-01</output-dir>
@@ -4059,55 +4059,55 @@
                 <output-dir compare="Text">query-issue236</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-15">
-        <output-dir compare="Text">open-closed-15</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-16">
-        <output-dir compare="Text">open-closed-16</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-17">
-        <output-dir compare="Text">open-closed-17</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-19">
-        <output-dir compare="Text">open-closed-19</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-20">
-        <output-dir compare="Text">open-closed-20</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-21">
-        <output-dir compare="Text">open-closed-21</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-22">
-        <output-dir compare="Text">open-closed-22</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-15">
+            <output-dir compare="Text">open-closed-15</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-16">
+            <output-dir compare="Text">open-closed-16</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-17">
+            <output-dir compare="Text">open-closed-17</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-19">
+            <output-dir compare="Text">open-closed-19</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-20">
+            <output-dir compare="Text">open-closed-20</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-21">
+            <output-dir compare="Text">open-closed-21</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-22">
+            <output-dir compare="Text">open-closed-22</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="open-closed">
             <compilation-unit name="open-closed-24">
                 <output-dir compare="Text">open-closed-24</output-dir>
@@ -4123,25 +4123,25 @@
                 <output-dir compare="Text">open-closed-26</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-28">
-        <output-dir compare="Text">open-closed-28</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-28">
+            <output-dir compare="Text">open-closed-28</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="open-closed">
             <compilation-unit name="open-closed-29">
                 <output-dir compare="Text">open-closed-29</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="open-closed">
-      <compilation-unit name="open-closed-30">
-        <output-dir compare="Text">open-closed-30</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-30">
+            <output-dir compare="Text">open-closed-30</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="open-closed">
             <compilation-unit name="open-closed-31">
                 <output-dir compare="Text">open-closed-31</output-dir>
@@ -4267,20 +4267,20 @@
                 <output-dir compare="Text">everysat_01</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="quantifiers">
-      <compilation-unit name="everysat_02">
-        <output-dir compare="Text">everysat_02</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="quantifiers">
-      <compilation-unit name="everysat_03">
-        <output-dir compare="Text">everysat_03</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="everysat_02">
+            <output-dir compare="Text">everysat_02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="everysat_03">
+            <output-dir compare="Text">everysat_03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="quantifiers">
             <compilation-unit name="everysat_04">
                 <output-dir compare="Text">everysat_04</output-dir>
@@ -4296,27 +4296,27 @@
                 <output-dir compare="Text">somesat_02</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="quantifiers">
-      <compilation-unit name="somesat_03">
-        <output-dir compare="Text">somesat_03</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="quantifiers">
-      <compilation-unit name="somesat_04">
-        <output-dir compare="Text">somesat_04</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--
-    <test-case FilePath="quantifiers">
-      <compilation-unit name="somesat_05">
-        <output-dir compare="Text">somesat_05</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="somesat_03">
+            <output-dir compare="Text">somesat_03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="somesat_04">
+            <output-dir compare="Text">somesat_04</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="somesat_05">
+            <output-dir compare="Text">somesat_05</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="quantifiers">
             <compilation-unit name="somesat_06">
                 <output-dir compare="Text">somesat_06</output-dir>
@@ -4455,25 +4455,25 @@
                 <output-dir compare="Text">spatial_types_01</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="scan">
-      <compilation-unit name="spatial_types_02">
-        <output-dir compare="Text">spatial_types_02</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="scan">
+          <compilation-unit name="spatial_types_02">
+            <output-dir compare="Text">spatial_types_02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="scan">
             <compilation-unit name="temp_types_01">
                 <output-dir compare="Text">temp_types_01</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="scan">
-      <compilation-unit name="temp_types_02">
-        <output-dir compare="Text">temp_types_02</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="scan">
+          <compilation-unit name="temp_types_02">
+            <output-dir compare="Text">temp_types_02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
     </test-group>
     <test-group name="semistructured">
         <test-case FilePath="semistructured">
@@ -4992,13 +4992,13 @@
                 <output-dir compare="Text">startwith02</output-dir>
             </compilation-unit>
         </test-case>
-    <!--
-    <test-case FilePath="string">
-      <compilation-unit name="startwith03">
-        <output-dir compare="Text">startwith03</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--
+        <test-case FilePath="string">
+          <compilation-unit name="startwith03">
+            <output-dir compare="Text">startwith03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="string">
             <compilation-unit name="strconcat01">
                 <output-dir compare="Text">strconcat01</output-dir>
@@ -5609,13 +5609,13 @@
                 <output-dir compare="Text">print_01</output-dir>
             </compilation-unit>
         </test-case>
-<!--  TODO(madhusudancs): Enable this test when REST API supports serialized output support.
-    <test-case FilePath="writers">
-      <compilation-unit name="serialized_01">
-        <output-dir compare="Text">serialized_01</output-dir>
-      </compilation-unit>
-    </test-case>
--->
+        <!--  TODO(madhusudancs): Enable this test when REST API supports serialized output support.
+            <test-case FilePath="writers">
+              <compilation-unit name="serialized_01">
+                <output-dir compare="Text">serialized_01</output-dir>
+              </compilation-unit>
+            </test-case>
+        -->
     </test-group>
     <test-group name="cross-dataverse">
         <test-case FilePath="cross-dataverse">
@@ -5643,13 +5643,13 @@
                 <output-dir compare="Text">cross-dv07</output-dir>
             </compilation-unit>
         </test-case>
-    <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
-    <test-case FilePath="cross-dataverse">
-      <compilation-unit name="cross-dv08">
-        <output-dir compare="Text">cross-dv08</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
+        <test-case FilePath="cross-dataverse">
+          <compilation-unit name="cross-dv08">
+            <output-dir compare="Text">cross-dv08</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="cross-dataverse">
             <compilation-unit name="cross-dv09">
                 <output-dir compare="Text">cross-dv09</output-dir>
@@ -5687,20 +5687,20 @@
                 <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
             </compilation-unit>
         </test-case>
-    <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
-    <test-case FilePath="cross-dataverse">
-      <compilation-unit name="cross-dv17">
-        <output-dir compare="Text">cross-dv17</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
-    <test-case FilePath="cross-dataverse">
-      <compilation-unit name="cross-dv18">
-        <output-dir compare="Text">cross-dv18</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
+        <test-case FilePath="cross-dataverse">
+          <compilation-unit name="cross-dv17">
+            <output-dir compare="Text">cross-dv17</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
+        <test-case FilePath="cross-dataverse">
+          <compilation-unit name="cross-dv18">
+            <output-dir compare="Text">cross-dv18</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="cross-dataverse">
             <compilation-unit name="cross-dv19">
                 <output-dir compare="Text">cross-dv19</output-dir>
@@ -5764,13 +5764,13 @@
                 <output-dir compare="Text">udf02</output-dir>
             </compilation-unit>
         </test-case>
-    <!-- causes NPE: Issue 200
-    <test-case FilePath="user-defined-functions">
-      <compilation-unit name="udf03">
-        <output-dir compare="Text">udf03</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!-- causes NPE: Issue 200
+        <test-case FilePath="user-defined-functions">
+          <compilation-unit name="udf03">
+            <output-dir compare="Text">udf03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="user-defined-functions">
             <compilation-unit name="udf04">
                 <output-dir compare="Text">udf04</output-dir>
@@ -5826,13 +5826,13 @@
                 <output-dir compare="Text">udf14</output-dir>
             </compilation-unit>
         </test-case>
-    <!-- Issue 166
-    <test-case FilePath="user-defined-functions">
-      <compilation-unit name="udf15">
-        <output-dir compare="Text">udf15</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!-- Issue 166
+        <test-case FilePath="user-defined-functions">
+          <compilation-unit name="udf15">
+            <output-dir compare="Text">udf15</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="user-defined-functions">
             <compilation-unit name="udf16">
                 <output-dir compare="Text">udf16</output-dir>
@@ -5873,20 +5873,20 @@
                 <output-dir compare="Text">udf23</output-dir>
             </compilation-unit>
         </test-case>
-    <!-- Issue 195
-    <test-case FilePath="user-defined-functions">
-      <compilation-unit name="udf24">
-        <output-dir compare="Text">udf24</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
-    <!-- Issue 218
-    <test-case FilePath="user-defined-functions">
-      <compilation-unit name="udf25">
-        <output-dir compare="Text">udf25</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!-- Issue 195
+        <test-case FilePath="user-defined-functions">
+          <compilation-unit name="udf24">
+            <output-dir compare="Text">udf24</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+        <!-- Issue 218
+        <test-case FilePath="user-defined-functions">
+          <compilation-unit name="udf25">
+            <output-dir compare="Text">udf25</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="user-defined-functions">
             <compilation-unit name="udf26">
                 <output-dir compare="Text">udf26</output-dir>
@@ -6090,13 +6090,13 @@
                 <output-dir compare="Text">feeds_05</output-dir>
             </compilation-unit>
         </test-case>
-    <!--Disable it because of sporadic failures. Raman will re-enable it.
-    <test-case FilePath="feeds">
-      <compilation-unit name="feeds_06">
-        <output-dir compare="Text">feeds_06</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
+        <!--Disable it because of sporadic failures. Raman will re-enable it.
+        <test-case FilePath="feeds">
+          <compilation-unit name="feeds_06">
+            <output-dir compare="Text">feeds_06</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
         <test-case FilePath="feeds">
             <compilation-unit name="feeds_07">
                 <output-dir compare="Text">feeds_07</output-dir>
@@ -6135,6 +6135,28 @@
             </compilation-unit>
         </test-case>
     </test-group>
+    <test-group>
+        <test-case FilePath="big-object">
+            <compilation-unit name="big_object_sort">
+                <output-dir compare="Text">big_object_sort</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="big-object">
+            <compilation-unit name="big_object_groupby">
+                <output-dir compare="Text">big_object_groupby</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="big-object">
+            <compilation-unit name="big_object_groupby-2">
+                <output-dir compare="Text">big_object_groupby-2</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="big-object">
+            <compilation-unit name="big_object_join">
+                <output-dir compare="Text">big_object_join</output-dir>
+            </compilation-unit>
+        </test-case>
+    </test-group>
     <test-group name="external-indexing">
         <test-case FilePath="external-indexing">
             <compilation-unit name="text-format">

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index 4673266..403d0c9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -19,6 +19,7 @@ import java.nio.ByteBuffer;
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
 import edu.uci.ics.asterix.common.transactions.ILogManager;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -48,8 +49,8 @@ public class AsterixLSMInsertDeleteOperatorNodePushable extends LSMIndexInsertUp
     @Override
     public void open() throws HyracksDataException {
         RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
-        writeBuffer = ctx.allocateFrame();
+        accessor = new FrameTupleAccessor(inputRecDesc);
+        writeBuffer = new VSizeFrame(ctx);
         writer.open();
         indexHelper.open();
         AbstractLSMIndex lsmIndex = (AbstractLSMIndex) indexHelper.getIndexInstance();
@@ -124,8 +125,9 @@ public class AsterixLSMInsertDeleteOperatorNodePushable extends LSMIndexInsertUp
             e.printStackTrace();
             throw new HyracksDataException(e);
         }
-        System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
-        FrameUtils.flushFrame(writeBuffer, writer);
+        writeBuffer.ensureFrameSize(buffer.capacity());
+        FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+        FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
index c90f43c..f0e20c1 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAdapter.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.asterix.external.dataset.adapter;
 
-import java.nio.ByteBuffer;
 import java.util.Map;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -23,12 +22,13 @@ import edu.uci.ics.asterix.external.dataset.adapter.IPullBasedFeedClient.InflowS
 import edu.uci.ics.asterix.metadata.feeds.FeedPolicyEnforcer;
 import edu.uci.ics.asterix.metadata.feeds.IPullBasedFeedAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 /**
  * Acts as an abstract class for all pull-based external data adapters. Captures
@@ -48,7 +48,7 @@ public abstract class PullBasedAdapter implements IPullBasedFeedAdapter {
     protected Map<String, String> configuration;
 
     private FrameTupleAppender appender;
-    private ByteBuffer frame;
+    private IFrame frame;
     private long tupleCount = 0;
     private final IHyracksTaskContext ctx;
     private int frameTupleCount = 0;
@@ -76,9 +76,8 @@ public abstract class PullBasedAdapter implements IPullBasedFeedAdapter {
 
     @Override
     public void start(int partition, IFrameWriter writer) throws Exception {
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        frame = ctx.allocateFrame();
-        appender.reset(frame, true);
+        frame = new VSizeFrame(ctx);
+        appender = new FrameTupleAppender(frame);
 
         pullBasedFeedClient = getFeedClient(partition);
         InflowState inflowState = null;
@@ -98,14 +97,14 @@ public abstract class PullBasedAdapter implements IPullBasedFeedAdapter {
                         if (LOGGER.isLoggable(Level.INFO)) {
                             LOGGER.info("Reached end of feed");
                         }
-                        FrameUtils.flushFrame(frame, writer);
+                        appender.flush(writer, true);
                         tupleCount += frameTupleCount;
                         frameTupleCount = 0;
                         continueIngestion = false;
                         break;
                     case DATA_NOT_AVAILABLE:
                         if (frameTupleCount > 0) {
-                            FrameUtils.flushFrame(frame, writer);
+                            appender.flush(writer, true);
                             tupleCount += frameTupleCount;
                             frameTupleCount = 0;
                         }
@@ -134,8 +133,7 @@ public abstract class PullBasedAdapter implements IPullBasedFeedAdapter {
 
     private void appendTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
                     tupleBuilder.getSize())) {
                 throw new IllegalStateException();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
index 8fe4c70..96f0a89 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AbstractIndexingTupleParser.java
@@ -16,7 +16,6 @@ package edu.uci.ics.asterix.external.indexing.dataflow;
 
 import java.io.DataOutput;
 import java.io.InputStream;
-import java.nio.ByteBuffer;
 
 import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -25,12 +24,12 @@ import edu.uci.ics.asterix.om.base.AMutableInt64;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 
 public abstract class AbstractIndexingTupleParser implements ITupleParser{
@@ -38,7 +37,6 @@ public abstract class AbstractIndexingTupleParser implements ITupleParser{
     protected ArrayTupleBuilder tb;
     protected DataOutput dos;
     protected final FrameTupleAppender appender;
-    protected final ByteBuffer frame;
     protected final ARecordType recType;
     protected final IHyracksTaskContext ctx;
     protected final IAsterixHDFSRecordParser deserializer;
@@ -51,8 +49,7 @@ public abstract class AbstractIndexingTupleParser implements ITupleParser{
     protected final ISerializerDeserializer longSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
     
     public AbstractIndexingTupleParser(IHyracksTaskContext ctx, ARecordType recType, IAsterixHDFSRecordParser deserializer) throws HyracksDataException {
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        frame = ctx.allocateFrame();
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         this.recType = recType;
         this.ctx = ctx;
         this.deserializer = deserializer;
@@ -61,7 +58,6 @@ public abstract class AbstractIndexingTupleParser implements ITupleParser{
     @Override
     public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
         AbstractHDFSReader inReader = (AbstractHDFSReader) in;
-        appender.reset(frame, true);
         Object record;
         try {
             inReader.initialize();
@@ -75,9 +71,7 @@ public abstract class AbstractIndexingTupleParser implements ITupleParser{
                 addTupleToFrame(writer);
                 record = inReader.readNext();
             }
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
@@ -88,8 +82,7 @@ public abstract class AbstractIndexingTupleParser implements ITupleParser{
 
     protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException("Record is too big to fit in a frame");
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
index 78baa4a..56b1ee9 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedControlledTupleParser.java
@@ -27,6 +27,7 @@ import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
@@ -36,7 +37,6 @@ import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 
 /**
@@ -49,7 +49,6 @@ public class AdmOrDelimitedControlledTupleParser implements IControlledTuplePars
     private ArrayTupleBuilder tb;
     private transient DataOutput dos;
     private final FrameTupleAppender appender;
-    private final ByteBuffer frame;
     protected final ARecordType recType;
     private IDataParser parser;
     private boolean propagateInput;
@@ -78,9 +77,8 @@ public class AdmOrDelimitedControlledTupleParser implements IControlledTuplePars
         this.propagatedFields = propagatedFields;
         this.ridFields = ridFields;
         this.parser = parser;
-        this.tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecDesc);
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        frame = ctx.allocateFrame();
+        this.tupleAccessor = new FrameTupleAccessor(inRecDesc);
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         if (propagateInput) {
             tb = new ArrayTupleBuilder(propagatedFields.length + 1);
         } else {
@@ -88,7 +86,6 @@ public class AdmOrDelimitedControlledTupleParser implements IControlledTuplePars
         }
         frameTuple = new FrameTupleReference();
         dos = tb.getDataOutput();
-        appender.reset(frame, true);
         bbis = new ByteBufferInputStream();
         dis = new DataInputStream(bbis);
         nullByte = ATypeTag.NULL.serialize();
@@ -110,9 +107,7 @@ public class AdmOrDelimitedControlledTupleParser implements IControlledTuplePars
     public void close(IFrameWriter writer) throws Exception {
         try {
             in.close();
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
@@ -233,8 +228,7 @@ public class AdmOrDelimitedControlledTupleParser implements IControlledTuplePars
 
     protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException();
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
index 74009e0..ed1f2b6 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/AdmOrDelimitedIndexingTupleParser.java
@@ -15,7 +15,6 @@
 package edu.uci.ics.asterix.external.indexing.dataflow;
 
 import java.io.InputStream;
-import java.nio.ByteBuffer;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.external.indexing.input.AbstractHDFSReader;
@@ -26,19 +25,18 @@ import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.runtime.operators.file.IDataParser;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
 
 public class AdmOrDelimitedIndexingTupleParser implements ITupleParser {
 
     private ArrayTupleBuilder tb;
     private final FrameTupleAppender appender;
-    private final ByteBuffer frame;
     private final ARecordType recType;
     private final IDataParser parser;
     private final AMutableInt32 aMutableInt = new AMutableInt32(0);
@@ -55,16 +53,14 @@ public class AdmOrDelimitedIndexingTupleParser implements ITupleParser {
             throws HyracksDataException {
         this.parser = parser;
         this.recType = recType;
-        appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         tb = new ArrayTupleBuilder(3);
-        frame = ctx.allocateFrame();
     }
 
     @Override
     public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
         // Cast the input stream to a record reader
         AbstractHDFSReader inReader = (AbstractHDFSReader) in;
-        appender.reset(frame, true);
         try {
             parser.initialize(in, recType, true);
             while (true) {
@@ -76,9 +72,7 @@ public class AdmOrDelimitedIndexingTupleParser implements ITupleParser {
                 appendIndexingData(tb, inReader);
                 addTupleToFrame(writer);
             }
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            appender.flush(writer, true);
         } catch (AsterixException ae) {
             throw new HyracksDataException(ae);
         } catch (Exception ioe) {
@@ -97,8 +91,7 @@ public class AdmOrDelimitedIndexingTupleParser implements ITupleParser {
 
     private void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new IllegalStateException("Record is too big to fit in a frame");
             }


[06/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby-2/groupby.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby-2/groupby.1.adm b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby-2/groupby.1.adm
new file mode 100644
index 0000000..5d4a27e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby-2/groupby.1.adm
@@ -0,0 +1,6006 @@
+[ { "id": 1i32, "length": 32768, "comment": "egular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular co
 urts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts abo
 ve theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theeg
 ular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular cou
 rts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts abov
 e theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegu
 lar courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular cour
 ts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above
  theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegul
 ar courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular court
 s above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above 
 theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegula
 r courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts
  above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above t
 heegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular
  courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts 
 above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above th
 eegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular 
 courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts a
 bove theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above the
 egular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular c
 ourts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts ab
 ove theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above thee
 gular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular co
 urts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts abo
 ve theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theeg
 ular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular cou
 rts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts abov
 e theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegu
 lar courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular cour
 ts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above
  theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegul
 ar courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts above theegular courts ab" }
+, { "id": 1i32, "length": 34, "comment": "ly final dependencies: slyly bold " }
+, { "id": 1i32, "length": 17, "comment": "arefully slyly ex" }
+, { "id": 1i32, "length": 29, "comment": "riously. regular, express dep" }
+, { "id": 1i32, "length": 24, "comment": " pending foxes. slyly re" }
+, { "id": 1i32, "length": 23, "comment": "lites. fluffily even de" }
+, { "id": 2i32, "length": 31, "comment": "ven requests. deposits breach a" }
+, { "id": 3i32, "length": 22, "comment": " unusual accounts. eve" }
+, { "id": 3i32, "length": 21, "comment": "y. fluffily pending d" }
+, { "id": 3i32, "length": 22, "comment": "ages nag slyly pending" }
+, { "id": 3i32, "length": 25, "comment": "ges sleep after the caref" }
+, { "id": 3i32, "length": 16, "comment": "nal foxes wake. " }
+, { "id": 3i32, "length": 35, "comment": "ongside of the furiously brave acco" }
+, { "id": 4i32, "length": 38, "comment": "- quickly regular packages sleep. idly" }
+, { "id": 5i32, "length": 37, "comment": "sts use slyly quickly special instruc" }
+, { "id": 5i32, "length": 18, "comment": "ts wake furiously " }
+, { "id": 5i32, "length": 27, "comment": "eodolites. fluffily unusual" }
+, { "id": 6i32, "length": 25, "comment": "p furiously special foxes" }
+, { "id": 7i32, "length": 30, "comment": ". slyly special requests haggl" }
+, { "id": 7i32, "length": 16, "comment": "es. instructions" }
+, { "id": 7i32, "length": 39, "comment": "ns haggle carefully ironic deposits. bl" }
+, { "id": 7i32, "length": 13, "comment": "ithely regula" }
+, { "id": 7i32, "length": 15, "comment": " unusual reques" }
+, { "id": 7i32, "length": 42, "comment": "jole. excuses wake carefully alongside of " }
+, { "id": 7i32, "length": 30, "comment": "ss pinto beans wake against th" }
+, { "id": 32i32, "length": 24, "comment": " gifts cajole carefully." }
+, { "id": 32i32, "length": 17, "comment": "e slyly final pac" }
+, { "id": 32i32, "length": 42, "comment": "symptotes nag according to the ironic depo" }
+, { "id": 32i32, "length": 39, "comment": " express accounts wake according to the" }
+, { "id": 32i32, "length": 35, "comment": "lithely regular deposits. fluffily " }
+, { "id": 32i32, "length": 18, "comment": "sleep quickly. req" }
+, { "id": 33i32, "length": 28, "comment": "unusual packages doubt caref" }
+, { "id": 33i32, "length": 34, "comment": "ng to the furiously ironic package" }
+, { "id": 33i32, "length": 21, "comment": ". stealthily bold exc" }
+, { "id": 33i32, "length": 17, "comment": "gular theodolites" }
+, { "id": 34i32, "length": 15, "comment": "ar foxes sleep " }
+, { "id": 34i32, "length": 13, "comment": "thely slyly p" }
+, { "id": 34i32, "length": 31, "comment": "nic accounts. deposits are alon" }
+, { "id": 35i32, "length": 13, "comment": " quickly unti" }
+, { "id": 35i32, "length": 15, "comment": ", regular tithe" }
+, { "id": 35i32, "length": 32, "comment": ". silent, unusual deposits boost" }
+, { "id": 35i32, "length": 16, "comment": "ly alongside of " }
+, { "id": 35i32, "length": 29, "comment": "s are carefully against the f" }
+, { "id": 35i32, "length": 23, "comment": " the carefully regular " }
+, { "id": 36i32, "length": 25, "comment": " careful courts. special " }
+, { "id": 37i32, "length": 10, "comment": "iously ste" }
+, { "id": 37i32, "length": 42, "comment": "luffily regular requests. slyly final acco" }
+, { "id": 37i32, "length": 22, "comment": "the final requests. ca" }
+, { "id": 38i32, "length": 34, "comment": "s. blithely unusual theodolites am" }
+, { "id": 39i32, "length": 18, "comment": "eodolites. careful" }
+, { "id": 39i32, "length": 18, "comment": "quickly ironic fox" }
+, { "id": 39i32, "length": 13, "comment": "yly regular i" }
+, { "id": 39i32, "length": 30, "comment": "ckages across the slyly silent" }
+, { "id": 39i32, "length": 14, "comment": "he carefully e" }
+, { "id": 39i32, "length": 43, "comment": "heodolites sleep silently pending foxes. ac" }
+, { "id": 64i32, "length": 31, "comment": "ch slyly final, thin platelets." }
+, { "id": 65i32, "length": 18, "comment": " ideas. special, r" }
+, { "id": 65i32, "length": 38, "comment": "pending deposits nag even packages. ca" }
+, { "id": 65i32, "length": 43, "comment": "bove the even packages. accounts nag carefu" }
+, { "id": 66i32, "length": 39, "comment": "ut the unusual accounts sleep at the bo" }
+, { "id": 66i32, "length": 11, "comment": " regular de" }
+, { "id": 67i32, "length": 21, "comment": " cajole thinly expres" }
+, { "id": 67i32, "length": 18, "comment": "ly regular deposit" }
+, { "id": 67i32, "length": 41, "comment": "se quickly above the even, express reques" }
+, { "id": 67i32, "length": 11, "comment": "ultipliers " }
+, { "id": 67i32, "length": 32, "comment": "y unusual packages thrash pinto " }
+, { "id": 67i32, "length": 21, "comment": " even packages cajole" }
+, { "id": 68i32, "length": 42, "comment": "fully special instructions cajole. furious" }
+, { "id": 68i32, "length": 43, "comment": "egular dependencies affix ironically along " }
+, { "id": 68i32, "length": 35, "comment": "eposits nag special ideas. furiousl" }
+, { "id": 68i32, "length": 28, "comment": " excuses integrate fluffily " }
+, { "id": 68i32, "length": 37, "comment": " requests are unusual, regular pinto " }
+, { "id": 68i32, "length": 32, "comment": "ccounts. deposits use. furiously" }
+, { "id": 68i32, "length": 27, "comment": "oxes are slyly blithely fin" }
+, { "id": 69i32, "length": 20, "comment": "final, pending instr" }
+, { "id": 69i32, "length": 17, "comment": "nding accounts ca" }
+, { "id": 69i32, "length": 42, "comment": "regular epitaphs. carefully even ideas hag" }
+, { "id": 69i32, "length": 17, "comment": " blithely final d" }
+, { "id": 69i32, "length": 24, "comment": "s sleep carefully bold, " }
+, { "id": 69i32, "length": 19, "comment": "tect regular, speci" }
+, { "id": 70i32, "length": 33, "comment": "ggle. carefully pending dependenc" }
+, { "id": 70i32, "length": 19, "comment": "lyly special packag" }
+, { "id": 70i32, "length": 17, "comment": "n accounts are. q" }
+, { "id": 70i32, "length": 32, "comment": " packages wake pending accounts." }
+, { "id": 70i32, "length": 39, "comment": "quickly. fluffily unusual theodolites c" }
+, { "id": 70i32, "length": 30, "comment": "alongside of the deposits. fur" }
+, { "id": 71i32, "length": 35, "comment": " ironic packages believe blithely a" }
+, { "id": 71i32, "length": 10, "comment": "s cajole. " }
+, { "id": 71i32, "length": 11, "comment": "ckly. slyly" }
+, { "id": 71i32, "length": 32, "comment": "l accounts sleep across the pack" }
+, { "id": 71i32, "length": 35, "comment": " serve quickly fluffily bold deposi" }
+, { "id": 71i32, "length": 31, "comment": "y. pinto beans haggle after the" }
+, { "id": 96i32, "length": 28, "comment": "e quickly even ideas. furiou" }
+, { "id": 96i32, "length": 18, "comment": "ep-- carefully reg" }
+, { "id": 97i32, "length": 41, "comment": "gifts. furiously ironic packages cajole. " }
+, { "id": 97i32, "length": 34, "comment": "ayers cajole against the furiously" }
+, { "id": 97i32, "length": 32, "comment": "ic requests boost carefully quic" }
+, { "id": 98i32, "length": 28, "comment": " pending, regular accounts s" }
+, { "id": 98i32, "length": 30, "comment": ". unusual instructions against" }
+, { "id": 98i32, "length": 41, "comment": " cajole furiously. blithely ironic ideas " }
+, { "id": 98i32, "length": 32768, "comment": " carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly iro
 nic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic i
 deas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas 
 carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas caref
 ully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully.
  quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quic
 kly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly i
 ronic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic
  ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic idea
 s carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas car
 efully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefull
 y. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. qu
 ickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly
  ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly iron
 ic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic id
 eas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas c
 arefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefu
 lly. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. 
 quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quick
 ly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ir
 onic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic 
 ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas
  carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas care
 fully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully
 . quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. qui
 ckly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly 
 ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironi
 c ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ide
 as carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas ca
 refully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas careful
 ly. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. q
 uickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickl
 y ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas carefully. quickly ironic ideas" }
+, { "id": 99i32, "length": 35, "comment": "ests cajole fluffily waters. blithe" }
+, { "id": 99i32, "length": 14, "comment": "slyly. slyly e" }
+, { "id": 99i32, "length": 31, "comment": "kages are fluffily furiously ir" }
+, { "id": 99i32, "length": 11, "comment": "kages. requ" }
+, { "id": 100i32, "length": 29, "comment": "nto beans alongside of the fi" }
+, { "id": 100i32, "length": 17, "comment": "sts haggle. slowl" }
+, { "id": 100i32, "length": 16, "comment": "nd the quickly s" }
+, { "id": 100i32, "length": 28, "comment": "y. furiously ironic ideas gr" }
+, { "id": 100i32, "length": 19, "comment": "ular accounts. even" }
+, { "id": 101i32, "length": 17, "comment": ". quickly regular" }
+, { "id": 101i32, "length": 34, "comment": "ts-- final packages sleep furiousl" }
+, { "id": 101i32, "length": 38, "comment": "tes. blithely pending dolphins x-ray f" }
+, { "id": 102i32, "length": 21, "comment": "eposits cajole across" }
+, { "id": 102i32, "length": 35, "comment": "final packages. carefully even excu" }
+, { "id": 102i32, "length": 19, "comment": "bits. ironic accoun" }
+, { "id": 102i32, "length": 36, "comment": "ully across the ideas. final deposit" }
+, { "id": 103i32, "length": 20, "comment": "cajole. carefully ex" }
+, { "id": 103i32, "length": 12, "comment": "ironic accou" }
+, { "id": 103i32, "length": 36, "comment": "kages doze. special, regular deposit" }
+, { "id": 103i32, "length": 41, "comment": "ies. quickly ironic requests use blithely" }
+, { "id": 128i32, "length": 15, "comment": " cajole careful" }
+, { "id": 129i32, "length": 38, "comment": "uests. foxes cajole slyly after the ca" }
+, { "id": 129i32, "length": 30, "comment": "e carefully blithely bold dolp" }
+, { "id": 129i32, "length": 21, "comment": "quests. express ideas" }
+, { "id": 129i32, "length": 25, "comment": "sts nag bravely. fluffily" }
+, { "id": 129i32, "length": 20, "comment": "e. fluffily regular " }
+, { "id": 129i32, "length": 17, "comment": "packages are care" }
+, { "id": 129i32, "length": 32, "comment": "uietly bold theodolites. fluffil" }
+, { "id": 130i32, "length": 31, "comment": " pending dolphins sleep furious" }
+, { "id": 130i32, "length": 28, "comment": " requests. final instruction" }
+, { "id": 130i32, "length": 20, "comment": "thily about the ruth" }
+, { "id": 130i32, "length": 25, "comment": " slyly ironic decoys abou" }
+, { "id": 130i32, "length": 29, "comment": "lithely alongside of the regu" }
+, { "id": 131i32, "length": 30, "comment": "ironic, bold accounts. careful" }
+, { "id": 131i32, "length": 22, "comment": " are carefully slyly i" }
+, { "id": 131i32, "length": 42, "comment": "ending requests. final, ironic pearls slee" }
+, { "id": 132i32, "length": 19, "comment": "d instructions hagg" }
+, { "id": 132i32, "length": 25, "comment": "ges. platelets wake furio" }
+, { "id": 132i32, "length": 26, "comment": "refully blithely bold acco" }
+, { "id": 132i32, "length": 21, "comment": "y pending theodolites" }
+, { "id": 133i32, "length": 31, "comment": " the carefully regular theodoli" }
+, { "id": 133i32, "length": 28, "comment": "ts cajole fluffily quickly i" }
+, { "id": 133i32, "length": 27, "comment": "yly even gifts after the sl" }
+, { "id": 133i32, "length": 29, "comment": "e quickly across the dolphins" }
+, { "id": 134i32, "length": 18, "comment": "s. quickly regular" }
+, { "id": 134i32, "length": 24, "comment": " among the pending depos" }
+, { "id": 134i32, "length": 16, "comment": "lyly regular pac" }
+, { "id": 134i32, "length": 27, "comment": "ajole furiously. instructio" }
+, { "id": 134i32, "length": 12, "comment": "nts are quic" }
+, { "id": 134i32, "length": 43, "comment": "s! carefully unusual requests boost careful" }
+, { "id": 135i32, "length": 21, "comment": "ctions wake slyly abo" }
+, { "id": 135i32, "length": 22, "comment": "nal ideas. final instr" }
+, { "id": 135i32, "length": 22, "comment": "theodolites. quickly p" }
+, { "id": 135i32, "length": 38, "comment": " deposits believe. furiously regular p" }
+, { "id": 135i32, "length": 38, "comment": "counts doze against the blithely ironi" }
+, { "id": 135i32, "length": 24, "comment": "ptotes boost slowly care" }
+, { "id": 160i32, "length": 23, "comment": "ncies about the request" }
+, { "id": 160i32, "length": 37, "comment": "old, ironic deposits are quickly abov" }
+, { "id": 160i32, "length": 39, "comment": "st sleep even gifts. dependencies along" }
+, { "id": 161i32, "length": 29, "comment": ", regular sheaves sleep along" }
+, { "id": 162i32, "length": 25, "comment": "es! final somas integrate" }
+, { "id": 163i32, "length": 35, "comment": "inal requests. even pinto beans hag" }
+, { "id": 163i32, "length": 22, "comment": "ously express dependen" }
+, { "id": 163i32, "length": 21, "comment": "tructions integrate b" }
+, { "id": 163i32, "length": 32, "comment": "al, bold dependencies wake. iron" }
+, { "id": 163i32, "length": 11, "comment": " must belie" }
+, { "id": 163i32, "length": 26, "comment": "ly blithe accounts cajole " }
+, { "id": 164i32, "length": 22, "comment": "ayers wake carefully a" }
+, { "id": 164i32, "length": 38, "comment": "s. blithely special courts are blithel" }
+, { "id": 164i32, "length": 41, "comment": "ress packages haggle ideas. blithely spec" }
+, { "id": 164i32, "length": 13, "comment": "ts wake again" }
+, { "id": 164i32, "length": 42, "comment": "counts cajole fluffily regular packages. b" }
+, { "id": 164i32, "length": 40, "comment": "side of the slyly unusual theodolites. f" }
+, { "id": 164i32, "length": 23, "comment": "y carefully regular dep" }
+, { "id": 165i32, "length": 23, "comment": "riously requests. depos" }
+, { "id": 165i32, "length": 37, "comment": "uses sleep slyly ruthlessly regular a" }
+, { "id": 165i32, "length": 21, "comment": "jole slyly according " }
+, { "id": 165i32, "length": 21, "comment": " bold packages mainta" }
+, { "id": 165i32, "length": 27, "comment": "around the ironic, even orb" }
+, { "id": 166i32, "length": 34, "comment": "hily along the blithely pending fo" }
+, { "id": 166i32, "length": 29, "comment": "fully above the blithely fina" }
+, { "id": 166i32, "length": 25, "comment": "lar frays wake blithely a" }
+, { "id": 166i32, "length": 17, "comment": "e carefully bold " }
+, { "id": 167i32, "length": 31, "comment": "eans affix furiously-- packages" }
+, { "id": 167i32, "length": 16, "comment": "sly during the u" }
+, { "id": 192i32, "length": 19, "comment": ". carefully regular" }
+, { "id": 192i32, "length": 25, "comment": "equests. ideas sleep idea" }
+, { "id": 192i32, "length": 42, "comment": "ly pending theodolites haggle quickly fluf" }
+, { "id": 192i32, "length": 39, "comment": "s. dependencies nag furiously alongside" }
+, { "id": 192i32, "length": 11, "comment": "tes. carefu" }
+, { "id": 192i32, "length": 31, "comment": "he ironic requests haggle about" }
+, { "id": 193i32, "length": 25, "comment": "ffily. regular packages d" }
+, { "id": 193i32, "length": 35, "comment": "ly even accounts wake blithely bold" }
+, { "id": 193i32, "length": 30, "comment": "against the fluffily regular d" }
+, { "id": 194i32, "length": 35, "comment": " regular theodolites. regular, iron" }
+, { "id": 194i32, "length": 14, "comment": "about the blit" }
+, { "id": 194i32, "length": 31, "comment": "accounts detect quickly dogged " }
+, { "id": 194i32, "length": 24, "comment": "uriously unusual excuses" }
+, { "id": 194i32, "length": 15, "comment": " regular deposi" }
+, { "id": 194i32, "length": 38, "comment": "pecial packages wake after the slyly r" }
+, { "id": 194i32, "length": 32768, "comment": "y regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy reg
 ular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy r
 egular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy
  regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiou
 sy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furi
 ousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. fu
 riousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. 
 furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests
 . furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. furiousy regular requests. 

<TRUNCATED>


[10/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/data/big-object/customer.tbl.big
----------------------------------------------------------------------
diff --git a/asterix-app/data/big-object/customer.tbl.big b/asterix-app/data/big-object/customer.tbl.big
new file mode 100644
index 0000000..543b5db
--- /dev/null
+++ b/asterix-app/data/big-object/customer.tbl.big
@@ -0,0 +1,150 @@
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaph
 s nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag
  eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto 
 the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the e
 ven, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, 
 regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regul
 ar platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular pl
 atelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platele
 ts. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. r
 egular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regula
 r, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ir
 onic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic 
 epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epita
 phs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs n
 ag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag et
 o the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the
  even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even
 , regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, reg
 ular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular 
 platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular plate
 lets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets.
  regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regu
 lar, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, 
 ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironi
 c epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epi
 taphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs
  nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag 
 eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto t
 he even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the ev
 en, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, r
 egular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regula
 r platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular pla
 telets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. regular, ironic epitaphs nag eto the even, regular platelets. |
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|
+6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|
+7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|
+8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|
+9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|
+10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|
+11|Customer#000000011|PkWS 3HlXqwTuzrKg633BEi|23|33-464-151-3439|-272.60|BUILDING|ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. request
 s sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requ
 ests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. r
 equests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages
 . requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. cka
 ges. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. 
 ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto bean
 s. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto b
 eans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pint
 o beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular p
 into beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regula
 r pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly reg
 ular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly 
 regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the sly
 ly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the 
 slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above t
 he slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise abov
 e the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise a
 bove the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promis
 e above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans pro
 mise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans 
 promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto bea
 ns promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto 
 beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pin
 to beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even 
 pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly ev
 en pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly
  even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quic
 kly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. q
 uickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly
 . quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep sl
 yly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep
  slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. ckages. requests sleep slyly. quickly even pinto beans promise above|
+12|Customer#000000012|9PWKuhzT4Zr1Q|13|23-791-276-1263|3396.49|HOUSEHOLD| to the carefully final braids. blithely regular requests nag. ironic theodolites boost quickly along|
+13|Customer#000000013|nsXQu0oVjD7PM659uC3SRSp|3|13-761-547-5974|3857.34|BUILDING|ounts sleep carefully after the close frays. carefully bold notornis use ironic requests. blithely|
+14|Customer#000000014|KXkletMlL2JQEA |1|11-845-129-3851|5266.30|FURNITURE|, ironic packages across the unus|
+15|Customer#000000015|YtWggXoOLdwdo7b0y,BZaGUQMLJMX1Y,EC,6Dn|23|33-687-542-7601|2788.52|HOUSEHOLD| platelets. regular deposits detect asymptotes. blithely unusual packages nag slyly at the fluf|
+16|Customer#000000016|cYiaeMLZSMAOQ2 d0W,|10|20-781-609-3107|4681.03|FURNITURE|kly silent courts. thinly regular theodolites sleep fluffily after |
+17|Customer#000000017|izrh 6jdqtp2eqdtbkswDD8SG4SzXruMfIXyR7|2|12-970-682-3487|6.34|AUTOMOBILE|packages wake! blithely even pint|
+18|Customer#000000018|3txGO AiuFux3zT0Z9NYaFRnZt|6|16-155-215-1315|5494.43|BUILDING|s sleep. carefully even instructions nag furiously alongside of t|
+19|Customer#000000019|uc,3bHIx84H,wdrmLOjVsiqXCq2tr|18|28-396-526-5053|8914.71|HOUSEHOLD| nag. furiously careful packages are slyly at the accounts. furiously regular in|
+20|Customer#000000020|JrPk8Pqplj4Ne|22|32-957-234-8742|7603.40|FURNITURE|g alongside of the special excuses-- fluffily enticing packages wake |
+21|Customer#000000021|XYmVpr9yAHDEn|8|18-902-614-8344|1428.25|MACHINERY| quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickl
 y final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrat
 e blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u qu
 ickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts inte
 grate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously 
 u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts 
 integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiou
 sly u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accou
 nts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely fu
 riously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final a
 ccounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithel
 y furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly fin
 al accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate bli
 thely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly
  final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate
  blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u qui
 ckly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integ
 rate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u
  quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts i
 ntegrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furious
 ly u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely furiously u quickly final accounts integrate blithely

<TRUNCATED>

[11/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Apply the https://asterix-gerrit.ics.uci.edu/#/c/234/ API changes to
Asterix level.

Change-Id: I5459e877707a1494fc1bebf03d4457a7427e9e0f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/259
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>


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

Branch: refs/heads/master
Commit: 65100727ffebcbd774b524f0bee8dc4c2df438e0
Parents: e05df7b
Author: JavierJia <ji...@gmail.com>
Authored: Wed Jun 17 16:19:25 2015 -0700
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Wed Jun 17 20:45:37 2015 -0700

----------------------------------------------------------------------
 .../operators/physical/CommitRuntime.java       |    2 +-
 asterix-app/data/big-object/customer.tbl.big    |  150 +
 asterix-app/data/big-object/lineitem.tbl.big    | 6005 +++++++++++++++++
 asterix-app/data/big-object/order.tbl.big       | 1500 +++++
 .../uci/ics/asterix/result/ResultReader.java    |   11 +-
 .../edu/uci/ics/asterix/result/ResultUtils.java |   19 +-
 ...ConstantTupleSourceOperatorNodePushable.java |    9 +-
 .../resources/asterix-build-configuration.xml   |    6 +-
 .../ics/asterix/test/runtime/ExecutionTest.java |   11 +-
 .../src/test/resources/runtimets/only.xml       |   23 +
 .../big_object_groupby.1.ddl.aql                |   66 +
 .../big_object_groupby.2.update.aql             |    7 +
 .../big_object_groupby.3.query.aql              |   16 +
 .../big_object_groupby.1.ddl.aql                |   66 +
 .../big_object_groupby.2.update.aql             |    7 +
 .../big_object_groupby.3.query.aql              |   13 +
 .../big_object_join/big_object_join.1.ddl.aql   |   66 +
 .../big_object_join.2.update.aql                |    7 +
 .../big_object_join/big_object_join.3.query.aql |   20 +
 .../big_object_sort/big_object_sort.1.ddl.aql   |   66 +
 .../big_object_sort.2.update.aql                |    7 +
 .../big_object_sort/big_object_sort.3.query.aql |   18 +
 .../big_object_groupby-2/groupby.1.adm          | 6006 ++++++++++++++++++
 .../big-object/big_object_groupby/groupby.1.adm |  201 +
 .../big-object/big_object_join/join.1.adm       | 1501 +++++
 .../big-object/big_object_sort/sort.1.adm       | 1501 +++++
 .../src/test/resources/runtimets/testsuite.xml  | 1648 ++---
 ...erixLSMInsertDeleteOperatorNodePushable.java |   10 +-
 .../dataset/adapter/PullBasedAdapter.java       |   18 +-
 .../dataflow/AbstractIndexingTupleParser.java   |   15 +-
 .../AdmOrDelimitedControlledTupleParser.java    |   16 +-
 .../AdmOrDelimitedIndexingTupleParser.java      |   15 +-
 .../dataflow/HDFSObjectTupleParser.java         |   15 +-
 .../dataflow/RCFileControlledTupleParser.java   |   27 +-
 .../dataflow/SeqOrTxtControlledTupleParser.java |   16 +-
 ...rnalIndexBulkModifyOperatorNodePushable.java |    2 +-
 .../ConditionalPushTupleParserFactory.java      |   12 +-
 .../feeds/FeedIntakeOperatorNodePushable.java   |    5 +-
 .../feeds/FeedMetaOperatorDescriptor.java       |    2 +-
 ...ExternalBTreeSearchOperatorNodePushable.java |    7 +-
 ...ExternalRTreeSearchOperatorNodePushable.java |    7 +-
 .../operators/file/AbstractTupleParser.java     |   16 +-
 .../testframework/context/TestCaseContext.java  |    1 +
 ...ControlledFileSystemBasedAdapterFactory.java |    6 +-
 44 files changed, 18181 insertions(+), 961 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
index bed18fe..0ece791 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -134,6 +134,6 @@ public class CommitRuntime implements IPushRuntime {
 
     @Override
     public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
-        this.frameTupleAccessor = new FrameTupleAccessor(hyracksTaskCtx.getFrameSize(), recordDescriptor);
+        this.frameTupleAccessor = new FrameTupleAccessor(recordDescriptor);
     }
 }
\ No newline at end of file


[07/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
index ad21dd1..a64f350 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
@@ -14,12 +14,9 @@
  */
 package edu.uci.ics.asterix.result;
 
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
@@ -47,15 +44,15 @@ public class ResultReader {
 
     public void open(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
         reader = hyracksDataset.createReader(jobId, resultSetId);
-        frameTupleAccessor = new ResultFrameTupleAccessor(FRAME_SIZE);
+        frameTupleAccessor = new ResultFrameTupleAccessor();
     }
 
     public Status getStatus() {
         return reader.getResultStatus();
     }
 
-    public int read(ByteBuffer buffer) throws HyracksDataException {
-        return reader.read(buffer);
+    public int read(IFrame frame) throws HyracksDataException {
+        return reader.read(frame);
     }
 
     public IFrameTupleAccessor getFrameTupleAccessor() {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
index 326697f..3a4fd5f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultUtils.java
@@ -20,7 +20,6 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
@@ -37,8 +36,11 @@ import edu.uci.ics.asterix.api.common.SessionConfig.OutputFormat;
 import edu.uci.ics.asterix.api.http.servlet.APIServlet;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 public class ResultUtils {
@@ -84,13 +86,14 @@ public class ResultUtils {
         conf.out().print("\r\n");
     }
 
+    public static FrameManager resultDisplayFrameMgr = new FrameManager(ResultReader.FRAME_SIZE);
+
     public static void displayResults(ResultReader resultReader, SessionConfig conf)
             throws HyracksDataException {
         IFrameTupleAccessor fta = resultReader.getFrameTupleAccessor();
 
-        ByteBuffer buffer = ByteBuffer.allocate(ResultReader.FRAME_SIZE);
-        buffer.clear();
-        int bytesRead = resultReader.read(buffer);
+        IFrame frame = new VSizeFrame(resultDisplayFrameMgr);
+        int bytesRead = resultReader.read(frame);
         ByteBufferInputStream bbis = new ByteBufferInputStream();
 
         // Whether we need to separate top-level ADM instances with commas
@@ -122,13 +125,13 @@ public class ResultUtils {
         if (bytesRead > 0) {
             do {
                 try {
-                    fta.reset(buffer);
+                    fta.reset(frame.getBuffer());
                     int last = fta.getTupleCount();
                     String result;
                     for (int tIndex = 0; tIndex < last; tIndex++) {
                         int start = fta.getTupleStartOffset(tIndex);
                         int length = fta.getTupleEndOffset(tIndex) - start;
-                        bbis.setByteBuffer(buffer, start);
+                        bbis.setByteBuffer(frame.getBuffer(), start);
                         byte[] recordBytes = new byte[length];
                         int numread = bbis.read(recordBytes, 0, length);
                         if (conf.fmt() == OutputFormat.CSV) {
@@ -146,7 +149,7 @@ public class ResultUtils {
                             conf.out().print("\r\n");
                         }
                     }
-                    buffer.clear();
+                    frame.getBuffer().clear();
                 } finally {
                     try {
                         bbis.close();
@@ -154,7 +157,7 @@ public class ResultUtils {
                         throw new HyracksDataException(e);
                     }
                 }
-            } while (resultReader.read(buffer) > 0);
+            } while (resultReader.read(frame) > 0);
         }
 
         conf.out().flush();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java b/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 4cfb8c6..4ee37ba 100644
--- a/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
+++ b/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -15,8 +15,7 @@
 
 package edu.uci.ics.hyracks.dataflow.std.misc;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -41,14 +40,12 @@ public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutput
 
     @Override
     public void initialize() throws HyracksDataException {
-        ByteBuffer writeBuffer = ctx.allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(writeBuffer, true);
+        FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
         if (fieldSlots != null && tupleData != null && tupleSize > 0)
             appender.append(fieldSlots, tupleData, 0, tupleSize);
         writer.open();
         try {
-            FrameUtils.flushFrame(writeBuffer, writer);
+            appender.flush(writer, true);
         }
         finally {
             writer.close();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/main/resources/asterix-build-configuration.xml
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/resources/asterix-build-configuration.xml b/asterix-app/src/main/resources/asterix-build-configuration.xml
index 03f2f0e..9ac89e7 100644
--- a/asterix-app/src/main/resources/asterix-build-configuration.xml
+++ b/asterix-app/src/main/resources/asterix-build-configuration.xml
@@ -47,15 +47,15 @@
 	</property>
 	<property>
 		<name>compiler.sortmemory</name>
-		<value>98304</value>
+		<value>163840</value>
 	</property>
 	<property>
 		<name>compiler.groupmemory</name>
-		<value>98304</value>
+		<value>163840</value>
 	</property>
 	<property>
 		<name>compiler.joinmemory</name>
-		<value>131072</value>
+		<value>163840</value>
 	</property>
 	<property>
 		<name>storage.buffercache.pagesize</name>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
index 15f0c0a..d29f881 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
@@ -113,12 +113,21 @@ public class ExecutionTest {
 
     @Parameters
     public static Collection<Object[]> tests() throws Exception {
+        Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.ONLY_TESTSUITE_XML_NAME);
+        if (testArgs.size() == 0){
+            testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
+        }
+        return testArgs;
+    }
+
+    private static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
         Collection<Object[]> testArgs = new ArrayList<Object[]>();
         TestCaseContext.Builder b = new TestCaseContext.Builder();
-        for (TestCaseContext ctx : b.build(new File(PATH_BASE))) {
+        for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
             testArgs.add(new Object[] { ctx });
         }
         return testArgs;
+
     }
 
     private TestCaseContext tcCtx;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/only.xml
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/only.xml b/asterix-app/src/test/resources/runtimets/only.xml
new file mode 100644
index 0000000..a0cd439
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/only.xml
@@ -0,0 +1,23 @@
+<!--
+ ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License");
+ ! you may not use this file except in compliance with the License.
+ ! you may obtain a copy of the License from
+ !
+ !     http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing, software
+ ! distributed under the License is distributed on an "AS IS" BASIS,
+ ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ ! See the License for the specific language governing permissions and
+ ! limitations under the License.
+ !-->
+<test-suite
+    xmlns="urn:xml.testframework.asterix.ics.uci.edu"
+    ResultOffsetPath="results"
+    QueryOffsetPath="queries"
+    QueryFileExtension=".aql">
+    <test-group name="debug">
+
+    </test-group>
+</test-suite>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.1.ddl.aql
new file mode 100644
index 0000000..0584e28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.1.ddl.aql
@@ -0,0 +1,66 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineType as closed {
+  l_orderkey: int32,
+  l_partkey: int32,
+  l_suppkey: int32,
+  l_linenumber: int32,
+  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 type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create type CustomerType as closed {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+
+create external dataset Line(LineType)
+using localfs
+(("path"="nc1://data/big-object/lineitem.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Order(OrderType)
+using localfs
+(("path"="nc1://data/big-object/order.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Customer(CustomerType)
+using localfs
+(("path"="nc1://data/big-object/customer.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.2.update.aql
new file mode 100644
index 0000000..8c04dcc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.2.update.aql
@@ -0,0 +1,7 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql
new file mode 100644
index 0000000..95b3837
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql
@@ -0,0 +1,16 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+
+use dataverse test;
+
+for $i in dataset('Line')
+group by $comment := $i.l_comment, $id := $i.l_orderkey with $i
+order by $id
+return {
+    "id": $id,
+    "length": string-length($comment),
+    "comment": $comment}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.1.ddl.aql
new file mode 100644
index 0000000..0584e28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.1.ddl.aql
@@ -0,0 +1,66 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineType as closed {
+  l_orderkey: int32,
+  l_partkey: int32,
+  l_suppkey: int32,
+  l_linenumber: int32,
+  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 type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create type CustomerType as closed {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+
+create external dataset Line(LineType)
+using localfs
+(("path"="nc1://data/big-object/lineitem.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Order(OrderType)
+using localfs
+(("path"="nc1://data/big-object/order.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Customer(CustomerType)
+using localfs
+(("path"="nc1://data/big-object/customer.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.2.update.aql
new file mode 100644
index 0000000..8c04dcc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.2.update.aql
@@ -0,0 +1,7 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.3.query.aql
new file mode 100644
index 0000000..3fd4360
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby/big_object_groupby.3.query.aql
@@ -0,0 +1,13 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+
+use dataverse test;
+
+for $i in dataset('Line')
+order by $i.l_partkey, $i.l_shipdate
+group by $partkey := $i.l_partkey with $i
+return { "partkey": $partkey, "lines": $i}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.1.ddl.aql
new file mode 100644
index 0000000..0584e28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.1.ddl.aql
@@ -0,0 +1,66 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineType as closed {
+  l_orderkey: int32,
+  l_partkey: int32,
+  l_suppkey: int32,
+  l_linenumber: int32,
+  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 type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create type CustomerType as closed {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+
+create external dataset Line(LineType)
+using localfs
+(("path"="nc1://data/big-object/lineitem.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Order(OrderType)
+using localfs
+(("path"="nc1://data/big-object/order.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Customer(CustomerType)
+using localfs
+(("path"="nc1://data/big-object/customer.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.2.update.aql
new file mode 100644
index 0000000..8c04dcc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.2.update.aql
@@ -0,0 +1,7 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.3.query.aql
new file mode 100644
index 0000000..8b57b08
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_join/big_object_join.3.query.aql
@@ -0,0 +1,20 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+
+use dataverse test;
+
+for $c in dataset('Customer')
+for $o in dataset('Order')
+where $c.c_custkey = $o.o_custkey
+return {
+  "c_custkey": $c.c_custkey,
+  "o_orderkey": $o.o_orderkey,
+  "len_c_comment": string-length($c.c_comment),
+  "len_o_comment": string-length($o.o_comment),
+  "c_comment": $c.c_comment
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.1.ddl.aql
new file mode 100644
index 0000000..0584e28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.1.ddl.aql
@@ -0,0 +1,66 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineType as closed {
+  l_orderkey: int32,
+  l_partkey: int32,
+  l_suppkey: int32,
+  l_linenumber: int32,
+  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 type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create type CustomerType as closed {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+
+create external dataset Line(LineType)
+using localfs
+(("path"="nc1://data/big-object/lineitem.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Order(OrderType)
+using localfs
+(("path"="nc1://data/big-object/order.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Customer(CustomerType)
+using localfs
+(("path"="nc1://data/big-object/customer.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.2.update.aql
new file mode 100644
index 0000000..8c04dcc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.2.update.aql
@@ -0,0 +1,7 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.3.query.aql
new file mode 100644
index 0000000..c8107ff
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_sort/big_object_sort.3.query.aql
@@ -0,0 +1,18 @@
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+
+use dataverse test;
+
+for $o in dataset('Order')
+order by $o.o_custkey, $o.o_orderkey
+return {
+    "custkey": $o.o_custkey,
+    "orderkey": $o.o_orderkey,
+    "len-comment": string-length($o.o_comment),
+    "comment": $o.o_comment
+}
+


[05/11] incubator-asterixdb git commit: VariableSizeFrame(VSizeFrame) support for Asterix (Runtime Only)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/65100727/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby/groupby.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby/groupby.1.adm b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby/groupby.1.adm
new file mode 100644
index 0000000..5756361
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/big-object/big_object_groupby/groupby.1.adm
@@ -0,0 +1,201 @@
+[ { "partkey": 6i32, "lines": [ { "l_orderkey": 4483i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 1i32, "l_quantity": 32.0d, "l_extendedprice": 28992.0d, "l_discount": 0.07d, "l_tax": 0.07d, "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": 801i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 1i32, "l_quantity": 13.0d, "l_extendedprice": 11778.0d, "l_discount": 0.1d, "l_tax": 0.02d, "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": 2689i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 45.0d, "l_extendedprice": 40770.0d, "l_discount": 0.02d, "l
 _tax": 0.04d, "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": 3043i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 2i32, "l_quantity": 15.0d, "l_extendedprice": 13590.0d, "l_discount": 0.03d, "l_tax": 0.05d, "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": 2054i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 5i32, "l_quantity": 40.0d, "l_extendedprice": 36240.0d, "l_discount": 0.08d, "l_tax": 0.06d, "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": 4195i32, "l_partkey": 6i32, "l_suppkey": 9i32, "l_linenumber": 1i32, "l_quantity": 14.0d, "l_extendedprice": 12684.0d, "l_discount": 0.09d, "l_tax": 0.04d, "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": 4612i32, "l_partkey": 6i32, "l_suppkey": 9i32, "l_linenumber": 1i32, "l_quantity": 20.0d, "l_extendedprice": 18120.0d, "l_discount": 0.02d, "l_tax": 0.03d, "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": 1542i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 3i32, "l_quantity": 18.0d, "l_extendedprice": 16
 308.0d, "l_discount": 0.05d, "l_tax": 0.05d, "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": 2370i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 4i32, "l_quantity": 21.0d, "l_extendedprice": 19026.0d, "l_discount": 0.04d, "l_tax": 0.01d, "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": 5957i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 6i32, "l_quantity": 41.0d, "l_extendedprice": 37146.0d, "l_discount": 0.1d, "l_tax": 0.07d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-31", "l_shipinstruc
 t": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "es across the regular requests maint" }, { "l_orderkey": 290i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 35.0d, "l_extendedprice": 31710.0d, "l_discount": 0.01d, "l_tax": 0.02d, "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": 5760i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 2i32, "l_quantity": 24.0d, "l_extendedprice": 21744.0d, "l_discount": 0.04d, "l_tax": 0.05d, "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": 4870i32, "l_partkey": 6i32, "l_suppkey": 9i32, "l_linenumber": 4i32
 , "l_quantity": 4.0d, "l_extendedprice": 3624.0d, "l_discount": 0.03d, "l_tax": 0.08d, "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": 5223i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 3i32, "l_quantity": 19.0d, "l_extendedprice": 17214.0d, "l_discount": 0.04d, "l_tax": 0.01d, "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": 4485i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 5i32, "l_quantity": 47.0d, "l_extendedprice": 42582.0d, "l_discount": 0.08d, "l_tax": 0.04d, "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": 3329i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 2i32, "l_quantity": 9.0d, "l_extendedprice": 8154.0d, "l_discount": 0.0d, "l_tax": 0.02d, "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": 3137i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 2i32, "l_quantity": 4.0d, "l_extendedprice": 3624.0d, "l_discount": 0.06d, "l_tax": 0.04d, "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": 4387i32, "l_partkey": 6i32, "l_suppkey": 3i3
 2, "l_linenumber": 6i32, "l_quantity": 40.0d, "l_extendedprice": 36240.0d, "l_discount": 0.02d, "l_tax": 0.04d, "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": 4676i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 2i32, "l_quantity": 33.0d, "l_extendedprice": 29898.0d, "l_discount": 0.08d, "l_tax": 0.05d, "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": 2049i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 4i32, "l_quantity": 39.0d, "l_extendedprice": 35334.0d, "l_discount": 0.02d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_comm
 itdate": "1996-01-21", "l_receiptdate": "1996-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the even pinto beans " }, { "l_orderkey": 1284i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 2i32, "l_quantity": 4.0d, "l_extendedprice": 3624.0d, "l_discount": 0.07d, "l_tax": 0.06d, "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": 2276i32, "l_partkey": 6i32, "l_suppkey": 9i32, "l_linenumber": 6i32, "l_quantity": 4.0d, "l_extendedprice": 3624.0d, "l_discount": 0.1d, "l_tax": 0.03d, "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": 768i32, "l_partkey": 6i32, "l_suppkey": 
 1i32, "l_linenumber": 3i32, "l_quantity": 30.0d, "l_extendedprice": 27180.0d, "l_discount": 0.06d, "l_tax": 0.05d, "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": 1827i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 7i32, "l_quantity": 38.0d, "l_extendedprice": 34428.0d, "l_discount": 0.05d, "l_tax": 0.01d, "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": 2179i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 4i32, "l_quantity": 24.0d, "l_extendedprice": 21744.0d, "l_discount": 0.04d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-2
 6", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " cajole carefully. " }, { "l_orderkey": 4005i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 5i32, "l_quantity": 14.0d, "l_extendedprice": 12684.0d, "l_discount": 0.09d, "l_tax": 0.08d, "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": 1220i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 4i32, "l_quantity": 36.0d, "l_extendedprice": 32616.0d, "l_discount": 0.07d, "l_tax": 0.03d, "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": 3426
 i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 4i32, "l_quantity": 9.0d, "l_extendedprice": 8154.0d, "l_discount": 0.09d, "l_tax": 0.05d, "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": 260i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 4i32, "l_quantity": 29.0d, "l_extendedprice": 26274.0d, "l_discount": 0.1d, "l_tax": 0.06d, "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": 4036i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 46.0d, "l_extendedprice": 41676.0d, "l_discount": 0.09d, "l_tax": 0.0d, "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": 5124i32, "l_partkey": 6i32, "l_suppkey": 3i32, "l_linenumber": 2i32, "l_quantity": 41.0d, "l_extendedprice": 37146.0d, "l_discount": 0.02d, "l_tax": 0.06d, "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": 1638i32, "l_partkey": 6i32, "l_suppkey": 7i32, "l_linenumber": 1i32, "l_quantity": 46.0d, "l_extendedprice": 41676.0d, "l_discount": 0.03d, "l_tax": 0.02d, "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": 5125i32, "l_partkey": 6i32, "l_suppkey": 9i32, "l_linenumber": 1i32, "l_quantity": 38.0d, "l_extendedprice": 34428.0d, "l_discount": 0.09d, "l_tax": 0.05d, "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": 1124i32, "l_partkey": 6i32, "l_suppkey": 1i32, "l_linenumber": 2i32, "l_quantity": 13.0d, "l_extendedprice": 11778.0d, "l_discount": 0.05d, "l_tax": 0.04d, "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 " } ] }
+, { "partkey": 11i32, "lines": [ { "l_orderkey": 4800i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 3i32, "l_quantity": 21.0d, "l_extendedprice": 19131.21d, "l_discount": 0.09d, "l_tax": 0.05d, "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": 5858i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 7i32, "l_quantity": 50.0d, "l_extendedprice": 45550.5d, "l_discount": 0.06d, "l_tax": 0.0d, "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": 3237i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 1i32, "l_quantity": 11.0d, "l_extendedprice": 10021.11d, "l_discount": 0.02d, "l_tax": 0.07d, "l_returnfla
 g": "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": 2054i32, "l_partkey": 11i32, "l_suppkey": 1i32, "l_linenumber": 7i32, "l_quantity": 4.0d, "l_extendedprice": 3644.04d, "l_discount": 0.0d, "l_tax": 0.08d, "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": 2147i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 4i32, "l_quantity": 11.0d, "l_extendedprice": 10021.11d, "l_discount": 0.06d, "l_tax": 0.07d, "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": 4935i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 3i32, "l_quantity": 24.0d, "l_extendedprice": 21864.24d, "l_discount": 0.06d, "l_tax": 0.04d, "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": 1509i32, "l_partkey": 11i32, "l_suppkey": 2i32, "l_linenumber": 2i32, "l_quantity": 46.0d, "l_extendedprice": 41906.46d, "l_discount": 0.08d, "l_tax": 0.02d, "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": 773i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 2i32, "l_quantity": 31.0d, "l_extendedprice": 28241.31d, "l_discount": 0.02d, "l_tax": 0.06d, "l_return
 flag": "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": 2466i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 4i32, "l_quantity": 29.0d, "l_extendedprice": 26419.29d, "l_discount": 0.04d, "l_tax": 0.04d, "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": 5698i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 1i32, "l_quantity": 30.0d, "l_extendedprice": 27330.3d, "l_discount": 0.01d, "l_tax": 0.05d, "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": 4389i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 5i32, "l_quantity": 22.0d, "l_extendedprice": 20042.22d, "l_discount": 0.08d, "l_tax": 0.0d, "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": 998i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 4i32, "l_quantity": 6.0d, "l_extendedprice": 5466.06d, "l_discount": 0.09d, "l_tax": 0.05d, "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": 928i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 7i32, "l_quantity": 11.0d, "l_extendedprice": 10021.11d, "l_discount": 0.0
 d, "l_tax": 0.01d, "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": 3460i32, "l_partkey": 11i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 40.0d, "l_extendedprice": 36440.4d, "l_discount": 0.1d, "l_tax": 0.06d, "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": 2979i32, "l_partkey": 11i32, "l_suppkey": 2i32, "l_linenumber": 2i32, "l_quantity": 47.0d, "l_extendedprice": 42817.47d, "l_discount": 0.05d, "l_tax": 0.0d, "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": 4551i32, "l_partkey": 11i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 6.0d, "l_extendedprice": 5466.06d, "l_discount": 0.08d, "l_tax": 0.08d, "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": 103i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 2i32, "l_quantity": 37.0d, "l_extendedprice": 33707.37d, "l_discount": 0.02d, "l_tax": 0.07d, "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": 5573i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 3i32, "l_quantity": 46.0d, "l_extended
 price": 41906.46d, "l_discount": 0.06d, "l_tax": 0.01d, "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": 3872i32, "l_partkey": 11i32, "l_suppkey": 2i32, "l_linenumber": 4i32, "l_quantity": 41.0d, "l_extendedprice": 37351.41d, "l_discount": 0.07d, "l_tax": 0.03d, "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": 2469i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 3i32, "l_quantity": 48.0d, "l_extendedprice": 43728.48d, "l_discount": 0.05d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-03", "l_receiptdate": "1997-01-15", "l_ship
 instruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "riously even theodolites u" }, { "l_orderkey": 4934i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 7i32, "l_quantity": 2.0d, "l_extendedprice": 1822.02d, "l_discount": 0.1d, "l_tax": 0.06d, "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": 3584i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 1i32, "l_quantity": 4.0d, "l_extendedprice": 3644.04d, "l_discount": 0.04d, "l_tax": 0.08d, "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": 4866i32, "l_partkey": 11i32, "l_suppkey": 8i32, "l_linenumber": 1i32, "l_quantity": 9.
 0d, "l_extendedprice": 8199.09d, "l_discount": 0.01d, "l_tax": 0.05d, "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": 5187i32, "l_partkey": 11i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 49.0d, "l_extendedprice": 44639.49d, "l_discount": 0.04d, "l_tax": 0.06d, "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": 198i32, "l_partkey": 11i32, "l_suppkey": 5i32, "l_linenumber": 4i32, "l_quantity": 35.0d, "l_extendedprice": 31885.35d, "l_discount": 0.08d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "19
 98-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": 4192i32, "l_partkey": 11i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 36.0d, "l_extendedprice": 32796.36d, "l_discount": 0.06d, "l_tax": 0.08d, "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": 4263i32, "l_partkey": 11i32, "l_suppkey": 1i32, "l_linenumber": 3i32, "l_quantity": 38.0d, "l_extendedprice": 34618.38d, "l_discount": 0.01d, "l_tax": 0.01d, "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": 5633i32, "l_partkey": 11i32, "l_suppk
 ey": 5i32, "l_linenumber": 7i32, "l_quantity": 39.0d, "l_extendedprice": 35529.39d, "l_discount": 0.02d, "l_tax": 0.08d, "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" } ] }
+, { "partkey": 12i32, "lines": [ { "l_orderkey": 130i32, "l_partkey": 12i32, "l_suppkey": 3i32, "l_linenumber": 3i32, "l_quantity": 18.0d, "l_extendedprice": 16416.18d, "l_discount": 0.04d, "l_tax": 0.08d, "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": 322i32, "l_partkey": 12i32, "l_suppkey": 2i32, "l_linenumber": 5i32, "l_quantity": 35.0d, "l_extendedprice": 31920.35d, "l_discount": 0.07d, "l_tax": 0.06d, "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": 2497i32, "l_partkey": 12i32, "l_suppkey": 2i32, "l_linenumber": 1i32, "l_quantity": 34.0d, "l_extendedprice": 31008.
 34d, "l_discount": 0.02d, "l_tax": 0.03d, "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": 1345i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 2i32, "l_quantity": 37.0d, "l_extendedprice": 33744.37d, "l_discount": 0.1d, "l_tax": 0.07d, "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": 4771i32, "l_partkey": 12i32, "l_suppkey": 3i32, "l_linenumber": 3i32, "l_quantity": 5.0d, "l_extendedprice": 4560.05d, "l_discount": 0.06d, "l_tax": 0.08d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1993-01-19", "l_receiptdate": "1993-01-26", "l_shipinst
 ruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar, quiet accounts nag furiously express id" }, { "l_orderkey": 3204i32, "l_partkey": 12i32, "l_suppkey": 2i32, "l_linenumber": 1i32, "l_quantity": 10.0d, "l_extendedprice": 9120.1d, "l_discount": 0.1d, "l_tax": 0.07d, "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": 2435i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 3i32, "l_quantity": 24.0d, "l_extendedprice": 21888.24d, "l_discount": 0.07d, "l_tax": 0.08d, "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": 2274i32, "l_partkey": 12i32, "l_suppkey": 6i32, "l_linenumber": 1i32, "l_quantity": 18
 .0d, "l_extendedprice": 16416.18d, "l_discount": 0.04d, "l_tax": 0.03d, "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": 807i32, "l_partkey": 12i32, "l_suppkey": 2i32, "l_linenumber": 6i32, "l_quantity": 11.0d, "l_extendedprice": 10032.11d, "l_discount": 0.02d, "l_tax": 0.04d, "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": 5729i32, "l_partkey": 12i32, "l_suppkey": 3i32, "l_linenumber": 3i32, "l_quantity": 50.0d, "l_extendedprice": 45600.5d, "l_discount": 0.0d, "l_tax": 0.05d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1994-12-31", "l_receiptdate": "1994-12-24", "l_sh
 ipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly special sentiments. car" }, { "l_orderkey": 5920i32, "l_partkey": 12i32, "l_suppkey": 2i32, "l_linenumber": 4i32, "l_quantity": 28.0d, "l_extendedprice": 25536.28d, "l_discount": 0.06d, "l_tax": 0.02d, "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": 359i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 2i32, "l_quantity": 18.0d, "l_extendedprice": 16416.18d, "l_discount": 0.0d, "l_tax": 0.03d, "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": 5636i32, "l_partkey": 12i32, "l_suppkey": 3i32, "l_linenumber": 6
 i32, "l_quantity": 33.0d, "l_extendedprice": 30096.33d, "l_discount": 0.06d, "l_tax": 0.04d, "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": 928i32, "l_partkey": 12i32, "l_suppkey": 3i32, "l_linenumber": 5i32, "l_quantity": 38.0d, "l_extendedprice": 34656.38d, "l_discount": 0.02d, "l_tax": 0.08d, "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": 2311i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 6i32, "l_quantity": 32.0d, "l_extendedprice": 29184.32d, "l_discount": 0.01d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-19", "l_commitdate": "1995-06-26", "l_receiptda
 te": "1995-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "sts along the slyly" }, { "l_orderkey": 32i32, "l_partkey": 12i32, "l_suppkey": 6i32, "l_linenumber": 6i32, "l_quantity": 6.0d, "l_extendedprice": 5472.06d, "l_discount": 0.04d, "l_tax": 0.03d, "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": 4038i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 2i32, "l_quantity": 37.0d, "l_extendedprice": 33744.37d, "l_discount": 0.04d, "l_tax": 0.03d, "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": 3715i32, "l_partkey": 12i32, "l_suppkey": 3i32, "l_linenumber": 3i32, "l_quanti
 ty": 37.0d, "l_extendedprice": 33744.37d, "l_discount": 0.05d, "l_tax": 0.02d, "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": 1797i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 3i32, "l_quantity": 21.0d, "l_extendedprice": 19152.21d, "l_discount": 0.02d, "l_tax": 0.01d, "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": 1123i32, "l_partkey": 12i32, "l_suppkey": 2i32, "l_linenumber": 1i32, "l_quantity": 10.0d, "l_extendedprice": 9120.1d, "l_discount": 0.05d, "l_tax": 0.08d, "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": 1763i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 1i32, "l_quantity": 22.0d, "l_extendedprice": 20064.22d, "l_discount": 0.09d, "l_tax": 0.06d, "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": 1156i32, "l_partkey": 12i32, "l_suppkey": 2i32, "l_linenumber": 3i32, "l_quantity": 29.0d, "l_extendedprice": 26448.29d, "l_discount": 0.09d, "l_tax": 0.06d, "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": 5762i32, "l_partkey": 12i32, "l_suppkey": 6i32, "l
 _linenumber": 6i32, "l_quantity": 12.0d, "l_extendedprice": 10944.12d, "l_discount": 0.0d, "l_tax": 0.06d, "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": 3239i32, "l_partkey": 12i32, "l_suppkey": 9i32, "l_linenumber": 5i32, "l_quantity": 31.0d, "l_extendedprice": 28272.31d, "l_discount": 0.1d, "l_tax": 0.08d, "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" } ] }
+, { "partkey": 14i32, "lines": [ { "l_orderkey": 5028i32, "l_partkey": 14i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 15.0d, "l_extendedprice": 13710.15d, "l_discount": 0.07d, "l_tax": 0.07d, "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": 3232i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 1i32, "l_quantity": 22.0d, "l_extendedprice": 20108.22d, "l_discount": 0.1d, "l_tax": 0.01d, "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": 2279i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 1i32, "l_quantity": 12.0d, "l_extendedprice": 10968.12d, "l_discount": 0.07d,
  "l_tax": 0.08d, "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": 5792i32, "l_partkey": 14i32, "l_suppkey": 8i32, "l_linenumber": 4i32, "l_quantity": 14.0d, "l_extendedprice": 12796.14d, "l_discount": 0.09d, "l_tax": 0.02d, "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": 2020i32, "l_partkey": 14i32, "l_suppkey": 4i32, "l_linenumber": 3i32, "l_quantity": 30.0d, "l_extendedprice": 27420.3d, "l_discount": 0.07d, "l_tax": 0.04d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-08", "l_commitdate": "1993-08-11", "l_receiptdate": "1993-09-29", "l_shipinstruct": "TAKE BAC
 K RETURN", "l_shipmode": "AIR", "l_comment": "ly about the blithely ironic foxes. bold" }, { "l_orderkey": 2368i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 2i32, "l_quantity": 32.0d, "l_extendedprice": 29248.32d, "l_discount": 0.03d, "l_tax": 0.0d, "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": 1953i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 2i32, "l_quantity": 35.0d, "l_extendedprice": 31990.35d, "l_discount": 0.06d, "l_tax": 0.06d, "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": 2466i32, "l_partkey": 14i32, "l_suppkey": 1i32, "l_linenumber": 3i32, "l_quant
 ity": 29.0d, "l_extendedprice": 26506.29d, "l_discount": 0.1d, "l_tax": 0.07d, "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": 3751i32, "l_partkey": 14i32, "l_suppkey": 4i32, "l_linenumber": 4i32, "l_quantity": 39.0d, "l_extendedprice": 35646.39d, "l_discount": 0.07d, "l_tax": 0.01d, "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": 3042i32, "l_partkey": 14i32, "l_suppkey": 8i32, "l_linenumber": 3i32, "l_quantity": 34.0d, "l_extendedprice": 31076.34d, "l_discount": 0.04d, "l_tax": 0.0d, "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": 3399i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 4i32, "l_quantity": 21.0d, "l_extendedprice": 19194.21d, "l_discount": 0.09d, "l_tax": 0.06d, "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": 5253i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 3i32, "l_quantity": 9.0d, "l_extendedprice": 8226.09d, "l_discount": 0.03d, "l_tax": 0.08d, "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":
  4929i32, "l_partkey": 14i32, "l_suppkey": 1i32, "l_linenumber": 1i32, "l_quantity": 20.0d, "l_extendedprice": 18280.2d, "l_discount": 0.0d, "l_tax": 0.04d, "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": 4548i32, "l_partkey": 14i32, "l_suppkey": 8i32, "l_linenumber": 1i32, "l_quantity": 21.0d, "l_extendedprice": 19194.21d, "l_discount": 0.1d, "l_tax": 0.05d, "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": 3425i32, "l_partkey": 14i32, "l_suppkey": 4i32, "l_linenumber": 3i32, "l_quantity": 8.0d, "l_extendedprice": 7312.08d, "l_discount": 0.06d, "l_tax": 0.08d, "l_returnflag": "N", "l_linesta
 tus": "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": 3781i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 1i32, "l_quantity": 48.0d, "l_extendedprice": 43872.48d, "l_discount": 0.02d, "l_tax": 0.06d, "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": 1510i32, "l_partkey": 14i32, "l_suppkey": 5i32, "l_linenumber": 6i32, "l_quantity": 3.0d, "l_extendedprice": 2742.03d, "l_discount": 0.05d, "l_tax": 0.02d, "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 sl
 yly regular pin" }, { "l_orderkey": 3426i32, "l_partkey": 14i32, "l_suppkey": 4i32, "l_linenumber": 2i32, "l_quantity": 19.0d, "l_extendedprice": 17366.19d, "l_discount": 0.1d, "l_tax": 0.08d, "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": 3047i32, "l_partkey": 14i32, "l_suppkey": 1i32, "l_linenumber": 2i32, "l_quantity": 23.0d, "l_extendedprice": 21022.23d, "l_discount": 0.0d, "l_tax": 0.04d, "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": 453i32, "l_partkey": 14i32, "l_suppkey": 1i32, "l_linenumber": 3i32, "l_quantity": 38.0d, "l_extendedprice": 34732.38d, "l_discount": 0.1d, "l_tax": 0.01
 d, "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": 5731i32, "l_partkey": 14i32, "l_suppkey": 1i32, "l_linenumber": 4i32, "l_quantity": 6.0d, "l_extendedprice": 5484.06d, "l_discount": 0.03d, "l_tax": 0.06d, "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": 1858i32, "l_partkey": 14i32, "l_suppkey": 8i32, "l_linenumber": 1i32, "l_quantity": 33.0d, "l_extendedprice": 30162.33d, "l_discount": 0.01d, "l_tax": 0.02d, "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": 4322i32, "l_partkey": 14i32, "l_suppkey": 4i32, "l_linenumber": 7i32, "l_quantity": 34.0d, "l_extendedprice": 31076.34d, "l_discount": 0.05d, "l_tax": 0.0d, "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": 899i32, "l_partkey": 14i32, "l_suppkey": 1i32, "l_linenumber": 7i32, "l_quantity": 11.0d, "l_extendedprice": 10054.11d, "l_discount": 0.02d, "l_tax": 0.08d, "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": 2981i32, "l_partkey": 14i32, "l_suppkey": 4i32, "l_linenumber": 1i32, "l_quantity": 17.0d, "l_extendedprice": 15538.17d, "l_discount": 0.03d, "l_tax"
 : 0.05d, "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" } ] }
+, { "partkey": 21i32, "lines": [ { "l_orderkey": 549i32, "l_partkey": 21i32, "l_suppkey": 4i32, "l_linenumber": 4i32, "l_quantity": 18.0d, "l_extendedprice": 16578.36d, "l_discount": 0.09d, "l_tax": 0.01d, "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": 4581i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 3i32, "l_quantity": 46.0d, "l_extendedprice": 42366.92d, "l_discount": 0.04d, "l_tax": 0.04d, "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": 481i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 2i32, "l_quantity": 19.0d, "l_extendedprice": 17499.38d, "l_discou
 nt": 0.08d, "l_tax": 0.01d, "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": 3331i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 2i32, "l_quantity": 38.0d, "l_extendedprice": 34998.76d, "l_discount": 0.06d, "l_tax": 0.04d, "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": 1604i32, "l_partkey": 21i32, "l_suppkey": 4i32, "l_linenumber": 5i32, "l_quantity": 23.0d, "l_extendedprice": 21183.46d, "l_discount": 0.08d, "l_tax": 0.05d, "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": 1921i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 1i32, "l_quantity": 9.0d, "l_extendedprice": 8289.18d, "l_discount": 0.08d, "l_tax": 0.0d, "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": 2530i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 1i32, "l_quantity": 9.0d, "l_extendedprice": 8289.18d, "l_discount": 0.09d, "l_tax": 0.03d, "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": 2949i32, "l_partkey": 21i32, "l_suppkey": 6i32, "l_linenumber": 1i32, "l_quantity": 4.0d, "l_ext
 endedprice": 3684.08d, "l_discount": 0.06d, "l_tax": 0.06d, "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": 1985i32, "l_partkey": 21i32, "l_suppkey": 6i32, "l_linenumber": 2i32, "l_quantity": 50.0d, "l_extendedprice": 46051.0d, "l_discount": 0.04d, "l_tax": 0.02d, "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": 1287i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 6i32, "l_quantity": 26.0d, "l_extendedprice": 23946.52d, "l_discount": 0.03d, "l_tax": 0.08d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-09-27", "l_re
 ceiptdate": "1994-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egular foxes. theodolites nag along t" }, { "l_orderkey": 3841i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 2i32, "l_quantity": 31.0d, "l_extendedprice": 28551.62d, "l_discount": 0.09d, "l_tax": 0.03d, "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": 4642i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 3i32, "l_quantity": 10.0d, "l_extendedprice": 9210.2d, "l_discount": 0.04d, "l_tax": 0.02d, "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": 43
 27i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 4i32, "l_quantity": 8.0d, "l_extendedprice": 7368.16d, "l_discount": 0.04d, "l_tax": 0.08d, "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": 1827i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 6i32, "l_quantity": 7.0d, "l_extendedprice": 6447.14d, "l_discount": 0.1d, "l_tax": 0.02d, "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": 5573i32, "l_partkey": 21i32, "l_suppkey": 6i32, "l_linenumber": 1i32, "l_quantity": 32.0d, "l_extendedprice": 29472.64d, "l_discount": 0.05d, "l_tax": 0.07d, "l_returnflag": "N", "l_linesta
 tus": "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": 39i32, "l_partkey": 21i32, "l_suppkey": 6i32, "l_linenumber": 4i32, "l_quantity": 32.0d, "l_extendedprice": 29472.64d, "l_discount": 0.07d, "l_tax": 0.05d, "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": 3718i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 1i32, "l_quantity": 40.0d, "l_extendedprice": 36840.8d, "l_discount": 0.01d, "l_tax": 0.04d, "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": 67i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 2i32, "l_quantity": 12.0d, "l_extendedprice": 11052.24d, "l_discount": 0.09d, "l_tax": 0.05d, "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": 160i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 3i32, "l_quantity": 34.0d, "l_extendedprice": 31314.68d, "l_discount": 0.01d, "l_tax": 0.05d, "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": 4064i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 5i32, "l_quantity": 12.0d, "l_extendedprice": 11052.24d, "l_discount": 0.08d, "l_
 tax": 0.08d, "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": 1698i32, "l_partkey": 21i32, "l_suppkey": 6i32, "l_linenumber": 3i32, "l_quantity": 22.0d, "l_extendedprice": 20262.44d, "l_discount": 0.03d, "l_tax": 0.04d, "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": 2468i32, "l_partkey": 21i32, "l_suppkey": 10i32, "l_linenumber": 2i32, "l_quantity": 43.0d, "l_extendedprice": 39603.86d, "l_discount": 0.0d, "l_tax": 0.04d, "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": 1667i32, "l_partkey": 21i32, "l_suppkey": 4i32, "l_linenumber": 1i32, "l_quantity": 6.0d, "l_extendedprice": 5526.12d, "l_discount": 0.04d, "l_tax": 0.02d, "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": 2917i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 2i32, "l_quantity": 20.0d, "l_extendedprice": 18420.4d, "l_discount": 0.06d, "l_tax": 0.03d, "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": 3303i32, "l_partkey": 21i32, "l_suppkey": 2i32, "l_linenumber": 2i32, "l_quantity": 15.0d, "l_extendedprice": 13815.3d, "l_di
 scount": 0.04d, "l_tax": 0.06d, "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": 1024i32, "l_partkey": 21i32, "l_suppkey": 4i32, "l_linenumber": 5i32, "l_quantity": 49.0d, "l_extendedprice": 45129.98d, "l_discount": 0.02d, "l_tax": 0.04d, "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 " } ] }
+, { "partkey": 23i32, "lines": [ { "l_orderkey": 2786i32, "l_partkey": 23i32, "l_suppkey": 4i32, "l_linenumber": 4i32, "l_quantity": 24.0d, "l_extendedprice": 22152.48d, "l_discount": 0.05d, "l_tax": 0.02d, "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": 1856i32, "l_partkey": 23i32, "l_suppkey": 6i32, "l_linenumber": 6i32, "l_quantity": 36.0d, "l_extendedprice": 33228.72d, "l_discount": 0.03d, "l_tax": 0.05d, "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": 1282i32, "l_partkey": 23i32, "l_suppkey": 4i32, "l_linenumber": 1i32, "l_quantity": 14.0d, "l_ext
 endedprice": 12922.28d, "l_discount": 0.04d, "l_tax": 0.02d, "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": 37i32, "l_partkey": 23i32, "l_suppkey": 8i32, "l_linenumber": 1i32, "l_quantity": 40.0d, "l_extendedprice": 36920.8d, "l_discount": 0.09d, "l_tax": 0.03d, "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": 2209i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 1i32, "l_quantity": 40.0d, "l_extendedprice": 36920.8d, "l_discount": 0.05d, "l_tax": 0.01d, "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": 2566i32, "l_partkey": 23i32, "l_suppkey": 8i32, "l_linenumber": 3i32, "l_quantity": 18.0d, "l_extendedprice": 16614.36d, "l_discount": 0.09d, "l_tax": 0.02d, "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": 4896i32, "l_partkey": 23i32, "l_suppkey": 4i32, "l_linenumber": 4i32, "l_quantity": 5.0d, "l_extendedprice": 4615.1d, "l_discount": 0.08d, "l_tax": 0.02d, "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": 1412i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 3i3
 2, "l_quantity": 2.0d, "l_extendedprice": 1846.04d, "l_discount": 0.1d, "l_tax": 0.07d, "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": 1382i32, "l_partkey": 23i32, "l_suppkey": 4i32, "l_linenumber": 7i32, "l_quantity": 5.0d, "l_extendedprice": 4615.1d, "l_discount": 0.07d, "l_tax": 0.01d, "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": 2759i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 4i32, "l_quantity": 31.0d, "l_extendedprice": 28613.62d, "l_discount": 0.02d, "l_tax": 0.05d, "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": 2496i32, "l_partkey": 23i32, "l_suppkey": 4i32, "l_linenumber": 2i32, "l_quantity": 39.0d, "l_extendedprice": 35997.78d, "l_discount": 0.03d, "l_tax": 0.0d, "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": 5347i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 3i32, "l_quantity": 34.0d, "l_extendedprice": 31382.68d, "l_discount": 0.06d, "l_tax": 0.0d, "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": 1025i3
 2, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 3i32, "l_quantity": 25.0d, "l_extendedprice": 23075.5d, "l_discount": 0.06d, "l_tax": 0.05d, "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": 1767i32, "l_partkey": 23i32, "l_suppkey": 8i32, "l_linenumber": 4i32, "l_quantity": 50.0d, "l_extendedprice": 46151.0d, "l_discount": 0.01d, "l_tax": 0.02d, "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": 512i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 2i32, "l_quantity": 37.0d, "l_extendedprice": 34151.74d, "l_discount": 0.01d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_ship
 date": "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": 4640i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 4i32, "l_quantity": 36.0d, "l_extendedprice": 33228.72d, "l_discount": 0.06d, "l_tax": 0.08d, "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": 2598i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 4i32, "l_quantity": 19.0d, "l_extendedprice": 17537.38d, "l_discount": 0.02d, "l_tax": 0.0d, "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. ev
 en accounts" }, { "l_orderkey": 5987i32, "l_partkey": 23i32, "l_suppkey": 2i32, "l_linenumber": 1i32, "l_quantity": 1.0d, "l_extendedprice": 923.02d, "l_discount": 0.01d, "l_tax": 0.04d, "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 agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggl
 e furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously a
 grefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully fina
 l excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses hag
 gle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously
  agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully fi
 nal excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses h
 aggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furious
 ly agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully 
 final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses
  haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furio
 usly agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefull
 y final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excus
 es haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle fur
 iously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefu
 lly final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final exc
 uses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle f
 uriously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agre
 fully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final e
 xcuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle
  furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously ag
 refully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final
  excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses hagg
 le furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously 
 agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully fin
 al excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses ha
 ggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furiously agrefully final excuses haggle furi

<TRUNCATED>