Initial Algebricks integration
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_algebricks_integration@854 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/pom.xml b/hyracks-algebricks/hyracks-algebricks-compiler/pom.xml
new file mode 100644
index 0000000..ca8c534
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/pom.xml
@@ -0,0 +1,36 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-algebricks-compiler</artifactId>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks-rewriter</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks-core</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
new file mode 100644
index 0000000..276398d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -0,0 +1,219 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.compiler.api;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.INormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.ITypeTraitProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+
+public abstract class AbstractCompilerFactoryBuilder {
+
+ protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
+ protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+ protected ITypeTraitProvider typeTraitProvider;
+ protected ISerializerDeserializerProvider serializerDeserializerProvider;
+ protected IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
+ protected IBinaryComparatorFactoryProvider comparatorFactoryProvider;
+ protected IBinaryBooleanInspector binaryBooleanInspector;
+ protected IBinaryIntegerInspector binaryIntegerInspector;
+ protected IPrinterFactoryProvider printerProvider;
+ protected ILogicalExpressionJobGen exprJobGen;
+ protected IExpressionTypeComputer expressionTypeComputer;
+ protected INullableTypeComputer nullableTypeComputer;
+ protected IExpressionEvalSizeComputer expressionEvalSizeComputer;
+ protected INullWriterFactory nullWriterFactory;
+ protected INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
+ protected IPartialAggregationTypeComputer partialAggregationTypeComputer;
+ protected IMergeAggregationExpressionFactory mergeAggregationExpressionFactory;
+ protected PhysicalOptimizationConfig physicalOptimizationConfig = new PhysicalOptimizationConfig();
+ protected AlgebricksPartitionConstraint clusterLocations;
+ protected int frameSize = -1;
+
+ public abstract ICompilerFactory create();
+
+ public void setLogicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites) {
+ this.logicalRewrites = logicalRewrites;
+ }
+
+ public void setPhysicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites) {
+ this.physicalRewrites = physicalRewrites;
+ }
+
+ public void setTypeTraitProvider(ITypeTraitProvider typeTraitProvider) {
+ this.typeTraitProvider = typeTraitProvider;
+ }
+
+ public ITypeTraitProvider getTypeTraitProvider() {
+ return typeTraitProvider;
+ }
+
+ public void setSerializerDeserializerProvider(ISerializerDeserializerProvider serializerDeserializerProvider) {
+ this.serializerDeserializerProvider = serializerDeserializerProvider;
+ }
+
+ public ISerializerDeserializerProvider getSerializerDeserializerProvider() {
+ return serializerDeserializerProvider;
+ }
+
+ public void setHashFunctionFactoryProvider(IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider) {
+ this.hashFunctionFactoryProvider = hashFunctionFactoryProvider;
+ }
+
+ public IBinaryHashFunctionFactoryProvider getHashFunctionFactoryProvider() {
+ return hashFunctionFactoryProvider;
+ }
+
+ public void setComparatorFactoryProvider(IBinaryComparatorFactoryProvider comparatorFactoryProvider) {
+ this.comparatorFactoryProvider = comparatorFactoryProvider;
+ }
+
+ public IBinaryComparatorFactoryProvider getComparatorFactoryProvider() {
+ return comparatorFactoryProvider;
+ }
+
+ public void setBinaryBooleanInspector(IBinaryBooleanInspector binaryBooleanInspector) {
+ this.binaryBooleanInspector = binaryBooleanInspector;
+ }
+
+ public IBinaryBooleanInspector getBinaryBooleanInspector() {
+ return binaryBooleanInspector;
+ }
+
+ public void setBinaryIntegerInspector(IBinaryIntegerInspector binaryIntegerInspector) {
+ this.binaryIntegerInspector = binaryIntegerInspector;
+ }
+
+ public IBinaryIntegerInspector getBinaryIntegerInspector() {
+ return binaryIntegerInspector;
+ }
+
+ public void setPrinterProvider(IPrinterFactoryProvider printerProvider) {
+ this.printerProvider = printerProvider;
+ }
+
+ public IPrinterFactoryProvider getPrinterProvider() {
+ return printerProvider;
+ }
+
+ public void setExprJobGen(ILogicalExpressionJobGen exprJobGen) {
+ this.exprJobGen = exprJobGen;
+ }
+
+ public ILogicalExpressionJobGen getExprJobGen() {
+ return exprJobGen;
+ }
+
+ public void setExpressionTypeComputer(IExpressionTypeComputer expressionTypeComputer) {
+ this.expressionTypeComputer = expressionTypeComputer;
+ }
+
+ public IExpressionTypeComputer getExpressionTypeComputer() {
+ return expressionTypeComputer;
+ }
+
+ public void setClusterLocations(AlgebricksPartitionConstraint clusterLocations) {
+ this.clusterLocations = clusterLocations;
+ }
+
+ public AlgebricksPartitionConstraint getClusterLocations() {
+ return clusterLocations;
+ }
+
+ public void setNullWriterFactory(INullWriterFactory nullWriterFactory) {
+ this.nullWriterFactory = nullWriterFactory;
+ }
+
+ public INullWriterFactory getNullWriterFactory() {
+ return nullWriterFactory;
+ }
+
+ public void setExpressionEvalSizeComputer(IExpressionEvalSizeComputer expressionEvalSizeComputer) {
+ this.expressionEvalSizeComputer = expressionEvalSizeComputer;
+ }
+
+ public IExpressionEvalSizeComputer getExpressionEvalSizeComputer() {
+ return expressionEvalSizeComputer;
+ }
+
+ public void setNormalizedKeyComputerFactoryProvider(
+ INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider) {
+ this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
+ }
+
+ public INormalizedKeyComputerFactoryProvider getNormalizedKeyComputerFactoryProvider() {
+ return normalizedKeyComputerFactoryProvider;
+ }
+
+ public void setFrameSize(int frameSize) {
+ this.frameSize = frameSize;
+ }
+
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ public IPartialAggregationTypeComputer getPartialAggregationTypeComputer() {
+ return partialAggregationTypeComputer;
+ }
+
+ public void setPartialAggregationTypeComputer(IPartialAggregationTypeComputer partialAggregationTypeComputer) {
+ this.partialAggregationTypeComputer = partialAggregationTypeComputer;
+ }
+
+ public IMergeAggregationExpressionFactory getIMergeAggregationExpressionFactory() {
+ return mergeAggregationExpressionFactory;
+ }
+
+ public void setIMergeAggregationExpressionFactory(
+ IMergeAggregationExpressionFactory mergeAggregationExpressionFactory) {
+ this.mergeAggregationExpressionFactory = mergeAggregationExpressionFactory;
+ }
+
+ public PhysicalOptimizationConfig getPhysicalOptimizationConfig() {
+ return physicalOptimizationConfig;
+ }
+
+ public void setPhysicalOptimizationConfig(PhysicalOptimizationConfig physicalOptimizationConfig) {
+ this.physicalOptimizationConfig = physicalOptimizationConfig;
+ }
+
+ public void setNullableTypeComputer(INullableTypeComputer nullableTypeComputer) {
+ this.nullableTypeComputer = nullableTypeComputer;
+ }
+
+ public INullableTypeComputer getNullableTypeComputer() {
+ return nullableTypeComputer;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
new file mode 100644
index 0000000..9245772
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.compiler.api;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.PlanCompiler;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class HeuristicCompilerFactoryBuilder extends AbstractCompilerFactoryBuilder {
+
+ public static class DefaultOptimizationContextFactory implements IOptimizationContextFactory {
+
+ public static final DefaultOptimizationContextFactory INSTANCE = new DefaultOptimizationContextFactory();
+
+ private DefaultOptimizationContextFactory() {
+ }
+
+ @Override
+ public IOptimizationContext createOptimizationContext(int varCounter, int frameSize,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
+ IExpressionTypeComputer expressionTypeComputer, INullableTypeComputer nullableTypeComputer,
+ PhysicalOptimizationConfig physicalOptimizationConfig) {
+ return new AlgebricksOptimizationContext(varCounter, frameSize, expressionEvalSizeComputer,
+ mergeAggregationExpressionFactory, expressionTypeComputer, nullableTypeComputer,
+ physicalOptimizationConfig);
+ }
+ }
+
+ private IOptimizationContextFactory optCtxFactory;
+
+ public HeuristicCompilerFactoryBuilder() {
+ this.optCtxFactory = DefaultOptimizationContextFactory.INSTANCE;
+ }
+
+ public HeuristicCompilerFactoryBuilder(IOptimizationContextFactory optCtxFactory) {
+ this.optCtxFactory = optCtxFactory;
+ }
+
+ @Override
+ public ICompilerFactory create() {
+ return new ICompilerFactory() {
+ @Override
+ public ICompiler createCompiler(final ILogicalPlan plan, final IMetadataProvider<?, ?> metadata,
+ int varCounter) {
+ final IOptimizationContext oc = optCtxFactory.createOptimizationContext(varCounter, frameSize,
+ expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
+ nullableTypeComputer, physicalOptimizationConfig);
+ oc.setMetadataDeclarations(metadata);
+ final HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
+ return new ICompiler() {
+
+ @Override
+ public void optimize() throws AlgebricksException {
+ opt.optimize();
+ }
+
+ @Override
+ public JobSpecification createJob(Object appContext) throws AlgebricksException {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Starting Job Generation.\n");
+ JobGenContext context = new JobGenContext(null, metadata, appContext,
+ serializerDeserializerProvider, hashFunctionFactoryProvider, comparatorFactoryProvider,
+ typeTraitProvider, binaryBooleanInspector, binaryIntegerInspector, printerProvider,
+ nullWriterFactory, normalizedKeyComputerFactoryProvider, exprJobGen,
+ expressionTypeComputer, nullableTypeComputer, oc, expressionEvalSizeComputer,
+ partialAggregationTypeComputer, frameSize, clusterLocations);
+ PlanCompiler pc = new PlanCompiler(context);
+ return pc.compilePlan(plan, null);
+ }
+ };
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompiler.java b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompiler.java
new file mode 100644
index 0000000..654337a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompiler.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.compiler.api;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public interface ICompiler {
+ public void optimize() throws AlgebricksException;
+
+ public JobSpecification createJob(Object appContext) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompilerFactory.java b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompilerFactory.java
new file mode 100644
index 0000000..f831e8f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompilerFactory.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.compiler.api;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public interface ICompilerFactory {
+ ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/PrioritizedRuleController.java b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/PrioritizedRuleController.java
new file mode 100644
index 0000000..570bb2c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/PrioritizedRuleController.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ *
+ * Runs each rule until it produces no changes. Then the whole collection of
+ * rules is run again until no change is made.
+ *
+ *
+ * @author Nicola
+ *
+ */
+
+public class PrioritizedRuleController extends AbstractRuleController {
+
+ public PrioritizedRuleController() {
+ super();
+ }
+
+ @Override
+ public boolean rewriteWithRuleCollection(LogicalOperatorReference root, Collection<IAlgebraicRewriteRule> rules)
+ throws AlgebricksException {
+ boolean anyRuleFired = false;
+ boolean anyChange = false;
+ do {
+ anyChange = false;
+ for (IAlgebraicRewriteRule r : rules) {
+ while (true) {
+ boolean ruleFired = rewriteOperatorRef(root, r);
+ if (ruleFired) {
+ anyChange = true;
+ anyRuleFired = true;
+ } else {
+ break; // go to next rule
+ }
+ }
+ }
+ } while (anyChange);
+ return anyRuleFired;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFixpointRuleController.java b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFixpointRuleController.java
new file mode 100644
index 0000000..4658041
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFixpointRuleController.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ *
+ * Runs rules sequentially (round-robin), until one iteration over all rules
+ * produces no change.
+ *
+ *
+ * @author Nicola
+ *
+ */
+public class SequentialFixpointRuleController extends AbstractRuleController {
+
+ private boolean fullDfs;
+
+ public SequentialFixpointRuleController(boolean fullDfs) {
+ super();
+ this.fullDfs = fullDfs;
+ }
+
+ @Override
+ public boolean rewriteWithRuleCollection(LogicalOperatorReference root,
+ Collection<IAlgebraicRewriteRule> ruleCollection) throws AlgebricksException {
+ boolean anyRuleFired = false;
+ boolean anyChange = false;
+ do {
+ anyChange = false;
+ for (IAlgebraicRewriteRule rule : ruleCollection) {
+ boolean ruleFired = rewriteOperatorRef(root, rule, true, fullDfs);
+ if (ruleFired) {
+ anyChange = true;
+ anyRuleFired = true;
+ }
+ }
+ } while (anyChange);
+ return anyRuleFired;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialOnceRuleController.java b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialOnceRuleController.java
new file mode 100644
index 0000000..69281a1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialOnceRuleController.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class SequentialOnceRuleController extends AbstractRuleController {
+
+ private final boolean enterNestedPlans;
+
+ public SequentialOnceRuleController(boolean enterNestedPlans) {
+ super();
+ this.enterNestedPlans = enterNestedPlans;
+ }
+
+ @Override
+ public boolean rewriteWithRuleCollection(LogicalOperatorReference root, Collection<IAlgebraicRewriteRule> rules)
+ throws AlgebricksException {
+ boolean fired = false;
+ for (IAlgebraicRewriteRule rule : rules) {
+ if (rewriteOperatorRef(root, rule, enterNestedPlans, true)) {
+ fired = true;
+ }
+ }
+ return fired;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/pom.xml b/hyracks-algebricks/hyracks-algebricks-core/pom.xml
new file mode 100644
index 0000000..b6132bd
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/pom.xml
@@ -0,0 +1,41 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-algebricks-core</artifactId>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-btree</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-rtree</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-std</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/Counter.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/Counter.java
new file mode 100644
index 0000000..b27bf84
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/Counter.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+public class Counter {
+ private int counter = 0;
+
+ public Counter() {
+ }
+
+ public Counter(int initial) {
+ counter = initial;
+ }
+
+ public int get() {
+ return counter;
+ }
+
+ public void inc() {
+ ++counter;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/EquivalenceClass.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/EquivalenceClass.java
new file mode 100644
index 0000000..fe3511e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/EquivalenceClass.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public final class EquivalenceClass {
+ private List<LogicalVariable> members;
+ private ILogicalExpression constRepresentative;
+ private LogicalVariable variableRepresentative;
+ private boolean representativeIsConst;
+
+ public EquivalenceClass(List<LogicalVariable> members, ILogicalExpression constRepresentative) {
+ this.members = members;
+ this.constRepresentative = constRepresentative;
+ representativeIsConst = true;
+ }
+
+ public EquivalenceClass(List<LogicalVariable> members, LogicalVariable variableRepresentative) {
+ this.members = members;
+ this.variableRepresentative = variableRepresentative;
+ representativeIsConst = false;
+ }
+
+ public boolean representativeIsConst() {
+ return representativeIsConst;
+ }
+
+ public List<LogicalVariable> getMembers() {
+ return members;
+ }
+
+ public boolean contains(LogicalVariable var) {
+ return members.contains(var);
+ }
+
+ public ILogicalExpression getConstRepresentative() {
+ return constRepresentative;
+ }
+
+ public LogicalVariable getVariableRepresentative() {
+ return variableRepresentative;
+ }
+
+ public void setConstRepresentative(ILogicalExpression constRepresentative) {
+ this.constRepresentative = constRepresentative;
+ this.representativeIsConst = true;
+ }
+
+ public void setVariableRepresentative(LogicalVariable variableRepresentative) {
+ this.variableRepresentative = variableRepresentative;
+ this.representativeIsConst = false;
+ }
+
+ public void merge(EquivalenceClass ec2) {
+ members.addAll(ec2.getMembers());
+ if (!representativeIsConst && ec2.representativeIsConst()) {
+ representativeIsConst = true;
+ constRepresentative = ec2.getConstRepresentative();
+ }
+ }
+
+ public void addMember(LogicalVariable v) {
+ members.add(v);
+ }
+
+ public EquivalenceClass cloneEquivalenceClass() {
+ List<LogicalVariable> membersClone = new LinkedList<LogicalVariable>();
+ membersClone.addAll(members);
+ EquivalenceClass ec;
+ if (representativeIsConst()) {
+ ec = new EquivalenceClass(membersClone, constRepresentative);
+ } else {
+ ec = new EquivalenceClass(membersClone, variableRepresentative);
+ }
+ return ec;
+ }
+
+ @Override
+ public String toString() {
+ return "(<" + (representativeIsConst ? constRepresentative : variableRepresentative) + "> " + members + ")";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof EquivalenceClass)) {
+ return false;
+ } else {
+ EquivalenceClass ec = (EquivalenceClass) obj;
+ if (!members.equals(ec.getMembers())) {
+ return false;
+ }
+ if (representativeIsConst) {
+ return ec.representativeIsConst() && (constRepresentative.equals(ec.getConstRepresentative()));
+ } else {
+ return !ec.representativeIsConst() && (variableRepresentative.equals(ec.getVariableRepresentative()));
+ }
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IHyracksJobBuilder.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IHyracksJobBuilder.java
new file mode 100644
index 0000000..19c85c6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IHyracksJobBuilder.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public interface IHyracksJobBuilder {
+ public enum TargetConstraint {
+ ONE,
+ SAME_COUNT
+ }
+
+ public void contributeHyracksOperator(ILogicalOperator op, IOperatorDescriptor opDesc);
+
+ public void contributeAlgebricksPartitionConstraint(IOperatorDescriptor opDesc, AlgebricksPartitionConstraint apc);
+
+ public void contributeMicroOperator(ILogicalOperator op, IPushRuntimeFactory runtime, RecordDescriptor recDesc);
+
+ public void contributeMicroOperator(ILogicalOperator op, IPushRuntimeFactory runtime, RecordDescriptor recDesc,
+ AlgebricksPartitionConstraint pc);
+
+ /**
+ * inputs are numbered starting from 0
+ */
+ public void contributeGraphEdge(ILogicalOperator src, int srcOutputIndex, ILogicalOperator dest, int destInputIndex);
+
+ public void contributeConnector(ILogicalOperator exchgOp, IConnectorDescriptor conn);
+
+ public void contributeConnectorWithTargetConstraint(ILogicalOperator exchgOp, IConnectorDescriptor conn,
+ TargetConstraint numberOfTargetPartitions);
+
+ public JobSpecification getJobSpec();
+
+ /**
+ * to be called only after all the graph information is added
+ */
+ public void buildSpec(List<ILogicalOperator> roots) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
new file mode 100644
index 0000000..2d19944
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface ILogicalExpression {
+
+ public LogicalExpressionTag getExpressionTag();
+
+ public abstract <R, T> R accept(ILogicalExpressionVisitor<R, T> visitor, T arg) throws AlgebricksException;
+
+ public void getUsedVariables(Collection<LogicalVariable> vars);
+
+ public void substituteVar(LogicalVariable v1, LogicalVariable v2);
+
+ // constraints (e.g., FDs, equivalences)
+
+ /**
+ * @param fds
+ * Output argument: functional dependencies that can be inferred
+ * from this expression.
+ * @param equivClasses
+ * Output argument: Equivalence classes that can be inferred from
+ * this expression.
+ */
+ public void getConstraintsAndEquivClasses(Collection<FunctionalDependency> fds,
+ Map<LogicalVariable, EquivalenceClass> equivClasses);
+
+ /**
+ * @param fds
+ * Output argument: functional dependencies that can be inferred
+ * from this expression.
+ * @param outerVars
+ * Input argument: variables coming from outer branch(es), e.g.,
+ * the left branch of a left outer join.
+ */
+ public void getConstraintsForOuterJoin(Collection<FunctionalDependency> fds, Collection<LogicalVariable> outerVars);
+
+ /**
+ * @param conjs
+ * Output argument: a list of expression whose conjunction, in
+ * any order, can replace the current expression.
+ * @return true if the expression can be broken in at least two conjuncts,
+ * false otherwise.
+ */
+ public boolean splitIntoConjuncts(List<LogicalExpressionReference> conjs);
+
+ public abstract ILogicalExpression cloneExpression();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
new file mode 100644
index 0000000..198d762
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface ILogicalOperator {
+
+ public List<LogicalOperatorReference> getInputs();
+
+ boolean hasInputs();
+
+ public void recomputeSchema() throws AlgebricksException;
+
+ public List<LogicalVariable> getSchema();
+
+ /*
+ *
+ * support for visitors
+ */
+
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException;
+
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException;
+
+ public boolean isMap();
+
+ public Map<String, Object> getAnnotations();
+
+ public void removeAnnotation(String annotationName);
+
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException;
+
+ // variables
+
+ /**
+ * Get the variable propogation policy from this operator's input to its
+ * output.
+ *
+ * @return The VariablePropogationPolicy.
+ */
+ public VariablePropagationPolicy getVariablePropagationPolicy();
+
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException;
+
+ public IVariableTypeEnvironment computeInputTypeEnvironment(ITypingContext ctx) throws AlgebricksException;
+
+ // structural properties
+
+ /**
+ * @return for each child, one vector of required physical properties
+ */
+
+ public PhysicalRequirements getRequiredPhysicalPropertiesForChildren(IPhysicalPropertiesVector requiredProperties);
+
+ /**
+ * @return the physical properties that this operator delivers, based on
+ * what its children deliver
+ */
+
+ public IPhysicalPropertiesVector getDeliveredPhysicalProperties();
+
+ public void computeDeliveredPhysicalProperties(IOptimizationContext context) throws AlgebricksException;
+
+ public void setHostQueryContext(Object context);
+
+ public Object getHostQueryContext();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalPlan.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalPlan.java
new file mode 100644
index 0000000..bf0f548
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalPlan.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import java.util.List;
+
+public interface ILogicalPlan {
+ public List<LogicalOperatorReference> getRoots();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
new file mode 100644
index 0000000..5a616c6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+
+public interface ILogicalPlanAndMetadata {
+ public ILogicalPlan getPlan();
+
+ public IMetadataProvider<?, ?> getMetadataProvider();
+
+ public AlgebricksPartitionConstraint getClusterLocations();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IOptimizationContext.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
new file mode 100644
index 0000000..4daaceb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableEvalSizeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+
+public interface IOptimizationContext extends ITypingContext {
+
+ public abstract int getVarCounter();
+
+ // -1 if unknown
+ public abstract int getFrameSize();
+
+ public abstract void setVarCounter(int varCounter);
+
+ public abstract LogicalVariable newVar();
+
+ public abstract IMetadataProvider<?, ?> getMetadataProvider();
+
+ public abstract void setMetadataDeclarations(IMetadataProvider<?, ?> metadataProvider);
+
+ public abstract boolean checkIfInDontApplySet(IAlgebraicRewriteRule rule, ILogicalOperator op);
+
+ public abstract void addToDontApplySet(IAlgebraicRewriteRule rule, ILogicalOperator op);
+
+ /*
+ * returns true if op1 and op2 have already been compared
+ */
+ public abstract boolean checkAndAddToAlreadyCompared(ILogicalOperator op1, ILogicalOperator op2);
+
+ public abstract void addNotToBeInlinedVar(LogicalVariable var);
+
+ public abstract boolean shouldNotBeInlined(LogicalVariable var);
+
+ public abstract void addPrimaryKey(FunctionalDependency pk);
+
+ public abstract List<LogicalVariable> findPrimaryKey(LogicalVariable recordVar);
+
+ public abstract void putEquivalenceClassMap(ILogicalOperator op, Map<LogicalVariable, EquivalenceClass> eqClassMap);
+
+ public abstract Map<LogicalVariable, EquivalenceClass> getEquivalenceClassMap(ILogicalOperator op);
+
+ public abstract void putFDList(ILogicalOperator op, List<FunctionalDependency> fdList);
+
+ public abstract List<FunctionalDependency> getFDList(ILogicalOperator op);
+
+ public abstract void putLogicalPropertiesVector(ILogicalOperator op, ILogicalPropertiesVector v);
+
+ public abstract ILogicalPropertiesVector getLogicalPropertiesVector(ILogicalOperator op);
+
+ public abstract IExpressionEvalSizeComputer getExpressionEvalSizeComputer();
+
+ public abstract IVariableEvalSizeEnvironment getVariableEvalSizeEnvironment();
+
+ public abstract IMergeAggregationExpressionFactory getMergeAggregationExpressionFactory();
+
+ public abstract PhysicalOptimizationConfig getPhysicalOptimizationConfig();
+
+ public abstract void invalidateTypeEnvironmentForOperator(ILogicalOperator op);
+
+ public abstract void computeAndSetTypeEnvironmentForOperator(ILogicalOperator op) throws AlgebricksException;
+
+ public abstract void updatePrimaryKeys(Map<LogicalVariable, LogicalVariable> mappedVars);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java
new file mode 100644
index 0000000..46d2dc3
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IPhysicalOperator {
+
+ public PhysicalOperatorTag getOperatorTag();
+
+ /**
+ * @param op
+ * the logical operator this physical operator annotates
+ * @param reqdByParent
+ * parent's requirements, which are not enforced for now, as we
+ * only explore one plan
+ * @return for each child, one vector of required physical properties
+ */
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent);
+
+ /**
+ * @return the physical properties that this operator delivers, based on
+ * what its children deliver
+ */
+ public IPhysicalPropertiesVector getDeliveredProperties();
+
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException;
+
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException;
+
+ public void disableJobGenBelowMe();
+
+ public boolean isJobGenDisabledBelowMe();
+
+ public boolean isMicroOperator();
+
+ public void setHostQueryContext(Object context);
+
+ public Object getHostQueryContext();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalExpressionReference.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalExpressionReference.java
new file mode 100644
index 0000000..6ab0cb9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalExpressionReference.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+public final class LogicalExpressionReference {
+ private ILogicalExpression expression;
+
+ public LogicalExpressionReference(ILogicalExpression expression) {
+ this.expression = expression;
+ }
+
+ public ILogicalExpression getExpression() {
+ return expression;
+ }
+
+ public void setExpression(ILogicalExpression expression) {
+ this.expression = expression;
+ }
+
+ public String toString() {
+ return expression.toString();
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalExpressionTag.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalExpressionTag.java
new file mode 100644
index 0000000..166b357
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalExpressionTag.java
@@ -0,0 +1,19 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+public enum LogicalExpressionTag {
+ FUNCTION_CALL, VARIABLE, CONSTANT
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorReference.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorReference.java
new file mode 100644
index 0000000..ce47ff8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorReference.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+public final class LogicalOperatorReference {
+ private ILogicalOperator operator;
+
+ public LogicalOperatorReference() {
+ }
+
+ public LogicalOperatorReference(ILogicalOperator operator) {
+ this.operator = operator;
+ }
+
+ public ILogicalOperator getOperator() {
+ return operator;
+ }
+
+ public void setOperator(ILogicalOperator operator) {
+ this.operator = operator;
+ }
+
+ @Override
+ public String toString() {
+ return operator == null ? "" : operator.toString();
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
new file mode 100644
index 0000000..01fa9da
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+public enum LogicalOperatorTag {
+ AGGREGATE,
+ ASSIGN,
+ CLUSTER,
+ DATASOURCESCAN,
+ DISTINCT,
+ GROUP,
+ EMPTYTUPLESOURCE,
+ EXCHANGE,
+ INNERJOIN,
+ LEFTOUTERJOIN,
+ LIMIT,
+ DIE,
+ NESTEDTUPLESOURCE,
+ ORDER,
+ PROJECT,
+ PARTITIONINGSPLIT,
+ REPLICATE,
+ RUNNINGAGGREGATE,
+ SCRIPT,
+ SELECT,
+ SINK,
+ SUBPLAN,
+ UNIONALL,
+ UNNEST,
+ UNNEST_MAP,
+ WRITE,
+ WRITE_RESULT,
+ INSERT_DELETE,
+ INDEX_INSERT_DELETE,
+ UPDATE
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalVariable.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalVariable.java
new file mode 100644
index 0000000..0a88459
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalVariable.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+/**
+ * Represents a logical variable in an asterix logical plan.
+ *
+ * @author Vinayak R. Borkar [vborkar@ics.uci.edu]
+ */
+public final class LogicalVariable {
+ private final int id;
+
+ public LogicalVariable(int id) {
+ this.id = id;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ @Override
+ public String toString() {
+ return "$$" + id;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof LogicalVariable)) {
+ return false;
+ } else {
+ return id == ((LogicalVariable) obj).getId();
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return id;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
new file mode 100644
index 0000000..655645f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
@@ -0,0 +1,12 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+public interface OperatorAnnotations {
+ // hints
+ public static final String USE_HASH_GROUP_BY = "USE_HASH_GROUP_BY"; // -->
+ public static final String USE_EXTERNAL_GROUP_BY = "USE_EXTERNAL_GROUP_BY"; // -->
+ // Boolean
+ public static final String CARDINALITY = "CARDINALITY"; // -->
+ // Integer
+ public static final String MAX_NUMBER_FRAMES = "MAX_NUMBER_FRAMES"; // -->
+ // Integer
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
new file mode 100644
index 0000000..48f320b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.base;
+
+public enum PhysicalOperatorTag {
+ AGGREGATE,
+ ASSIGN,
+ BROADCAST_EXCHANGE,
+ BTREE_SEARCH,
+ DATASOURCE_SCAN,
+ EMPTY_TUPLE_SOURCE,
+ EXTERNAL_GROUP_BY,
+ IN_MEMORY_HASH_JOIN,
+ HASH_GROUP_BY,
+ HASH_PARTITION_EXCHANGE,
+ HASH_PARTITION_MERGE_EXCHANGE,
+ HYBRID_HASH_JOIN,
+ HDFS_READER,
+ IN_MEMORY_STABLE_SORT,
+ MICRO_PRE_CLUSTERED_GROUP_BY,
+ NESTED_LOOP,
+ NESTED_TUPLE_SOURCE,
+ ONE_TO_ONE_EXCHANGE,
+ PRE_SORTED_DISTINCT_BY,
+ PRE_CLUSTERED_GROUP_BY,
+ RANGE_PARTITION_EXCHANGE,
+ RANDOM_MERGE_EXCHANGE,
+ RTREE_SEARCH,
+ RUNNING_AGGREGATE,
+ SORT_MERGE_EXCHANGE,
+ SINK,
+ SINK_WRITE,
+ SPLIT,
+ STABLE_SORT,
+ STREAM_LIMIT,
+ STREAM_DIE,
+ STREAM_SELECT,
+ STREAM_PROJECT,
+ STRING_STREAM_SCRIPT,
+ SUBPLAN,
+ UNION_ALL,
+ UNNEST,
+ WRITE_RESULT,
+ INSERT_DELETE,
+ INDEX_INSERT_DELETE,
+ UPDATE
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IAWriter.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IAWriter.java
new file mode 100644
index 0000000..d29fab6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IAWriter.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public interface IAWriter {
+
+ public void init() throws AlgebricksException;
+
+ public void printTuple(FrameTupleAccessor tAccess, int tIdx) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IAWriterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IAWriterFactory.java
new file mode 100644
index 0000000..200f58f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IAWriterFactory.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public interface IAWriterFactory extends Serializable {
+ public IAWriter createWriter(int[] fields, PrintStream ps, IPrinterFactory[] printerFactories,
+ RecordDescriptor inputRecordDescriptor);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryBooleanInspector.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryBooleanInspector.java
new file mode 100644
index 0000000..60e5050
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryBooleanInspector.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import java.io.Serializable;
+
+public interface IBinaryBooleanInspector extends Serializable {
+ public boolean getBooleanValue(byte[] bytes, int offset, int length);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryComparatorFactoryProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryComparatorFactoryProvider.java
new file mode 100644
index 0000000..bb4b734
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryComparatorFactoryProvider.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public interface IBinaryComparatorFactoryProvider {
+ /**
+ * only ASC and DESC are currently supported for orderKind (FUNCTIONCALL can
+ * be ignored)
+ */
+ public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, OrderKind orderKind)
+ throws AlgebricksException;
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryHashFunctionFactoryProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryHashFunctionFactoryProvider.java
new file mode 100644
index 0000000..2e1f0fa
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryHashFunctionFactoryProvider.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public interface IBinaryHashFunctionFactoryProvider {
+ public IBinaryHashFunctionFactory getBinaryHashFunctionFactory(Object type) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryIntegerInspector.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryIntegerInspector.java
new file mode 100644
index 0000000..3a2fe29
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IBinaryIntegerInspector.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import java.io.Serializable;
+
+public interface IBinaryIntegerInspector extends Serializable {
+ public int getIntegerValue(byte[] bytes, int offset, int length);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/INormalizedKeyComputerFactoryProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/INormalizedKeyComputerFactoryProvider.java
new file mode 100644
index 0000000..1883ef5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/INormalizedKeyComputerFactoryProvider.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public interface INormalizedKeyComputerFactoryProvider {
+ public INormalizedKeyComputerFactory getNormalizedKeyComputerFactory(Object type, OrderKind order);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinter.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinter.java
new file mode 100644
index 0000000..89de415
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinter.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IPrinter {
+ public void init() throws AlgebricksException;
+
+ public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinterFactory.java
new file mode 100644
index 0000000..871fe43
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinterFactory.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import java.io.Serializable;
+
+public interface IPrinterFactory extends Serializable {
+ public IPrinter createPrinter();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinterFactoryProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinterFactoryProvider.java
new file mode 100644
index 0000000..e2e2f1c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/IPrinterFactoryProvider.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IPrinterFactoryProvider {
+ public IPrinterFactory getPrinterFactory(Object type) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ISerializerDeserializerProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ISerializerDeserializerProvider.java
new file mode 100644
index 0000000..66d696c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ISerializerDeserializerProvider.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+
+public interface ISerializerDeserializerProvider {
+ @SuppressWarnings("unchecked")
+ public ISerializerDeserializer getSerializerDeserializer(Object type) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ITypeTraitProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ITypeTraitProvider.java
new file mode 100644
index 0000000..98034b6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ITypeTraitProvider.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+
+public interface ITypeTraitProvider {
+ public ITypeTrait getTypeTrait(Object type);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
new file mode 100644
index 0000000..6dcaa31
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
@@ -0,0 +1,321 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+
+public abstract class AbstractFunctionCallExpression extends AbstractLogicalExpression {
+
+ public enum FunctionKind {
+ SCALAR, STATEFUL, AGGREGATE, UNNEST
+ }
+
+ protected IFunctionInfo finfo;
+ final private List<LogicalExpressionReference> arguments;
+ private Object[] opaqueParameters;
+ private final FunctionKind kind;
+ private Map<Object, IExpressionAnnotation> annotationMap = new HashMap<Object, IExpressionAnnotation>();
+
+ public AbstractFunctionCallExpression(FunctionKind kind, IFunctionInfo finfo,
+ List<LogicalExpressionReference> arguments) {
+ this.kind = kind;
+ this.finfo = finfo;
+ this.arguments = arguments;
+ }
+
+ public AbstractFunctionCallExpression(FunctionKind kind, IFunctionInfo finfo) {
+ this.kind = kind;
+ this.finfo = finfo;
+ this.arguments = new ArrayList<LogicalExpressionReference>();
+ }
+
+ public AbstractFunctionCallExpression(FunctionKind kind, IFunctionInfo finfo,
+ LogicalExpressionReference... expressions) {
+ this(kind, finfo);
+ for (LogicalExpressionReference e : expressions) {
+ this.arguments.add(e);
+ }
+ }
+
+ public void setOpaqueParameters(Object[] opaqueParameters) {
+ this.opaqueParameters = opaqueParameters;
+ }
+
+ public Object[] getOpaqueParameters() {
+ return opaqueParameters;
+ }
+
+ public FunctionKind getKind() {
+ return kind;
+ }
+
+ protected List<LogicalExpressionReference> cloneArguments() {
+ List<LogicalExpressionReference> clonedArgs = new ArrayList<LogicalExpressionReference>(arguments.size());
+ for (LogicalExpressionReference e : arguments) {
+ ILogicalExpression e2 = ((AbstractLogicalExpression) e.getExpression()).cloneExpression();
+ clonedArgs.add(new LogicalExpressionReference(e2));
+ }
+ return clonedArgs;
+ }
+
+ public FunctionIdentifier getFunctionIdentifier() {
+ return finfo.getFunctionIdentifier();
+ }
+
+ public IFunctionInfo getFunctionInfo() {
+ return finfo;
+ }
+
+ public void setFunctionInfo(IFunctionInfo finfo) {
+ this.finfo = finfo;
+ }
+
+ public List<LogicalExpressionReference> getArguments() {
+ return arguments;
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("function-call: " + finfo.getFunctionIdentifier() + ", Args:[");
+ // + arguments;
+ boolean first = true;
+ for (LogicalExpressionReference ref : arguments) {
+ if (first) {
+ first = false;
+ } else {
+ sb.append(", ");
+ }
+ sb.append(ref.getExpression());
+ }
+ sb.append("]");
+ return sb.toString();
+ }
+
+ @Override
+ public LogicalExpressionTag getExpressionTag() {
+ return LogicalExpressionTag.FUNCTION_CALL;
+ }
+
+ @Override
+ public void getUsedVariables(Collection<LogicalVariable> vars) {
+ for (LogicalExpressionReference arg : arguments) {
+ arg.getExpression().getUsedVariables(vars);
+ }
+ }
+
+ @Override
+ public void substituteVar(LogicalVariable v1, LogicalVariable v2) {
+ for (LogicalExpressionReference arg : arguments) {
+ arg.getExpression().substituteVar(v1, v2);
+ }
+ }
+
+ @Override
+ public void getConstraintsAndEquivClasses(Collection<FunctionalDependency> fds,
+ Map<LogicalVariable, EquivalenceClass> equivClasses) {
+ FunctionIdentifier funId = getFunctionIdentifier();
+ if (funId == AlgebricksBuiltinFunctions.AND) {
+ for (LogicalExpressionReference a : arguments) {
+ a.getExpression().getConstraintsAndEquivClasses(fds, equivClasses);
+ }
+ } else if (funId == AlgebricksBuiltinFunctions.EQ) {
+ ILogicalExpression opLeft = arguments.get(0).getExpression();
+ ILogicalExpression opRight = arguments.get(1).getExpression();
+ if (opLeft.getExpressionTag() == LogicalExpressionTag.CONSTANT
+ && opRight.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ ConstantExpression op1 = (ConstantExpression) opLeft;
+ VariableReferenceExpression op2 = (VariableReferenceExpression) opRight;
+ getFDsAndEquivClassesForEqWithConstant(op1, op2, fds, equivClasses);
+ } else if (opLeft.getExpressionTag() == LogicalExpressionTag.VARIABLE
+ && opRight.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression op1 = (VariableReferenceExpression) opLeft;
+ VariableReferenceExpression op2 = (VariableReferenceExpression) opRight;
+ getFDsAndEquivClassesForColumnEq(op1, op2, fds, equivClasses);
+ }
+ }
+ }
+
+ @Override
+ public void getConstraintsForOuterJoin(Collection<FunctionalDependency> fds, Collection<LogicalVariable> outerVars) {
+ FunctionIdentifier funId = getFunctionIdentifier();
+ if (funId == AlgebricksBuiltinFunctions.AND) {
+ for (LogicalExpressionReference a : arguments) {
+ a.getExpression().getConstraintsForOuterJoin(fds, outerVars);
+ }
+ } else if (funId == AlgebricksBuiltinFunctions.EQ) {
+ ILogicalExpression opLeft = arguments.get(0).getExpression();
+ ILogicalExpression opRight = arguments.get(1).getExpression();
+ if (opLeft.getExpressionTag() == LogicalExpressionTag.VARIABLE
+ && opRight.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable var1 = ((VariableReferenceExpression) opLeft).getVariableReference();
+ LogicalVariable var2 = ((VariableReferenceExpression) opRight).getVariableReference();
+ if (outerVars.contains(var1)) {
+ addFD(fds, var1, var2);
+ }
+ if (outerVars.contains(var2)) {
+ addFD(fds, var2, var1);
+ }
+ }
+ }
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof AbstractFunctionCallExpression)) {
+ return false;
+ } else {
+ AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) obj;
+ boolean equal = getFunctionIdentifier().equals(fce.getFunctionIdentifier());
+ if (!equal)
+ return false;
+ for (int i = 0; i < arguments.size(); i++) {
+ ILogicalExpression argument = arguments.get(i).getExpression();
+ ILogicalExpression fceArgument = fce.getArguments().get(i).getExpression();
+ if (!argument.equals(fceArgument))
+ return false;
+ }
+ return true;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ int h = finfo.hashCode();
+ for (LogicalExpressionReference e : arguments) {
+ h = h * 41 + e.getExpression().hashCode();
+ }
+ return h;
+ }
+
+ @Override
+ public boolean splitIntoConjuncts(List<LogicalExpressionReference> conjs) {
+ if (getFunctionIdentifier() != AlgebricksBuiltinFunctions.AND || arguments.size() <= 1) {
+ return false;
+ } else {
+ conjs.addAll(arguments);
+ return true;
+ }
+ }
+
+ public Map<Object, IExpressionAnnotation> getAnnotations() {
+ return annotationMap;
+ }
+
+ protected Map<Object, IExpressionAnnotation> cloneAnnotations() {
+ Map<Object, IExpressionAnnotation> m = new HashMap<Object, IExpressionAnnotation>();
+ for (Object k : annotationMap.keySet()) {
+ IExpressionAnnotation annot2 = annotationMap.get(k).copy();
+ m.put(k, annot2);
+ }
+ return m;
+ }
+
+ private final static void addFD(Collection<FunctionalDependency> fds, LogicalVariable var1, LogicalVariable var2) {
+ LinkedList<LogicalVariable> set1 = new LinkedList<LogicalVariable>();
+ set1.add(var1);
+ LinkedList<LogicalVariable> set2 = new LinkedList<LogicalVariable>();
+ set2.add(var2);
+ FunctionalDependency fd1 = new FunctionalDependency(set1, set2);
+ fds.add(fd1);
+ }
+
+ private final static void getFDsAndEquivClassesForEqWithConstant(ConstantExpression c,
+ VariableReferenceExpression v, Collection<FunctionalDependency> fds,
+ Map<LogicalVariable, EquivalenceClass> equivClasses) {
+ LogicalVariable var = v.getVariableReference();
+ LinkedList<LogicalVariable> head = new LinkedList<LogicalVariable>();
+ // empty set in the head
+ LinkedList<LogicalVariable> tail = new LinkedList<LogicalVariable>();
+ tail.add(var);
+ FunctionalDependency fd = new FunctionalDependency(head, tail);
+ fds.add(fd);
+
+ EquivalenceClass ec = equivClasses.get(var);
+ if (ec == null) {
+ LinkedList<LogicalVariable> members = new LinkedList<LogicalVariable>();
+ members.add(var);
+ EquivalenceClass eclass = new EquivalenceClass(members, c);
+ equivClasses.put(var, eclass);
+ } else {
+ if (ec.representativeIsConst()) {
+ ILogicalExpression c1 = ec.getConstRepresentative();
+ if (!c1.equals(c)) {
+ // here I could also rewrite to FALSE
+ return;
+ }
+ }
+ ec.setConstRepresentative(c);
+ }
+ }
+
+ /*
+ * Obs.: mgmt. of equiv. classes should use a more efficient data
+ * structure,if we are to implem. cost-bazed optim.
+ */
+ private final static void getFDsAndEquivClassesForColumnEq(VariableReferenceExpression v1,
+ VariableReferenceExpression v2, Collection<FunctionalDependency> fds,
+ Map<LogicalVariable, EquivalenceClass> equivClasses) {
+ LogicalVariable var1 = v1.getVariableReference();
+ LogicalVariable var2 = v2.getVariableReference();
+ LinkedList<LogicalVariable> set1 = new LinkedList<LogicalVariable>();
+ set1.add(var1);
+ LinkedList<LogicalVariable> set2 = new LinkedList<LogicalVariable>();
+ set2.add(var2);
+ FunctionalDependency fd1 = new FunctionalDependency(set1, set2);
+ FunctionalDependency fd2 = new FunctionalDependency(set2, set1);
+ fds.add(fd1);
+ fds.add(fd2);
+
+ EquivalenceClass ec1 = equivClasses.get(var1);
+ EquivalenceClass ec2 = equivClasses.get(var2);
+ if (ec1 == null && ec2 == null) {
+ LinkedList<LogicalVariable> members = new LinkedList<LogicalVariable>();
+ members.add(var1);
+ members.add(var2);
+ EquivalenceClass ec = new EquivalenceClass(members, var1);
+ equivClasses.put(var1, ec);
+ equivClasses.put(var2, ec);
+ } else if (ec1 == null && ec2 != null) {
+ ec2.addMember(var1);
+ equivClasses.put(var1, ec2);
+ } else if (ec2 == null && ec1 != null) {
+ ec1.addMember(var2);
+ equivClasses.put(var2, ec1);
+ } else {
+ ec1.merge(ec2);
+ for (LogicalVariable w : equivClasses.keySet()) {
+ if (ec2.getMembers().contains(w)) {
+ equivClasses.put(w, ec1);
+ }
+ }
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
new file mode 100644
index 0000000..f9b30cc
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.Collection;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+
+public abstract class AbstractLogicalExpression implements ILogicalExpression {
+
+ @Override
+ public void getConstraintsAndEquivClasses(Collection<FunctionalDependency> fds,
+ Map<LogicalVariable, EquivalenceClass> equivClasses) {
+ // do nothing
+ }
+
+ @Override
+ public void getConstraintsForOuterJoin(Collection<FunctionalDependency> fds,
+ Collection<LogicalVariable> innerSideVars) {
+ // do nothing
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
new file mode 100644
index 0000000..84a421b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+/**
+ *
+ * An aggregate function may be executed in a "two step" mode. First the
+ * "step-one" aggregates are run and then the results are passed to the
+ * "step-two" aggregators. The convention is the following:
+ *
+ * 1. The step-one aggregate must be able to accept the same arguments as the
+ * original aggregate function call.
+ *
+ * 2. The step-two aggregate must be a unary function that accepts as input the
+ * output of the step-one aggregate.
+ *
+ */
+
+public class AggregateFunctionCallExpression extends AbstractFunctionCallExpression {
+
+ private boolean twoStep;
+ private IFunctionInfo stepOneAggregate;
+ private IFunctionInfo stepTwoAggregate;
+
+ public AggregateFunctionCallExpression(IFunctionInfo finfo, boolean isTwoStep) {
+ super(FunctionKind.AGGREGATE, finfo);
+ this.twoStep = isTwoStep;
+ }
+
+ public AggregateFunctionCallExpression(IFunctionInfo finfo, boolean isTwoStep,
+ List<LogicalExpressionReference> arguments) {
+ super(FunctionKind.AGGREGATE, finfo, arguments);
+ this.twoStep = isTwoStep;
+ }
+
+ public AggregateFunctionCallExpression(IFunctionInfo finfo, boolean isTwoStep,
+ LogicalExpressionReference... expressions) {
+ super(FunctionKind.AGGREGATE, finfo, expressions);
+ this.twoStep = isTwoStep;
+ }
+
+ public boolean isTwoStep() {
+ return twoStep;
+ }
+
+ public void setTwoStep(boolean twoStep) {
+ this.twoStep = twoStep;
+ }
+
+ @Override
+ public AggregateFunctionCallExpression cloneExpression() {
+ cloneAnnotations();
+ List<LogicalExpressionReference> clonedArgs = cloneArguments();
+ AggregateFunctionCallExpression fun = new AggregateFunctionCallExpression(finfo, twoStep, clonedArgs);
+ fun.setStepTwoAggregate(stepTwoAggregate);
+ fun.setStepOneAggregate(stepOneAggregate);
+ return fun;
+ }
+
+ public void setStepOneAggregate(IFunctionInfo stepOneAggregate) {
+ this.stepOneAggregate = stepOneAggregate;
+ }
+
+ public IFunctionInfo getStepOneAggregate() {
+ return stepOneAggregate;
+ }
+
+ public void setStepTwoAggregate(IFunctionInfo stepTwoAggregate) {
+ this.stepTwoAggregate = stepTwoAggregate;
+ }
+
+ public IFunctionInfo getStepTwoAggregate() {
+ return stepTwoAggregate;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalExpressionVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitAggregateFunctionCallExpression(this, arg);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/BroadcastExpressionAnnotation.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/BroadcastExpressionAnnotation.java
new file mode 100644
index 0000000..ff88912
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/BroadcastExpressionAnnotation.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+public class BroadcastExpressionAnnotation implements IExpressionAnnotation {
+
+ public static final String BROADCAST_ANNOTATION_KEY = "broadcast";
+
+ public enum BroadcastSide {
+ LEFT,
+ RIGHT
+ };
+
+ private BroadcastSide side;
+
+ @Override
+ public Object getObject() {
+ return side;
+ }
+
+ @Override
+ public void setObject(Object side) {
+ this.side = (BroadcastSide) side;
+ }
+
+ @Override
+ public IExpressionAnnotation copy() {
+ BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+ bcast.side = side;
+ return bcast;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
new file mode 100644
index 0000000..800fee7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public final class ConstantExpression extends AbstractLogicalExpression {
+ private IAlgebricksConstantValue value;
+
+ public final static ConstantExpression TRUE = new ConstantExpression(new IAlgebricksConstantValue() {
+
+ @Override
+ public boolean isTrue() {
+ return true;
+ }
+
+ @Override
+ public boolean isNull() {
+ return false;
+ }
+
+ @Override
+ public boolean isFalse() {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "TRUE";
+ }
+ });
+ public final static ConstantExpression FALSE = new ConstantExpression(new IAlgebricksConstantValue() {
+
+ @Override
+ public boolean isTrue() {
+ return false;
+ }
+
+ @Override
+ public boolean isNull() {
+ return false;
+ }
+
+ @Override
+ public boolean isFalse() {
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "FALSE";
+ }
+ });
+ public final static ConstantExpression NULL = new ConstantExpression(new IAlgebricksConstantValue() {
+
+ @Override
+ public boolean isTrue() {
+ return false;
+ }
+
+ @Override
+ public boolean isNull() {
+ return true;
+ }
+
+ @Override
+ public boolean isFalse() {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "NULL";
+ }
+ });
+
+ private Map<Object, IExpressionAnnotation> annotationMap = new HashMap<Object, IExpressionAnnotation>();
+
+ public ConstantExpression(IAlgebricksConstantValue value) {
+ this.value = value;
+ }
+
+ public IAlgebricksConstantValue getValue() {
+ return value;
+ }
+
+ public void setValue(IAlgebricksConstantValue value) {
+ this.value = value;
+ }
+
+ @Override
+ public LogicalExpressionTag getExpressionTag() {
+ return LogicalExpressionTag.CONSTANT;
+ }
+
+ @Override
+ public String toString() {
+ return value.toString();
+ }
+
+ @Override
+ public void getUsedVariables(Collection<LogicalVariable> vars) {
+ // do nothing
+ }
+
+ @Override
+ public void substituteVar(LogicalVariable v1, LogicalVariable v2) {
+ // do nothing
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof ConstantExpression)) {
+ return false;
+ } else {
+ return value.equals(((ConstantExpression) obj).getValue());
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return value.hashCode();
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalExpressionVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitConstantExpression(this, arg);
+ }
+
+ @Override
+ public AbstractLogicalExpression cloneExpression() {
+ Map<Object, IExpressionAnnotation> m = new HashMap<Object, IExpressionAnnotation>();
+ for (Object k : annotationMap.keySet()) {
+ IExpressionAnnotation annot2 = annotationMap.get(k).copy();
+ m.put(k, annot2);
+ }
+ ConstantExpression c = new ConstantExpression(value);
+ c.annotationMap = m;
+ return c;
+ }
+
+ public Map<Object, IExpressionAnnotation> getAnnotations() {
+ return annotationMap;
+ }
+
+ @Override
+ public boolean splitIntoConjuncts(List<LogicalExpressionReference> conjs) {
+ return false;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java
new file mode 100644
index 0000000..6870a38
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+public class ExpressionAnnotationNoCopyImpl implements IExpressionAnnotation {
+
+ private Object object;
+
+ @Override
+ public IExpressionAnnotation copy() {
+ return this;
+ }
+
+ @Override
+ public Object getObject() {
+ return object;
+ }
+
+ @Override
+ public void setObject(Object object) {
+ this.object = object;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IAlgebricksConstantValue.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IAlgebricksConstantValue.java
new file mode 100644
index 0000000..b6c7dbd
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IAlgebricksConstantValue.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+public interface IAlgebricksConstantValue {
+ public boolean isNull();
+
+ public boolean isTrue();
+
+ public boolean isFalse();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionAnnotation.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionAnnotation.java
new file mode 100644
index 0000000..15e543b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionAnnotation.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+public interface IExpressionAnnotation {
+ public Object getObject();
+
+ public void setObject(Object object);
+
+ public IExpressionAnnotation copy();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionEvalSizeComputer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionEvalSizeComputer.java
new file mode 100644
index 0000000..8dd807e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionEvalSizeComputer.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IExpressionEvalSizeComputer {
+ // size in bytes, or -1 if unknown
+ public int getEvalSize(ILogicalExpression expr, IVariableEvalSizeEnvironment env) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionTypeComputer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionTypeComputer.java
new file mode 100644
index 0000000..72861ae
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IExpressionTypeComputer.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IExpressionTypeComputer {
+ public Object getType(ILogicalExpression expr, IMetadataProvider<?, ?> metadataProvider,
+ IVariableTypeEnvironment env) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ILogicalExpressionJobGen.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ILogicalExpressionJobGen.java
new file mode 100644
index 0000000..babfede
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ILogicalExpressionJobGen.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.ISerializableAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface ILogicalExpressionJobGen {
+
+ public IEvaluatorFactory createEvaluatorFactory(ILogicalExpression expr, IVariableTypeEnvironment env,
+ IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException;
+
+ public IAggregateFunctionFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException;
+
+ public ISerializableAggregateFunctionFactory createSerializableAggregateFunctionFactory(
+ AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
+ JobGenContext context) throws AlgebricksException;
+
+ public IRunningAggregateFunctionFactory createRunningAggregateFunctionFactory(StatefulFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException;
+
+ public IUnnestingFunctionFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException;
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IMergeAggregationExpressionFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IMergeAggregationExpressionFactory.java
new file mode 100644
index 0000000..36a414c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IMergeAggregationExpressionFactory.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IMergeAggregationExpressionFactory {
+ ILogicalExpression createMergeAggregation(ILogicalExpression expr, IOptimizationContext env)
+ throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/INullableTypeComputer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/INullableTypeComputer.java
new file mode 100644
index 0000000..3290e35
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/INullableTypeComputer.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface INullableTypeComputer {
+ public Object makeNullableType(Object type) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IPartialAggregationTypeComputer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IPartialAggregationTypeComputer.java
new file mode 100644
index 0000000..88e34e9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IPartialAggregationTypeComputer.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IPartialAggregationTypeComputer {
+ public Object getType(ILogicalExpression expr, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IVariableEvalSizeEnvironment.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IVariableEvalSizeEnvironment.java
new file mode 100644
index 0000000..f6799b0
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IVariableEvalSizeEnvironment.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public interface IVariableEvalSizeEnvironment {
+ // size in bytes or -1 if unknown
+ public int getVariableEvalSize(LogicalVariable var);
+
+ public void setVariableEvalSize(LogicalVariable var, int size);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
new file mode 100644
index 0000000..0827b97
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IVariableTypeEnvironment {
+ public Object getVarType(LogicalVariable var) throws AlgebricksException;
+
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables) throws AlgebricksException;
+
+ public void setVarType(LogicalVariable var, Object type);
+
+ public Object getType(ILogicalExpression expr) throws AlgebricksException;
+
+ public boolean substituteProducedVariable(LogicalVariable v1, LogicalVariable v2) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
new file mode 100644
index 0000000..9b1e2dc
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class ScalarFunctionCallExpression extends AbstractFunctionCallExpression {
+
+ public ScalarFunctionCallExpression(IFunctionInfo finfo) {
+ super(FunctionKind.SCALAR, finfo);
+ }
+
+ public ScalarFunctionCallExpression(IFunctionInfo finfo, List<LogicalExpressionReference> arguments) {
+ super(FunctionKind.SCALAR, finfo, arguments);
+ }
+
+ public ScalarFunctionCallExpression(IFunctionInfo finfo, LogicalExpressionReference... expressions) {
+ super(FunctionKind.SCALAR, finfo, expressions);
+ }
+
+ @Override
+ public ScalarFunctionCallExpression cloneExpression() {
+ cloneAnnotations();
+ List<LogicalExpressionReference> clonedArgs = cloneArguments();
+ return new ScalarFunctionCallExpression(finfo, clonedArgs);
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalExpressionVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitScalarFunctionCallExpression(this, arg);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
new file mode 100644
index 0000000..67c3682
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPropertiesComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class StatefulFunctionCallExpression extends AbstractFunctionCallExpression {
+
+ private final IPropertiesComputer propertiesComputer;
+
+ public StatefulFunctionCallExpression(IFunctionInfo finfo, IPropertiesComputer propertiesComputer) {
+ super(FunctionKind.STATEFUL, finfo);
+ this.propertiesComputer = propertiesComputer;
+ }
+
+ public StatefulFunctionCallExpression(IFunctionInfo finfo, IPropertiesComputer propertiesComputer,
+ List<LogicalExpressionReference> arguments) {
+ super(FunctionKind.STATEFUL, finfo, arguments);
+ this.propertiesComputer = propertiesComputer;
+ }
+
+ public StatefulFunctionCallExpression(IFunctionInfo finfo, IPropertiesComputer propertiesComputer,
+ LogicalExpressionReference... expressions) {
+ super(FunctionKind.STATEFUL, finfo, expressions);
+ this.propertiesComputer = propertiesComputer;
+ }
+
+ @Override
+ public StatefulFunctionCallExpression cloneExpression() {
+ cloneAnnotations();
+ List<LogicalExpressionReference> clonedArgs = cloneArguments();
+ return new StatefulFunctionCallExpression(finfo, propertiesComputer, clonedArgs);
+ }
+
+ // can be null
+ public IPartitioningProperty getRequiredPartitioningProperty() {
+ return propertiesComputer.computePartitioningProperty(this);
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalExpressionVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitStatefulFunctionCallExpression(this, arg);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
new file mode 100644
index 0000000..25fa964
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class UnnestingFunctionCallExpression extends AbstractFunctionCallExpression {
+
+ private boolean returnsUniqueValues;
+
+ public UnnestingFunctionCallExpression(IFunctionInfo finfo) {
+ super(FunctionKind.UNNEST, finfo);
+ }
+
+ public UnnestingFunctionCallExpression(IFunctionInfo finfo, List<LogicalExpressionReference> arguments) {
+ super(FunctionKind.UNNEST, finfo, arguments);
+ }
+
+ public UnnestingFunctionCallExpression(IFunctionInfo finfo, LogicalExpressionReference... expressions) {
+ super(FunctionKind.UNNEST, finfo, expressions);
+ }
+
+ @Override
+ public UnnestingFunctionCallExpression cloneExpression() {
+ cloneAnnotations();
+ List<LogicalExpressionReference> clonedArgs = cloneArguments();
+ UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(finfo, clonedArgs);
+ ufce.setReturnsUniqueValues(returnsUniqueValues);
+ return ufce;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalExpressionVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitUnnestingFunctionCallExpression(this, arg);
+ }
+
+ public void setReturnsUniqueValues(boolean returnsUniqueValues) {
+ this.returnsUniqueValues = returnsUniqueValues;
+ }
+
+ public boolean returnsUniqueValues() {
+ return returnsUniqueValues;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
new file mode 100644
index 0000000..07d423b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.expressions;
+
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class VariableReferenceExpression extends AbstractLogicalExpression {
+ private LogicalVariable variable;
+
+ public VariableReferenceExpression(LogicalVariable variable) {
+ this.variable = variable;
+ }
+
+ public LogicalVariable getVariableReference() {
+ return variable;
+ }
+
+ public void setVariable(LogicalVariable variable) {
+ this.variable = variable;
+ }
+
+ @Override
+ public LogicalExpressionTag getExpressionTag() {
+ return LogicalExpressionTag.VARIABLE;
+ }
+
+ @Override
+ public String toString() {
+ return variable.toString();
+ }
+
+ @Override
+ public void getUsedVariables(Collection<LogicalVariable> vars) {
+ // if (!vars.contains(variable)) {
+ vars.add(variable);
+ // }
+ }
+
+ @Override
+ public void substituteVar(LogicalVariable v1, LogicalVariable v2) {
+ if (variable.equals(v1)) {
+ variable = v2;
+ }
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof VariableReferenceExpression)) {
+ return false;
+ } else {
+ return variable.equals(((VariableReferenceExpression) obj).getVariableReference());
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return variable.getId();
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalExpressionVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitVariableReferenceExpression(this, arg);
+ }
+
+ @Override
+ public AbstractLogicalExpression cloneExpression() {
+ return new VariableReferenceExpression(variable);
+ }
+
+ @Override
+ public boolean splitIntoConjuncts(List<LogicalExpressionReference> conjs) {
+ return false;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/AlgebricksBuiltinFunctions.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/AlgebricksBuiltinFunctions.java
new file mode 100644
index 0000000..7511155
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/AlgebricksBuiltinFunctions.java
@@ -0,0 +1,117 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.functions;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+public class AlgebricksBuiltinFunctions {
+ public enum ComparisonKind {
+ EQ,
+ LE,
+ GE,
+ LT,
+ GT,
+ NEQ
+ }
+
+ private static Map<FunctionIdentifier, FunctionIdentifier> algebricksBuiltinFunctions = new HashMap<FunctionIdentifier, FunctionIdentifier>();
+
+ private static Map<FunctionIdentifier, IFunctionInfo> _finfos = new HashMap<FunctionIdentifier, IFunctionInfo>();
+
+ public static final String ALGEBRICKS_NS = "algebricks";
+
+ // comparisons
+ public final static FunctionIdentifier EQ = new FunctionIdentifier(ALGEBRICKS_NS, "eq", 2, true);
+ public final static FunctionIdentifier LE = new FunctionIdentifier(ALGEBRICKS_NS, "le", 2, true);
+ public final static FunctionIdentifier GE = new FunctionIdentifier(ALGEBRICKS_NS, "ge", 2, true);
+ public final static FunctionIdentifier LT = new FunctionIdentifier(ALGEBRICKS_NS, "lt", 2, true);
+ public final static FunctionIdentifier GT = new FunctionIdentifier(ALGEBRICKS_NS, "gt", 2, true);
+ public final static FunctionIdentifier NEQ = new FunctionIdentifier(ALGEBRICKS_NS, "neq", 2, true);
+
+ // booleans
+ public final static FunctionIdentifier NOT = new FunctionIdentifier(ALGEBRICKS_NS, "not", 1, true);
+ public final static FunctionIdentifier AND = new FunctionIdentifier(ALGEBRICKS_NS, "and",
+ FunctionIdentifier.VARARGS, true);
+ public final static FunctionIdentifier OR = new FunctionIdentifier(ALGEBRICKS_NS, "or", FunctionIdentifier.VARARGS,
+ true);
+
+ // numerics
+ public final static FunctionIdentifier NUMERIC_ADD = new FunctionIdentifier(ALGEBRICKS_NS, "numeric-add", 2, true);
+
+ // nulls
+ public final static FunctionIdentifier IS_NULL = new FunctionIdentifier(ALGEBRICKS_NS, "is-null", 1, true);
+
+ static {
+ // comparisons
+ add(EQ);
+ add(LE);
+ add(GE);
+ add(LT);
+ add(GT);
+ add(NEQ);
+ // booleans
+ add(NOT);
+ add(AND);
+ add(OR);
+ // numerics
+ add(NUMERIC_ADD);
+ // nulls
+ add(IS_NULL);
+ for (FunctionIdentifier fi : algebricksBuiltinFunctions.values()) {
+ _finfos.put(fi, new FunctionInfoImpl(fi));
+ }
+ }
+
+ private static void add(FunctionIdentifier fi) {
+ algebricksBuiltinFunctions.put(fi, fi);
+ }
+
+ public static final boolean isAlgebricksBuiltinFunction(FunctionIdentifier fi) {
+ return algebricksBuiltinFunctions.get(fi) != null;
+ }
+
+ public static final Collection<FunctionIdentifier> getAlgebricksBuiltinFunctions() {
+ return algebricksBuiltinFunctions.values();
+ }
+
+ public static final FunctionIdentifier getBuiltinFunctionIdentifier(FunctionIdentifier fi) {
+ return algebricksBuiltinFunctions.get(fi);
+ }
+
+ private static final Map<FunctionIdentifier, ComparisonKind> comparisonFunctions = new HashMap<FunctionIdentifier, ComparisonKind>();
+ static {
+ comparisonFunctions.put(AlgebricksBuiltinFunctions.EQ, ComparisonKind.EQ);
+ comparisonFunctions.put(AlgebricksBuiltinFunctions.LE, ComparisonKind.LE);
+ comparisonFunctions.put(AlgebricksBuiltinFunctions.GE, ComparisonKind.GE);
+ comparisonFunctions.put(AlgebricksBuiltinFunctions.LT, ComparisonKind.LT);
+ comparisonFunctions.put(AlgebricksBuiltinFunctions.GT, ComparisonKind.GT);
+ comparisonFunctions.put(AlgebricksBuiltinFunctions.NEQ, ComparisonKind.NEQ);
+ }
+
+ public static ComparisonKind getComparisonType(FunctionIdentifier fi) {
+ return comparisonFunctions.get(fi);
+ }
+
+ public static boolean isComparisonFunction(FunctionIdentifier fi) {
+ return comparisonFunctions.get(fi) != null;
+ }
+
+ public static IFunctionInfo getBuiltinFunctionInfo(FunctionIdentifier fi) {
+ return _finfos.get(fi);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/FunctionIdentifier.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/FunctionIdentifier.java
new file mode 100644
index 0000000..252f23c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/FunctionIdentifier.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.functions;
+
+public final class FunctionIdentifier {
+ final private String namespace;
+ final private String name;
+ final private boolean isBuiltin;
+ final private int arity;
+
+ public final static int VARARGS = -1;
+
+ public FunctionIdentifier(String namespace, String name, boolean isBuiltin) {
+ this.namespace = namespace;
+ this.name = name;
+ this.arity = VARARGS;
+ this.isBuiltin = isBuiltin;
+ }
+
+ public FunctionIdentifier(String namespace, String name, int arity, boolean isBuiltin) {
+ this.namespace = namespace;
+ this.name = name;
+ this.arity = arity;
+ this.isBuiltin = isBuiltin;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public boolean isBuiltin() {
+ return isBuiltin;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (super.equals(o)) {
+ return true;
+ }
+ if (o instanceof FunctionIdentifier) {
+ FunctionIdentifier ofi = (FunctionIdentifier) o;
+ return ofi.getNamespace().equals(getNamespace()) && ofi.name.equals(name);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return name.hashCode() + namespace.hashCode();
+ }
+
+ public String toString() {
+ return getNamespace() + ":" + name;
+ }
+
+ public int getArity() {
+ return arity;
+ }
+
+ public String getNamespace() {
+ return namespace;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/FunctionInfoImpl.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/FunctionInfoImpl.java
new file mode 100644
index 0000000..15521d6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/FunctionInfoImpl.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.functions;
+
+public class FunctionInfoImpl implements IFunctionInfo {
+
+ private final FunctionIdentifier fid;
+
+ public FunctionInfoImpl(FunctionIdentifier fid) {
+ this.fid = fid;
+ }
+
+ @Override
+ public FunctionIdentifier getFunctionIdentifier() {
+ return fid;
+ }
+
+ @Override
+ public Object getInfo() {
+ throw new IllegalStateException();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
new file mode 100644
index 0000000..57b7262
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.functions;
+
+public interface IFunctionInfo {
+ FunctionIdentifier getFunctionIdentifier();
+
+ Object getInfo();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSink.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSink.java
new file mode 100644
index 0000000..00b3554
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSink.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.metadata;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+
+public interface IDataSink {
+ public Object getId();
+
+ public Object[] getSchemaTypes();
+
+ public IPartitioningProperty getPartitioningProperty();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSource.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSource.java
new file mode 100644
index 0000000..22f9feb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSource.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.metadata;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+
+public interface IDataSource<T> {
+ public T getId();
+
+ public Object[] getSchemaTypes();
+
+ public IDataSourcePropertiesProvider getPropertiesProvider();
+
+ public void computeFDs(List<LogicalVariable> scanVariables, List<FunctionalDependency> fdList);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSourceIndex.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSourceIndex.java
new file mode 100644
index 0000000..90c8536
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSourceIndex.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.metadata;
+
+public interface IDataSourceIndex<I, T> {
+ public I getId();
+
+ public IDataSource<T> getDataSource();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSourcePropertiesProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSourcePropertiesProvider.java
new file mode 100644
index 0000000..500b968
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IDataSourcePropertiesProvider.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.metadata;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+
+public interface IDataSourcePropertiesProvider {
+ public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
new file mode 100644
index 0000000..221992e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.metadata;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public interface IMetadataProvider<S, I> {
+ public IDataSource<S> findDataSource(S id) throws AlgebricksException;
+
+ /**
+ * Obs: A scanner may choose to contribute a null
+ * AlgebricksPartitionConstraint and implement
+ * contributeSchedulingConstraints instead.
+ */
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
+
+ public boolean scannerOperatorIsLeaf(IDataSource<S> dataSource);
+
+ public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
+ int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
+ throws AlgebricksException;
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
+ IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
+ JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> dataSource,
+ IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> dataSource,
+ IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(
+ IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
+ List<LogicalVariable> secondaryKeys, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec) throws AlgebricksException;
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(
+ IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
+ List<LogicalVariable> secondaryKeys, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec) throws AlgebricksException;
+
+ public IDataSourceIndex<I, S> findDataSourceIndex(I indexId, S dataSourceId) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
new file mode 100644
index 0000000..438386f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+/**
+ * @author Nicola
+ *
+ */
+public abstract class AbstractAssignOperator extends AbstractLogicalOperator {
+ protected final List<LogicalVariable> variables;
+ protected final List<LogicalExpressionReference> expressions;
+
+ public AbstractAssignOperator() {
+ this.variables = new ArrayList<LogicalVariable>();
+ this.expressions = new ArrayList<LogicalExpressionReference>();
+ }
+
+ public AbstractAssignOperator(List<LogicalVariable> variables,
+ List<LogicalExpressionReference> expressions) {
+ this.variables = variables;
+ this.expressions = expressions;
+ }
+
+ public List<LogicalVariable> getVariables() {
+ return variables;
+ }
+
+ public List<LogicalExpressionReference> getExpressions() {
+ return expressions;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ schema.addAll(variables);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean modif = false;
+ for (int i = 0; i < expressions.size(); i++) {
+ if (visitor.transform(expressions.get(i))) {
+ modif = true;
+ }
+ }
+ return modif;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractBinaryJoinOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractBinaryJoinOperator.java
new file mode 100644
index 0000000..a6eb30f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractBinaryJoinOperator.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class AbstractBinaryJoinOperator extends AbstractLogicalOperator {
+ protected final LogicalExpressionReference condition;
+ protected JoinKind joinKind;
+
+ public enum JoinKind {
+ INNER, LEFT_OUTER
+ }
+
+ public AbstractBinaryJoinOperator(JoinKind joinKind, LogicalExpressionReference condition) {
+ this.joinKind = joinKind;
+ this.condition = condition;
+ }
+
+ public AbstractBinaryJoinOperator(JoinKind joinKind, LogicalExpressionReference condition, LogicalOperatorReference input1,
+ LogicalOperatorReference input2) {
+ this(joinKind, condition);
+ inputs.add(input1);
+ inputs.add(input2);
+ }
+
+ public LogicalExpressionReference getCondition() {
+ return condition;
+ }
+
+ public JoinKind getJoinKind() {
+ return joinKind;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ schema.addAll(inputs.get(1).getOperator().getSchema());
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.concat(VariablePropagationPolicy.ALL, VariablePropagationPolicy.ALL);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ return visitor.transform(condition);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
new file mode 100644
index 0000000..9b1931d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
@@ -0,0 +1,193 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class AbstractLogicalOperator implements ILogicalOperator {
+
+ /*********************************************************************
+ * UNPARTITIONED, the input data is not partitioned
+ * PARTITIONED, the input data is partitioned, the operator is executed on
+ * each partition and may receive input from other partitions (e.g. if it is
+ * a join or an aggregate)
+ * LOCAL, the input data is partitioned, the operator is executed on each
+ * partition and only processes data from that partition
+ */
+
+ public static enum ExecutionMode {
+ UNPARTITIONED,
+ PARTITIONED,
+ LOCAL
+ }
+
+ private AbstractLogicalOperator.ExecutionMode mode = AbstractLogicalOperator.ExecutionMode.UNPARTITIONED;
+ protected IPhysicalOperator physicalOperator;
+ private final Map<String, Object> annotations = new HashMap<String, Object>();
+ private boolean bJobGenEnabled = true;
+ private Object hostQueryContext;
+
+ final protected List<LogicalOperatorReference> inputs;
+ // protected List<LogicalOperatorReference> outputs;
+ protected List<LogicalVariable> schema;
+
+ public AbstractLogicalOperator() {
+ inputs = new ArrayList<LogicalOperatorReference>();
+ // outputs = new ArrayList<LogicalOperatorReference>();
+ }
+
+ public abstract LogicalOperatorTag getOperatorTag();
+
+ public ExecutionMode getExecutionMode() {
+ return mode;
+ }
+
+ public void setExecutionMode(ExecutionMode mode) {
+ this.mode = mode;
+ }
+
+ @Override
+ public List<LogicalVariable> getSchema() {
+ return schema;
+ }
+
+ public void setPhysicalOperator(IPhysicalOperator physicalOp) {
+ this.physicalOperator = physicalOp;
+ }
+
+ public IPhysicalOperator getPhysicalOperator() {
+ return physicalOperator;
+ }
+
+ /**
+ * @return for each child, one vector of required physical properties
+ */
+
+ @Override
+ public final PhysicalRequirements getRequiredPhysicalPropertiesForChildren(
+ IPhysicalPropertiesVector requiredProperties) {
+ return physicalOperator.getRequiredPropertiesForChildren(this, requiredProperties);
+ }
+
+ /**
+ * @return the physical properties that this operator delivers, based on
+ * what its children deliver
+ */
+
+ @Override
+ public final IPhysicalPropertiesVector getDeliveredPhysicalProperties() {
+ return physicalOperator.getDeliveredProperties();
+ }
+
+ @Override
+ public final void computeDeliveredPhysicalProperties(IOptimizationContext context) throws AlgebricksException {
+ physicalOperator.computeDeliveredProperties(this, context);
+ }
+
+ @Override
+ public final List<LogicalOperatorReference> getInputs() {
+ return inputs;
+ }
+
+ // @Override
+ // public final List<LogicalOperatorReference> getOutputs() {
+ // return outputs;
+ // }
+
+ @Override
+ public final boolean hasInputs() {
+ return !inputs.isEmpty();
+ }
+
+ public boolean hasNestedPlans() {
+ return false;
+ }
+
+ @Override
+ public Map<String, Object> getAnnotations() {
+ return annotations;
+ }
+
+ @Override
+ public void removeAnnotation(String annotationName) {
+ annotations.remove(annotationName);
+ }
+
+ @Override
+ public final void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ if (bJobGenEnabled) {
+ if (physicalOperator == null) {
+ throw new AlgebricksException("Physical operator not set for operator: " + this);
+ }
+ physicalOperator.contributeRuntimeOperator(builder, context, this, propagatedSchema, inputSchemas,
+ outerPlanSchema);
+ }
+ }
+
+ public void disableJobGen() {
+ bJobGenEnabled = false;
+ }
+
+ public boolean isJobGenEnabled() {
+ return bJobGenEnabled;
+ }
+
+ public void setHostQueryContext(Object context) {
+ this.hostQueryContext = context;
+ }
+
+ public Object getHostQueryContext() {
+ return hostQueryContext;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeInputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+ protected IVariableTypeEnvironment createPropagatingAllInputsTypeEnvironment(ITypingContext ctx) {
+ // return createPropagatingAllInputsTypeEnvironment(ctx);
+ int n = inputs.size();
+ ITypeEnvPointer[] envPointers = new ITypeEnvPointer[n];
+ for (int i = 0; i < n; i++) {
+ envPointers[i] = new OpRefTypeEnvPointer(inputs.get(i), ctx);
+ }
+ return new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getNullableTypeComputer(),
+ ctx.getMetadataProvider(), TypePropagationPolicy.ALL, envPointers);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractOperatorWithNestedPlans.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractOperatorWithNestedPlans.java
new file mode 100644
index 0000000..ec4e6ef
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractOperatorWithNestedPlans.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public abstract class AbstractOperatorWithNestedPlans extends AbstractLogicalOperator {
+ protected final List<ILogicalPlan> nestedPlans;
+
+ public AbstractOperatorWithNestedPlans() {
+ nestedPlans = new ArrayList<ILogicalPlan>();
+ }
+
+ public AbstractOperatorWithNestedPlans(List<ILogicalPlan> nestedPlans) {
+ this.nestedPlans = nestedPlans;
+ }
+
+ public List<ILogicalPlan> getNestedPlans() {
+ return nestedPlans;
+ }
+
+ @Override
+ public boolean hasNestedPlans() {
+ return true;
+ }
+
+ public LinkedList<LogicalOperatorReference> allRootsInReverseOrder() {
+ LinkedList<LogicalOperatorReference> allRoots = new LinkedList<LogicalOperatorReference>();
+ for (ILogicalPlan p : nestedPlans) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ allRoots.addFirst(r);
+ }
+ }
+ return allRoots;
+ }
+
+ //
+ // @Override
+ // public void computeConstraintsAndEquivClasses() {
+ // for (ILogicalPlan p : nestedPlans) {
+ // for (LogicalOperatorReference r : p.getRoots()) {
+ // AbstractLogicalOperator op = (AbstractLogicalOperator) r.getOperator();
+ // equivalenceClasses.putAll(op.getEquivalenceClasses());
+ // functionalDependencies.addAll(op.getFDs());
+ // }
+ // }
+ // }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ for (ILogicalPlan p : nestedPlans) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ schema.addAll(r.getOperator().getSchema());
+ }
+ }
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ public abstract void getUsedVariablesExceptNestedPlans(Collection<LogicalVariable> vars);
+
+ public abstract void getProducedVariablesExceptNestedPlans(Collection<LogicalVariable> vars);
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
new file mode 100644
index 0000000..e5132ef
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class AbstractScanOperator extends AbstractLogicalOperator {
+ protected List<LogicalVariable> variables;
+
+ public AbstractScanOperator(List<LogicalVariable> variables) {
+ this.variables = variables;
+ }
+
+ public List<LogicalVariable> getVariables() {
+ return variables;
+ }
+
+ public void setVariables(List<LogicalVariable> variables) {
+ this.variables = variables;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ schema.addAll(variables);
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ if (sources.length > 0) {
+ target.addAllVariables(sources[0]);
+ }
+ for (LogicalVariable v : variables) {
+ target.addVariable(v);
+ }
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestOperator.java
new file mode 100644
index 0000000..020d83c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestOperator.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class AbstractUnnestOperator extends AbstractScanOperator {
+
+ protected final LogicalExpressionReference expression;
+
+ public AbstractUnnestOperator(List<LogicalVariable> variables, LogicalExpressionReference expression) {
+ super(variables);
+ this.expression = expression;
+ }
+
+ public LogicalExpressionReference getExpressionRef() {
+ return expression;
+ }
+
+ @Override
+ public boolean isMap() {
+ return true;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ return visitor.transform(expression);
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
new file mode 100644
index 0000000..ba7c970
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
@@ -0,0 +1,82 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class AggregateOperator extends AbstractAssignOperator {
+
+ // private ArrayList<AggregateFunctionCallExpression> expressions;
+ // TODO type safe list of expressions
+ private List<LogicalExpressionReference> mergeExpressions;
+
+ public AggregateOperator(List<LogicalVariable> variables, List<LogicalExpressionReference> expressions) {
+ super(variables, expressions);
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.AGGREGATE;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitAggregateOperator(this, arg);
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ for (LogicalVariable v : variables) {
+ target.addVariable(v);
+ }
+ }
+ };
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(variables);
+ }
+
+ public void setMergeExpressions(List<LogicalExpressionReference> merges) {
+ mergeExpressions = merges;
+ }
+
+ public List<LogicalExpressionReference> getMergeExpressions() {
+ return mergeExpressions;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx
+ .getMetadataProvider());
+ IVariableTypeEnvironment env2 = ctx.getOutputTypeEnvironment(inputs.get(0).getOperator());
+ int n = variables.size();
+ for (int i = 0; i < n; i++) {
+ Object t = ctx.getExpressionTypeComputer().getType(expressions.get(i).getExpression(), ctx.getMetadataProvider(),
+ env2);
+ env.setVarType(variables.get(i), t);
+ }
+ return env;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
new file mode 100644
index 0000000..db57c4b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+/**
+ *
+ * It corresponds to the Map operator in other algebras.
+ *
+ * @author Nicola
+ *
+ */
+
+public class AssignOperator extends AbstractAssignOperator {
+
+ public AssignOperator(List<LogicalVariable> vars, List<LogicalExpressionReference> exprs) {
+ super(vars, exprs);
+ }
+
+ public AssignOperator(LogicalVariable var, LogicalExpressionReference expr) {
+ super();
+ this.variables.add(var);
+ this.expressions.add(expr);
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.ASSIGN;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitAssignOperator(this, arg);
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ target.addAllVariables(sources[0]);
+ for (LogicalVariable v : variables) {
+ target.addVariable(v);
+ }
+ }
+ };
+
+ }
+
+ @Override
+ public boolean isMap() {
+ return true;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
+ int n = variables.size();
+ for (int i = 0; i < n; i++) {
+ env.setVarType(variables.get(i), ctx.getExpressionTypeComputer().getType(
+ expressions.get(i).getExpression(), ctx.getMetadataProvider(), env));
+ }
+ return env;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
new file mode 100644
index 0000000..bcc1192
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class DataSourceScanOperator extends AbstractScanOperator {
+ private IDataSource<?> dataSource;
+
+ private List<LogicalVariable> projectVars;
+
+ private boolean projectPushed = false;
+
+ public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource) {
+ super(variables);
+ this.dataSource = dataSource;
+ projectVars = new ArrayList<LogicalVariable>();
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.DATASOURCESCAN;
+ }
+
+ public IDataSource<?> getDataSource() {
+ return dataSource;
+ }
+
+ @Override
+ public <R, S> R accept(ILogicalOperatorVisitor<R, S> visitor, S arg) throws AlgebricksException {
+ return visitor.visitDataScanOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ public void addProjectVariables(Collection<LogicalVariable> vars) {
+ projectVars.addAll(vars);
+ projectPushed = true;
+ }
+
+ public List<LogicalVariable> getProjectVariables() {
+ return projectVars;
+ }
+
+ public boolean isProjectPushed() {
+ return projectPushed;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ if (sources.length > 0) {
+ target.addAllVariables(sources[0]);
+ }
+ List<LogicalVariable> outputVariables = projectPushed ? projectVars : variables;
+ for (LogicalVariable v : outputVariables) {
+ target.addVariable(v);
+ }
+ }
+ };
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
+ Object[] types = dataSource.getSchemaTypes();
+ int i = 0;
+ for (LogicalVariable v : variables) {
+ env.setVarType(v, types[i]);
+ ++i;
+ }
+ return env;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DieOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DieOperator.java
new file mode 100644
index 0000000..fe22219
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DieOperator.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class DieOperator extends AbstractLogicalOperator {
+
+ private final LogicalExpressionReference afterObjects; // mandatory
+
+ public DieOperator(ILogicalExpression maxObjectsExpr) {
+ this.afterObjects = new LogicalExpressionReference(maxObjectsExpr);
+ }
+
+ public LogicalExpressionReference getAfterObjects() {
+ return afterObjects;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitDieOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = false;
+ if (visitor.transform(afterObjects)) {
+ b = true;
+ }
+ return b;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.DIE;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean isMap() {
+ return true;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
new file mode 100644
index 0000000..593e505
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class DistinctOperator extends AbstractLogicalOperator {
+ private final List<LogicalExpressionReference> expressions;
+
+ public DistinctOperator(List<LogicalExpressionReference> expressions) {
+ this.expressions = expressions;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.DISTINCT;
+ }
+
+ public List<LogicalExpressionReference> getExpressions() {
+ return expressions;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean changed = false;
+ for (LogicalExpressionReference e : expressions) {
+ if (visitor.transform(e)) {
+ changed = true;
+ }
+ }
+ return changed;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitDistinctOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ public List<LogicalVariable> getDistinctByVarList() {
+ List<LogicalVariable> varList = new ArrayList<LogicalVariable>(expressions.size());
+ for (LogicalExpressionReference eRef : expressions) {
+ ILogicalExpression e = eRef.getExpression();
+ if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression v = (VariableReferenceExpression) e;
+ varList.add(v.getVariableReference());
+ }
+ }
+ return varList;
+ }
+
+ public boolean isDistinctByVar(LogicalVariable var) {
+ for (LogicalExpressionReference eRef : expressions) {
+ ILogicalExpression e = eRef.getExpression();
+ if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression v = (VariableReferenceExpression) e;
+ if (v.getVariableReference() == var) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
new file mode 100644
index 0000000..732dfbe
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class EmptyTupleSourceOperator extends AbstractLogicalOperator {
+
+ // public final static EmptyTupleSourceOperator INSTANCE = new
+ // EmptyTupleSourceOperator();
+
+ public EmptyTupleSourceOperator() {
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.EMPTYTUPLESOURCE;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.NONE;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) {
+ // do nothing
+ return false;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitEmptyTupleSourceOperator(this, arg);
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(final ITypingContext ctx) throws AlgebricksException {
+ return new IVariableTypeEnvironment() {
+
+ @Override
+ public void setVarType(LogicalVariable var, Object type) {
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Object getType(ILogicalExpression expr) throws AlgebricksException {
+ return ctx.getExpressionTypeComputer().getType(expr, ctx.getMetadataProvider(), this);
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables)
+ throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public boolean substituteProducedVariable(LogicalVariable v1, LogicalVariable v2)
+ throws AlgebricksException {
+ return false;
+ }
+ };
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExchangeOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExchangeOperator.java
new file mode 100644
index 0000000..f0899c9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExchangeOperator.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class ExchangeOperator extends AbstractLogicalOperator {
+
+ public ExchangeOperator() {
+ super();
+ setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitExchangeOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) {
+ // do nothing
+ return false;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.EXCHANGE;
+ }
+
+ @Override
+ public void recomputeSchema() throws AlgebricksException {
+ AbstractLogicalOperator cld = (AbstractLogicalOperator) inputs.get(0).getOperator();
+ OperatorPropertiesUtil.computeSchemaRecIfNull(cld);
+ List<LogicalVariable> inputSchema = inputs.get(0).getOperator().getSchema();
+ schema = new ArrayList<LogicalVariable>(inputSchema);
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean isMap() {
+ return true;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/GroupByOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/GroupByOperator.java
new file mode 100644
index 0000000..9bc37d4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/GroupByOperator.java
@@ -0,0 +1,276 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class GroupByOperator extends AbstractOperatorWithNestedPlans {
+ // If the LogicalVariable in a pair is null, it means that the GroupBy is
+ // only grouping by the expression, without producing a new variable.
+ private final List<Pair<LogicalVariable, LogicalExpressionReference>> gByList;
+ private final List<Pair<LogicalVariable, LogicalExpressionReference>> decorList;
+
+ // In decorList, if the variable (first member of the pair) is null, the
+ // second member of the pair is variable reference which is propagated.
+
+ public GroupByOperator() {
+ super();
+ gByList = new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>();
+ decorList = new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>();
+ }
+
+ public GroupByOperator(List<Pair<LogicalVariable, LogicalExpressionReference>> groupByList,
+ List<Pair<LogicalVariable, LogicalExpressionReference>> decorList, List<ILogicalPlan> nestedPlans) {
+ super(nestedPlans);
+ this.decorList = decorList;
+ this.gByList = groupByList;
+ }
+
+ public void addGbyExpression(LogicalVariable variable, ILogicalExpression expression) {
+ this.gByList.add(new Pair<LogicalVariable, LogicalExpressionReference>(variable,
+ new LogicalExpressionReference(expression)));
+ }
+
+ public void addDecorExpression(LogicalVariable variable, ILogicalExpression expression) {
+ this.decorList.add(new Pair<LogicalVariable, LogicalExpressionReference>(variable,
+ new LogicalExpressionReference(expression)));
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.GROUP;
+ }
+
+ public List<Pair<LogicalVariable, LogicalExpressionReference>> getGroupByList() {
+ return gByList;
+ }
+
+ public String gByListToString() {
+ return veListToString(gByList);
+ }
+
+ public String decorListToString() {
+ return veListToString(decorList);
+ }
+
+ public List<LogicalVariable> getGbyVarList() {
+ List<LogicalVariable> varList = new ArrayList<LogicalVariable>(gByList.size());
+ for (Pair<LogicalVariable, LogicalExpressionReference> ve : gByList) {
+ ILogicalExpression expr = ve.second.getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ varList.add(v.getVariableReference());
+ }
+ }
+ return varList;
+ }
+
+ public static String veListToString(List<Pair<LogicalVariable, LogicalExpressionReference>> vePairList) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ boolean fst = true;
+ for (Pair<LogicalVariable, LogicalExpressionReference> ve : vePairList) {
+ if (fst) {
+ fst = false;
+ } else {
+ sb.append("; ");
+ }
+ if (ve.first != null) {
+ sb.append(ve.first + " := " + ve.second);
+ } else {
+ sb.append(ve.second.getExpression());
+ }
+ }
+ sb.append("]");
+ return sb.toString();
+ }
+
+ @Override
+ public void recomputeSchema() {
+ super.recomputeSchema();
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gByList) {
+ schema.add(p.first);
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : decorList) {
+ schema.add(getDecorVariable(p));
+ }
+ }
+
+ @Override
+ public void getProducedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+ // super.getProducedVariables(vars);
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gByList) {
+ if (p.first != null) {
+ vars.add(p.first);
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : decorList) {
+ if (p.first != null) {
+ vars.add(p.first);
+ }
+ }
+ }
+
+ @Override
+ public void getUsedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> g : gByList) {
+ g.second.getExpression().getUsedVariables(vars);
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> g : decorList) {
+ g.second.getExpression().getUsedVariables(vars);
+ }
+ // super.getUsedVariables(vars);
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gByList) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (p.first != null) {
+ target.addVariable(p.first);
+ } else {
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new AlgebricksException("hash group-by expects variable references.");
+ }
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ target.addVariable(v.getVariableReference());
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : decorList) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new AlgebricksException("pre-sorted group-by expects variable references.");
+ }
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ LogicalVariable decor = v.getVariableReference();
+ if (p.first != null) {
+ target.addVariable(p.first);
+ } else {
+ target.addVariable(decor);
+ }
+ }
+
+ }
+ };
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = false;
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gByList) {
+ if (visitor.transform(p.second)) {
+ b = true;
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : decorList) {
+ if (visitor.transform(p.second)) {
+ b = true;
+ }
+ }
+ return b;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitGroupByOperator(this, arg);
+ }
+
+ public static LogicalVariable getDecorVariable(Pair<LogicalVariable, LogicalExpressionReference> p) {
+ if (p.first != null) {
+ return p.first;
+ } else {
+ VariableReferenceExpression e = (VariableReferenceExpression) p.second.getExpression();
+ return e.getVariableReference();
+ }
+ }
+
+ public List<Pair<LogicalVariable, LogicalExpressionReference>> getDecorList() {
+ return decorList;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ int n = 0;
+ for (ILogicalPlan p : nestedPlans) {
+ n += p.getRoots().size();
+ }
+ ITypeEnvPointer[] envPointers = new ITypeEnvPointer[n];
+ int i = 0;
+ for (ILogicalPlan p : nestedPlans) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ envPointers[i] = new OpRefTypeEnvPointer(r, ctx);
+ i++;
+ }
+ }
+ IVariableTypeEnvironment env = new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx
+ .getNullableTypeComputer(), ctx.getMetadataProvider(), TypePropagationPolicy.ALL, envPointers);
+ ILogicalOperator child = inputs.get(0).getOperator();
+ IVariableTypeEnvironment env2 = ctx.getOutputTypeEnvironment(child);
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : getGroupByList()) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (p.first != null) {
+ env.setVarType(p.first, env2.getType(expr));
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v1 = ((VariableReferenceExpression) expr).getVariableReference();
+ env.setVarType(v1, env2.getVarType(v1));
+ }
+ } else {
+ VariableReferenceExpression vre = (VariableReferenceExpression) p.second.getExpression();
+ LogicalVariable v2 = vre.getVariableReference();
+ env.setVarType(v2, env2.getVarType(v2));
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : getDecorList()) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (p.first != null) {
+ env.setVarType(p.first, env2.getType(expr));
+ } else {
+ VariableReferenceExpression vre = (VariableReferenceExpression) p.second.getExpression();
+ LogicalVariable v2 = vre.getVariableReference();
+ env.setVarType(v2, env2.getVarType(v2));
+ }
+ }
+ return env;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IOperatorSchema.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IOperatorSchema.java
new file mode 100644
index 0000000..043ac8e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IOperatorSchema.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public interface IOperatorSchema extends Iterable<LogicalVariable> {
+ public void addAllVariables(IOperatorSchema source);
+
+ public void addAllNewVariables(IOperatorSchema source);
+
+ public int addVariable(LogicalVariable var);
+
+ public int findVariable(LogicalVariable var);
+
+ public LogicalVariable getVariable(int index);
+
+ public int getSize();
+
+ public void clear();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
new file mode 100644
index 0000000..ca032a2
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
@@ -0,0 +1,97 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class IndexInsertDeleteOperator extends AbstractLogicalOperator {
+
+ private final IDataSourceIndex<?, ?> dataSourceIndex;
+ private final List<LogicalExpressionReference> primaryKeyExprs;
+ private final List<LogicalExpressionReference> secondaryKeyExprs;
+ private final Kind operation;
+
+ public IndexInsertDeleteOperator(IDataSourceIndex<?, ?> dataSourceIndex,
+ List<LogicalExpressionReference> primaryKeyExprs, List<LogicalExpressionReference> secondaryKeyExprs,
+ Kind operation) {
+ this.dataSourceIndex = dataSourceIndex;
+ this.primaryKeyExprs = primaryKeyExprs;
+ this.secondaryKeyExprs = secondaryKeyExprs;
+ this.operation = operation;
+ }
+
+ @Override
+ public void recomputeSchema() throws AlgebricksException {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = false;
+ for (int i = 0; i < primaryKeyExprs.size(); i++) {
+ if (visitor.transform(primaryKeyExprs.get(i))) {
+ b = true;
+ }
+ }
+ for (int i = 0; i < secondaryKeyExprs.size(); i++) {
+ if (visitor.transform(secondaryKeyExprs.get(i))) {
+ b = true;
+ }
+ }
+ return b;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitIndexInsertDeleteOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.INDEX_INSERT_DELETE;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+ public List<LogicalExpressionReference> getPrimaryKeyExpressions() {
+ return primaryKeyExprs;
+ }
+
+ public IDataSourceIndex<?, ?> getDataSourceIndex() {
+ return dataSourceIndex;
+ }
+
+ public List<LogicalExpressionReference> getSecondaryKeyExpressions() {
+ return secondaryKeyExprs;
+ }
+
+ public Kind getOperation() {
+ return operation;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/InnerJoinOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/InnerJoinOperator.java
new file mode 100644
index 0000000..e9d8c2c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/InnerJoinOperator.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class InnerJoinOperator extends AbstractBinaryJoinOperator {
+
+ public InnerJoinOperator(LogicalExpressionReference condition) {
+ super(JoinKind.INNER, condition);
+ }
+
+ public InnerJoinOperator(LogicalExpressionReference condition, LogicalOperatorReference input1,
+ LogicalOperatorReference input2) {
+ super(JoinKind.INNER, condition, input1, input2);
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.INNERJOIN;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitInnerJoinOperator(this, arg);
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
new file mode 100644
index 0000000..be69e96
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
@@ -0,0 +1,95 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class InsertDeleteOperator extends AbstractLogicalOperator {
+
+ public enum Kind {
+ INSERT, DELETE
+ }
+
+ private final IDataSource<?> dataSource;
+ private final LogicalExpressionReference payloadExpr;
+ private final List<LogicalExpressionReference> primaryKeyExprs;
+ private final Kind operation;
+
+ public InsertDeleteOperator(IDataSource<?> dataSource, LogicalExpressionReference payload,
+ List<LogicalExpressionReference> primaryKeyExprs, Kind operation) {
+ this.dataSource = dataSource;
+ this.payloadExpr = payload;
+ this.primaryKeyExprs = primaryKeyExprs;
+ this.operation = operation;
+ }
+
+ @Override
+ public void recomputeSchema() throws AlgebricksException {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = false;
+ b = visitor.transform(payloadExpr);
+ for (int i = 0; i < primaryKeyExprs.size(); i++) {
+ if (visitor.transform(primaryKeyExprs.get(i))) {
+ b = true;
+ }
+ }
+ return b;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitInsertDeleteOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.INSERT_DELETE;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+ public List<LogicalExpressionReference> getPrimaryKeyExpressions() {
+ return primaryKeyExprs;
+ }
+
+ public IDataSource<?> getDataSource() {
+ return dataSource;
+ }
+
+ public LogicalExpressionReference getPayloadExpression() {
+ return payloadExpr;
+ }
+
+ public Kind getOperation() {
+ return operation;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
new file mode 100644
index 0000000..8166b1b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class LeftOuterJoinOperator extends AbstractBinaryJoinOperator {
+
+ public LeftOuterJoinOperator(LogicalExpressionReference condition) {
+ super(JoinKind.LEFT_OUTER, condition);
+ }
+
+ public LeftOuterJoinOperator(LogicalExpressionReference condition, LogicalOperatorReference input1,
+ LogicalOperatorReference input2) {
+ super(JoinKind.LEFT_OUTER, condition, input1, input2);
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.LEFTOUTERJOIN;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitLeftOuterJoinOperator(this, arg);
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ int n = inputs.size();
+ ITypeEnvPointer[] envPointers = new ITypeEnvPointer[n];
+ for (int i = 0; i < n; i++) {
+ envPointers[i] = new OpRefTypeEnvPointer(inputs.get(i), ctx);
+ }
+ return new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getNullableTypeComputer(), ctx
+ .getMetadataProvider(), TypePropagationPolicy.LEFT_OUTER, envPointers);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LimitOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LimitOperator.java
new file mode 100644
index 0000000..5fedfdc
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/LimitOperator.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class LimitOperator extends AbstractLogicalOperator {
+
+ private final LogicalExpressionReference maxObjects; // mandatory
+ private final LogicalExpressionReference offset; // optional
+ private boolean topmost;
+
+ public LimitOperator(ILogicalExpression maxObjectsExpr, ILogicalExpression offsetExpr, boolean topmost) {
+ this.maxObjects = new LogicalExpressionReference(maxObjectsExpr);
+ this.offset = new LogicalExpressionReference(offsetExpr);
+ this.topmost = topmost;
+ }
+
+ public LimitOperator(ILogicalExpression maxObjectsExpr, boolean topmost) {
+ this(maxObjectsExpr, null, topmost);
+ }
+
+ public LimitOperator(ILogicalExpression maxObjects, ILogicalExpression offset) {
+ this(maxObjects, offset, true);
+ }
+
+ public LimitOperator(ILogicalExpression maxObjects) {
+ this(maxObjects, null, true);
+ }
+
+ public LogicalExpressionReference getMaxObjects() {
+ return maxObjects;
+ }
+
+ public LogicalExpressionReference getOffset() {
+ return offset;
+ }
+
+ public boolean isTopmostLimitOp() {
+ return topmost;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitLimitOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = false;
+ if (visitor.transform(maxObjects)) {
+ b = true;
+ }
+ if (offset.getExpression() != null) {
+ if (visitor.transform(offset)) {
+ b = true;
+ }
+ }
+ return b;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.LIMIT;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean isMap() {
+ return true;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/NestedTupleSourceOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/NestedTupleSourceOperator.java
new file mode 100644
index 0000000..fc17ce4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/NestedTupleSourceOperator.java
@@ -0,0 +1,103 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class NestedTupleSourceOperator extends AbstractLogicalOperator {
+ private final LogicalOperatorReference dataSourceReference;
+
+ public NestedTupleSourceOperator(LogicalOperatorReference dataSourceReference) {
+ this.dataSourceReference = dataSourceReference;
+ }
+
+ public ILogicalOperator getSourceOperator() {
+ return dataSourceReference.getOperator().getInputs().get(0).getOperator();
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.NESTEDTUPLESOURCE;
+ }
+
+ public LogicalOperatorReference getDataSourceReference() {
+ return dataSourceReference;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ ILogicalOperator topOp = dataSourceReference.getOperator();
+ for (LogicalOperatorReference i : topOp.getInputs()) {
+ schema.addAll(i.getOperator().getSchema());
+ }
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) {
+ // do nothing
+ return false;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitNestedTupleSourceOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(final ITypingContext ctx) throws AlgebricksException {
+ ITypeEnvPointer[] p = new ITypeEnvPointer[1];
+ p[0] = new ITypeEnvPointer() {
+
+ @Override
+ public IVariableTypeEnvironment getTypeEnv() {
+ ILogicalOperator op = dataSourceReference.getOperator().getInputs().get(0).getOperator();
+ return ctx.getOutputTypeEnvironment(op);
+ }
+ };
+ return new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getNullableTypeComputer(),
+ ctx.getMetadataProvider(), TypePropagationPolicy.ALL, p);
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeInputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return computeOutputTypeEnvironment(ctx);
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/OrderOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/OrderOperator.java
new file mode 100644
index 0000000..9c3cef5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/OrderOperator.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class OrderOperator extends AbstractLogicalOperator {
+
+ public interface IOrder {
+ public enum OrderKind {
+ FUNCTIONCALL,
+ ASC,
+ DESC
+ };
+
+ public LogicalExpressionReference getExpressionRef();
+
+ public OrderKind getKind();
+ }
+
+ public static IOrder ASC_ORDER = new IOrder() {
+
+ @Override
+ public LogicalExpressionReference getExpressionRef() {
+ return null;
+ }
+
+ @Override
+ public OrderKind getKind() {
+ return OrderKind.ASC;
+ }
+
+ };
+
+ public static IOrder DESC_ORDER = new IOrder() {
+
+ @Override
+ public LogicalExpressionReference getExpressionRef() {
+ return null;
+ }
+
+ @Override
+ public OrderKind getKind() {
+ return OrderKind.DESC;
+ }
+ };
+
+ public class FunOrder implements IOrder {
+ private final LogicalExpressionReference f;
+
+ public FunOrder(LogicalExpressionReference f) {
+ this.f = f;
+ }
+
+ @Override
+ public LogicalExpressionReference getExpressionRef() {
+ return f;
+ }
+
+ @Override
+ public OrderKind getKind() {
+ return OrderKind.FUNCTIONCALL;
+ }
+
+ };
+
+ private final List<Pair<IOrder, LogicalExpressionReference>> orderExpressions;
+
+ // These are pairs of type (comparison, expr) where comparison is
+ // ASC or DESC or a boolean function of arity 2 that can take as
+ // arguments results of expr.
+
+ public OrderOperator() {
+ orderExpressions = new ArrayList<Pair<IOrder, LogicalExpressionReference>>();
+ }
+
+ public OrderOperator(List<Pair<IOrder, LogicalExpressionReference>> orderExpressions) {
+ this.orderExpressions = orderExpressions;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.ORDER;
+ }
+
+ public List<Pair<IOrder, LogicalExpressionReference>> getOrderExpressions() {
+ return orderExpressions;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = false;
+ for (Pair<IOrder, LogicalExpressionReference> p : orderExpressions) {
+ if (p.first.getKind() == OrderKind.FUNCTIONCALL) {
+ FunOrder fo = (FunOrder) p.first;
+ LogicalExpressionReference r1 = fo.getExpressionRef();
+ if (visitor.transform(r1)) {
+ b = true;
+ }
+ }
+ if (visitor.transform(p.second)) {
+ b = true;
+ }
+ }
+ return b;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitOrderOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/PartitioningSplitOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/PartitioningSplitOperator.java
new file mode 100644
index 0000000..dc0e4df
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/PartitioningSplitOperator.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class PartitioningSplitOperator extends AbstractLogicalOperator {
+
+ private LogicalExpressionReference[] expressions;
+ private boolean hasDefault;
+
+ public PartitioningSplitOperator(LogicalExpressionReference[] exprList, boolean defaultBranchDefined) {
+ expressions = exprList;
+ hasDefault = defaultBranchDefined;
+ }
+
+ public LogicalExpressionReference[] getExpressions() {
+ return expressions;
+ }
+
+ public boolean hasDefault() {
+ return hasDefault;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.PARTITIONINGSPLIT;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = false;
+ for (int i = 0; i < expressions.length; i++) {
+ if (visitor.transform(expressions[i])) {
+ b = true;
+ }
+ }
+ return b;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitPartitioningSplitOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ProjectOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ProjectOperator.java
new file mode 100644
index 0000000..736ae1d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ProjectOperator.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FilteredVariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class ProjectOperator extends AbstractLogicalOperator {
+
+ private final List<LogicalVariable> variables;
+
+ public ProjectOperator(List<LogicalVariable> variables) {
+ this.variables = variables;
+ }
+
+ public ProjectOperator(LogicalVariable v) {
+ this.variables = new ArrayList<LogicalVariable>(1);
+ this.getVariables().add(v);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) {
+ // do nothing
+ return false;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.PROJECT;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>(getVariables());
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new FilteredVariablePropagationPolicy(getVariables());
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitProjectOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return true;
+ }
+
+ public List<LogicalVariable> getVariables() {
+ return variables;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java
new file mode 100644
index 0000000..0f0c624
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class ReplicateOperator extends AbstractLogicalOperator {
+
+ private int outputArity = 2;
+
+ public ReplicateOperator(int outputArity) {
+ this.outputArity = outputArity;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.REPLICATE;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitReplicateOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>(inputs.get(0).getOperator().getSchema());
+ }
+
+ public void substituteVar(LogicalVariable v1, LogicalVariable v2) {
+ // do nothing
+ }
+
+ public int getOutputArity() {
+ return outputArity;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java
new file mode 100644
index 0000000..d9be7d4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class RunningAggregateOperator extends AbstractAssignOperator {
+
+ public RunningAggregateOperator(List<LogicalVariable> variables, List<LogicalExpressionReference> expressions) {
+ super(variables, expressions);
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.RUNNINGAGGREGATE;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitRunningAggregateOperator(this, arg);
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ target.addAllVariables(sources[0]);
+ for (LogicalVariable v : variables) {
+ target.addVariable(v);
+ }
+ }
+ };
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
+ int n = variables.size();
+ for (int i = 0; i < n; i++) {
+ env.setVarType(
+ variables.get(i),
+ ctx.getExpressionTypeComputer().getType(expressions.get(i).getExpression(),
+ ctx.getMetadataProvider(), env));
+ }
+ return env;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ScriptOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ScriptOperator.java
new file mode 100644
index 0000000..dc2795c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ScriptOperator.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.scripting.IScriptDescription;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class ScriptOperator extends AbstractLogicalOperator {
+
+ private ArrayList<LogicalVariable> inputVariables;
+ private ArrayList<LogicalVariable> outputVariables;
+ private IScriptDescription scriptDesc;
+
+ public ScriptOperator(IScriptDescription scriptDesc, ArrayList<LogicalVariable> inputVariables,
+ ArrayList<LogicalVariable> outputVariables) {
+ this.inputVariables = inputVariables;
+ this.outputVariables = outputVariables;
+ this.scriptDesc = scriptDesc;
+ }
+
+ public ArrayList<LogicalVariable> getInputVariables() {
+ return inputVariables;
+ }
+
+ public ArrayList<LogicalVariable> getOutputVariables() {
+ return outputVariables;
+ }
+
+ public IScriptDescription getScriptDescription() {
+ return scriptDesc;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.SCRIPT;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitScriptOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ for (LogicalVariable v : outputVariables) {
+ target.addVariable(v);
+ }
+ }
+ };
+
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ this.schema = outputVariables;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
+ ctx.getMetadataProvider());
+ for (Pair<LogicalVariable, Object> p : scriptDesc.getVarTypePairs()) {
+ env.setVarType(p.first, p.second);
+ }
+ return env;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
new file mode 100644
index 0000000..93db0e9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class SelectOperator extends AbstractLogicalOperator {
+ private final LogicalExpressionReference condition;
+
+ public SelectOperator(LogicalExpressionReference condition) {
+ this.condition = condition;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.SELECT;
+ }
+
+ public LogicalExpressionReference getCondition() {
+ return condition;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ return visitor.transform(condition);
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitSelectOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return true;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ ITypeEnvPointer[] envPointers = new ITypeEnvPointer[1];
+ envPointers[0] = new OpRefTypeEnvPointer(inputs.get(0), ctx);
+ PropagatingTypeEnvironment env = new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
+ ctx.getNullableTypeComputer(), ctx.getMetadataProvider(), TypePropagationPolicy.ALL, envPointers);
+ if (condition.getExpression().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) condition.getExpression();
+ if (f1.getFunctionIdentifier() == AlgebricksBuiltinFunctions.NOT) {
+ ILogicalExpression a1 = f1.getArguments().get(0).getExpression();
+ if (a1.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+ if (f2.getFunctionIdentifier() == AlgebricksBuiltinFunctions.IS_NULL) {
+ ILogicalExpression a2 = f2.getArguments().get(0).getExpression();
+ if (a2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable var = ((VariableReferenceExpression) a2).getVariableReference();
+ env.getNonNullVariables().add(var);
+ }
+ }
+ }
+ }
+ }
+ return env;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SinkOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SinkOperator.java
new file mode 100644
index 0000000..a40df28
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SinkOperator.java
@@ -0,0 +1,59 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class SinkOperator extends AbstractLogicalOperator {
+
+ @Override
+ public void recomputeSchema() throws AlgebricksException {
+ schema = new ArrayList<LogicalVariable>(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitSinkOperator(this, arg);
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ ITypeEnvPointer[] envPointers = new ITypeEnvPointer[1];
+ envPointers[0] = new OpRefTypeEnvPointer(inputs.get(0), ctx);
+ PropagatingTypeEnvironment env = new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
+ ctx.getNullableTypeComputer(), ctx.getMetadataProvider(), TypePropagationPolicy.ALL, envPointers);
+ return env;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.SINK;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SubplanOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SubplanOperator.java
new file mode 100644
index 0000000..a60d58d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/SubplanOperator.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class SubplanOperator extends AbstractOperatorWithNestedPlans {
+
+ public SubplanOperator() {
+ super();
+ }
+
+ public SubplanOperator(List<ILogicalPlan> plans) {
+ super(plans);
+ }
+
+ public SubplanOperator(ILogicalOperator planRoot) {
+ ArrayList<LogicalOperatorReference> roots = new ArrayList<LogicalOperatorReference>(1);
+ roots.add(new LogicalOperatorReference(planRoot));
+ nestedPlans.add(new ALogicalPlanImpl(roots));
+ }
+
+ public void setRootOp(LogicalOperatorReference opRef) {
+ ILogicalPlan p = new ALogicalPlanImpl(opRef);
+ nestedPlans.add(p);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) {
+ // do nothing
+ return false;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.SUBPLAN;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ADDNEWVARIABLES;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitSubplanOperator(this, arg);
+ }
+
+ @Override
+ public void getProducedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+ // do nothing
+ }
+
+ @Override
+ public void getUsedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+ // do nothing
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ int n = 0;
+ for (ILogicalPlan p : nestedPlans) {
+ n += p.getRoots().size();
+ }
+ ITypeEnvPointer[] envPointers = new ITypeEnvPointer[n + 1];
+ envPointers[0] = new OpRefTypeEnvPointer(inputs.get(0), ctx);
+ int i = 1;
+ for (ILogicalPlan p : nestedPlans) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ envPointers[i] = new OpRefTypeEnvPointer(r, ctx);
+ i++;
+ }
+ }
+ return new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getNullableTypeComputer(),
+ ctx.getMetadataProvider(), TypePropagationPolicy.LEFT_OUTER, envPointers);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnionAllOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnionAllOperator.java
new file mode 100644
index 0000000..9b78ae3
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnionAllOperator.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class UnionAllOperator extends AbstractLogicalOperator {
+
+ // (left-var, right-var, out-var)
+ private List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap;
+
+ public UnionAllOperator(List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap) {
+ this.varMap = varMap;
+ }
+
+ public List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> getVariableMappings() {
+ return varMap;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.UNIONALL;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
+ target.addVariable(t.third);
+ }
+ }
+ };
+
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitUnionOperator(this, arg);
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ for (LogicalVariable v1 : inputs.get(0).getOperator().getSchema()) {
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
+ if (t.first.equals(v1)) {
+ schema.add(t.third);
+ } else {
+ schema.add(v1);
+ }
+ }
+ }
+ for (LogicalVariable v2 : inputs.get(1).getOperator().getSchema()) {
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
+ if (t.second.equals(v2)) {
+ schema.add(t.third);
+ } else {
+ schema.add(v2);
+ }
+ }
+ }
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx
+ .getMetadataProvider());
+ IVariableTypeEnvironment envLeft = ctx.getOutputTypeEnvironment(inputs.get(0).getOperator());
+ if (envLeft == null) {
+ throw new AlgebricksException("Left input types for union operator are not computed.");
+ }
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
+ Object t1 = envLeft.getVarType(t.first);
+ if (t1 == null) {
+ throw new AlgebricksException("Failed typing union operator: no type for variable " + t.first);
+ }
+ env.setVarType(t.third, t1);
+ }
+ return env;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
new file mode 100644
index 0000000..7056d83
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+/**
+ * This operator may go away after we add indexes to Algebricks.
+ *
+ */
+public class UnnestMapOperator extends AbstractUnnestOperator {
+
+ private final List<Object> variableTypes; // TODO: get rid of this and
+
+ // deprecate UnnestMap
+
+ public UnnestMapOperator(List<LogicalVariable> variables, LogicalExpressionReference expression,
+ List<Object> variableTypes) {
+ super(variables, expression);
+ this.variableTypes = variableTypes;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.UNNEST_MAP;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitUnnestMapOperator(this, arg);
+ }
+
+ /**
+ *
+ * UnnestMap doesn't propagate input variables, because currently it is only
+ * used to search indexes. In the future, it would be nice to have the
+ * choice to propagate input variables or not.
+ *
+ */
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return new VariablePropagationPolicy() {
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ for (LogicalVariable v : variables) {
+ target.addVariable(v);
+ }
+ }
+ };
+ }
+
+ public List<Object> getVariableTypes() {
+ return variableTypes;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
+ ctx.getMetadataProvider());
+ int n = variables.size();
+ for (int i = 0; i < n; i++) {
+ env.setVarType(variables.get(i), variableTypes.get(i));
+ }
+ return env;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestOperator.java
new file mode 100644
index 0000000..613c83b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestOperator.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class UnnestOperator extends AbstractUnnestOperator {
+
+ private LogicalVariable positionalVariable;
+ private Object positionalVariableType;
+
+ public UnnestOperator(LogicalVariable variable, LogicalExpressionReference expression) {
+ super(makeSingletonList(variable), expression);
+ }
+
+ public UnnestOperator(LogicalVariable variable, LogicalExpressionReference expression,
+ LogicalVariable positionalVariable, Object positionalVariableType) {
+ this(variable, expression);
+ this.setPositionalVariable(positionalVariable);
+ this.setPositionalVariableType(positionalVariableType);
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.UNNEST;
+ }
+
+ public LogicalVariable getVariable() {
+ return variables.get(0);
+ }
+
+ public void setPositionalVariable(LogicalVariable positionalVariable) {
+ this.positionalVariable = positionalVariable;
+ }
+
+ public LogicalVariable getPositionalVariable() {
+ return positionalVariable;
+ }
+
+ public void setPositionalVariableType(Object positionalVariableType) {
+ this.positionalVariableType = positionalVariableType;
+ }
+
+ public Object getPositionalVariableType() {
+ return positionalVariableType;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitUnnestOperator(this, arg);
+ }
+
+ private static <E> ArrayList<E> makeSingletonList(E item) {
+ ArrayList<E> array = new ArrayList<E>(1);
+ array.add(item);
+ return array;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ IVariableTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
+ Object t = env.getType(expression.getExpression());
+ env.setVarType(variables.get(0), t);
+ if (positionalVariable != null) {
+ env.setVarType(positionalVariable, positionalVariableType);
+ }
+ return env;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UpdateOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UpdateOperator.java
new file mode 100644
index 0000000..4f94f34
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UpdateOperator.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class UpdateOperator extends AbstractLogicalOperator {
+
+ @Override
+ public void recomputeSchema() throws AlgebricksException {
+ // TODO Auto-generated method stub
+
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public boolean isMap() {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/WriteOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/WriteOperator.java
new file mode 100644
index 0000000..c3559f1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/WriteOperator.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class WriteOperator extends AbstractLogicalOperator {
+ private List<LogicalExpressionReference> expressions;
+ private IDataSink dataSink;
+
+ public WriteOperator(List<LogicalExpressionReference> expressions, IDataSink dataSink) {
+ this.expressions = expressions;
+ this.dataSink = dataSink;
+ }
+
+ public List<LogicalExpressionReference> getExpressions() {
+ return expressions;
+ }
+
+ public IDataSink getDataSink() {
+ return dataSink;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.WRITE;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitWriteOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean modif = false;
+ for (int i = 0; i < expressions.size(); i++) {
+ boolean b = visitor.transform(expressions.get(i));
+ if (b) {
+ modif = true;
+ }
+ }
+ return modif;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean isMap() {
+ return false; // actually depends on the physical op.
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/WriteResultOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/WriteResultOperator.java
new file mode 100644
index 0000000..e8150d2
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/WriteResultOperator.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class WriteResultOperator extends AbstractLogicalOperator {
+
+ private IDataSource<?> dataSource;
+ private LogicalExpressionReference payloadExpr;
+ private List<LogicalExpressionReference> keyExprs;
+
+ public WriteResultOperator(IDataSource<?> dataSource, LogicalExpressionReference payload,
+ List<LogicalExpressionReference> keyExprs) {
+ this.dataSource = dataSource;
+ this.payloadExpr = payload;
+ this.keyExprs = keyExprs;
+ }
+
+ public IDataSource<?> getDataSource() {
+ return dataSource;
+ }
+
+ public LogicalExpressionReference getPayloadExpression() {
+ return payloadExpr;
+ }
+
+ public List<LogicalExpressionReference> getKeyExpressions() {
+ return keyExprs;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.WRITE_RESULT;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitWriteResultOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean b = visitor.transform(payloadExpr);
+ for (int i = 0; i < keyExprs.size(); i++) {
+ if (visitor.transform(keyExprs.get(i))) {
+ b = true;
+ }
+ }
+ return b;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean isMap() {
+ return false;
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getOperator().getSchema());
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
new file mode 100644
index 0000000..8ea552a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -0,0 +1,640 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class FDsAndEquivClassesVisitor implements ILogicalOperatorVisitor<Void, IOptimizationContext> {
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ ctx.putEquivalenceClassMap(op, new HashMap<LogicalVariable, EquivalenceClass>());
+ ctx.putFDList(op, new ArrayList<FunctionalDependency>());
+ return null;
+ }
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ ILogicalOperator inp1 = op.getInputs().get(0).getOperator();
+ Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+ ctx.putEquivalenceClassMap(op, eqClasses);
+ List<LogicalVariable> used = new ArrayList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, used);
+ List<FunctionalDependency> fds1 = getOrComputeFDs(inp1, ctx);
+ List<FunctionalDependency> eFds = new ArrayList<FunctionalDependency>(fds1.size());
+ for (FunctionalDependency fd : fds1) {
+ if (fd.getTail().containsAll(used)) {
+ List<LogicalVariable> hd = new ArrayList<LogicalVariable>(fd.getHead());
+ List<LogicalVariable> tl = new ArrayList<LogicalVariable>(fd.getTail());
+ tl.addAll(op.getVariables());
+ FunctionalDependency fd2 = new FunctionalDependency(hd, tl);
+ eFds.add(fd2);
+ } else {
+ eFds.add(fd);
+ }
+ }
+ ctx.putFDList(op, eFds);
+ return null;
+ }
+
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ ILogicalOperator inp1 = op.getInputs().get(0).getOperator();
+ Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+ ctx.putEquivalenceClassMap(op, eqClasses);
+ List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>(getOrComputeFDs(inp1, ctx));
+ ctx.putFDList(op, fds);
+ op.getDataSource().computeFDs(op.getVariables(), fds);
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ ILogicalOperator op0 = op.getInputs().get(0).getOperator();
+ List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+ ctx.putFDList(op, functionalDependencies);
+ for (FunctionalDependency inherited : getOrComputeFDs(op0, ctx)) {
+ boolean isCoveredByDistinctByVars = true;
+ for (LogicalVariable v : inherited.getHead()) {
+ if (!op.isDistinctByVar(v)) {
+ isCoveredByDistinctByVars = false;
+ }
+ }
+ if (isCoveredByDistinctByVars) {
+ List<LogicalVariable> newTail = new ArrayList<LogicalVariable>();
+ for (LogicalVariable v2 : inherited.getTail()) {
+ if (op.isDistinctByVar(v2)) {
+ newTail.add(v2);
+ }
+ }
+ if (!newTail.isEmpty()) {
+ List<LogicalVariable> newHead = new ArrayList<LogicalVariable>(inherited.getHead());
+ FunctionalDependency newFd = new FunctionalDependency(newHead, newTail);
+ functionalDependencies.add(newFd);
+ }
+ }
+ }
+ Set<LogicalVariable> gbySet = new HashSet<LogicalVariable>();
+ List<LogicalExpressionReference> expressions = op.getExpressions();
+ for (LogicalExpressionReference pRef : expressions) {
+ ILogicalExpression p = pRef.getExpression();
+ if (p.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression v = (VariableReferenceExpression) p;
+ gbySet.add(v.getVariableReference());
+ }
+ }
+ LocalGroupingProperty lgp = new LocalGroupingProperty(gbySet);
+
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses = getOrComputeEqClasses(op0, ctx);
+ ctx.putEquivalenceClassMap(op, equivalenceClasses);
+
+ lgp.normalizeGroupingColumns(equivalenceClasses, functionalDependencies);
+ Set<LogicalVariable> normSet = lgp.getColumnSet();
+ List<LogicalExpressionReference> newDistinctByList = new ArrayList<LogicalExpressionReference>();
+ for (LogicalExpressionReference p2Ref : expressions) {
+ ILogicalExpression p2 = p2Ref.getExpression();
+ if (p2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression var2 = (VariableReferenceExpression) p2;
+ LogicalVariable v2 = var2.getVariableReference();
+ if (normSet.contains(v2)) {
+ newDistinctByList.add(p2Ref);
+ }
+ } else {
+ newDistinctByList.add(p2Ref);
+ }
+ }
+ expressions.clear();
+ expressions.addAll(newDistinctByList);
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ ctx.putEquivalenceClassMap(op, new HashMap<LogicalVariable, EquivalenceClass>());
+ ctx.putFDList(op, new ArrayList<FunctionalDependency>());
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+ List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+ ctx.putEquivalenceClassMap(op, equivalenceClasses);
+ ctx.putFDList(op, functionalDependencies);
+
+ List<FunctionalDependency> inheritedFDs = new ArrayList<FunctionalDependency>();
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ ILogicalOperator op2 = r.getOperator();
+ equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
+ inheritedFDs.addAll(getOrComputeFDs(op2, ctx));
+ }
+ }
+
+ ILogicalOperator op0 = op.getInputs().get(0).getOperator();
+ inheritedFDs.addAll(getOrComputeFDs(op0, ctx));
+ Map<LogicalVariable, EquivalenceClass> inheritedEcs = getOrComputeEqClasses(op0, ctx);
+ for (FunctionalDependency inherited : inheritedFDs) {
+ boolean isCoveredByGbyOrDecorVars = true;
+ List<LogicalVariable> newHead = new ArrayList<LogicalVariable>(inherited.getHead().size());
+ for (LogicalVariable v : inherited.getHead()) {
+ LogicalVariable vnew = getNewGbyVar(op, v);
+ if (vnew == null) {
+ vnew = getNewDecorVar(op, v);
+ if (vnew == null) {
+ isCoveredByGbyOrDecorVars = false;
+ }
+ break;
+ }
+ newHead.add(vnew);
+ }
+
+ if (isCoveredByGbyOrDecorVars) {
+ List<LogicalVariable> newTail = new ArrayList<LogicalVariable>();
+ for (LogicalVariable v2 : inherited.getTail()) {
+ LogicalVariable v3 = getNewGbyVar(op, v2);
+ if (v3 != null) {
+ newTail.add(v3);
+ }
+ }
+ if (!newTail.isEmpty()) {
+ FunctionalDependency newFd = new FunctionalDependency(newHead, newTail);
+ functionalDependencies.add(newFd);
+ }
+ }
+ }
+
+ List<LogicalVariable> premiseGby = new LinkedList<LogicalVariable>();
+ List<Pair<LogicalVariable, LogicalExpressionReference>> gByList = op.getGroupByList();
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gByList) {
+ premiseGby.add(p.first);
+ }
+
+ List<Pair<LogicalVariable, LogicalExpressionReference>> decorList = op.getDecorList();
+
+ LinkedList<LogicalVariable> conclDecor = new LinkedList<LogicalVariable>();
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : decorList) {
+ conclDecor.add(GroupByOperator.getDecorVariable(p));
+ }
+ if (!conclDecor.isEmpty()) {
+ functionalDependencies.add(new FunctionalDependency(premiseGby, conclDecor));
+ }
+
+ Set<LogicalVariable> gbySet = new HashSet<LogicalVariable>();
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gByList) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ gbySet.add(v.getVariableReference());
+ }
+ }
+ LocalGroupingProperty lgp = new LocalGroupingProperty(gbySet);
+ lgp.normalizeGroupingColumns(inheritedEcs, inheritedFDs);
+ Set<LogicalVariable> normSet = lgp.getColumnSet();
+ List<Pair<LogicalVariable, LogicalExpressionReference>> newGbyList = new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>();
+ boolean changed = false;
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gByList) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ LogicalVariable v2 = varRef.getVariableReference();
+ EquivalenceClass ec2 = inheritedEcs.get(v2);
+ LogicalVariable v3;
+ if (ec2 != null && !ec2.representativeIsConst()) {
+ v3 = ec2.getVariableRepresentative();
+ } else {
+ v3 = v2;
+ }
+ if (normSet.contains(v3)) {
+ newGbyList.add(p);
+ } else {
+ changed = true;
+ decorList.add(p);
+ }
+ } else {
+ newGbyList.add(p);
+ }
+ }
+ if (changed) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Group-by list changed from "
+ + GroupByOperator.veListToString(gByList) + " to " + GroupByOperator.veListToString(newGbyList)
+ + ".\n");
+ }
+ gByList.clear();
+ gByList.addAll(newGbyList);
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+ List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+ ctx.putEquivalenceClassMap(op, equivalenceClasses);
+ ctx.putFDList(op, functionalDependencies);
+ ILogicalOperator op0 = op.getInputs().get(0).getOperator();
+ ILogicalOperator op1 = op.getInputs().get(1).getOperator();
+ functionalDependencies.addAll(getOrComputeFDs(op0, ctx));
+ functionalDependencies.addAll(getOrComputeFDs(op1, ctx));
+ equivalenceClasses.putAll(getOrComputeEqClasses(op0, ctx));
+ equivalenceClasses.putAll(getOrComputeEqClasses(op1, ctx));
+ ILogicalExpression expr = op.getCondition().getExpression();
+ expr.getConstraintsAndEquivClasses(functionalDependencies, equivalenceClasses);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+ List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+ ctx.putEquivalenceClassMap(op, equivalenceClasses);
+ ctx.putFDList(op, functionalDependencies);
+ ILogicalOperator opLeft = op.getInputs().get(0).getOperator();
+ ILogicalOperator opRight = op.getInputs().get(1).getOperator();
+ functionalDependencies.addAll(getOrComputeFDs(opLeft, ctx));
+ functionalDependencies.addAll(getOrComputeFDs(opRight, ctx));
+ equivalenceClasses.putAll(getOrComputeEqClasses(opLeft, ctx));
+ equivalenceClasses.putAll(getOrComputeEqClasses(opRight, ctx));
+
+ Collection<LogicalVariable> leftSideVars;
+ if (opLeft.getSchema() == null) {
+ leftSideVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(opLeft, leftSideVars);
+ // actually, not all produced vars. are visible (due to projection)
+ // so using cached schema is better and faster
+ } else {
+ leftSideVars = opLeft.getSchema();
+ }
+ ILogicalExpression expr = op.getCondition().getExpression();
+ expr.getConstraintsForOuterJoin(functionalDependencies, leftSideVars);
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitDieOperator(DieOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) op.getDataSourceReference().getOperator();
+ ILogicalOperator inp1 = op1.getInputs().get(0).getOperator();
+ Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+ ctx.putEquivalenceClassMap(op, eqClasses);
+ List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>(getOrComputeFDs(inp1, ctx));
+ if (op1.getOperatorTag() == LogicalOperatorTag.GROUP) {
+ GroupByOperator gby = (GroupByOperator) op1;
+ LinkedList<LogicalVariable> tail = new LinkedList<LogicalVariable>();
+ for (LogicalVariable v : gby.getGbyVarList()) {
+ tail.add(v);
+ // all values for gby vars. are the same
+ }
+ FunctionalDependency gbyfd = new FunctionalDependency(new LinkedList<LogicalVariable>(), tail);
+ fds.add(gbyfd);
+ }
+ ctx.putFDList(op, fds);
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ throw new NotImplementedException();
+ }
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClassesForUsedVars(op, ctx, op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ propagateFDsAndEquivClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ propagateFDsAndEquivClassesForUsedVars(op, ctx, op.getInputVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+ List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+ ctx.putEquivalenceClassMap(op, equivalenceClasses);
+ ctx.putFDList(op, functionalDependencies);
+ ILogicalOperator op0 = op.getInputs().get(0).getOperator();
+ functionalDependencies.addAll(getOrComputeFDs(op0, ctx));
+ equivalenceClasses.putAll(getOrComputeEqClasses(op0, ctx));
+ ILogicalExpression expr = op.getCondition().getExpression();
+ expr.getConstraintsAndEquivClasses(functionalDependencies, equivalenceClasses);
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+ List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+ ctx.putEquivalenceClassMap(op, equivalenceClasses);
+ ctx.putFDList(op, functionalDependencies);
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ ILogicalOperator op2 = r.getOperator();
+ equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
+ functionalDependencies.addAll(getOrComputeFDs(op2, ctx));
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ setEmptyFDsEqClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ fdsEqClassesForAbstractUnnestOperator(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ fdsEqClassesForAbstractUnnestOperator(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ // propagateFDsAndEquivClasses(op, ctx);
+ setEmptyFDsEqClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ // propagateFDsAndEquivClasses(op, ctx);
+ setEmptyFDsEqClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteOperator(InsertDeleteOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ setEmptyFDsEqClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ setEmptyFDsEqClasses(op, ctx);
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ setEmptyFDsEqClasses(op, ctx);
+ return null;
+ }
+
+ private void propagateFDsAndEquivClasses(ILogicalOperator op, IOptimizationContext ctx) throws AlgebricksException {
+ ILogicalOperator inp1 = op.getInputs().get(0).getOperator();
+ Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+ ctx.putEquivalenceClassMap(op, eqClasses);
+ List<FunctionalDependency> fds = getOrComputeFDs(inp1, ctx);
+ ctx.putFDList(op, fds);
+ }
+
+ private Map<LogicalVariable, EquivalenceClass> getOrComputeEqClasses(ILogicalOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ Map<LogicalVariable, EquivalenceClass> eqClasses = ctx.getEquivalenceClassMap(op);
+ if (eqClasses == null) {
+ op.accept(this, ctx);
+ eqClasses = ctx.getEquivalenceClassMap(op);
+ }
+ return eqClasses;
+ }
+
+ private List<FunctionalDependency> getOrComputeFDs(ILogicalOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ List<FunctionalDependency> fds = ctx.getFDList(op);
+ if (fds == null) {
+ op.accept(this, ctx);
+ fds = ctx.getFDList(op);
+ }
+ return fds;
+ }
+
+ private void propagateFDsAndEquivClassesForUsedVars(ILogicalOperator op, IOptimizationContext ctx,
+ List<LogicalVariable> usedVariables) throws AlgebricksException {
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ Map<LogicalVariable, EquivalenceClass> eqClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+ ctx.putEquivalenceClassMap(op, eqClasses);
+ List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>();
+ ctx.putFDList(op, fds);
+
+ Map<LogicalVariable, EquivalenceClass> chldClasses = getOrComputeEqClasses(op2, ctx);
+ for (LogicalVariable v : usedVariables) {
+ EquivalenceClass ec = eqClasses.get(v);
+ if (ec == null) {
+ EquivalenceClass oc = chldClasses.get(v);
+ if (oc == null) {
+ continue;
+ }
+ List<LogicalVariable> m = new LinkedList<LogicalVariable>();
+ for (LogicalVariable v2 : oc.getMembers()) {
+ if (usedVariables.contains(v2)) {
+ m.add(v2);
+ }
+ }
+ EquivalenceClass nc;
+ if (oc.representativeIsConst()) {
+ nc = new EquivalenceClass(m, oc.getConstRepresentative());
+ } else if (m.contains(oc.getVariableRepresentative())) {
+ nc = new EquivalenceClass(m, oc.getVariableRepresentative());
+ } else {
+ nc = new EquivalenceClass(m, v);
+ }
+ for (LogicalVariable v3 : m) {
+ eqClasses.put(v3, nc);
+ }
+ }
+ }
+
+ List<FunctionalDependency> chldFds = getOrComputeFDs(op2, ctx);
+ for (FunctionalDependency fd : chldFds) {
+ if (!usedVariables.containsAll(fd.getHead())) {
+ continue;
+ }
+ List<LogicalVariable> tl = new LinkedList<LogicalVariable>();
+ for (LogicalVariable v : fd.getTail()) {
+ if (usedVariables.contains(v)) {
+ tl.add(v);
+ }
+ }
+ if (!tl.isEmpty()) {
+ FunctionalDependency newFd = new FunctionalDependency(fd.getHead(), tl);
+ fds.add(newFd);
+ }
+ }
+ }
+
+ private void fdsEqClassesForAbstractUnnestOperator(AbstractUnnestOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ ILogicalOperator inp1 = op.getInputs().get(0).getOperator();
+ Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+ ctx.putEquivalenceClassMap(op, eqClasses);
+ List<FunctionalDependency> fds = getOrComputeFDs(inp1, ctx);
+ ctx.putFDList(op, fds);
+
+ ILogicalExpression expr = op.getExpressionRef().getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression afe = (AbstractFunctionCallExpression) expr;
+ if (afe.getKind() == FunctionKind.UNNEST && ((UnnestingFunctionCallExpression) afe).returnsUniqueValues()) {
+ List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(op, vars);
+ ArrayList<LogicalVariable> h = new ArrayList<LogicalVariable>();
+ h.addAll(op.getVariables());
+ FunctionalDependency fd = new FunctionalDependency(h, vars);
+ fds.add(fd);
+ }
+ }
+ }
+
+ public static void setEmptyFDsEqClasses(ILogicalOperator op, IOptimizationContext ctx) {
+ Map<LogicalVariable, EquivalenceClass> eqClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+ ctx.putEquivalenceClassMap(op, eqClasses);
+ List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>();
+ ctx.putFDList(op, fds);
+ }
+
+ private LogicalVariable getNewGbyVar(GroupByOperator g, LogicalVariable v) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : g.getGroupByList()) {
+ ILogicalExpression e = p.second.getExpression();
+ if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v2 = ((VariableReferenceExpression) e).getVariableReference();
+ if (v2 == v) {
+ return p.first;
+ }
+ }
+ }
+ return null;
+ }
+
+ private LogicalVariable getNewDecorVar(GroupByOperator g, LogicalVariable v) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : g.getDecorList()) {
+ ILogicalExpression e = p.second.getExpression();
+ if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v2 = ((VariableReferenceExpression) e).getVariableReference();
+ if (v2 == v) {
+ return (p.first != null) ? p.first : v2;
+ }
+ }
+ }
+ return null;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
new file mode 100644
index 0000000..694324c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -0,0 +1,811 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class IsomorphismOperatorVisitor implements ILogicalOperatorVisitor<Boolean, ILogicalOperator> {
+
+ private Map<LogicalVariable, LogicalVariable> variableMapping = new HashMap<LogicalVariable, LogicalVariable>();
+
+ public IsomorphismOperatorVisitor() {
+ }
+
+ @Override
+ public Boolean visitAggregateOperator(AggregateOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.AGGREGATE)
+ return Boolean.FALSE;
+ AggregateOperator aggOpArg = (AggregateOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(
+ getPairList(op.getVariables(), op.getExpressions()),
+ getPairList(aggOpArg.getVariables(), aggOpArg.getExpressions()));
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitRunningAggregateOperator(RunningAggregateOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.RUNNINGAGGREGATE)
+ return Boolean.FALSE;
+ RunningAggregateOperator aggOpArg = (RunningAggregateOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(
+ getPairList(op.getVariables(), op.getExpressions()),
+ getPairList(aggOpArg.getVariables(), aggOpArg.getExpressions()));
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) copyAndSubstituteVar(op, arg);
+ if (aop.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE)
+ return Boolean.FALSE;
+ return Boolean.TRUE;
+ }
+
+ @Override
+ public Boolean visitGroupByOperator(GroupByOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ // require the same physical operator, otherwise delivers different data
+ // properties
+ if (aop.getOperatorTag() != LogicalOperatorTag.GROUP
+ || aop.getPhysicalOperator().getOperatorTag() != op.getPhysicalOperator().getOperatorTag())
+ return Boolean.FALSE;
+
+ List<Pair<LogicalVariable, LogicalExpressionReference>> keyLists = op.getGroupByList();
+ GroupByOperator gbyOpArg = (GroupByOperator) copyAndSubstituteVar(op, arg);
+ List<Pair<LogicalVariable, LogicalExpressionReference>> keyListsArg = gbyOpArg.getGroupByList();
+
+ List<Pair<LogicalVariable, ILogicalExpression>> listLeft = new ArrayList<Pair<LogicalVariable, ILogicalExpression>>();
+ List<Pair<LogicalVariable, ILogicalExpression>> listRight = new ArrayList<Pair<LogicalVariable, ILogicalExpression>>();
+
+ for (Pair<LogicalVariable, LogicalExpressionReference> pair : keyLists)
+ listLeft.add(new Pair<LogicalVariable, ILogicalExpression>(pair.first, pair.second.getExpression()));
+ for (Pair<LogicalVariable, LogicalExpressionReference> pair : keyListsArg)
+ listRight.add(new Pair<LogicalVariable, ILogicalExpression>(pair.first, pair.second.getExpression()));
+
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(listLeft, listRight);
+
+ if (!isomorphic)
+ return Boolean.FALSE;
+ int sizeOp = op.getNestedPlans().size();
+ int sizeArg = gbyOpArg.getNestedPlans().size();
+ if (sizeOp != sizeArg)
+ return Boolean.FALSE;
+
+ GroupByOperator argOp = (GroupByOperator) arg;
+ List<ILogicalPlan> plans = op.getNestedPlans();
+ List<ILogicalPlan> plansArg = argOp.getNestedPlans();
+ for (int i = 0; i < plans.size(); i++) {
+ List<LogicalOperatorReference> roots = plans.get(i).getRoots();
+ List<LogicalOperatorReference> rootsArg = plansArg.get(i).getRoots();
+ if (roots.size() != rootsArg.size())
+ return Boolean.FALSE;
+ for (int j = 0; j < roots.size(); j++) {
+ ILogicalOperator topOp1 = roots.get(j).getOperator();
+ ILogicalOperator topOp2 = rootsArg.get(j).getOperator();
+ isomorphic = this.checkBottomUp(topOp1, topOp2);
+ if (!isomorphic)
+ return Boolean.FALSE;
+ }
+ }
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitLimitOperator(LimitOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.LIMIT)
+ return Boolean.FALSE;
+ LimitOperator limitOpArg = (LimitOperator) copyAndSubstituteVar(op, arg);
+ if (op.getOffset() != limitOpArg.getOffset())
+ return Boolean.FALSE;
+ boolean isomorphic = op.getMaxObjects().getExpression().equals(limitOpArg.getMaxObjects().getExpression());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitDieOperator(DieOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.DIE)
+ return Boolean.FALSE;
+ DieOperator dieOpArg = (DieOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = op.getAfterObjects().getExpression().equals(dieOpArg.getAfterObjects().getExpression());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitInnerJoinOperator(InnerJoinOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.INNERJOIN)
+ return Boolean.FALSE;
+ InnerJoinOperator joinOpArg = (InnerJoinOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = op.getCondition().getExpression().equals(joinOpArg.getCondition().getExpression());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitLeftOuterJoinOperator(LeftOuterJoinOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN)
+ return Boolean.FALSE;
+ LeftOuterJoinOperator joinOpArg = (LeftOuterJoinOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = op.getCondition().getExpression().equals(joinOpArg.getCondition().getExpression());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitNestedTupleSourceOperator(NestedTupleSourceOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE)
+ return Boolean.FALSE;
+ return Boolean.TRUE;
+ }
+
+ @Override
+ public Boolean visitOrderOperator(OrderOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.ORDER)
+ return Boolean.FALSE;
+ OrderOperator orderOpArg = (OrderOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = compareIOrderAndExpressions(op.getOrderExpressions(), orderOpArg.getOrderExpressions());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitAssignOperator(AssignOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.ASSIGN)
+ return Boolean.FALSE;
+ AssignOperator assignOpArg = (AssignOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(
+ getPairList(op.getVariables(), op.getExpressions()),
+ getPairList(assignOpArg.getVariables(), assignOpArg.getExpressions()));
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitSelectOperator(SelectOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.SELECT)
+ return Boolean.FALSE;
+ SelectOperator selectOpArg = (SelectOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = op.getCondition().getExpression().equals(selectOpArg.getCondition().getExpression());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitProjectOperator(ProjectOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.PROJECT)
+ return Boolean.FALSE;
+ ProjectOperator projectOpArg = (ProjectOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getVariables(), projectOpArg.getVariables());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitPartitioningSplitOperator(PartitioningSplitOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.PARTITIONINGSPLIT)
+ return Boolean.FALSE;
+ PartitioningSplitOperator partitionOpArg = (PartitioningSplitOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = compareExpressions(Arrays.asList(op.getExpressions()),
+ Arrays.asList(partitionOpArg.getExpressions()));
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitReplicateOperator(ReplicateOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.REPLICATE)
+ return Boolean.FALSE;
+ return Boolean.TRUE;
+ }
+
+ @Override
+ public Boolean visitScriptOperator(ScriptOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.SCRIPT)
+ return Boolean.FALSE;
+ ScriptOperator scriptOpArg = (ScriptOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = op.getScriptDescription().equals(scriptOpArg.getScriptDescription());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitSubplanOperator(SubplanOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.SUBPLAN)
+ return Boolean.FALSE;
+ SubplanOperator subplanOpArg = (SubplanOperator) copyAndSubstituteVar(op, arg);
+ List<ILogicalPlan> plans = op.getNestedPlans();
+ List<ILogicalPlan> plansArg = subplanOpArg.getNestedPlans();
+ for (int i = 0; i < plans.size(); i++) {
+ List<LogicalOperatorReference> roots = plans.get(i).getRoots();
+ List<LogicalOperatorReference> rootsArg = plansArg.get(i).getRoots();
+ if (roots.size() == rootsArg.size())
+ return Boolean.FALSE;
+ for (int j = 0; j < roots.size(); j++) {
+ ILogicalOperator topOp1 = roots.get(j).getOperator();
+ ILogicalOperator topOp2 = rootsArg.get(j).getOperator();
+ boolean isomorphic = this.checkBottomUp(topOp1, topOp2);
+ if (!isomorphic)
+ return Boolean.FALSE;
+ }
+ }
+ return Boolean.TRUE;
+ }
+
+ @Override
+ public Boolean visitUnionOperator(UnionAllOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.UNIONALL)
+ return Boolean.FALSE;
+ UnionAllOperator unionOpArg = (UnionAllOperator) copyAndSubstituteVar(op, arg);
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> mapping = op.getVariableMappings();
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> mappingArg = unionOpArg.getVariableMappings();
+ if (mapping.size() != mappingArg.size())
+ return Boolean.FALSE;
+ return VariableUtilities.varListEqualUnordered(mapping, mappingArg);
+ }
+
+ @Override
+ public Boolean visitUnnestOperator(UnnestOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.UNNEST)
+ return Boolean.FALSE;
+ UnnestOperator unnestOpArg = (UnnestOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getVariables(), unnestOpArg.getVariables())
+ && variableEqual(op.getPositionalVariable(), unnestOpArg.getPositionalVariable());
+ if (!isomorphic)
+ return Boolean.FALSE;
+ isomorphic = op.getExpressionRef().getExpression().equals(unnestOpArg.getExpressionRef().getExpression());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.UNNEST_MAP)
+ return Boolean.FALSE;
+ UnnestOperator unnestOpArg = (UnnestOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getVariables(), unnestOpArg.getVariables());
+ if (!isomorphic)
+ return Boolean.FALSE;
+ isomorphic = op.getExpressionRef().getExpression().equals(unnestOpArg.getExpressionRef().getExpression());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitDataScanOperator(DataSourceScanOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN)
+ return Boolean.FALSE;
+ DataSourceScanOperator argScan = (DataSourceScanOperator) arg;
+ if (!argScan.getDataSource().toString().equals(op.getDataSource().toString()))
+ return Boolean.FALSE;
+ DataSourceScanOperator scanOpArg = (DataSourceScanOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getVariables(), scanOpArg.getVariables())
+ && op.getDataSource().toString().equals(scanOpArg.getDataSource().toString());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitDistinctOperator(DistinctOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.DISTINCT)
+ return Boolean.FALSE;
+ DistinctOperator distinctOpArg = (DistinctOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = compareExpressions(op.getExpressions(), distinctOpArg.getExpressions());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitExchangeOperator(ExchangeOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.EXCHANGE)
+ return Boolean.FALSE;
+ // require the same partition property
+ if (!(op.getPhysicalOperator().getOperatorTag() == aop.getPhysicalOperator().getOperatorTag()))
+ return Boolean.FALSE;
+ variableMapping.clear();
+ IsomorphismUtilities.mapVariablesTopDown(op, arg, variableMapping);
+ IPhysicalPropertiesVector properties = op.getPhysicalOperator().getDeliveredProperties();
+ IPhysicalPropertiesVector propertiesArg = aop.getPhysicalOperator().getDeliveredProperties();
+ if (properties == null && propertiesArg == null)
+ return Boolean.TRUE;
+ if (properties == null || propertiesArg == null)
+ return Boolean.FALSE;
+ IPartitioningProperty partProp = properties.getPartitioningProperty();
+ IPartitioningProperty partPropArg = propertiesArg.getPartitioningProperty();
+ if (!partProp.getPartitioningType().equals(partPropArg.getPartitioningType()))
+ return Boolean.FALSE;
+ List<LogicalVariable> columns = new ArrayList<LogicalVariable>();
+ partProp.getColumns(columns);
+ List<LogicalVariable> columnsArg = new ArrayList<LogicalVariable>();
+ partPropArg.getColumns(columnsArg);
+ if (columns.size() != columnsArg.size())
+ return Boolean.FALSE;
+ if (columns.size() == 0)
+ return Boolean.TRUE;
+ for (int i = 0; i < columnsArg.size(); i++) {
+ LogicalVariable rightVar = columnsArg.get(i);
+ LogicalVariable leftVar = variableMapping.get(rightVar);
+ if (leftVar != null)
+ columnsArg.set(i, leftVar);
+ }
+ return VariableUtilities.varListEqualUnordered(columns, columnsArg);
+ }
+
+ @Override
+ public Boolean visitWriteOperator(WriteOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.WRITE)
+ return Boolean.FALSE;
+ WriteOperator writeOpArg = (WriteOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), writeOpArg.getSchema());
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitWriteResultOperator(WriteResultOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT)
+ return Boolean.FALSE;
+ WriteResultOperator writeOpArg = (WriteResultOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), writeOpArg.getSchema());
+ if (!op.getDataSource().equals(writeOpArg.getDataSource()))
+ isomorphic = false;
+ if (!op.getPayloadExpression().equals(writeOpArg.getPayloadExpression()))
+ isomorphic = false;
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitInsertDeleteOperator(InsertDeleteOperator op, ILogicalOperator arg) throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE)
+ return Boolean.FALSE;
+ InsertDeleteOperator insertOpArg = (InsertDeleteOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), insertOpArg.getSchema());
+ if (!op.getDataSource().equals(insertOpArg.getDataSource()))
+ isomorphic = false;
+ if (!op.getPayloadExpression().equals(insertOpArg.getPayloadExpression()))
+ isomorphic = false;
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.INDEX_INSERT_DELETE)
+ return Boolean.FALSE;
+ IndexInsertDeleteOperator insertOpArg = (IndexInsertDeleteOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), insertOpArg.getSchema());
+ if (!op.getDataSourceIndex().equals(insertOpArg.getDataSourceIndex()))
+ isomorphic = false;
+ return isomorphic;
+ }
+
+ @Override
+ public Boolean visitSinkOperator(SinkOperator op, ILogicalOperator arg) throws AlgebricksException {
+ return true;
+ }
+
+ private Boolean compareExpressions(List<LogicalExpressionReference> opExprs,
+ List<LogicalExpressionReference> argExprs) {
+ if (opExprs.size() != argExprs.size())
+ return Boolean.FALSE;
+ for (int i = 0; i < opExprs.size(); i++) {
+ boolean isomorphic = opExprs.get(i).getExpression().equals(argExprs.get(i).getExpression());
+ if (!isomorphic)
+ return Boolean.FALSE;
+ }
+ return Boolean.TRUE;
+ }
+
+ private Boolean compareIOrderAndExpressions(List<Pair<IOrder, LogicalExpressionReference>> opOrderExprs,
+ List<Pair<IOrder, LogicalExpressionReference>> argOrderExprs) {
+ if (opOrderExprs.size() != argOrderExprs.size())
+ return Boolean.FALSE;
+ for (int i = 0; i < opOrderExprs.size(); i++) {
+ boolean isomorphic = opOrderExprs.get(i).first.equals(argOrderExprs.get(i).first);
+ if (!isomorphic)
+ return Boolean.FALSE;
+ isomorphic = opOrderExprs.get(i).second.getExpression().equals(argOrderExprs.get(i).second.getExpression());
+ if (!isomorphic)
+ return Boolean.FALSE;
+ }
+ return Boolean.TRUE;
+ }
+
+ private Boolean checkBottomUp(ILogicalOperator op1, ILogicalOperator op2) throws AlgebricksException {
+ List<LogicalOperatorReference> inputs1 = op1.getInputs();
+ List<LogicalOperatorReference> inputs2 = op2.getInputs();
+ if (inputs1.size() != inputs2.size())
+ return Boolean.FALSE;
+ for (int i = 0; i < inputs1.size(); i++) {
+ ILogicalOperator input1 = inputs1.get(i).getOperator();
+ ILogicalOperator input2 = inputs2.get(i).getOperator();
+ boolean isomorphic = checkBottomUp(input1, input2);
+ if (!isomorphic)
+ return Boolean.FALSE;
+ }
+ return IsomorphismUtilities.isOperatorIsomorphic(op1, op2);
+ }
+
+ private ILogicalOperator copyAndSubstituteVar(ILogicalOperator op, ILogicalOperator argOp)
+ throws AlgebricksException {
+ ILogicalOperator newOp = IsomorphismOperatorVisitor.deepCopy(argOp);
+ variableMapping.clear();
+ IsomorphismUtilities.mapVariablesTopDown(op, argOp, variableMapping);
+
+ List<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
+ if (argOp.getInputs().size() > 0)
+ for (int i = 0; i < argOp.getInputs().size(); i++)
+ VariableUtilities.getLiveVariables(argOp.getInputs().get(i).getOperator(), liveVars);
+ List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(argOp, producedVars);
+ List<LogicalVariable> producedVarsNew = new ArrayList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op, producedVarsNew);
+
+ if (producedVars.size() != producedVarsNew.size())
+ return newOp;
+ for (Entry<LogicalVariable, LogicalVariable> map : variableMapping.entrySet()) {
+ if (liveVars.contains(map.getKey())) {
+ VariableUtilities.substituteVariables(newOp, map.getKey(), map.getValue(), null);
+ }
+ }
+ for (int i = 0; i < producedVars.size(); i++)
+ VariableUtilities.substituteVariables(newOp, producedVars.get(i), producedVarsNew.get(i), null);
+ return newOp;
+ }
+
+ public List<Pair<LogicalVariable, ILogicalExpression>> getPairList(List<LogicalVariable> vars,
+ List<LogicalExpressionReference> exprs) throws AlgebricksException {
+ List<Pair<LogicalVariable, ILogicalExpression>> list = new ArrayList<Pair<LogicalVariable, ILogicalExpression>>();
+ if (vars.size() != exprs.size())
+ throw new AlgebricksException("variable list size does not equal to expression list size ");
+ for (int i = 0; i < vars.size(); i++) {
+ list.add(new Pair<LogicalVariable, ILogicalExpression>(vars.get(i), exprs.get(i).getExpression()));
+ }
+ return list;
+ }
+
+ private static ILogicalOperator deepCopy(ILogicalOperator op) throws AlgebricksException {
+ OperatorDeepCopyVisitor visitor = new OperatorDeepCopyVisitor();
+ return op.accept(visitor, null);
+ }
+
+ private static ILogicalPlan deepCopy(ILogicalPlan plan) throws AlgebricksException {
+ List<LogicalOperatorReference> roots = plan.getRoots();
+ List<LogicalOperatorReference> newRoots = new ArrayList<LogicalOperatorReference>();
+ for (LogicalOperatorReference opRef : roots)
+ newRoots.add(new LogicalOperatorReference(bottomUpCopyOperators(opRef.getOperator())));
+ return new ALogicalPlanImpl(newRoots);
+ }
+
+ private static ILogicalOperator bottomUpCopyOperators(ILogicalOperator op) throws AlgebricksException {
+ ILogicalOperator newOp = deepCopy(op);
+ newOp.getInputs().clear();
+ for (LogicalOperatorReference child : op.getInputs())
+ newOp.getInputs().add(new LogicalOperatorReference(bottomUpCopyOperators(child.getOperator())));
+ return newOp;
+ }
+
+ private static boolean variableEqual(LogicalVariable var, LogicalVariable varArg) {
+ if (var == null && varArg == null)
+ return true;
+ if (var.equals(varArg))
+ return true;
+ else
+ return false;
+ }
+
+ private static class OperatorDeepCopyVisitor implements ILogicalOperatorVisitor<ILogicalOperator, Void> {
+
+ @Override
+ public ILogicalOperator visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalVariable> newList = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalExpressionReference> newExpressions = new ArrayList<LogicalExpressionReference>();
+ newList.addAll(op.getVariables());
+ deepCopyExpressionRefs(newExpressions, op.getExpressions());
+ return new AggregateOperator(newList, newExpressions);
+ }
+
+ @Override
+ public ILogicalOperator visitRunningAggregateOperator(RunningAggregateOperator op, Void arg)
+ throws AlgebricksException {
+ ArrayList<LogicalVariable> newList = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalExpressionReference> newExpressions = new ArrayList<LogicalExpressionReference>();
+ newList.addAll(op.getVariables());
+ deepCopyExpressionRefs(newExpressions, op.getExpressions());
+ return new RunningAggregateOperator(newList, newExpressions);
+ }
+
+ @Override
+ public ILogicalOperator visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg)
+ throws AlgebricksException {
+ return new EmptyTupleSourceOperator();
+ }
+
+ @Override
+ public ILogicalOperator visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+ List<Pair<LogicalVariable, LogicalExpressionReference>> groupByList = new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>();
+ List<Pair<LogicalVariable, LogicalExpressionReference>> decoList = new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>();
+ ArrayList<ILogicalPlan> newSubplans = new ArrayList<ILogicalPlan>();
+ for (Pair<LogicalVariable, LogicalExpressionReference> pair : op.getGroupByList())
+ groupByList.add(new Pair<LogicalVariable, LogicalExpressionReference>(pair.first,
+ deepCopyExpressionRef(pair.second)));
+ for (Pair<LogicalVariable, LogicalExpressionReference> pair : op.getDecorList())
+ decoList.add(new Pair<LogicalVariable, LogicalExpressionReference>(pair.first,
+ deepCopyExpressionRef(pair.second)));
+ for (ILogicalPlan plan : op.getNestedPlans()) {
+ newSubplans.add(IsomorphismOperatorVisitor.deepCopy(plan));
+ }
+ return new GroupByOperator(groupByList, decoList, newSubplans);
+ }
+
+ @Override
+ public ILogicalOperator visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
+ return new LimitOperator(deepCopyExpressionRef(op.getMaxObjects()).getExpression(), deepCopyExpressionRef(
+ op.getOffset()).getExpression(), op.isTopmostLimitOp());
+ }
+
+ @Override
+ public ILogicalOperator visitDieOperator(DieOperator op, Void arg) throws AlgebricksException {
+ return new DieOperator(deepCopyExpressionRef(op.getAfterObjects()).getExpression());
+ }
+
+ @Override
+ public ILogicalOperator visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
+ return new InnerJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0), op
+ .getInputs().get(1));
+ }
+
+ @Override
+ public ILogicalOperator visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg)
+ throws AlgebricksException {
+ return new LeftOuterJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0), op
+ .getInputs().get(1));
+ }
+
+ @Override
+ public ILogicalOperator visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg)
+ throws AlgebricksException {
+ return new NestedTupleSourceOperator(null);
+ }
+
+ @Override
+ public ILogicalOperator visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {
+ return new OrderOperator(this.deepCopyOrderAndExpression(op.getOrderExpressions()));
+ }
+
+ @Override
+ public ILogicalOperator visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalVariable> newList = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalExpressionReference> newExpressions = new ArrayList<LogicalExpressionReference>();
+ newList.addAll(op.getVariables());
+ deepCopyExpressionRefs(newExpressions, op.getExpressions());
+ return new AssignOperator(newList, newExpressions);
+ }
+
+ @Override
+ public ILogicalOperator visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
+ return new SelectOperator(deepCopyExpressionRef(op.getCondition()));
+ }
+
+ @Override
+ public ILogicalOperator visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalVariable> newList = new ArrayList<LogicalVariable>();
+ newList.addAll(op.getVariables());
+ return new ProjectOperator(newList);
+ }
+
+ @Override
+ public ILogicalOperator visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg)
+ throws AlgebricksException {
+ ArrayList<LogicalExpressionReference> newExpressions = new ArrayList<LogicalExpressionReference>();
+ deepCopyExpressionRefs(newExpressions, Arrays.asList(op.getExpressions()));
+ return new PartitioningSplitOperator(newExpressions.toArray(new LogicalExpressionReference[0]),
+ op.hasDefault());
+ }
+
+ @Override
+ public ILogicalOperator visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+ return new ReplicateOperator(op.getOutputArity());
+ }
+
+ @Override
+ public ILogicalOperator visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalVariable> newInputList = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> newOutputList = new ArrayList<LogicalVariable>();
+ newInputList.addAll(op.getInputVariables());
+ newOutputList.addAll(op.getOutputVariables());
+ return new ScriptOperator(op.getScriptDescription(), newInputList, newOutputList);
+ }
+
+ @Override
+ public ILogicalOperator visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+ ArrayList<ILogicalPlan> newSubplans = new ArrayList<ILogicalPlan>();
+ for (ILogicalPlan plan : op.getNestedPlans()) {
+ newSubplans.add(IsomorphismOperatorVisitor.deepCopy(plan));
+ }
+ return new SubplanOperator(newSubplans);
+ }
+
+ @Override
+ public ILogicalOperator visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> newVarMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = op.getVariableMappings();
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple : varMap)
+ newVarMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(triple.first,
+ triple.second, triple.third));
+ return new UnionAllOperator(newVarMap);
+ }
+
+ @Override
+ public ILogicalOperator visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {
+ return new UnnestOperator(op.getVariable(), deepCopyExpressionRef(op.getExpressionRef()),
+ op.getPositionalVariable(), op.getPositionalVariableType());
+ }
+
+ @Override
+ public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalVariable> newInputList = new ArrayList<LogicalVariable>();
+ newInputList.addAll(op.getVariables());
+ return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
+ new ArrayList<Object>(op.getVariableTypes()));
+ }
+
+ @Override
+ public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalVariable> newInputList = new ArrayList<LogicalVariable>();
+ newInputList.addAll(op.getVariables());
+ return new DataSourceScanOperator(newInputList, op.getDataSource());
+ }
+
+ @Override
+ public ILogicalOperator visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalExpressionReference> newExpressions = new ArrayList<LogicalExpressionReference>();
+ deepCopyExpressionRefs(newExpressions, op.getExpressions());
+ return new DistinctOperator(newExpressions);
+ }
+
+ @Override
+ public ILogicalOperator visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+ return new ExchangeOperator();
+ }
+
+ @Override
+ public ILogicalOperator visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalExpressionReference> newExpressions = new ArrayList<LogicalExpressionReference>();
+ deepCopyExpressionRefs(newExpressions, op.getExpressions());
+ return new WriteOperator(newExpressions, op.getDataSink());
+ }
+
+ @Override
+ public ILogicalOperator visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
+ ArrayList<LogicalExpressionReference> newKeyExpressions = new ArrayList<LogicalExpressionReference>();
+ deepCopyExpressionRefs(newKeyExpressions, op.getKeyExpressions());
+ return new WriteResultOperator(op.getDataSource(), deepCopyExpressionRef(op.getPayloadExpression()),
+ newKeyExpressions);
+ }
+
+ @Override
+ public ILogicalOperator visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+ List<LogicalExpressionReference> newKeyExpressions = new ArrayList<LogicalExpressionReference>();
+ deepCopyExpressionRefs(newKeyExpressions, op.getPrimaryKeyExpressions());
+ return new InsertDeleteOperator(op.getDataSource(), deepCopyExpressionRef(op.getPayloadExpression()),
+ newKeyExpressions, op.getOperation());
+ }
+
+ @Override
+ public ILogicalOperator visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg)
+ throws AlgebricksException {
+ List<LogicalExpressionReference> newPrimaryKeyExpressions = new ArrayList<LogicalExpressionReference>();
+ deepCopyExpressionRefs(newPrimaryKeyExpressions, op.getPrimaryKeyExpressions());
+ List<LogicalExpressionReference> newSecondaryKeyExpressions = new ArrayList<LogicalExpressionReference>();
+ deepCopyExpressionRefs(newSecondaryKeyExpressions, op.getSecondaryKeyExpressions());
+ return new IndexInsertDeleteOperator(op.getDataSourceIndex(), newPrimaryKeyExpressions,
+ newSecondaryKeyExpressions, op.getOperation());
+ }
+
+ @Override
+ public ILogicalOperator visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
+ return new SinkOperator();
+ }
+
+ private void deepCopyExpressionRefs(List<LogicalExpressionReference> newExprs,
+ List<LogicalExpressionReference> oldExprs) {
+ for (LogicalExpressionReference oldExpr : oldExprs)
+ newExprs.add(new LogicalExpressionReference(((AbstractLogicalExpression) oldExpr.getExpression())
+ .cloneExpression()));
+ }
+
+ private LogicalExpressionReference deepCopyExpressionRef(LogicalExpressionReference oldExpr) {
+ return new LogicalExpressionReference(
+ ((AbstractLogicalExpression) oldExpr.getExpression()).cloneExpression());
+ }
+
+ private List<Pair<IOrder, LogicalExpressionReference>> deepCopyOrderAndExpression(
+ List<Pair<IOrder, LogicalExpressionReference>> ordersAndExprs) {
+ List<Pair<IOrder, LogicalExpressionReference>> newOrdersAndExprs = new ArrayList<Pair<IOrder, LogicalExpressionReference>>();
+ for (Pair<IOrder, LogicalExpressionReference> pair : ordersAndExprs)
+ newOrdersAndExprs.add(new Pair<IOrder, LogicalExpressionReference>(pair.first,
+ deepCopyExpressionRef(pair.second)));
+ return newOrdersAndExprs;
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismUtilities.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismUtilities.java
new file mode 100644
index 0000000..21271fd
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismUtilities.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class IsomorphismUtilities {
+
+ public static void mapVariablesTopDown(ILogicalOperator op, ILogicalOperator arg,
+ Map<LogicalVariable, LogicalVariable> variableMapping) throws AlgebricksException {
+ IsomorphismVariableMappingVisitor visitor = new IsomorphismVariableMappingVisitor(variableMapping);
+ op.accept(visitor, arg);
+ }
+
+ public static boolean isOperatorIsomorphic(ILogicalOperator op, ILogicalOperator arg) throws AlgebricksException {
+ IsomorphismOperatorVisitor visitor = new IsomorphismOperatorVisitor();
+ return op.accept(visitor, arg).booleanValue();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
new file mode 100644
index 0000000..214d6da
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -0,0 +1,416 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractAssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class IsomorphismVariableMappingVisitor implements ILogicalOperatorVisitor<Void, ILogicalOperator> {
+
+ private Map<LogicalVariable, LogicalVariable> variableMapping;
+
+ public IsomorphismVariableMappingVisitor(Map<LogicalVariable, LogicalVariable> variableMapping) {
+ this.variableMapping = variableMapping;
+ }
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapChildren(op, arg);
+ mapVariablesForAbstractAssign(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ mapChildren(op, arg);
+ mapVariablesForAbstractAssign(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapChildren(op, arg);
+ mapVariablesForGroupBy(op, arg);
+ mapVariablesInNestedPlans(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitDieOperator(DieOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ ILogicalOperator inputToCreator1 = op.getSourceOperator();
+ NestedTupleSourceOperator nts = (NestedTupleSourceOperator) arg;
+ ILogicalOperator inputToCreator2 = nts.getSourceOperator();
+ inputToCreator1.accept(this, inputToCreator2);
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapChildren(op, arg);
+ mapVariablesForAbstractAssign(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapChildren(op, arg);
+ mapVariablesInNestedPlans(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapChildren(op, arg);
+ mapVariablesForUnion(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteOperator(InsertDeleteOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ private void mapChildren(ILogicalOperator op, ILogicalOperator opArg) throws AlgebricksException {
+ List<LogicalOperatorReference> inputs = op.getInputs();
+ List<LogicalOperatorReference> inputsArg = opArg.getInputs();
+ if (inputs.size() != inputsArg.size())
+ throw new AlgebricksException("children are not isomoprhic");
+ for (int i = 0; i < inputs.size(); i++) {
+ ILogicalOperator input = inputs.get(i).getOperator();
+ ILogicalOperator inputArg = inputsArg.get(i).getOperator();
+ input.accept(this, inputArg);
+ }
+ }
+
+ private void mapVariables(ILogicalOperator left, ILogicalOperator right) throws AlgebricksException {
+ List<LogicalVariable> producedVarLeft = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> producedVarRight = new ArrayList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(left, producedVarLeft);
+ VariableUtilities.getProducedVariables(right, producedVarRight);
+ mapVariables(producedVarLeft, producedVarRight);
+ }
+
+ private void mapVariables(List<LogicalVariable> variablesLeft, List<LogicalVariable> variablesRight) {
+ if (variablesLeft.size() != variablesRight.size())
+ return;
+ int size = variablesLeft.size();
+ for (int i = 0; i < size; i++) {
+ LogicalVariable left = variablesLeft.get(i);
+ LogicalVariable right = variablesRight.get(i);
+ variableMapping.put(right, left);
+ }
+ }
+
+ private void mapVariablesForAbstractAssign(ILogicalOperator left, ILogicalOperator right)
+ throws AlgebricksException {
+ AbstractAssignOperator leftOp = (AbstractAssignOperator) left;
+ AbstractAssignOperator rightOp = (AbstractAssignOperator) right;
+ List<LogicalVariable> producedVarLeft = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> producedVarRight = new ArrayList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(left, producedVarLeft);
+ VariableUtilities.getProducedVariables(right, producedVarRight);
+ mapVariablesForAbstractAssign(producedVarLeft, leftOp.getExpressions(), producedVarRight,
+ rightOp.getExpressions());
+ }
+
+ private void mapVariablesForGroupBy(ILogicalOperator left, ILogicalOperator right) throws AlgebricksException {
+ GroupByOperator leftOp = (GroupByOperator) left;
+ GroupByOperator rightOp = (GroupByOperator) right;
+ List<Pair<LogicalVariable, LogicalExpressionReference>> leftPairs = leftOp.getGroupByList();
+ List<Pair<LogicalVariable, LogicalExpressionReference>> rightPairs = rightOp.getGroupByList();
+ mapVarExprPairList(leftPairs, rightPairs);
+ leftPairs = leftOp.getDecorList();
+ rightPairs = rightOp.getDecorList();
+ mapVarExprPairList(leftPairs, rightPairs);
+ }
+
+ private void mapVarExprPairList(List<Pair<LogicalVariable, LogicalExpressionReference>> leftPairs,
+ List<Pair<LogicalVariable, LogicalExpressionReference>> rightPairs) {
+ if (leftPairs.size() != rightPairs.size())
+ return;
+ for (int i = 0; i < leftPairs.size(); i++) {
+ ILogicalExpression exprLeft = leftPairs.get(i).second.getExpression();
+ LogicalVariable leftVar = leftPairs.get(i).first;
+ for (int j = 0; j < leftPairs.size(); j++) {
+ ILogicalExpression exprRight = copyExpressionAndSubtituteVars(rightPairs.get(j).second).getExpression();
+ if (exprLeft.equals(exprRight)) {
+ LogicalVariable rightVar = rightPairs.get(j).first;
+ if (rightVar != null && leftVar != null) {
+ variableMapping.put(rightVar, leftVar);
+ }
+ break;
+ }
+ }
+ }
+ }
+
+ private void mapVariablesForAbstractAssign(List<LogicalVariable> variablesLeft,
+ List<LogicalExpressionReference> exprsLeft, List<LogicalVariable> variablesRight,
+ List<LogicalExpressionReference> exprsRight) {
+ if (variablesLeft.size() != variablesRight.size())
+ return;
+ int size = variablesLeft.size();
+ for (int i = 0; i < size; i++) {
+ ILogicalExpression exprLeft = exprsLeft.get(i).getExpression();
+ LogicalVariable left = variablesLeft.get(i);
+ for (int j = 0; j < size; j++) {
+ ILogicalExpression exprRight = copyExpressionAndSubtituteVars(exprsRight.get(j)).getExpression();
+ if (exprLeft.equals(exprRight)) {
+ LogicalVariable right = variablesRight.get(j);
+ variableMapping.put(right, left);
+ break;
+ }
+ }
+ }
+ }
+
+ private void mapVariablesInNestedPlans(ILogicalOperator opOrigin, ILogicalOperator arg) throws AlgebricksException {
+ AbstractOperatorWithNestedPlans op = (AbstractOperatorWithNestedPlans) opOrigin;
+ AbstractOperatorWithNestedPlans argOp = (AbstractOperatorWithNestedPlans) arg;
+ List<ILogicalPlan> plans = op.getNestedPlans();
+ List<ILogicalPlan> plansArg = argOp.getNestedPlans();
+ if (plans.size() != plansArg.size())
+ return;
+ for (int i = 0; i < plans.size(); i++) {
+ List<LogicalOperatorReference> roots = plans.get(i).getRoots();
+ List<LogicalOperatorReference> rootsArg = plansArg.get(i).getRoots();
+ if (roots.size() != rootsArg.size())
+ return;
+ for (int j = 0; j < roots.size(); j++) {
+ ILogicalOperator topOp1 = roots.get(j).getOperator();
+ ILogicalOperator topOp2 = rootsArg.get(j).getOperator();
+ topOp1.accept(this, topOp2);
+ }
+ }
+ }
+
+ private void mapVariablesStandard(ILogicalOperator op, ILogicalOperator arg) throws AlgebricksException {
+ mapChildren(op, arg);
+ mapVariables(op, arg);
+ }
+
+ private LogicalExpressionReference copyExpressionAndSubtituteVars(LogicalExpressionReference expr) {
+ ILogicalExpression copy = ((AbstractLogicalExpression) expr.getExpression()).cloneExpression();
+ for (Entry<LogicalVariable, LogicalVariable> entry : variableMapping.entrySet())
+ copy.substituteVar(entry.getKey(), entry.getValue());
+ return new LogicalExpressionReference(copy);
+ }
+
+ private void mapVariablesForUnion(ILogicalOperator op, ILogicalOperator arg) {
+ UnionAllOperator union = (UnionAllOperator) op;
+ UnionAllOperator unionArg = (UnionAllOperator) arg;
+ mapVarTripleList(union.getVariableMappings(), unionArg.getVariableMappings());
+ }
+
+ private void mapVarTripleList(List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> leftTriples,
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> rightTriples) {
+ if (leftTriples.size() != rightTriples.size())
+ return;
+ for (int i = 0; i < leftTriples.size(); i++) {
+ LogicalVariable leftFirstVar = leftTriples.get(i).first;
+ LogicalVariable leftSecondVar = leftTriples.get(i).second;
+ LogicalVariable leftThirdVar = leftTriples.get(i).third;
+ for (int j = 0; j < rightTriples.size(); j++) {
+ LogicalVariable rightFirstVar = rightTriples.get(j).first;
+ LogicalVariable rightSecondVar = rightTriples.get(j).second;
+ LogicalVariable rightThirdVar = rightTriples.get(j).third;
+ if (varEquivalent(leftFirstVar, rightFirstVar) && varEquivalent(leftSecondVar, rightSecondVar)) {
+ variableMapping.put(rightThirdVar, leftThirdVar);
+ break;
+ }
+ }
+ }
+ }
+
+ private boolean varEquivalent(LogicalVariable left, LogicalVariable right) {
+ if (variableMapping.get(right) == null)
+ return false;
+ if (variableMapping.get(right).equals(left))
+ return true;
+ else
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
new file mode 100644
index 0000000..950772d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -0,0 +1,320 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableEvalSizeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractAssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LogicalPropertiesVectorImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+
+public class LogicalPropertiesVisitor implements ILogicalOperatorVisitor<Void, IOptimizationContext> {
+
+ public static void computeLogicalPropertiesDFS(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ LogicalPropertiesVisitor visitor = new LogicalPropertiesVisitor();
+ computeLogicalPropertiesRec(op, visitor, context);
+ }
+
+ private static void computeLogicalPropertiesRec(ILogicalOperator op, LogicalPropertiesVisitor visitor,
+ IOptimizationContext context) throws AlgebricksException {
+ for (LogicalOperatorReference ref : op.getInputs()) {
+ computeLogicalPropertiesRec(ref.getOperator(), visitor, context);
+ }
+ op.accept(visitor, context);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest("Logical properties visitor for " + op + ": "
+ + context.getLogicalPropertiesVector(op) + "\n");
+ }
+ }
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, IOptimizationContext context) throws AlgebricksException {
+ visitAssignment(op, context);
+ return null;
+ }
+
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, IOptimizationContext arg)
+ throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, IOptimizationContext arg) throws AlgebricksException {
+ propagateCardinalityAndFrameNumber(op, arg);
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, IOptimizationContext arg)
+ throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitDieOperator(DieOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, IOptimizationContext arg)
+ throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, IOptimizationContext arg) throws AlgebricksException {
+ Object annot1 = op.getAnnotations().get(OperatorAnnotations.CARDINALITY);
+ if (annot1 == null) {
+ return null;
+ }
+ Integer m = (Integer) annot1;
+ LogicalPropertiesVectorImpl v = new LogicalPropertiesVectorImpl();
+ v.setNumberOfTuples(m);
+ Object annot2 = op.getAnnotations().get(OperatorAnnotations.MAX_NUMBER_FRAMES);
+ if (annot2 != null) {
+ Integer f = (Integer) annot2;
+ v.setMaxOutputFrames(f);
+ }
+ arg.putLogicalPropertiesVector(op, v);
+ return null;
+ }
+
+ @Override
+ public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, IOptimizationContext arg)
+ throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, IOptimizationContext context) throws AlgebricksException {
+ propagateCardinalityAndFrameNumber(op, context);
+ return null;
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ visitAssignment(op, context);
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteOperator(InsertDeleteOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, IOptimizationContext arg)
+ throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, IOptimizationContext arg) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ private LogicalPropertiesVectorImpl propagateCardinality(ILogicalOperator op, IOptimizationContext context) {
+ ILogicalOperator op0 = op.getInputs().get(0).getOperator();
+ ILogicalPropertiesVector v0 = context.getLogicalPropertiesVector(op0);
+ if (v0 == null) {
+ return null;
+ }
+ LogicalPropertiesVectorImpl v = new LogicalPropertiesVectorImpl();
+ v.setNumberOfTuples(v0.getNumberOfTuples());
+ context.putLogicalPropertiesVector(op, v);
+ return v;
+ }
+
+ private void visitAssignment(AbstractAssignOperator op, IOptimizationContext context) throws AlgebricksException {
+ LogicalPropertiesVectorImpl v = propagateCardinality(op, context);
+ if (v != null && v.getNumberOfTuples() != null) {
+ IVariableEvalSizeEnvironment varSizeEnv = context.getVariableEvalSizeEnvironment();
+ IExpressionEvalSizeComputer evalSize = context.getExpressionEvalSizeComputer();
+ if (evalSize != null) {
+ ILogicalOperator op0 = op.getInputs().get(0).getOperator();
+ ILogicalPropertiesVector v0 = context.getLogicalPropertiesVector(op0);
+ if (v0 != null) {
+ long frames0 = v0.getMaxOutputFrames();
+ long overhead = 0; // added per tuple
+ for (LogicalExpressionReference exprRef : op.getExpressions()) {
+ int sz = evalSize.getEvalSize(exprRef.getExpression(), varSizeEnv);
+ if (sz == -1) {
+ return;
+ }
+ overhead += sz;
+ }
+ int frameSize = context.getFrameSize();
+ if (frameSize > 0) {
+ long sz = frames0 * frameSize + overhead * v.getNumberOfTuples();
+ int frames1 = (int) (sz / frameSize);
+ if (sz % frameSize > 0) {
+ frames1++;
+ }
+ v.setMaxOutputFrames(frames1);
+ }
+ }
+ }
+ }
+ }
+
+ public void propagateCardinalityAndFrameNumber(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ LogicalPropertiesVectorImpl v = propagateCardinality(op, context);
+ // propagate also max number of frames (conservatively)
+ ILogicalOperator op0 = op.getInputs().get(0).getOperator();
+ ILogicalPropertiesVector v0 = context.getLogicalPropertiesVector(op0);
+ if (v0 != null) {
+ v.setMaxOutputFrames(v0.getMaxOutputFrames());
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
new file mode 100644
index 0000000..1e991f1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -0,0 +1,246 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class ProducedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+ private Collection<LogicalVariable> producedVariables;
+
+ public ProducedVariableVisitor(Collection<LogicalVariable> producedVariables) throws AlgebricksException {
+ this.producedVariables = producedVariables;
+ }
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {
+ producedVariables.addAll(op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {
+ producedVariables.addAll(op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+ producedVariables.addAll(op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, Void arg) {
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ VariableUtilities.getLiveVariables(r.getOperator(), producedVariables);
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : op.getGroupByList()) {
+ if (p.first != null) {
+ producedVariables.add(p.first);
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : op.getDecorList()) {
+ if (p.first != null) {
+ producedVariables.add(p.first);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitDieOperator(DieOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {
+ producedVariables.addAll(op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
+ List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, usedVariables);
+ for (LogicalVariable v : op.getOutputVariables()) {
+ if (!usedVariables.contains(v)) {
+ producedVariables.add(v);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ VariableUtilities.getLiveVariables(r.getOperator(), producedVariables);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : op.getVariableMappings()) {
+ producedVariables.add(t.third);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+ producedVariables.addAll(op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {
+ producedVariables.addAll(op.getVariables());
+ LogicalVariable positionalVariable = op.getPositionalVariable();
+ if (positionalVariable != null) {
+ if (!producedVariables.contains(positionalVariable))
+ producedVariables.add(positionalVariable);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
new file mode 100644
index 0000000..be03961
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -0,0 +1,270 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class SchemaVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+ private Collection<LogicalVariable> schemaVariables;
+
+ public SchemaVariableVisitor(Collection<LogicalVariable> schemaVariables) {
+ this.schemaVariables = schemaVariables;
+ }
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {
+ schemaVariables.addAll(op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ VariableUtilities.getLiveVariables(r.getOperator(), schemaVariables);
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : op.getGroupByList()) {
+ if (p.first != null) {
+ schemaVariables.add(p.first);
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : op.getDecorList()) {
+ if (p.first != null) {
+ schemaVariables.add(p.first);
+ } else {
+ ILogicalExpression e = p.second.getExpression();
+ if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ schemaVariables.add(((VariableReferenceExpression) e).getVariableReference());
+ }
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitDieOperator(DieOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {
+ VariableUtilities.getLiveVariables(op.getSourceOperator(), schemaVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
+ schemaVariables.addAll(op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {
+ // VariableUtilities.getProducedVariables(op, schemaVariables);
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
+ schemaVariables.addAll(op.getOutputVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+ for (LogicalOperatorReference c : op.getInputs()) {
+ VariableUtilities.getLiveVariables(c.getOperator(), schemaVariables);
+ }
+ VariableUtilities.getProducedVariables(op, schemaVariables);
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ VariableUtilities.getLiveVariables(r.getOperator(), schemaVariables);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {
+ VariableUtilities.getProducedVariables(op, schemaVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ private void standardLayout(ILogicalOperator op) throws AlgebricksException {
+ for (LogicalOperatorReference c : op.getInputs()) {
+ VariableUtilities.getLiveVariables(c.getOperator(), schemaVariables);
+ }
+ VariableUtilities.getProducedVariables(op, schemaVariables);
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
new file mode 100644
index 0000000..d463805
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -0,0 +1,412 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class SubstituteVariableVisitor implements ILogicalOperatorVisitor<Void, Pair<LogicalVariable, LogicalVariable>> {
+
+ private final boolean goThroughNts;
+ private final ITypingContext ctx;
+
+ public SubstituteVariableVisitor(boolean goThroughNts, ITypingContext ctx) {
+ this.goThroughNts = goThroughNts;
+ this.ctx = ctx;
+ }
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<LogicalVariable> variables = op.getVariables();
+ int n = variables.size();
+ for (int i = 0; i < n; i++) {
+ if (variables.get(i).equals(pair.first)) {
+ variables.set(i, pair.second);
+ } else {
+ op.getExpressions().get(i).getExpression().substituteVar(pair.first, pair.second);
+ }
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<LogicalVariable> variables = op.getVariables();
+ int n = variables.size();
+ for (int i = 0; i < n; i++) {
+ if (variables.get(i).equals(pair.first)) {
+ variables.set(i, pair.second);
+ } else {
+ op.getExpressions().get(i).getExpression().substituteVar(pair.first, pair.second);
+ }
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<LogicalVariable> variables = op.getVariables();
+ for (int i = 0; i < variables.size(); i++) {
+ if (variables.get(i) == pair.first) {
+ variables.set(i, pair.second);
+ return null;
+ }
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ for (LogicalExpressionReference eRef : op.getExpressions()) {
+ eRef.getExpression().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair) {
+ // does not use any variable
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, Pair<LogicalVariable, LogicalVariable> pair) {
+ // does not use any variable
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ subst(pair.first, pair.second, op.getGroupByList());
+ subst(pair.first, pair.second, op.getDecorList());
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getOperator(), pair.first,
+ pair.second, goThroughNts, ctx);
+ }
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ op.getCondition().getExpression().substituteVar(pair.first, pair.second);
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ op.getCondition().getExpression().substituteVar(pair.first, pair.second);
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ op.getMaxObjects().getExpression().substituteVar(pair.first, pair.second);
+ ILogicalExpression offset = op.getOffset().getExpression();
+ if (offset != null) {
+ offset.substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitDieOperator(DieOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ op.getAfterObjects().getExpression().substituteVar(pair.first, pair.second);
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ for (Pair<IOrder, LogicalExpressionReference> oe : op.getOrderExpressions()) {
+ oe.second.getExpression().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ for (LogicalExpressionReference e : op.getExpressions()) {
+ e.getExpression().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<LogicalVariable> usedVariables = op.getVariables();
+ int n = usedVariables.size();
+ for (int i = 0; i < n; i++) {
+ LogicalVariable v = usedVariables.get(i);
+ if (v.equals(pair.first)) {
+ usedVariables.set(i, pair.second);
+ }
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<LogicalVariable> variables = op.getVariables();
+ int n = variables.size();
+ for (int i = 0; i < n; i++) {
+ if (variables.get(i).equals(pair.first)) {
+ variables.set(i, pair.second);
+ } else {
+ op.getExpressions().get(i).getExpression().substituteVar(pair.first, pair.second);
+ }
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ substInArray(op.getInputVariables(), pair.first, pair.second);
+ substInArray(op.getOutputVariables(), pair.first, pair.second);
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, Pair<LogicalVariable, LogicalVariable> pair) {
+ op.getCondition().getExpression().substituteVar(pair.first, pair.second);
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getOperator(), pair.first,
+ pair.second, goThroughNts, ctx);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = op.getVariableMappings();
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
+ if (t.first.equals(pair.first)) {
+ t.first = pair.second;
+ }
+ if (t.second.equals(pair.first)) {
+ t.second = pair.second;
+ }
+ if (t.third.equals(pair.first)) {
+ t.third = pair.second;
+ }
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<LogicalVariable> variables = op.getVariables();
+ for (int i = 0; i < variables.size(); i++) {
+ if (variables.get(i) == pair.first) {
+ variables.set(i, pair.second);
+ return null;
+ }
+ }
+ op.getExpressionRef().getExpression().substituteVar(pair.first, pair.second);
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ List<LogicalVariable> variables = op.getVariables();
+ for (int i = 0; i < variables.size(); i++) {
+ if (variables.get(i) == pair.first) {
+ variables.set(i, pair.second);
+ return null;
+ }
+ }
+ op.getExpressionRef().getExpression().substituteVar(pair.first, pair.second);
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ for (LogicalExpressionReference e : op.getExpressions()) {
+ e.getExpression().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ op.getPayloadExpression().getExpression().substituteVar(pair.first, pair.second);
+ for (LogicalExpressionReference e : op.getKeyExpressions()) {
+ e.getExpression().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ private void subst(LogicalVariable v1, LogicalVariable v2,
+ List<Pair<LogicalVariable, LogicalExpressionReference>> varExprPairList) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> ve : varExprPairList) {
+ if (ve.first != null && ve.first.equals(v1)) {
+ ve.first = v2;
+ return;
+ }
+ ve.second.getExpression().substituteVar(v1, v2);
+ }
+ }
+
+ private void substInArray(ArrayList<LogicalVariable> varArray, LogicalVariable v1, LogicalVariable v2) {
+ for (int i = 0; i < varArray.size(); i++) {
+ LogicalVariable v = varArray.get(i);
+ if (v == v1) {
+ varArray.set(i, v2);
+ }
+ }
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+ throws AlgebricksException {
+ op.substituteVar(arg.first, arg.second);
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteOperator(InsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ op.getPayloadExpression().getExpression().substituteVar(pair.first, pair.second);
+ for (LogicalExpressionReference e : op.getPrimaryKeyExpressions()) {
+ e.getExpression().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ for (LogicalExpressionReference e : op.getPrimaryKeyExpressions()) {
+ e.getExpression().substituteVar(pair.first, pair.second);
+ }
+ for (LogicalExpressionReference e : op.getSecondaryKeyExpressions()) {
+ e.getExpression().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ return null;
+ }
+
+ private void substVarTypes(ILogicalOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+ throws AlgebricksException {
+ if (ctx == null) {
+ return;
+ }
+ IVariableTypeEnvironment env = ctx.getOutputTypeEnvironment(op);
+ env.substituteProducedVariable(arg.first, arg.second);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
new file mode 100644
index 0000000..08512d8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -0,0 +1,293 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+ private Collection<LogicalVariable> usedVariables;
+
+ public UsedVariableVisitor(Collection<LogicalVariable> usedVariables) {
+ this.usedVariables = usedVariables;
+ }
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, Void arg) {
+ for (LogicalExpressionReference exprRef : op.getExpressions()) {
+ exprRef.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, Void arg) {
+ for (LogicalExpressionReference exprRef : op.getExpressions()) {
+ exprRef.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) {
+ // does not use any variable
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, Void arg) {
+ for (LogicalExpressionReference eRef : op.getExpressions()) {
+ eRef.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) {
+ // does not use any variable
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, Void arg) {
+ // does not use any variable
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ VariableUtilities.getUsedVariablesInDescendantsAndSelf(r.getOperator(), usedVariables);
+ }
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> g : op.getGroupByList()) {
+ g.second.getExpression().getUsedVariables(usedVariables);
+ }
+ for (Pair<LogicalVariable, LogicalExpressionReference> g : op.getDecorList()) {
+ g.second.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) {
+ op.getCondition().getExpression().getUsedVariables(usedVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) {
+ op.getCondition().getExpression().getUsedVariables(usedVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, Void arg) {
+ op.getMaxObjects().getExpression().getUsedVariables(usedVariables);
+ ILogicalExpression offsetExpr = op.getOffset().getExpression();
+ if (offsetExpr != null) {
+ offsetExpr.getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitDieOperator(DieOperator op, Void arg) {
+ op.getAfterObjects().getExpression().getUsedVariables(usedVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) {
+ // does not use any variable
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, Void arg) {
+ for (Pair<IOrder, LogicalExpressionReference> oe : op.getOrderExpressions()) {
+ oe.second.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg) {
+ for (LogicalExpressionReference e : op.getExpressions()) {
+ e.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, Void arg) {
+ List<LogicalVariable> parameterVariables = op.getVariables();
+ for (LogicalVariable v : parameterVariables) {
+ if (!usedVariables.contains(v)) {
+ usedVariables.add(v);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) {
+ for (LogicalExpressionReference exprRef : op.getExpressions()) {
+ exprRef.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, Void arg) {
+ List<LogicalVariable> parameterVariables = op.getInputVariables();
+ for (LogicalVariable v : parameterVariables) {
+ if (!usedVariables.contains(v)) {
+ usedVariables.add(v);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, Void arg) {
+ op.getCondition().getExpression().getUsedVariables(usedVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ VariableUtilities.getUsedVariablesInDescendantsAndSelf(r.getOperator(), usedVariables);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, Void arg) {
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> m : op.getVariableMappings()) {
+ if (!usedVariables.contains(m.first)) {
+ usedVariables.add(m.first);
+ }
+ if (!usedVariables.contains(m.second)) {
+ usedVariables.add(m.second);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) {
+ op.getExpressionRef().getExpression().getUsedVariables(usedVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, Void arg) {
+ op.getExpressionRef().getExpression().getUsedVariables(usedVariables);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, Void arg) {
+ for (LogicalExpressionReference expr : op.getExpressions()) {
+ expr.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, Void arg) {
+ op.getPayloadExpression().getExpression().getUsedVariables(usedVariables);
+ for (LogicalExpressionReference e : op.getKeyExpressions()) {
+ e.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) {
+ op.getPayloadExpression().getExpression().getUsedVariables(usedVariables);
+ for (LogicalExpressionReference e : op.getPrimaryKeyExpressions()) {
+ e.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Void arg) {
+ for (LogicalExpressionReference e : op.getPrimaryKeyExpressions()) {
+ e.getExpression().getUsedVariables(usedVariables);
+ }
+ for (LogicalExpressionReference e : op.getSecondaryKeyExpressions()) {
+ e.getExpression().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, Void arg) {
+ return null;
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
new file mode 100644
index 0000000..a918bed
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class VariableUtilities {
+
+ public static void getUsedVariables(ILogicalOperator op, Collection<LogicalVariable> usedVariables)
+ throws AlgebricksException {
+ ILogicalOperatorVisitor<Void, Void> visitor = new UsedVariableVisitor(usedVariables);
+ op.accept(visitor, null);
+ }
+
+ public static void getProducedVariables(ILogicalOperator op, Collection<LogicalVariable> producedVariables)
+ throws AlgebricksException {
+ ILogicalOperatorVisitor<Void, Void> visitor = new ProducedVariableVisitor(producedVariables);
+ op.accept(visitor, null);
+ }
+
+ public static void getLiveVariables(ILogicalOperator op, Collection<LogicalVariable> schemaVariables)
+ throws AlgebricksException {
+ ILogicalOperatorVisitor<Void, Void> visitor = new SchemaVariableVisitor(schemaVariables);
+ op.accept(visitor, null);
+ }
+
+ public static void getUsedVariablesInDescendantsAndSelf(ILogicalOperator op, Collection<LogicalVariable> vars)
+ throws AlgebricksException {
+ // DFS traversal
+ VariableUtilities.getUsedVariables(op, vars);
+ for (LogicalOperatorReference c : op.getInputs()) {
+ getUsedVariablesInDescendantsAndSelf(c.getOperator(), vars);
+ }
+ }
+
+ public static void substituteVariables(ILogicalOperator op, LogicalVariable v1, LogicalVariable v2,
+ ITypingContext ctx) throws AlgebricksException {
+ substituteVariables(op, v1, v2, true, ctx);
+ }
+
+ public static void substituteVariables(ILogicalOperator op, LogicalVariable v1, LogicalVariable v2,
+ boolean goThroughNts, ITypingContext ctx) throws AlgebricksException {
+ ILogicalOperatorVisitor<Void, Pair<LogicalVariable, LogicalVariable>> visitor = new SubstituteVariableVisitor(
+ goThroughNts, ctx);
+ op.accept(visitor, new Pair<LogicalVariable, LogicalVariable>(v1, v2));
+ }
+
+ public static <T> boolean varListEqualUnordered(List<T> var, List<T> varArg) {
+ Set<T> varSet = new HashSet<T>();
+ Set<T> varArgSet = new HashSet<T>();
+ varSet.addAll(var);
+ varArgSet.addAll(varArg);
+ return varSet.equals(varArgSet);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java
new file mode 100644
index 0000000..766237c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractExchangePOperator extends AbstractPhysicalOperator {
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ Pair<IConnectorDescriptor, TargetConstraint> connPair = createConnectorDescriptor(builder.getJobSpec(), op,
+ opSchema, context);
+ builder.contributeConnectorWithTargetConstraint(op, connPair.first, connPair.second);
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ public abstract Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
new file mode 100644
index 0000000..61422af
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
@@ -0,0 +1,191 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public abstract class AbstractHashJoinPOperator extends AbstractJoinPOperator {
+
+ protected List<LogicalVariable> keysLeftBranch;
+ protected List<LogicalVariable> keysRightBranch;
+
+ public AbstractHashJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+ List<LogicalVariable> sideLeftOfEqualities, List<LogicalVariable> sideRightOfEqualities) {
+ super(kind, partitioningType);
+ this.keysLeftBranch = sideLeftOfEqualities;
+ this.keysRightBranch = sideRightOfEqualities;
+ }
+
+ public List<LogicalVariable> getKeysLeftBranch() {
+ return keysLeftBranch;
+ }
+
+ public List<LogicalVariable> getKeysRightBranch() {
+ return keysRightBranch;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context)
+ throws AlgebricksException {
+ IPartitioningProperty pp;
+ AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPhysicalPropertiesVector pv0 = op0.getPhysicalOperator().getDeliveredProperties();
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) op.getInputs().get(1).getOperator();
+ IPhysicalPropertiesVector pv1 = op1.getPhysicalOperator().getDeliveredProperties();
+
+ if (pv0 == null || pv1 == null) {
+ pp = null;
+ } else {
+ pp = pv0.getPartitioningProperty();
+ }
+ } else {
+ pp = null;
+ }
+ this.deliveredProperties = new StructuralPropertiesVector(pp, deliveredLocalProperties(iop, context));
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
+ IPhysicalPropertiesVector reqdByParent) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+ // In a cost-based optimizer, we would also try to propagate the
+ // parent's partitioning requirements.
+ AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+ IPartitioningProperty pp1 = null;
+ IPartitioningProperty pp2 = null;
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+ switch (partitioningType) {
+ case PAIRWISE: {
+ pp1 = new UnorderedPartitionedProperty(new HashSet<LogicalVariable>(keysLeftBranch), null);
+ pp2 = new UnorderedPartitionedProperty(new HashSet<LogicalVariable>(keysRightBranch), null);
+ break;
+ }
+ case BROADCAST: {
+ pp2 = new BroadcastPartitioningProperty(null);
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ pv[0] = new StructuralPropertiesVector(pp1, null);
+ pv[1] = new StructuralPropertiesVector(pp2, null);
+
+ IPartitioningRequirementsCoordinator prc;
+ switch (kind) {
+ case INNER: {
+ prc = IPartitioningRequirementsCoordinator.EQCLASS_PARTITIONING_COORDINATOR;
+ break;
+ }
+ case LEFT_OUTER: {
+ prc = new IPartitioningRequirementsCoordinator() {
+
+ @Override
+ public Pair<Boolean, IPartitioningProperty> coordinateRequirements(
+ IPartitioningProperty requirements, IPartitioningProperty firstDeliveredPartitioning,
+ ILogicalOperator op, IOptimizationContext context) throws AlgebricksException {
+ if (firstDeliveredPartitioning != null
+ && firstDeliveredPartitioning.getPartitioningType() == requirements
+ .getPartitioningType()) {
+ switch (requirements.getPartitioningType()) {
+ case UNORDERED_PARTITIONED: {
+ UnorderedPartitionedProperty upp1 = (UnorderedPartitionedProperty) firstDeliveredPartitioning;
+ Set<LogicalVariable> set1 = upp1.getColumnSet();
+ UnorderedPartitionedProperty uppreq = (UnorderedPartitionedProperty) requirements;
+ Set<LogicalVariable> modifuppreq = new HashSet<LogicalVariable>();
+ Map<LogicalVariable, EquivalenceClass> eqmap = context.getEquivalenceClassMap(op);
+ Set<LogicalVariable> covered = new HashSet<LogicalVariable>();
+ for (LogicalVariable r : uppreq.getColumnSet()) {
+ EquivalenceClass ecSnd = eqmap.get(r);
+ boolean found = false;
+ int j = 0;
+ for (LogicalVariable rvar : keysRightBranch) {
+ if (rvar == r || ecSnd != null && eqmap.get(rvar) == ecSnd) {
+ found = true;
+ break;
+ }
+ j++;
+ }
+ if (!found) {
+ throw new IllegalStateException("Did not find a variable equivalent to "
+ + r + " among " + keysRightBranch);
+ }
+ LogicalVariable v2 = keysLeftBranch.get(j);
+ EquivalenceClass ecFst = eqmap.get(v2);
+ for (LogicalVariable vset1 : set1) {
+ if (vset1 == v2 || ecFst != null && eqmap.get(vset1) == ecFst) {
+ covered.add(vset1);
+ modifuppreq.add(r);
+ break;
+ }
+ }
+ }
+ if (!covered.equals(set1)) {
+ throw new AlgebricksException("Could not modify " + requirements
+ + " to agree with partitioning property " + firstDeliveredPartitioning
+ + " delivered by previous input operator.");
+ }
+ UnorderedPartitionedProperty upp2 = new UnorderedPartitionedProperty(modifuppreq,
+ requirements.getNodeDomain());
+ return new Pair<Boolean, IPartitioningProperty>(false, upp2);
+ }
+ case ORDERED_PARTITIONED: {
+ throw new NotImplementedException();
+ }
+ }
+ }
+ return new Pair<Boolean, IPartitioningProperty>(true, requirements);
+ }
+ };
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+
+ return new PhysicalRequirements(pv, prc);
+ }
+
+ protected abstract List<ILocalStructuralProperty> deliveredLocalProperties(ILogicalOperator op,
+ IOptimizationContext context) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractJoinPOperator.java
new file mode 100644
index 0000000..68e3a51
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractJoinPOperator.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+
+public abstract class AbstractJoinPOperator extends AbstractPhysicalOperator {
+
+ public enum JoinPartitioningType {
+ PAIRWISE, BROADCAST
+ }
+
+ protected final JoinKind kind;
+ protected final JoinPartitioningType partitioningType;
+
+ public AbstractJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType) {
+ this.kind = kind;
+ this.partitioningType = partitioningType;
+ }
+
+ public JoinKind getKind() {
+ return kind;
+ }
+
+ public JoinPartitioningType getPartitioningType() {
+ return partitioningType;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
new file mode 100644
index 0000000..038793e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
@@ -0,0 +1,137 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.PlanCompiler;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractPhysicalOperator implements IPhysicalOperator {
+
+ protected IPhysicalPropertiesVector deliveredProperties;
+ private boolean disableJobGenBelow = false;
+ private Object hostQueryContext;
+
+ @Override
+ public final IPhysicalPropertiesVector getDeliveredProperties() {
+ return deliveredProperties;
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString();
+ }
+
+ public void setHostQueryContext(Object context) {
+ this.hostQueryContext = context;
+ }
+
+ public Object getHostQueryContext() {
+ return hostQueryContext;
+ }
+
+ protected PhysicalRequirements emptyUnaryRequirements() {
+ StructuralPropertiesVector[] req = new StructuralPropertiesVector[] { StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR };
+ return new PhysicalRequirements(req, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void disableJobGenBelowMe() {
+ this.disableJobGenBelow = true;
+ }
+
+ @Override
+ public boolean isJobGenDisabledBelowMe() {
+ return disableJobGenBelow;
+ }
+
+ protected void contributeOpDesc(IHyracksJobBuilder builder, AbstractLogicalOperator op, IOperatorDescriptor opDesc) {
+ if (op.getExecutionMode() == ExecutionMode.UNPARTITIONED) {
+ AlgebricksPartitionConstraint apc = new AlgebricksCountPartitionConstraint(1);
+ builder.contributeAlgebricksPartitionConstraint(opDesc, apc);
+ }
+ builder.contributeHyracksOperator(op, opDesc);
+ }
+
+ protected AlgebricksPipeline[] compileSubplans(IOperatorSchema outerPlanSchema,
+ AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, JobGenContext context)
+ throws AlgebricksException {
+ AlgebricksPipeline[] subplans = new AlgebricksPipeline[npOp.getNestedPlans().size()];
+ PlanCompiler pc = new PlanCompiler(context);
+ int i = 0;
+ for (ILogicalPlan p : npOp.getNestedPlans()) {
+ subplans[i++] = buildPipelineWithProjection(p, outerPlanSchema, npOp, opSchema, pc);
+ }
+ return subplans;
+ }
+
+ private AlgebricksPipeline buildPipelineWithProjection(ILogicalPlan p, IOperatorSchema outerPlanSchema,
+ AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, PlanCompiler pc) throws AlgebricksException {
+ if (p.getRoots().size() > 1) {
+ throw new NotImplementedException("Nested plans with several roots are not supported.");
+ }
+ JobSpecification nestedJob = pc.compilePlan(p, outerPlanSchema);
+ ILogicalOperator topOpInSubplan = p.getRoots().get(0).getOperator();
+ JobGenContext context = pc.getContext();
+ IOperatorSchema topOpInSubplanScm = context.getSchema(topOpInSubplan);
+ opSchema.addAllVariables(topOpInSubplanScm);
+
+ Map<OperatorDescriptorId, IOperatorDescriptor> opMap = nestedJob.getOperatorMap();
+ if (opMap.size() != 1) {
+ throw new AlgebricksException(
+ "Attempting to construct a nested plan with "
+ + opMap.size()
+ + " operator descriptors. Currently, nested plans can only consist in linear pipelines of Asterix micro operators.");
+ }
+
+ for (OperatorDescriptorId oid : opMap.keySet()) {
+ IOperatorDescriptor opd = opMap.get(oid);
+ if (!(opd instanceof AlgebricksMetaOperatorDescriptor)) {
+ throw new AlgebricksException(
+ "Can only generate Hyracks jobs for pipelinable Asterix nested plans, not for "
+ + opd.getClass().getName());
+ }
+ AlgebricksMetaOperatorDescriptor amod = (AlgebricksMetaOperatorDescriptor) opd;
+
+ return amod.getPipeline();
+ // we suppose that the top operator in the subplan already does the
+ // projection for us
+ }
+
+ throw new IllegalStateException();
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPreclusteredGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPreclusteredGroupByPOperator.java
new file mode 100644
index 0000000..1aee87b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPreclusteredGroupByPOperator.java
@@ -0,0 +1,238 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public abstract class AbstractPreclusteredGroupByPOperator extends AbstractPhysicalOperator {
+
+ protected List<LogicalVariable> columnList;
+
+ public AbstractPreclusteredGroupByPOperator(List<LogicalVariable> columnList) {
+ this.columnList = columnList;
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + columnList;
+ }
+
+ public List<LogicalVariable> getGbyColumns() {
+ return columnList;
+ }
+
+ public void setGbyColumns(List<LogicalVariable> gByColumns) {
+ this.columnList = gByColumns;
+ }
+
+ // Obs: We don't propagate properties corresponding to decors, since they
+ // are func. dep. on the group-by variables.
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ List<ILocalStructuralProperty> propsLocal = new LinkedList<ILocalStructuralProperty>();
+ GroupByOperator gby = (GroupByOperator) op;
+ ILogicalOperator op2 = gby.getInputs().get(0).getOperator();
+ IPhysicalPropertiesVector childProp = op2.getDeliveredPhysicalProperties();
+ IPartitioningProperty pp = childProp.getPartitioningProperty();
+ List<ILocalStructuralProperty> childLocals = childProp.getLocalProperties();
+ if (childLocals != null) {
+ for (ILocalStructuralProperty lsp : childLocals) {
+ boolean failed = false;
+ switch (lsp.getPropertyType()) {
+ case LOCAL_GROUPING_PROPERTY: {
+ LocalGroupingProperty lgp = (LocalGroupingProperty) lsp;
+ Set<LogicalVariable> colSet = new HashSet<LogicalVariable>();
+ for (LogicalVariable v : lgp.getColumnSet()) {
+ LogicalVariable v2 = getLhsGbyVar(gby, v);
+ if (v2 != null) {
+ colSet.add(v2);
+ } else {
+ failed = true;
+ }
+ }
+ if (!failed) {
+ propsLocal.add(new LocalGroupingProperty(colSet));
+ }
+ break;
+ }
+ case LOCAL_ORDER_PROPERTY: {
+ LocalOrderProperty lop = (LocalOrderProperty) lsp;
+ OrderColumn oc = lop.getOrderColumn();
+ LogicalVariable v2 = getLhsGbyVar(gby, oc.getColumn());
+ if (v2 != null) {
+ propsLocal.add(new LocalOrderProperty(new OrderColumn(v2, oc.getOrder())));
+ } else {
+ failed = true;
+ }
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ if (failed) {
+ break;
+ }
+ }
+ }
+ deliveredProperties = new StructuralPropertiesVector(pp, propsLocal);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+ List<ILocalStructuralProperty> localProps = null;
+
+ localProps = new ArrayList<ILocalStructuralProperty>(1);
+ Set<LogicalVariable> gbvars = new HashSet<LogicalVariable>(columnList);
+ LocalGroupingProperty groupProp = new LocalGroupingProperty(gbvars, new ArrayList<LogicalVariable>(columnList));
+
+ GroupByOperator gby = (GroupByOperator) op;
+ boolean goon = true;
+ for (ILogicalPlan p : gby.getNestedPlans()) {
+ // try to propagate secondary order requirements from nested
+ // groupings
+ for (LogicalOperatorReference r : p.getRoots()) {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) r.getOperator();
+ if (op1.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getOperator();
+ IPhysicalOperator pop2 = op2.getPhysicalOperator();
+ if (pop2 instanceof AbstractPreclusteredGroupByPOperator) {
+ List<LogicalVariable> sndOrder = ((AbstractPreclusteredGroupByPOperator) pop2).getGbyColumns();
+ groupProp.getColumnSet().addAll(sndOrder);
+ groupProp.getPreferredOrderEnforcer().addAll(sndOrder);
+ goon = false;
+ break;
+ }
+ }
+ }
+ if (!goon) {
+ break;
+ }
+ }
+
+ localProps.add(groupProp);
+
+ if (reqdByParent != null) {
+ // propagate parent requirements
+ List<ILocalStructuralProperty> lpPar = reqdByParent.getLocalProperties();
+ if (lpPar != null) {
+ boolean allOk = true;
+ List<ILocalStructuralProperty> props = new ArrayList<ILocalStructuralProperty>(lpPar.size());
+ for (ILocalStructuralProperty prop : lpPar) {
+ if (prop.getPropertyType() != PropertyType.LOCAL_ORDER_PROPERTY) {
+ allOk = false;
+ break;
+ }
+ LocalOrderProperty lop = (LocalOrderProperty) prop;
+ LogicalVariable ord = lop.getColumn();
+ Pair<LogicalVariable, LogicalExpressionReference> p = getGbyPairByRhsVar(gby, ord);
+ if (p == null) {
+ p = getDecorPairByRhsVar(gby, ord);
+ if (p == null) {
+ allOk = false;
+ break;
+ }
+ }
+ ILogicalExpression e = p.second.getExpression();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new IllegalStateException(
+ "Right hand side of group-by assignment should have been normalized to a variable reference.");
+ }
+ LogicalVariable v = ((VariableReferenceExpression) e).getVariableReference();
+ props.add(new LocalOrderProperty(new OrderColumn(v, lop.getOrder())));
+ }
+ List<FunctionalDependency> fdList = new ArrayList<FunctionalDependency>();
+ for (Pair<LogicalVariable, LogicalExpressionReference> decorPair : gby.getDecorList()) {
+ List<LogicalVariable> hd = gby.getGbyVarList();
+ List<LogicalVariable> tl = new ArrayList<LogicalVariable>(1);
+ tl.add(((VariableReferenceExpression) decorPair.second.getExpression()).getVariableReference());
+ fdList.add(new FunctionalDependency(hd, tl));
+ }
+ if (allOk
+ && PropertiesUtil.matchLocalProperties(localProps, props,
+ new HashMap<LogicalVariable, EquivalenceClass>(), fdList)) {
+ localProps = props;
+ }
+ }
+ }
+
+ IPartitioningProperty pp = null;
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) op;
+ if (aop.getExecutionMode() == ExecutionMode.PARTITIONED) {
+ pp = new UnorderedPartitionedProperty(new HashSet<LogicalVariable>(columnList), null);
+ }
+ pv[0] = new StructuralPropertiesVector(pp, localProps);
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ private static Pair<LogicalVariable, LogicalExpressionReference> getGbyPairByRhsVar(GroupByOperator gby,
+ LogicalVariable var) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> ve : gby.getGroupByList()) {
+ if (ve.first == var) {
+ return ve;
+ }
+ }
+ return null;
+ }
+
+ private static Pair<LogicalVariable, LogicalExpressionReference> getDecorPairByRhsVar(GroupByOperator gby,
+ LogicalVariable var) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> ve : gby.getDecorList()) {
+ if (ve.first == var) {
+ return ve;
+ }
+ }
+ return null;
+ }
+
+ private static LogicalVariable getLhsGbyVar(GroupByOperator gby, LogicalVariable var) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> ve : gby.getGroupByList()) {
+ ILogicalExpression e = ve.second.getExpression();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new IllegalStateException(
+ "Right hand side of group by assignment should have been normalized to a variable reference.");
+ }
+ LogicalVariable v = ((VariableReferenceExpression) e).getVariableReference();
+ if (v == var) {
+ return ve.first;
+ }
+ }
+ return null;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPropagatePropertiesForUsedVariablesPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPropagatePropertiesForUsedVariablesPOperator.java
new file mode 100644
index 0000000..2974753
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPropagatePropertiesForUsedVariablesPOperator.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+
+public abstract class AbstractPropagatePropertiesForUsedVariablesPOperator extends AbstractPhysicalOperator {
+
+ public void computeDeliveredPropertiesForUsedVariables(ILogicalOperator op, List<LogicalVariable> usedVariables) {
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ IPartitioningProperty pp = op2.getDeliveredPhysicalProperties().getPartitioningProperty();
+ List<ILocalStructuralProperty> downPropsLocal = op2.getDeliveredPhysicalProperties().getLocalProperties();
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
+ for (ILocalStructuralProperty lsp : downPropsLocal) {
+ LinkedList<LogicalVariable> cols = new LinkedList<LogicalVariable>();
+ lsp.getColumns(cols);
+ if (usedVariables.containsAll(cols)) {
+ propsLocal.add(lsp);
+ }
+ }
+ deliveredProperties = new StructuralPropertiesVector(pp, propsLocal);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractScanPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractScanPOperator.java
new file mode 100644
index 0000000..347d580
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractScanPOperator.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+
+public abstract class AbstractScanPOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
new file mode 100644
index 0000000..227ed73e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public abstract class AbstractStableSortPOperator extends AbstractPhysicalOperator {
+
+ protected OrderColumn[] sortColumns;
+ protected List<ILocalStructuralProperty> orderProps;
+
+ public AbstractStableSortPOperator() {
+ }
+
+ public OrderColumn[] getSortColumns() {
+ return sortColumns;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ // if (orderProps == null) { // to do caching, we need some mechanism to
+ // invalidate cache
+ computeLocalProperties(op);
+ // }
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ StructuralPropertiesVector childProp = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties();
+ deliveredProperties = new StructuralPropertiesVector(childProp.getPartitioningProperty(), orderProps);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
+ IPhysicalPropertiesVector reqdByParent) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+ if (orderProps == null) {
+ computeLocalProperties(op);
+ }
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(
+ IPartitioningProperty.UNPARTITIONED, orderProps) };
+ return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ } else {
+ return emptyUnaryRequirements();
+ }
+ }
+
+ public void computeLocalProperties(ILogicalOperator op) {
+ orderProps = new LinkedList<ILocalStructuralProperty>();
+
+ OrderOperator ord = (OrderOperator) op;
+ for (Pair<IOrder, LogicalExpressionReference> p : ord.getOrderExpressions()) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ LogicalVariable var = varRef.getVariableReference();
+ switch (p.first.getKind()) {
+ case ASC: {
+ orderProps.add(new LocalOrderProperty(new OrderColumn(var, OrderKind.ASC)));
+ break;
+ }
+ case DESC: {
+ orderProps.add(new LocalOrderProperty(new OrderColumn(var, OrderKind.DESC)));
+ break;
+ }
+ default: {
+ throw new NotImplementedException();
+ }
+ }
+ } else {
+ throw new IllegalStateException();
+ }
+ }
+
+ int n = orderProps.size();
+ sortColumns = new OrderColumn[n];
+ int i = 0;
+ for (ILocalStructuralProperty prop : orderProps) {
+ sortColumns[i++] = ((LocalOrderProperty) prop).getOrderColumn();
+ }
+ }
+
+ public List<ILocalStructuralProperty> getOrderProperties() {
+ return orderProps;
+ }
+
+ @Override
+ public String toString() {
+ if (orderProps == null) {
+ return getOperatorTag().toString();
+ } else {
+ return getOperatorTag().toString() + " " + orderProps;
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
new file mode 100644
index 0000000..c7215fe
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.AggregateRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class AggregatePOperator extends AbstractPhysicalOperator {
+
+ public AggregatePOperator() {
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.AGGREGATE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPhysicalPropertiesVector childProps = op2.getDeliveredPhysicalProperties();
+ deliveredProperties = new StructuralPropertiesVector(childProps.getPartitioningProperty(),
+ new ArrayList<ILocalStructuralProperty>(0));
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ AggregateOperator aggOp = (AggregateOperator) op;
+ List<LogicalVariable> variables = aggOp.getVariables();
+ List<LogicalExpressionReference> expressions = aggOp.getExpressions();
+ int[] outColumns = new int[variables.size()];
+ for (int i = 0; i < outColumns.length; i++) {
+ outColumns[i] = opSchema.findVariable(variables.get(i));
+ }
+ IAggregateFunctionFactory[] aggFactories = new IAggregateFunctionFactory[expressions.size()];
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ for (int i = 0; i < aggFactories.length; i++) {
+ AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) expressions.get(i)
+ .getExpression();
+ aggFactories[i] = exprJobGen.createAggregateFunctionFactory(aggFun, context.getTypeEnvironment(op
+ .getInputs().get(0).getOperator()), inputSchemas, context);
+ }
+
+ AggregateRuntimeFactory runtime = new AggregateRuntimeFactory(aggFactories);
+
+ // contribute one Asterix framewriter
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ builder.contributeMicroOperator(aggOp, runtime, recDesc);
+ // and contribute one edge from its child
+ ILogicalOperator src = aggOp.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, aggOp, 0);
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
new file mode 100644
index 0000000..276ada9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.AssignRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class AssignPOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.ASSIGN;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ AssignOperator assign = (AssignOperator) op;
+ List<LogicalVariable> variables = assign.getVariables();
+ List<LogicalExpressionReference> expressions = assign.getExpressions();
+ int[] outColumns = new int[variables.size()];
+ for (int i = 0; i < outColumns.length; i++) {
+ outColumns[i] = opSchema.findVariable(variables.get(i));
+ }
+ IEvaluatorFactory[] evalFactories = new IEvaluatorFactory[expressions.size()];
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ for (int i = 0; i < evalFactories.length; i++) {
+ evalFactories[i] = exprJobGen.createEvaluatorFactory(expressions.get(i).getExpression(), context
+ .getTypeEnvironment(op.getInputs().get(0).getOperator()), inputSchemas, context);
+ }
+
+ // TODO push projections into the operator
+ int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
+
+ AssignRuntimeFactory runtime = new AssignRuntimeFactory(outColumns, evalFactories, projectionList);
+
+ // contribute one Asterix framewriter
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ builder.contributeMicroOperator(assign, runtime, recDesc);
+ // and contribute one edge from its child
+ ILogicalOperator src = assign.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, assign, 0);
+
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/BroadcastPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/BroadcastPOperator.java
new file mode 100644
index 0000000..65b6452
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/BroadcastPOperator.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+
+public class BroadcastPOperator extends AbstractExchangePOperator {
+
+ private INodeDomain domain;
+
+ public BroadcastPOperator(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.BROADCAST_EXCHANGE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPartitioningProperty pp = new BroadcastPartitioningProperty(domain);
+ this.deliveredProperties = new StructuralPropertiesVector(pp, op2.getDeliveredPhysicalProperties()
+ .getLocalProperties());
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ IConnectorDescriptor conn = new MToNReplicatingConnectorDescriptor(spec);
+ return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
new file mode 100644
index 0000000..7bf4e49
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+
+@SuppressWarnings("rawtypes")
+public class DataSourceScanPOperator extends AbstractScanPOperator {
+
+ private IDataSource<?> dataSource;
+
+ public DataSourceScanPOperator(IDataSource<?> dataSource) {
+ this.dataSource = dataSource;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.DATASOURCE_SCAN;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ // partitioning properties
+ DataSourceScanOperator dssOp = (DataSourceScanOperator) op;
+ IDataSourcePropertiesProvider dspp = dataSource.getPropertiesProvider();
+ deliveredProperties = dspp.computePropertiesVector(dssOp.getVariables());
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ DataSourceScanOperator scan = (DataSourceScanOperator) op;
+ IMetadataProvider mp = context.getMetadataProvider();
+
+ List<LogicalVariable> vars = scan.getVariables();
+ List<LogicalVariable> projectVars = scan.getProjectVariables();
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = mp.getScannerRuntime(dataSource, vars,
+ projectVars, scan.isProjectPushed(), context, builder.getJobSpec());
+ builder.contributeHyracksOperator(scan, p.first);
+ if (p.second != null) {
+ builder.contributeAlgebricksPartitionConstraint(p.first, p.second);
+ }
+
+ ILogicalOperator srcExchange = scan.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(srcExchange, 0, scan, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
new file mode 100644
index 0000000..7fc76da
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.LinkedList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class EmptyTupleSourcePOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.EMPTY_TUPLE_SOURCE;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ deliveredProperties = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED,
+ new LinkedList<ILocalStructuralProperty>());
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return null;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ EmptyTupleSourceRuntimeFactory runtime = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+ builder.contributeMicroOperator(op, runtime, recDesc);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
new file mode 100644
index 0000000..a983ce7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
@@ -0,0 +1,244 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.ISerializableAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.OperatorSchemaImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.SerializableAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableGroupingTableFactory;
+
+public class ExternalGroupByPOperator extends AbstractPhysicalOperator {
+
+ private int tableSize = 0;
+ private int frameLimit = 0;
+ private List<LogicalVariable> columnSet = new ArrayList<LogicalVariable>();
+
+ public ExternalGroupByPOperator(List<Pair<LogicalVariable, LogicalExpressionReference>> gbyList, int frameLimit,
+ int tableSize) {
+ this.tableSize = tableSize;
+ this.frameLimit = frameLimit;
+ computeColumnSet(gbyList);
+ }
+
+ public void computeColumnSet(List<Pair<LogicalVariable, LogicalExpressionReference>> gbyList) {
+ columnSet.clear();
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gbyList) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ columnSet.add(v.getVariableReference());
+ }
+ }
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.EXTERNAL_GROUP_BY;
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + columnSet;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ public List<LogicalVariable> getGbyColumns() {
+ return columnSet;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ List<ILocalStructuralProperty> propsLocal = new LinkedList<ILocalStructuralProperty>();
+
+ GroupByOperator gOp = (GroupByOperator) op;
+ HashSet<LogicalVariable> columnSet = new HashSet<LogicalVariable>();
+
+ if (!columnSet.isEmpty()) {
+ propsLocal.add(new LocalGroupingProperty(columnSet));
+ }
+ for (ILogicalPlan p : gOp.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ ILogicalOperator rOp = r.getOperator();
+ propsLocal.addAll(rOp.getDeliveredPhysicalProperties().getLocalProperties());
+ }
+ }
+
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ IPhysicalPropertiesVector childProp = op2.getDeliveredPhysicalProperties();
+ deliveredProperties = new StructuralPropertiesVector(childProp.getPartitioningProperty(), propsLocal);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) op;
+ if (aop.getExecutionMode() == ExecutionMode.PARTITIONED) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+ pv[0] = new StructuralPropertiesVector(new UnorderedPartitionedProperty(new HashSet<LogicalVariable>(
+ columnSet), null), null);
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ } else {
+ return emptyUnaryRequirements();
+ }
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ List<LogicalVariable> gbyCols = getGbyColumns();
+ int keys[] = JobGenHelper.variablesToFieldIndexes(gbyCols, inputSchemas[0]);
+ GroupByOperator gby = (GroupByOperator) op;
+ int numFds = gby.getDecorList().size();
+ int fdColumns[] = new int[numFds];
+ int j = 0;
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gby.getDecorList()) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new AlgebricksException("pre-sorted group-by expects variable references.");
+ }
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ LogicalVariable decor = v.getVariableReference();
+ fdColumns[j++] = inputSchemas[0].findVariable(decor);
+ }
+
+ if (gby.getNestedPlans().size() != 1) {
+ throw new AlgebricksException(
+ "External group-by currently works only for one nested plan with one root containing"
+ + "an aggregate and a nested-tuple-source.");
+ }
+ ILogicalPlan p0 = gby.getNestedPlans().get(0);
+ if (p0.getRoots().size() != 1) {
+ throw new AlgebricksException(
+ "External group-by currently works only for one nested plan with one root containing"
+ + "an aggregate and a nested-tuple-source.");
+ }
+ LogicalOperatorReference r0 = p0.getRoots().get(0);
+ AggregateOperator aggOp = (AggregateOperator) r0.getOperator();
+
+ IPartialAggregationTypeComputer partialAggregationTypeComputer = context.getPartialAggregationTypeComputer();
+ List<Object> intermediateTypes = new ArrayList<Object>();
+ int n = aggOp.getExpressions().size();
+ ISerializableAggregateFunctionFactory[] aff = new ISerializableAggregateFunctionFactory[n];
+ int i = 0;
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ IVariableTypeEnvironment aggOpInputEnv = context.getTypeEnvironment(aggOp.getInputs().get(0).getOperator());
+ IVariableTypeEnvironment outputEnv = context.getTypeEnvironment(op);
+ for (LogicalExpressionReference exprRef : aggOp.getExpressions()) {
+ AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) exprRef.getExpression();
+ aff[i++] = exprJobGen.createSerializableAggregateFunctionFactory(aggFun, aggOpInputEnv, inputSchemas,
+ context);
+ intermediateTypes.add(partialAggregationTypeComputer.getType(aggFun, aggOpInputEnv,
+ context.getMetadataProvider()));
+ }
+
+ int[] keyAndDecFields = new int[keys.length + fdColumns.length];
+ for (i = 0; i < keys.length; ++i) {
+ keyAndDecFields[i] = keys[i];
+ }
+ for (i = 0; i < fdColumns.length; i++) {
+ keyAndDecFields[keys.length + i] = fdColumns[i];
+ }
+
+ List<LogicalVariable> keyAndDecVariables = new ArrayList<LogicalVariable>();
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gby.getGroupByList())
+ keyAndDecVariables.add(p.first);
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gby.getDecorList())
+ keyAndDecVariables.add(GroupByOperator.getDecorVariable(p));
+
+ for (LogicalVariable var : keyAndDecVariables)
+ aggOpInputEnv.setVarType(var, outputEnv.getVarType(var));
+
+ compileSubplans(inputSchemas[0], gby, opSchema, context);
+ JobSpecification spec = builder.getJobSpec();
+ IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(gbyCols,
+ aggOpInputEnv, context);
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ IBinaryHashFunctionFactory[] hashFunctionFactories = JobGenHelper.variablesToBinaryHashFunctionFactories(
+ gbyCols, aggOpInputEnv, context);
+
+ ISerializableAggregateFunctionFactory[] merges = new ISerializableAggregateFunctionFactory[n];
+ List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+ IOperatorSchema[] localInputSchemas = new IOperatorSchema[1];
+ localInputSchemas[0] = new OperatorSchemaImpl();
+ for (i = 0; i < n; i++) {
+ AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) aggOp.getMergeExpressions()
+ .get(i).getExpression();
+ aggFun.getUsedVariables(usedVars);
+ }
+ i = 0;
+ for (Object type : intermediateTypes) {
+ aggOpInputEnv.setVarType(usedVars.get(i++), type);
+ }
+ for (LogicalVariable keyVar : keyAndDecVariables)
+ localInputSchemas[0].addVariable(keyVar);
+ for (LogicalVariable usedVar : usedVars)
+ localInputSchemas[0].addVariable(usedVar);
+ for (i = 0; i < n; i++) {
+ AggregateFunctionCallExpression mergeFun = (AggregateFunctionCallExpression) aggOp.getMergeExpressions()
+ .get(i).getExpression();
+ merges[i] = exprJobGen.createSerializableAggregateFunctionFactory(mergeFun, aggOpInputEnv,
+ localInputSchemas, context);
+ }
+ IAggregatorDescriptorFactory aggregatorFactory = new SerializableAggregatorDescriptorFactory(aff);
+ IAggregatorDescriptorFactory mergeFactory = new SerializableAggregatorDescriptorFactory(merges);
+
+ ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories);
+ INormalizedKeyComputerFactory normalizedKeyFactory = JobGenHelper.variablesToAscNormalizedKeyComputerFactory(
+ gbyCols, aggOpInputEnv, context);
+ ExternalGroupOperatorDescriptor gbyOpDesc = new ExternalGroupOperatorDescriptor(spec, keyAndDecFields,
+ frameLimit, comparatorFactories, normalizedKeyFactory, aggregatorFactory, mergeFactory,
+ recordDescriptor, new HashSpillableGroupingTableFactory(tpcf, tableSize), false);
+
+ contributeOpDesc(builder, gby, gbyOpDesc);
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionExchangePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionExchangePOperator.java
new file mode 100644
index 0000000..823ac0b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionExchangePOperator.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.HashSet;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
+
+public class HashPartitionExchangePOperator extends AbstractExchangePOperator {
+
+ private List<LogicalVariable> hashFields;
+ private INodeDomain domain;
+
+ public HashPartitionExchangePOperator(List<LogicalVariable> hashFields, INodeDomain domain) {
+ this.hashFields = hashFields;
+ this.domain = domain;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.HASH_PARTITION_EXCHANGE;
+ }
+
+ public List<LogicalVariable> getHashFields() {
+ return hashFields;
+ }
+
+ public INodeDomain getDomain() {
+ return domain;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ IPartitioningProperty p = new UnorderedPartitionedProperty(new HashSet<LogicalVariable>(hashFields), domain);
+ this.deliveredProperties = new StructuralPropertiesVector(p, null);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + " " + hashFields;
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ int[] keys = new int[hashFields.size()];
+ IBinaryHashFunctionFactory[] hashFunctionFactories = new IBinaryHashFunctionFactory[hashFields.size()];
+ int i = 0;
+ IBinaryHashFunctionFactoryProvider hashFunProvider = context.getBinaryHashFunctionFactoryProvider();
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ for (LogicalVariable v : hashFields) {
+ keys[i] = opSchema.findVariable(v);
+ hashFunctionFactories[i] = hashFunProvider.getBinaryHashFunctionFactory(env.getVarType(v));
+ ++i;
+ }
+ ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories);
+ IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, tpcf);
+ return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
new file mode 100644
index 0000000..0a44b0a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+
+public class HashPartitionMergeExchangePOperator extends AbstractExchangePOperator {
+
+ private List<OrderColumn> orderColumns;
+ private List<LogicalVariable> partitionFields;
+ private INodeDomain domain;
+
+ public HashPartitionMergeExchangePOperator(List<OrderColumn> orderColumns, List<LogicalVariable> partitionFields,
+ INodeDomain domain) {
+ this.orderColumns = orderColumns;
+ this.partitionFields = partitionFields;
+ this.domain = domain;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.HASH_PARTITION_MERGE_EXCHANGE;
+ }
+
+ public List<OrderColumn> getOrderExpressions() {
+ return orderColumns;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ IPartitioningProperty p = new UnorderedPartitionedProperty(new HashSet<LogicalVariable>(partitionFields),
+ domain);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ List<ILocalStructuralProperty> op2Locals = op2.getDeliveredPhysicalProperties().getLocalProperties();
+ List<ILocalStructuralProperty> locals = new ArrayList<ILocalStructuralProperty>();
+ for (ILocalStructuralProperty prop : op2Locals) {
+ if (prop.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
+ locals.add(prop);
+ } else {
+ break;
+ }
+ }
+
+ this.deliveredProperties = new StructuralPropertiesVector(p, locals);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ List<ILocalStructuralProperty> orderProps = new LinkedList<ILocalStructuralProperty>();
+ for (OrderColumn oc : orderColumns) {
+ LogicalVariable var = oc.getColumn();
+ switch (oc.getOrder()) {
+ case ASC: {
+ orderProps.add(new LocalOrderProperty(new OrderColumn(var, OrderKind.ASC)));
+ break;
+ }
+ case DESC: {
+ orderProps.add(new LocalOrderProperty(new OrderColumn(var, OrderKind.DESC)));
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(null,
+ orderProps) };
+ return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + " MERGE:" + orderColumns + " HASH:" + partitionFields;
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ int[] keys = new int[partitionFields.size()];
+ IBinaryHashFunctionFactory[] hashFunctionFactories = new IBinaryHashFunctionFactory[partitionFields.size()];
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ {
+ int i = 0;
+ IBinaryHashFunctionFactoryProvider hashFunProvider = context.getBinaryHashFunctionFactoryProvider();
+ for (LogicalVariable v : partitionFields) {
+ keys[i] = opSchema.findVariable(v);
+ hashFunctionFactories[i] = hashFunProvider.getBinaryHashFunctionFactory(env.getVarType(v));
+ ++i;
+ }
+ }
+ ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories);
+
+ int n = orderColumns.size();
+ int[] sortFields = new int[n];
+ IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[n];
+ {
+ int j = 0;
+ for (OrderColumn oc : orderColumns) {
+ LogicalVariable var = oc.getColumn();
+ sortFields[j] = opSchema.findVariable(var);
+ Object type = env.getVarType(var);
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comparatorFactories[j] = bcfp.getBinaryComparatorFactory(type, oc.getOrder());
+ j++;
+ }
+ }
+
+ IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields,
+ comparatorFactories);
+ return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
new file mode 100644
index 0000000..378852b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
+
+public class HybridHashJoinPOperator extends AbstractHashJoinPOperator {
+
+ private final int memSizeInFrames;
+ private final int maxInputBuildSizeInFrames;
+ private final int aveRecordsPerFrame;
+ private final double fudgeFactor;
+
+ public HybridHashJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+ List<LogicalVariable> sideLeftOfEqualities, List<LogicalVariable> sideRightOfEqualities,
+ int memSizeInFrames, int maxInputSize0InFrames, int aveRecordsPerFrame, double fudgeFactor) {
+ super(kind, partitioningType, sideLeftOfEqualities, sideRightOfEqualities);
+ this.memSizeInFrames = memSizeInFrames;
+ this.maxInputBuildSizeInFrames = maxInputSize0InFrames;
+ this.aveRecordsPerFrame = aveRecordsPerFrame;
+ this.fudgeFactor = fudgeFactor;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.HYBRID_HASH_JOIN;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ public double getFudgeFactor() {
+ return fudgeFactor;
+ }
+
+ public int getMemSizeInFrames() {
+ return memSizeInFrames;
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + " " + keysLeftBranch + keysRightBranch;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ int[] keysLeft = JobGenHelper.variablesToFieldIndexes(keysLeftBranch, inputSchemas[0]);
+ int[] keysRight = JobGenHelper.variablesToFieldIndexes(keysRightBranch, inputSchemas[1]);
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ IBinaryHashFunctionFactory[] hashFunFactories = JobGenHelper.variablesToBinaryHashFunctionFactories(
+ keysLeftBranch, env, context);
+ IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[keysLeft.length];
+ int i = 0;
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ for (LogicalVariable v : keysLeftBranch) {
+ Object t = env.getVarType(v);
+ comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, OrderKind.ASC);
+ }
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ JobSpecification spec = builder.getJobSpec();
+ IOperatorDescriptor opDesc = null;
+ try {
+ switch (kind) {
+ case INNER: {
+ opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+ maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
+ hashFunFactories, comparatorFactories, recDescriptor);
+ break;
+ }
+ case LEFT_OUTER: {
+ INullWriterFactory[] nullWriterFactories = new INullWriterFactory[inputSchemas[1].getSize()];
+ for (int j = 0; j < nullWriterFactories.length; j++) {
+ nullWriterFactories[j] = context.getNullWriterFactory();
+ }
+ opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+ maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
+ hashFunFactories, comparatorFactories, recDescriptor, true, nullWriterFactories);
+ break;
+ }
+ default: {
+ throw new NotImplementedException();
+ }
+ }
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src1 = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src1, 0, op, 0);
+ ILogicalOperator src2 = op.getInputs().get(1).getOperator();
+ builder.contributeGraphEdge(src2, 0, op, 1);
+ }
+
+ @Override
+ protected List<ILocalStructuralProperty> deliveredLocalProperties(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ return new LinkedList<ILocalStructuralProperty>();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
new file mode 100644
index 0000000..e1606f0
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
+
+public class InMemoryHashJoinPOperator extends AbstractHashJoinPOperator {
+
+ private final int tableSize;
+
+ /**
+ * builds on the first operator and probes on the second.
+ */
+
+ public InMemoryHashJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+ List<LogicalVariable> sideLeftOfEqualities, List<LogicalVariable> sideRightOfEqualities, int tableSize) {
+ super(kind, partitioningType, sideLeftOfEqualities, sideRightOfEqualities);
+ this.tableSize = tableSize;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.IN_MEMORY_HASH_JOIN;
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + " " + keysLeftBranch + keysRightBranch;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ int[] keysLeft = JobGenHelper.variablesToFieldIndexes(keysLeftBranch, inputSchemas[0]);
+ int[] keysRight = JobGenHelper.variablesToFieldIndexes(keysRightBranch, inputSchemas[1]);
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ IBinaryHashFunctionFactory[] hashFunFactories = JobGenHelper.variablesToBinaryHashFunctionFactories(
+ keysLeftBranch, env, context);
+ IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[keysLeft.length];
+ int i = 0;
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ for (LogicalVariable v : keysLeftBranch) {
+ Object t = env.getVarType(v);
+ comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, OrderKind.ASC);
+ }
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ JobSpecification spec = builder.getJobSpec();
+ IOperatorDescriptor opDesc = null;
+
+ switch (kind) {
+ case INNER: {
+ opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, hashFunFactories,
+ comparatorFactories, recDescriptor, tableSize);
+ break;
+ }
+ case LEFT_OUTER: {
+ INullWriterFactory[] nullWriterFactories = new INullWriterFactory[inputSchemas[1].getSize()];
+ for (int j = 0; j < nullWriterFactories.length; j++) {
+ nullWriterFactories[j] = context.getNullWriterFactory();
+ }
+ opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, hashFunFactories,
+ comparatorFactories, recDescriptor, true, nullWriterFactories, tableSize);
+ break;
+ }
+ default: {
+ throw new NotImplementedException();
+ }
+ }
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src1 = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src1, 0, op, 0);
+ ILogicalOperator src2 = op.getInputs().get(1).getOperator();
+ builder.contributeGraphEdge(src2, 0, op, 1);
+ }
+
+ @Override
+ protected List<ILocalStructuralProperty> deliveredLocalProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPhysicalPropertiesVector pv0 = op0.getPhysicalOperator().getDeliveredProperties();
+ List<ILocalStructuralProperty> lp0 = pv0.getLocalProperties();
+ if (lp0 != null) {
+ // maintains the local properties on the probe side
+ return new LinkedList<ILocalStructuralProperty>(lp0);
+ }
+ return new LinkedList<ILocalStructuralProperty>();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryStableSortPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryStableSortPOperator.java
new file mode 100644
index 0000000..47b5cc4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryStableSortPOperator.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.INormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.sort.InMemorySortRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class InMemoryStableSortPOperator extends AbstractStableSortPOperator {
+
+ public InMemoryStableSortPOperator() {
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.IN_MEMORY_STABLE_SORT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ int n = sortColumns.length;
+ int[] sortFields = new int[n];
+ IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+ int i = 0;
+ INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+ INormalizedKeyComputerFactory nkcf = null;
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ for (OrderColumn oc : sortColumns) {
+ LogicalVariable var = oc.getColumn();
+ sortFields[i] = opSchema.findVariable(var);
+ Object type = env.getVarType(var);
+ OrderKind order = oc.getOrder();
+ if (i == 0 && nkcfProvider != null && type != null) {
+ nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order);
+ }
+
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder());
+ i++;
+ }
+
+ IPushRuntimeFactory runtime = new InMemorySortRuntimeFactory(sortFields, nkcf, comps, null);
+ builder.contributeMicroOperator(op, runtime, recDescriptor);
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
new file mode 100644
index 0000000..6e22ac3
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
@@ -0,0 +1,108 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class IndexInsertDeletePOperator extends AbstractPhysicalOperator {
+
+ private List<LogicalVariable> primaryKeys;
+ private List<LogicalVariable> secondaryKeys;
+ private IDataSourceIndex<?, ?> dataSourceIndex;
+
+ public IndexInsertDeletePOperator(List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ IDataSourceIndex<?, ?> dataSourceIndex) {
+ this.primaryKeys = primaryKeys;
+ this.secondaryKeys = secondaryKeys;
+ this.dataSourceIndex = dataSourceIndex;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.INDEX_INSERT_DELETE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ List<LogicalVariable> scanVariables = new ArrayList<LogicalVariable>();
+ scanVariables.addAll(primaryKeys);
+ scanVariables.add(new LogicalVariable(-1));
+ IPartitioningProperty pp = dataSourceIndex.getDataSource().getPropertiesProvider()
+ .computePropertiesVector(scanVariables).getPartitioningProperty();
+ List<ILocalStructuralProperty> orderProps = new LinkedList<ILocalStructuralProperty>();
+ for (LogicalVariable k : secondaryKeys) {
+ orderProps.add(new LocalOrderProperty(new OrderColumn(k, OrderKind.ASC)));
+ }
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp,
+ orderProps) };
+ return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ IndexInsertDeleteOperator insertDeleteOp = (IndexInsertDeleteOperator) op;
+ IMetadataProvider mp = context.getMetadataProvider();
+
+ JobSpecification spec = builder.getJobSpec();
+ RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getOperator(),
+ inputSchemas[0], context);
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
+ if (insertDeleteOp.getOperation() == Kind.INSERT)
+ runtimeAndConstraints = mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, primaryKeys,
+ secondaryKeys, inputDesc, context, spec);
+ else
+ runtimeAndConstraints = mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, primaryKeys,
+ secondaryKeys, inputDesc, context, spec);
+
+ builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
+ builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+ ILogicalOperator src = insertDeleteOp.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, insertDeleteOp, 0);
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
new file mode 100644
index 0000000..44b54c8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
@@ -0,0 +1,96 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+@SuppressWarnings("rawtypes")
+public class InsertDeletePOperator extends AbstractPhysicalOperator {
+
+ private LogicalVariable payload;
+ private List<LogicalVariable> keys;
+ private IDataSource<?> dataSource;
+
+ public InsertDeletePOperator(LogicalVariable payload, List<LogicalVariable> keys, IDataSource dataSource) {
+ this.payload = payload;
+ this.keys = keys;
+ this.dataSource = dataSource;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.INSERT_DELETE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ List<LogicalVariable> scanVariables = new ArrayList<LogicalVariable>();
+ scanVariables.addAll(keys);
+ scanVariables.add(new LogicalVariable(-1));
+ IPhysicalPropertiesVector r = dataSource.getPropertiesProvider().computePropertiesVector(scanVariables);
+ IPhysicalPropertiesVector[] requirements = new IPhysicalPropertiesVector[1];
+ requirements[0] = r;
+ return new PhysicalRequirements(requirements, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op;
+ IMetadataProvider mp = context.getMetadataProvider();
+ JobSpecification spec = builder.getJobSpec();
+ RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getOperator(),
+ inputSchemas[0], context);
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
+ if (insertDeleteOp.getOperation() == Kind.INSERT)
+ runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, keys, payload, inputDesc,
+ context, spec);
+ else
+ runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, keys, payload, inputDesc,
+ context, spec);
+
+ builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
+ builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+ ILogicalOperator src = insertDeleteOp.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, insertDeleteOp, 0);
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
new file mode 100644
index 0000000..8d424d1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
@@ -0,0 +1,79 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.NestedPlansAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.group.MicroPreClusteredGroupRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+
+public class MicroPreclusteredGroupByPOperator extends AbstractPreclusteredGroupByPOperator {
+
+ public MicroPreclusteredGroupByPOperator(List<LogicalVariable> columnList) {
+ super(columnList);
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.MICRO_PRE_CLUSTERED_GROUP_BY;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ int keys[] = JobGenHelper.variablesToFieldIndexes(columnList, inputSchemas[0]);
+ GroupByOperator gby = (GroupByOperator) op;
+ int numFds = gby.getDecorList().size();
+ int fdColumns[] = new int[numFds];
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ int j = 0;
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gby.getDecorList()) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new AlgebricksException("pre-sorted group-by expects variable references.");
+ }
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ LogicalVariable decor = v.getVariableReference();
+ fdColumns[j++] = inputSchemas[0].findVariable(decor);
+ }
+ // compile subplans and set the gby op. schema accordingly
+ AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], gby, opSchema, context);
+ IAccumulatingAggregatorFactory aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys,
+ fdColumns);
+
+ IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+ columnList, env, context);
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getOperator(),
+ inputSchemas[0], context);
+ MicroPreClusteredGroupRuntimeFactory runtime = new MicroPreClusteredGroupRuntimeFactory(keys,
+ comparatorFactories, aggregatorFactory, inputRecordDesc, recordDescriptor, null);
+ builder.contributeMicroOperator(gby, runtime, recordDescriptor);
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
new file mode 100644
index 0000000..6787158
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
@@ -0,0 +1,275 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+
+/**
+ *
+ * Left input is broadcast and preserves its local properties.
+ *
+ * Right input can be partitioned in any way.
+ *
+ */
+public class NLJoinPOperator extends AbstractJoinPOperator {
+
+ private final int memSize;
+
+ public NLJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType, int memSize) {
+ super(kind, partitioningType);
+ this.memSize = memSize;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.NESTED_LOOP;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context) {
+ if (partitioningType != JoinPartitioningType.BROADCAST) {
+ throw new NotImplementedException(partitioningType + " nested loop joins are not implemented.");
+ }
+
+ IPartitioningProperty pp;
+
+ AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(1).getOperator();
+ IPhysicalPropertiesVector pv1 = op2.getPhysicalOperator().getDeliveredProperties();
+ if (pv1 == null) {
+ pp = null;
+ } else {
+ pp = pv1.getPartitioningProperty();
+ }
+ } else {
+ pp = null;
+ }
+
+ List<ILocalStructuralProperty> localProps = new LinkedList<ILocalStructuralProperty>();
+ this.deliveredProperties = new StructuralPropertiesVector(pp, localProps);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ if (partitioningType != JoinPartitioningType.BROADCAST) {
+ throw new NotImplementedException(partitioningType + " nested loop joins are not implemented.");
+ }
+
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+ pv[0] = new StructuralPropertiesVector(new BroadcastPartitioningProperty(null), null);
+ pv[1] = new StructuralPropertiesVector(null, null);
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ IOperatorSchema[] conditionInputSchemas = new IOperatorSchema[1];
+ conditionInputSchemas[0] = propagatedSchema;
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ IEvaluatorFactory cond = exprJobGen.createEvaluatorFactory(join.getCondition().getExpression(), context
+ .getTypeEnvironment(op), conditionInputSchemas, context);
+ ITuplePairComparatorFactory comparatorFactory = new TuplePairEvaluatorFactory(cond, context
+ .getBinaryBooleanInspector());
+ JobSpecification spec = builder.getJobSpec();
+ IOperatorDescriptor opDesc = null;
+
+ switch (kind) {
+ case INNER: {
+ opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize);
+ break;
+ }
+ case LEFT_OUTER:
+ default: {
+ throw new NotImplementedException();
+ }
+ }
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src1 = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src1, 0, op, 0);
+ ILogicalOperator src2 = op.getInputs().get(1).getOperator();
+ builder.contributeGraphEdge(src2, 0, op, 1);
+ }
+
+ public static class TuplePairEvaluatorFactory implements ITuplePairComparatorFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final IEvaluatorFactory cond;
+ private final IBinaryBooleanInspector binaryBooleanInspector;
+
+ public TuplePairEvaluatorFactory(IEvaluatorFactory cond, IBinaryBooleanInspector binaryBooleanInspector) {
+ this.cond = cond;
+ this.binaryBooleanInspector = binaryBooleanInspector;
+ }
+
+ @Override
+ public synchronized ITuplePairComparator createTuplePairComparator() {
+ return new TuplePairEvaluator(cond, binaryBooleanInspector);
+ }
+
+ }
+
+ public static class TuplePairEvaluator implements ITuplePairComparator {
+
+ private IEvaluator condEvaluator;
+ private final IEvaluatorFactory condFactory;
+ private final CompositeFrameTupleReference compositeTupleRef;
+ private final FrameTupleReference leftRef;
+ private final FrameTupleReference rightRef;
+ private final ArrayBackedValueStorage evalOutput;
+ private final IBinaryBooleanInspector binaryBooleanInspector;
+
+ public TuplePairEvaluator(IEvaluatorFactory condFactory, IBinaryBooleanInspector binaryBooleanInspector) {
+ this.condFactory = condFactory;
+ this.binaryBooleanInspector = binaryBooleanInspector;
+ this.evalOutput = new ArrayBackedValueStorage();
+ this.leftRef = new FrameTupleReference();
+ this.rightRef = new FrameTupleReference();
+ this.compositeTupleRef = new CompositeFrameTupleReference(leftRef, rightRef);
+ }
+
+ @Override
+ public int compare(IFrameTupleAccessor outerAccessor, int outerIndex, IFrameTupleAccessor innerAccessor,
+ int innerIndex) throws HyracksDataException {
+ if (condEvaluator == null) {
+ try {
+ this.condEvaluator = condFactory.createEvaluator(evalOutput);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ compositeTupleRef.reset(outerAccessor, outerIndex, innerAccessor, innerIndex);
+ try {
+ condEvaluator.evaluate(compositeTupleRef);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ boolean result = binaryBooleanInspector.getBooleanValue(evalOutput.getBytes(), 0, evalOutput.getLength());
+ evalOutput.reset();
+ if (result)
+ return 0;
+ else
+ return 1;
+ }
+ }
+
+ public static class CompositeFrameTupleReference implements IFrameTupleReference {
+
+ private final FrameTupleReference refLeft;
+ private final FrameTupleReference refRight;
+
+ public CompositeFrameTupleReference(FrameTupleReference refLeft, FrameTupleReference refRight) {
+ this.refLeft = refLeft;
+ this.refRight = refRight;
+ }
+
+ public void reset(IFrameTupleAccessor outerAccessor, int outerIndex, IFrameTupleAccessor innerAccessor,
+ int innerIndex) {
+ refLeft.reset(outerAccessor, outerIndex);
+ refRight.reset(innerAccessor, innerIndex);
+ }
+
+ @Override
+ public int getFieldCount() {
+ return refLeft.getFieldCount() + refRight.getFieldCount();
+ }
+
+ @Override
+ public byte[] getFieldData(int fIdx) {
+ int leftFieldCount = refLeft.getFieldCount();
+ if (fIdx < leftFieldCount)
+ return refLeft.getFieldData(fIdx);
+ else
+ return refRight.getFieldData(fIdx - leftFieldCount);
+ }
+
+ @Override
+ public int getFieldStart(int fIdx) {
+ int leftFieldCount = refLeft.getFieldCount();
+ if (fIdx < leftFieldCount)
+ return refLeft.getFieldStart(fIdx);
+ else
+ return refRight.getFieldStart(fIdx - leftFieldCount);
+ }
+
+ @Override
+ public int getFieldLength(int fIdx) {
+ int leftFieldCount = refLeft.getFieldCount();
+ if (fIdx < leftFieldCount)
+ return refLeft.getFieldLength(fIdx);
+ else
+ return refRight.getFieldLength(fIdx - leftFieldCount);
+ }
+
+ @Override
+ public IFrameTupleAccessor getFrameTupleAccessor() {
+ throw new NotImplementedException();
+ }
+
+ @Override
+ public int getTupleIndex() {
+ throw new NotImplementedException();
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
new file mode 100644
index 0000000..321ecf4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.NestedTupleSourceRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class NestedTupleSourcePOperator extends AbstractPhysicalOperator {
+
+ public NestedTupleSourcePOperator() {
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.NESTED_TUPLE_SOURCE;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ LogicalOperatorReference dataSource = ((NestedTupleSourceOperator) op).getDataSourceReference();
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) dataSource.getOperator().getInputs().get(0)
+ .getOperator();
+ IPhysicalPropertiesVector inheritedProps = op2.getDeliveredPhysicalProperties();
+ deliveredProperties = (StructuralPropertiesVector) inheritedProps.clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return null;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ propagatedSchema.addAllVariables(outerPlanSchema);
+ NestedTupleSourceRuntimeFactory runtime = new NestedTupleSourceRuntimeFactory();
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ builder.contributeMicroOperator(op, runtime, recDesc);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
new file mode 100644
index 0000000..b7b4668
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+
+public class OneToOneExchangePOperator extends AbstractExchangePOperator {
+
+ public OneToOneExchangePOperator() {
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.ONE_TO_ONE_EXCHANGE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) {
+ IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
+ return new Pair<IConnectorDescriptor, TargetConstraint>(conn, TargetConstraint.SAME_COUNT);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
new file mode 100644
index 0000000..5fba08d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
+
+public class PreSortedDistinctByPOperator extends AbstractPhysicalOperator {
+
+ private List<LogicalVariable> columnList;
+
+ public PreSortedDistinctByPOperator(List<LogicalVariable> columnList) {
+ this.columnList = columnList;
+ }
+
+ public void setDistinctByColumns(List<LogicalVariable> distinctByColumns) {
+ this.columnList = distinctByColumns;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPartitioningProperty pp = op2.getDeliveredPhysicalProperties().getPartitioningProperty();
+ List<ILocalStructuralProperty> propsLocal = op2.getDeliveredPhysicalProperties().getLocalProperties();
+ deliveredProperties = new StructuralPropertiesVector(pp, propsLocal);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+ List<ILocalStructuralProperty> localProps = new ArrayList<ILocalStructuralProperty>(columnList.size());
+ for (LogicalVariable column : columnList) {
+ localProps.add(new LocalOrderProperty(new OrderColumn(column, OrderKind.ASC)));
+ }
+ IPartitioningProperty pp = null;
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) op;
+ if (aop.getExecutionMode() == ExecutionMode.PARTITIONED) {
+ pp = new UnorderedPartitionedProperty(new HashSet<LogicalVariable>(columnList), null);
+ }
+ pv[0] = new StructuralPropertiesVector(pp, localProps);
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+
+ JobSpecification spec = builder.getJobSpec();
+ int keys[] = JobGenHelper.variablesToFieldIndexes(columnList, inputSchemas[0]);
+ int sz = inputSchemas[0].getSize();
+ int fdSz = sz - columnList.size();
+ int[] fdColumns = new int[fdSz];
+ int j = 0;
+ for (LogicalVariable v : inputSchemas[0]) {
+ if (!columnList.contains(v)) {
+ fdColumns[j++] = inputSchemas[0].findVariable(v);
+ }
+ }
+ IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+ columnList, context.getTypeEnvironment(op), context);
+ IAggregateFunctionFactory[] aggFactories = new IAggregateFunctionFactory[] {};
+ IAccumulatingAggregatorFactory aggregatorFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(
+ aggFactories, keys, fdColumns);
+
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
+ comparatorFactories, aggregatorFactory, recordDescriptor);
+
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.PRE_SORTED_DISTINCT_BY;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
new file mode 100644
index 0000000..a539315
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.NestedPlansAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
+
+public class PreclusteredGroupByPOperator extends AbstractPreclusteredGroupByPOperator {
+
+ public PreclusteredGroupByPOperator(List<LogicalVariable> columnList) {
+ super(columnList);
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ int keys[] = JobGenHelper.variablesToFieldIndexes(columnList, inputSchemas[0]);
+ GroupByOperator gby = (GroupByOperator) op;
+ int numFds = gby.getDecorList().size();
+ int fdColumns[] = new int[numFds];
+ int j = 0;
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gby.getDecorList()) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new AlgebricksException("pre-sorted group-by expects variable references.");
+ }
+ VariableReferenceExpression v = (VariableReferenceExpression) expr;
+ LogicalVariable decor = v.getVariableReference();
+ fdColumns[j++] = inputSchemas[0].findVariable(decor);
+ }
+ // compile subplans and set the gby op. schema accordingly
+ AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], gby, opSchema, context);
+ IAccumulatingAggregatorFactory aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys,
+ fdColumns);
+
+ JobSpecification spec = builder.getJobSpec();
+ IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+ columnList, context.getTypeEnvironment(op), context);
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+
+ PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
+ comparatorFactories, aggregatorFactory, recordDescriptor);
+
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
new file mode 100644
index 0000000..abfa31f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+
+public class RandomMergeExchangePOperator extends AbstractExchangePOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.RANDOM_MERGE_EXCHANGE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ this.deliveredProperties = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED,
+ new ArrayList<ILocalStructuralProperty>(0));
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) {
+ IConnectorDescriptor conn = new MToNReplicatingConnectorDescriptor(spec);
+ return new Pair<IConnectorDescriptor, TargetConstraint>(conn, TargetConstraint.ONE);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
new file mode 100644
index 0000000..ff558ec
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class RangePartitionPOperator extends AbstractExchangePOperator {
+
+ private ArrayList<OrderColumn> partitioningFields;
+ private INodeDomain domain;
+
+ public RangePartitionPOperator(ArrayList<OrderColumn> partitioningFields, INodeDomain domain) {
+ this.partitioningFields = partitioningFields;
+ this.domain = domain;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.RANGE_PARTITION_EXCHANGE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ IPartitioningProperty p = new OrderedPartitionedProperty(partitioningFields, domain);
+ this.deliveredProperties = new StructuralPropertiesVector(p, new LinkedList<ILocalStructuralProperty>());
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ throw new NotImplementedException();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
new file mode 100644
index 0000000..ca7675c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
+
+public class ReplicatePOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SPLIT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ JobSpecification spec = builder.getJobSpec();
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+
+ ReplicateOperator rop = (ReplicateOperator) op;
+ int outputArity = rop.getOutputArity();
+
+ SplitOperatorDescriptor splitOpDesc = new SplitOperatorDescriptor(spec, recDescriptor, outputArity);
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, splitOpDesc);
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
new file mode 100644
index 0000000..87ccc05
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.RunningAggregateRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class RunningAggregatePOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.RUNNING_AGGREGATE;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ IPartitioningProperty pp = null;
+ RunningAggregateOperator ragg = (RunningAggregateOperator) op;
+ for (LogicalExpressionReference exprRef : ragg.getExpressions()) {
+ StatefulFunctionCallExpression f = (StatefulFunctionCallExpression) exprRef.getExpression();
+ IPartitioningProperty p = f.getRequiredPartitioningProperty();
+ if (p != null) {
+ if (pp == null) {
+ pp = p;
+ } else {
+ throw new IllegalStateException("Two stateful functions want to set partitioning requirements: "
+ + pp + " and " + p);
+ }
+ }
+ }
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp, null) };
+ return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ RunningAggregateOperator ragg = (RunningAggregateOperator) op;
+ List<LogicalVariable> variables = ragg.getVariables();
+ List<LogicalExpressionReference> expressions = ragg.getExpressions();
+ int[] outColumns = new int[variables.size()];
+ for (int i = 0; i < outColumns.length; i++) {
+ outColumns[i] = opSchema.findVariable(variables.get(i));
+ }
+ IRunningAggregateFunctionFactory[] runningAggFuns = new IRunningAggregateFunctionFactory[expressions.size()];
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ for (int i = 0; i < runningAggFuns.length; i++) {
+ StatefulFunctionCallExpression expr = (StatefulFunctionCallExpression) expressions.get(i).getExpression();
+ runningAggFuns[i] = exprJobGen.createRunningAggregateFunctionFactory(expr, context.getTypeEnvironment(op
+ .getInputs().get(0).getOperator()), inputSchemas, context);
+ }
+
+ // TODO push projections into the operator
+ int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
+
+ RunningAggregateRuntimeFactory runtime = new RunningAggregateRuntimeFactory(outColumns, runningAggFuns,
+ projectionList);
+
+ // contribute one Asterix framewriter
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ builder.contributeMicroOperator(ragg, runtime, recDesc);
+ // and contribute one edge from its child
+ ILogicalOperator src = ragg.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, ragg, 0);
+
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
new file mode 100644
index 0000000..fe501e2
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.SinkRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class SinkPOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SINK;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ SinkRuntimeFactory runtime = new SinkRuntimeFactory();
+ builder.contributeMicroOperator(op, runtime, recDesc);
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
new file mode 100644
index 0000000..ec664bb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class SinkWritePOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SINK_WRITE;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ WriteOperator write = (WriteOperator) op;
+ IDataSink sink = write.getDataSink();
+ IPartitioningProperty pp = sink.getPartitioningProperty();
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp, null) };
+ return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ WriteOperator write = (WriteOperator) op;
+ int[] columns = new int[write.getExpressions().size()];
+ int i = 0;
+ for (LogicalExpressionReference exprRef : write.getExpressions()) {
+ ILogicalExpression expr = exprRef.getExpression();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException("Only writing variable expressions is supported.");
+ }
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ LogicalVariable v = varRef.getVariableReference();
+ columns[i++] = inputSchemas[0].findVariable(v);
+ }
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getOperator(),
+ inputSchemas[0], context);
+
+ IPrinterFactory[] pf = JobGenHelper.mkPrinterFactories(inputSchemas[0], context.getTypeEnvironment(op),
+ context, columns);
+
+ IMetadataProvider<?, ?> mp = context.getMetadataProvider();
+
+ Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtime = mp.getWriteFileRuntime(write.getDataSink(),
+ columns, pf, inputDesc);
+
+ builder.contributeMicroOperator(write, runtime.first, recDesc, runtime.second);
+ ILogicalOperator src = write.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, write, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
new file mode 100644
index 0000000..185a671
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+
+public class SortMergeExchangePOperator extends AbstractExchangePOperator {
+
+ private OrderColumn[] sortColumns;
+
+ public SortMergeExchangePOperator(OrderColumn[] sortColumns) {
+ this.sortColumns = sortColumns;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SORT_MERGE_EXCHANGE;
+ }
+
+ public OrderColumn[] getSortColumns() {
+ return sortColumns;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append(getOperatorTag());
+ sb.append(" [");
+ sb.append(sortColumns[0]);
+ for (int i = 1; i < sortColumns.length; i++) {
+ sb.append(", " + sortColumns[i]);
+ }
+ sb.append(" ]");
+ return sb.toString();
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator inp1 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPhysicalPropertiesVector pv1 = inp1.getDeliveredPhysicalProperties();
+ if (pv1 == null) {
+ inp1.computeDeliveredPhysicalProperties(context);
+ pv1 = inp1.getDeliveredPhysicalProperties();
+ }
+ int sortCol = 0;
+ List<ILocalStructuralProperty> localProps = new ArrayList<ILocalStructuralProperty>(sortColumns.length);
+ for (ILocalStructuralProperty prop : pv1.getLocalProperties()) {
+ if (prop.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
+ LocalOrderProperty lop = (LocalOrderProperty) prop;
+ if (lop.getOrderColumn().equals(sortColumns[sortCol])) {
+ localProps.add(lop);
+ sortCol++;
+ if (sortCol == sortColumns.length) {
+ break;
+ }
+ }
+ } else {
+ break;
+ }
+ }
+ if (sortCol < sortColumns.length) {
+ localProps = null;
+ }
+ this.deliveredProperties = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED, localProps);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ List<ILocalStructuralProperty> localProps = new ArrayList<ILocalStructuralProperty>(sortColumns.length);
+ for (OrderColumn oc : sortColumns) {
+ localProps.add(new LocalOrderProperty(oc));
+ }
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(null,
+ localProps) };
+ return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(JobSpecification spec,
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ int n = sortColumns.length;
+ int[] sortFields = new int[n];
+ IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+ IBinaryHashFunctionFactory[] hashFuns = new IBinaryHashFunctionFactory[n];
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ for (int i = 0; i < n; i++) {
+ sortFields[i] = opSchema.findVariable(sortColumns[i].getColumn());
+ Object type = env.getVarType(sortColumns[i].getColumn());
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comps[i] = bcfp.getBinaryComparatorFactory(type, sortColumns[i].getOrder());
+ IBinaryHashFunctionFactoryProvider bhffp = context.getBinaryHashFunctionFactoryProvider();
+ hashFuns[i] = bhffp.getBinaryHashFunctionFactory(type);
+ }
+ ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(sortFields, hashFuns);
+ IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps);
+ return new Pair<IConnectorDescriptor, TargetConstraint>(conn, TargetConstraint.ONE);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
new file mode 100644
index 0000000..623bd90
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.INormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+
+/**
+ *
+ * This will always be attached to an {@link OrderOperator} logical operator.
+ *
+ *
+ */
+
+public class StableSortPOperator extends AbstractStableSortPOperator {
+
+ private int maxNumberOfFrames;
+
+ public StableSortPOperator(int maxNumberOfFrames) {
+ super();
+ this.maxNumberOfFrames = maxNumberOfFrames;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.STABLE_SORT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ JobSpecification spec = builder.getJobSpec();
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ int n = sortColumns.length;
+ int[] sortFields = new int[n];
+ IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+
+ INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+ INormalizedKeyComputerFactory nkcf = null;
+
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ int i = 0;
+ for (OrderColumn oc : sortColumns) {
+ LogicalVariable var = oc.getColumn();
+ sortFields[i] = opSchema.findVariable(var);
+ Object type = env.getVarType(var);
+ OrderKind order = oc.getOrder();
+ if (i == 0 && nkcfProvider != null && type != null) {
+ nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order);
+ }
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder());
+ i++;
+ }
+
+ ExternalSortOperatorDescriptor sortOpDesc = new ExternalSortOperatorDescriptor(spec, maxNumberOfFrames,
+ sortFields, nkcf, comps, recDescriptor);
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, sortOpDesc);
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamDiePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamDiePOperator.java
new file mode 100644
index 0000000..ee2cee3
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamDiePOperator.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StreamDieRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class StreamDiePOperator extends AbstractPhysicalOperator {
+
+ public StreamDiePOperator() {
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.STREAM_DIE;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+ pv[0] = (StructuralPropertiesVector) reqdByParent;
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ DieOperator die = (DieOperator) op;
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ IEvaluatorFactory afterObjectsFact = exprJobGen.createEvaluatorFactory(die.getAfterObjects().getExpression(),
+ env, inputSchemas, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ StreamDieRuntimeFactory runtime = new StreamDieRuntimeFactory(afterObjectsFact, null,
+ context.getBinaryIntegerInspector());
+ builder.contributeMicroOperator(die, runtime, recDesc);
+ ILogicalOperator src = die.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, die, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
new file mode 100644
index 0000000..a70d6c8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StreamLimitRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class StreamLimitPOperator extends AbstractPhysicalOperator {
+
+ private boolean global;
+
+ public StreamLimitPOperator(boolean global) {
+ this.global = global;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.STREAM_LIMIT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ if (global) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+ pv[0] = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED, null);
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ } else {
+ return emptyUnaryRequirements();
+ }
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ LimitOperator limit = (LimitOperator) op;
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ IEvaluatorFactory maxObjectsFact = exprJobGen.createEvaluatorFactory(limit.getMaxObjects().getExpression(),
+ env, inputSchemas, context);
+ ILogicalExpression offsetExpr = limit.getOffset().getExpression();
+ IEvaluatorFactory offsetFact = (offsetExpr == null) ? null : exprJobGen.createEvaluatorFactory(offsetExpr, env,
+ inputSchemas, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ StreamLimitRuntimeFactory runtime = new StreamLimitRuntimeFactory(maxObjectsFact, offsetFact, null, context
+ .getBinaryIntegerInspector());
+ builder.contributeMicroOperator(limit, runtime, recDesc);
+ // and contribute one edge from its child
+ ILogicalOperator src = limit.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, limit, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
new file mode 100644
index 0000000..e28d580
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StreamProjectRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class StreamProjectPOperator extends AbstractPropagatePropertiesForUsedVariablesPOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.STREAM_PROJECT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ ProjectOperator project = (ProjectOperator) op;
+ int[] projectionList = new int[project.getVariables().size()];
+ int i = 0;
+ for (LogicalVariable v : project.getVariables()) {
+ int pos = inputSchemas[0].findVariable(v);
+ if (pos < 0) {
+ throw new AlgebricksException("Could not find variable " + v + ".");
+ }
+ projectionList[i++] = pos;
+ }
+ StreamProjectRuntimeFactory runtime = new StreamProjectRuntimeFactory(projectionList);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ builder.contributeMicroOperator(project, runtime, recDesc);
+ ILogicalOperator src = project.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, project, 0);
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ProjectOperator p = (ProjectOperator) op;
+ computeDeliveredPropertiesForUsedVariables(p, p.getVariables());
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
new file mode 100644
index 0000000..4544e0e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StreamSelectRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class StreamSelectPOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.STREAM_SELECT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ILogicalOperator op2 = op.getInputs().get(0).getOperator();
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ SelectOperator select = (SelectOperator) op;
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ IEvaluatorFactory cond = exprJobGen.createEvaluatorFactory(select.getCondition().getExpression(),
+ context.getTypeEnvironment(op), inputSchemas, context);
+ StreamSelectRuntimeFactory runtime = new StreamSelectRuntimeFactory(cond, null,
+ context.getBinaryBooleanInspector());
+ // contribute one Asterix framewriter
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ builder.contributeMicroOperator(select, runtime, recDesc);
+ // and contribute one edge from its child
+ ILogicalOperator src = select.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, select, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
new file mode 100644
index 0000000..3ca3f40
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StringStreamingRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.scripting.IScriptDescription;
+import edu.uci.ics.hyracks.algebricks.core.algebra.scripting.StringStreamingScriptDescription;
+import edu.uci.ics.hyracks.algebricks.core.algebra.scripting.IScriptDescription.ScriptKind;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class StringStreamingScriptPOperator extends AbstractPropagatePropertiesForUsedVariablesPOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.STRING_STREAM_SCRIPT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ ScriptOperator scriptOp = (ScriptOperator) op;
+ IScriptDescription scriptDesc = scriptOp.getScriptDescription();
+ if (scriptDesc.getKind() != ScriptKind.STRING_STREAMING) {
+ throw new IllegalStateException();
+ }
+ StringStreamingScriptDescription sssd = (StringStreamingScriptDescription) scriptDesc;
+ StringStreamingRuntimeFactory runtime = new StringStreamingRuntimeFactory(sssd.getCommand(),
+ sssd.getPrinterFactories(), sssd.getFieldDelimiter(), sssd.getParserFactory());
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ builder.contributeMicroOperator(scriptOp, runtime, recDesc);
+ // and contribute one edge from its child
+ ILogicalOperator src = scriptOp.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, scriptOp, 0);
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ScriptOperator s = (ScriptOperator) op;
+ computeDeliveredPropertiesForUsedVariables(s, s.getInputVariables());
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
new file mode 100644
index 0000000..7852cdc
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta.SubplanRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class SubplanPOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SUBPLAN;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPhysicalPropertiesVector childsProperties = op2.getPhysicalOperator().getDeliveredProperties();
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
+ if (childsProperties.getLocalProperties() != null) {
+ propsLocal.addAll(childsProperties.getLocalProperties());
+ }
+ // ... get local properties for newly created variables...
+ SubplanOperator subplan = (SubplanOperator) op;
+ for (ILogicalPlan plan : subplan.getNestedPlans()) {
+ for (LogicalOperatorReference r : plan.getRoots()) {
+ AbstractLogicalOperator rOp = (AbstractLogicalOperator) r.getOperator();
+ propsLocal.addAll(rOp.getPhysicalOperator().getDeliveredProperties().getLocalProperties());
+ }
+ }
+
+ deliveredProperties = new StructuralPropertiesVector(childsProperties.getPartitioningProperty(), propsLocal);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ SubplanOperator subplan = (SubplanOperator) op;
+ if (subplan.getNestedPlans().size() != 1) {
+ throw new NotImplementedException("Subplan currently works only for one nested plan with one root.");
+ }
+ AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], subplan, opSchema, context);
+ assert (subplans.length == 1);
+ AlgebricksPipeline np = subplans[0];
+ RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getOperator(),
+ inputSchemas[0], context);
+ INullWriterFactory[] nullWriterFactories = new INullWriterFactory[np.getOutputWidth()];
+ for (int i = 0; i < nullWriterFactories.length; i++) {
+ nullWriterFactories[i] = context.getNullWriterFactory();
+ }
+ SubplanRuntimeFactory runtime = new SubplanRuntimeFactory(np, nullWriterFactories, inputRecordDesc, null);
+
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ builder.contributeMicroOperator(subplan, runtime, recDesc);
+
+ ILogicalOperator src = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, op, 0);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
new file mode 100644
index 0000000..1091365
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.union.UnionAllOperatorDescriptor;
+
+public class UnionAllPOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.UNION_ALL;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ IPartitioningProperty pp = op2.getDeliveredPhysicalProperties().getPartitioningProperty();
+ this.deliveredProperties = new StructuralPropertiesVector(pp, new ArrayList<ILocalStructuralProperty>(0));
+
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ StructuralPropertiesVector pv0 = StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
+ StructuralPropertiesVector pv1 = StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
+ return new PhysicalRequirements(new StructuralPropertiesVector[] { pv0, pv1 },
+ IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ UnionAllOperator unionOp = (UnionAllOperator) op;
+ int n = unionOp.getVariableMappings().size();
+ int[] leftColumns = new int[n];
+ int[] rightColumns = new int[n];
+ int i = 0;
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : unionOp.getVariableMappings()) {
+ int posLeft = inputSchemas[0].findVariable(t.first);
+ leftColumns[i] = posLeft;
+ int posRight = inputSchemas[1].findVariable(t.second);
+ rightColumns[i] = posRight;
+ ++i;
+ }
+
+ JobSpecification spec = builder.getJobSpec();
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+
+ // at algebricks level, union all only accepts two inputs, although at
+ // hyracks
+ // level, there is no restrictions
+ UnionAllOperatorDescriptor opDesc = new UnionAllOperatorDescriptor(spec, 2, recordDescriptor);
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+ ILogicalOperator src1 = op.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src1, 0, op, 0);
+ ILogicalOperator src2 = op.getInputs().get(1).getOperator();
+ builder.contributeGraphEdge(src2, 0, op, 1);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
new file mode 100644
index 0000000..5353316
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.UnnestRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class UnnestPOperator extends AbstractScanPOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.UNNEST;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ UnnestOperator unnest = (UnnestOperator) op;
+ if (unnest.getPositionalVariable() != null) {
+ throw new AlgebricksException("Cannot generate runtime for unnest with positional variable "
+ + unnest.getPositionalVariable());
+ }
+ int outCol = opSchema.findVariable(unnest.getVariable());
+ ILogicalExpression unnestExpr = unnest.getExpressionRef().getExpression();
+ ILogicalExpressionJobGen exprJobGen = context.getExpressionJobGen();
+ boolean exit = false;
+ if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ exit = true;
+ } else {
+ AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) unnestExpr;
+ if (fce.getKind() != FunctionKind.UNNEST) {
+ exit = true;
+ }
+ }
+ if (exit) {
+ throw new AlgebricksException("Unnest expression " + unnestExpr + " is not an unnesting function call.");
+ }
+ UnnestingFunctionCallExpression agg = (UnnestingFunctionCallExpression) unnestExpr;
+ IUnnestingFunctionFactory unnestingFactory = exprJobGen.createUnnestingFunctionFactory(agg, context
+ .getTypeEnvironment(op.getInputs().get(0).getOperator()), inputSchemas, context);
+ int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
+ UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
+ ILogicalOperator src = unnest.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, unnest, 0);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
new file mode 100644
index 0000000..676d6e6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
@@ -0,0 +1,89 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class WriteResultPOperator extends AbstractPhysicalOperator {
+
+ private LogicalVariable payload;
+ private List<LogicalVariable> keys;
+ private IDataSource<?> dataSource;
+
+ public WriteResultPOperator(IDataSource<?> dataSource, LogicalVariable payload, List<LogicalVariable> keys) {
+ this.dataSource = dataSource;
+ this.payload = payload;
+ this.keys = keys;
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + " " + payload + " " + keys;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.WRITE_RESULT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return true;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ List<LogicalVariable> scanVariables = new ArrayList<LogicalVariable>();
+ scanVariables.addAll(keys);
+ scanVariables.add(new LogicalVariable(-1));
+ IPhysicalPropertiesVector r = dataSource.getPropertiesProvider().computePropertiesVector(scanVariables);
+ IPhysicalPropertiesVector[] requirements = new IPhysicalPropertiesVector[1];
+ requirements[0] = r;
+ return new PhysicalRequirements(requirements, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ WriteResultOperator writeResultOp = (WriteResultOperator) op;
+ IMetadataProvider mp = context.getMetadataProvider();
+
+ JobSpecification spec = builder.getJobSpec();
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getWriteResultRuntime(
+ dataSource, propagatedSchema, keys, payload, context, spec);
+
+ builder.contributeHyracksOperator(writeResultOp, runtimeAndConstraints.first);
+ builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+ ILogicalOperator src = writeResultOp.getInputs().get(0).getOperator();
+ builder.contributeGraphEdge(src, 0, writeResultOp, 0);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/plan/ALogicalPlanImpl.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/plan/ALogicalPlanImpl.java
new file mode 100644
index 0000000..34fc909
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/plan/ALogicalPlanImpl.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.plan;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+
+/*
+ * Author: Guangqiang Li
+ * Created on Jul 9, 2009
+ */
+public class ALogicalPlanImpl implements ILogicalPlan {
+ private List<LogicalOperatorReference> roots;
+
+ public ALogicalPlanImpl() {
+ this.roots = new ArrayList<LogicalOperatorReference>();
+ }
+
+ public ALogicalPlanImpl(List<LogicalOperatorReference> roots) {
+ this.roots = roots;
+ }
+
+ public ALogicalPlanImpl(LogicalOperatorReference root) {
+ roots = new ArrayList<LogicalOperatorReference>(1);
+ roots.add(root);
+ }
+
+ public List<LogicalOperatorReference> getRoots() {
+ return roots;
+ }
+
+ public void setRoots(List<LogicalOperatorReference> roots) {
+ this.roots = roots;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
new file mode 100644
index 0000000..9a01651
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -0,0 +1,348 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class LogicalOperatorPrettyPrintVisitor implements ILogicalOperatorVisitor<String, Integer> {
+
+ public LogicalOperatorPrettyPrintVisitor() {
+ }
+
+ @Override
+ public String visitAggregateOperator(AggregateOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("aggregate ").append(op.getVariables()).append(" <- ");
+ pprintExprList(op.getExpressions(), buffer);
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitRunningAggregateOperator(RunningAggregateOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("running-aggregate ").append(op.getVariables()).append(" <- ");
+ pprintExprList(op.getExpressions(), buffer);
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("empty-tuple-source");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitGroupByOperator(GroupByOperator op, Integer indent) throws AlgebricksException {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("group by (").append(op.gByListToString()).append(") decor (")
+ .append(op.decorListToString()).append(") {");
+ printNestedPlans(op, indent, buffer);
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitDistinctOperator(DistinctOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("distinct " + "(");
+ pprintExprList(op.getExpressions(), buffer);
+ buffer.append(")");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitInnerJoinOperator(InnerJoinOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("join (").append(op.getCondition().getExpression()).append(")");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("left outer join (").append(op.getCondition().getExpression()).append(")");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("nested tuple source");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitOrderOperator(OrderOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("order ");
+ for (Pair<OrderOperator.IOrder, LogicalExpressionReference> p : op.getOrderExpressions()) {
+ String fst;
+ switch (p.first.getKind()) {
+ case ASC: {
+ fst = "ASC";
+ break;
+ }
+ case DESC: {
+ fst = "DESC";
+ break;
+ }
+ default: {
+ fst = p.first.getExpressionRef().toString();
+ }
+ }
+ buffer.append("(" + fst + ", " + p.second.getExpression() + ") ");
+ }
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitAssignOperator(AssignOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("assign ").append(op.getVariables()).append(" <- ");
+ pprintExprList(op.getExpressions(), buffer);
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitWriteOperator(WriteOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("write ").append(op.getExpressions());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitWriteResultOperator(WriteResultOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("load ").append(op.getDataSource()).append(" from ")
+ .append(op.getPayloadExpression()).append(" partitioned by ").append(op.getKeyExpressions().toString());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitSelectOperator(SelectOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("select " + "(" + op.getCondition().getExpression() + ")");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitProjectOperator(ProjectOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("project " + "(" + op.getVariables() + ")");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitPartitioningSplitOperator(PartitioningSplitOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("partitioning-split (" + op.getExpressions() + ")");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitSubplanOperator(SubplanOperator op, Integer indent) throws AlgebricksException {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("subplan {");
+ printNestedPlans(op, indent, buffer);
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitUnionOperator(UnionAllOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("union");
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> v : op.getVariableMappings()) {
+ buffer.append(" (" + v.first + ", " + v.second + ", " + v.third + ")");
+ }
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitUnnestOperator(UnnestOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("unnest " + op.getVariable());
+ if (op.getPositionalVariable() != null) {
+ buffer.append(" at " + op.getPositionalVariable());
+ }
+ buffer.append(" <- " + op.getExpressionRef().getExpression());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitUnnestMapOperator(UnnestMapOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append(
+ "unnest-map " + op.getVariables() + " <- " + op.getExpressionRef().getExpression());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitDataScanOperator(DataSourceScanOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append(
+ "data-scan " + op.getProjectVariables() + "<-" + op.getVariables() + " <- " + op.getDataSource());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitLimitOperator(LimitOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("limit " + op.getMaxObjects().getExpression());
+ ILogicalExpression offset = op.getOffset().getExpression();
+ if (offset != null) {
+ buffer.append(", " + offset);
+ }
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitDieOperator(DieOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("die after " + op.getAfterObjects().getExpression());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitExchangeOperator(ExchangeOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("exchange ");
+ return buffer.toString();
+ }
+
+ protected static final StringBuilder addIndent(StringBuilder buffer, int level) {
+ for (int i = 0; i < level; ++i) {
+ buffer.append(' ');
+ }
+ return buffer;
+ }
+
+ private void printNestedPlans(AbstractOperatorWithNestedPlans op, Integer indent, StringBuilder buffer)
+ throws AlgebricksException {
+ boolean first = true;
+ if (op.getNestedPlans().isEmpty()) {
+ buffer.append("}");
+ } else {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ // PrettyPrintUtil.indent(buffer, level + 10).append("var " +
+ // p.first + ":\n");
+ buffer.append("\n");
+ if (first) {
+ first = false;
+ } else {
+ addIndent(buffer, indent).append(" {\n");
+ }
+ PlanPrettyPrinter.printPlan(p, buffer, this, indent + 10);
+ addIndent(buffer, indent).append(" }");
+ }
+ }
+ }
+
+ @Override
+ public String visitScriptOperator(ScriptOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append(
+ "script (in: " + op.getInputVariables() + ") (out: " + op.getOutputVariables() + ")");
+ return buffer.toString();
+ }
+
+ private void pprintExprList(List<LogicalExpressionReference> expressions, StringBuilder buffer) {
+ buffer.append("[");
+ boolean first = true;
+ for (LogicalExpressionReference exprRef : expressions) {
+ if (first) {
+ first = false;
+ } else {
+ buffer.append(", ");
+ }
+ buffer.append(exprRef.getExpression());
+ }
+ buffer.append("]");
+ }
+
+ @Override
+ public String visitReplicateOperator(ReplicateOperator op, Integer indent) throws AlgebricksException {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("replicate ");
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitInsertDeleteOperator(InsertDeleteOperator op, Integer indent) throws AlgebricksException {
+ StringBuilder buffer = new StringBuilder();
+ String header = op.getOperation() == Kind.INSERT ? "insert into " : "delete from ";
+ addIndent(buffer, indent).append(header).append(op.getDataSource()).append(" from ")
+ .append(op.getPayloadExpression()).append(" partitioned by ")
+ .append(op.getPrimaryKeyExpressions().toString());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Integer indent)
+ throws AlgebricksException {
+ StringBuilder buffer = new StringBuilder();
+ String header = op.getOperation() == Kind.INSERT ? "insert into " : "delete from ";
+ addIndent(buffer, indent).append(header).append(op.getDataSourceIndex()).append(" from ")
+ .append(op.getSecondaryKeyExpressions().toString()).append(" ")
+ .append(op.getPrimaryKeyExpressions().toString());
+ return buffer.toString();
+ }
+
+ @Override
+ public String visitSinkOperator(SinkOperator op, Integer indent) throws AlgebricksException {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("sink");
+ return buffer.toString();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
new file mode 100644
index 0000000..91f28f6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class PlanPrettyPrinter {
+ public static void printPlan(ILogicalPlan plan, StringBuilder out, LogicalOperatorPrettyPrintVisitor pvisitor,
+ int indent) throws AlgebricksException {
+ for (LogicalOperatorReference root : plan.getRoots()) {
+ printOperator((AbstractLogicalOperator) root.getOperator(), out, pvisitor, indent);
+ }
+ }
+
+ public static void printPhysicalOps(ILogicalPlan plan, StringBuilder out, int indent) {
+ for (LogicalOperatorReference root : plan.getRoots()) {
+ printPhysicalOperator((AbstractLogicalOperator) root.getOperator(), indent, out);
+ }
+ }
+
+ public static void printOperator(AbstractLogicalOperator op, StringBuilder out,
+ LogicalOperatorPrettyPrintVisitor pvisitor, int indent) throws AlgebricksException {
+ out.append(op.accept(pvisitor, indent));
+ IPhysicalOperator pOp = op.getPhysicalOperator();
+
+ if (pOp != null) {
+ out.append("\n");
+ pad(out, indent);
+ appendln(out, "-- " + pOp.toString() + " |" + op.getExecutionMode() + "|");
+ } else {
+ appendln(out, " -- |" + op.getExecutionMode() + "|");
+ }
+
+ for (LogicalOperatorReference i : op.getInputs()) {
+ printOperator((AbstractLogicalOperator) i.getOperator(), out, pvisitor, indent + 2);
+ }
+
+ }
+
+ public static void printPhysicalOperator(AbstractLogicalOperator op, int indent, StringBuilder out) {
+ IPhysicalOperator pOp = op.getPhysicalOperator();
+ pad(out, indent);
+ appendln(out, "-- " + pOp.toString() + " |" + op.getExecutionMode() + "|");
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans opNest = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : opNest.getNestedPlans()) {
+ pad(out, indent + 8);
+ appendln(out, "{");
+ printPhysicalOps(p, out, indent + 10);
+ pad(out, indent + 8);
+ appendln(out, "}");
+ }
+ }
+
+ for (LogicalOperatorReference i : op.getInputs()) {
+ printPhysicalOperator((AbstractLogicalOperator) i.getOperator(), indent + 2, out);
+ }
+
+ }
+
+ private static void appendln(StringBuilder buf, String s) {
+ buf.append(s);
+ buf.append("\n");
+ }
+
+ private static void pad(StringBuilder buf, int indent) {
+ for (int i = 0; i < indent; ++i) {
+ buf.append(' ');
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/AbstractGroupingProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/AbstractGroupingProperty.java
new file mode 100644
index 0000000..c11131a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/AbstractGroupingProperty.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public abstract class AbstractGroupingProperty {
+ protected Set<LogicalVariable> columnSet;
+
+ public AbstractGroupingProperty(Set<LogicalVariable> columnSet) {
+ this.columnSet = columnSet;
+ }
+
+ public Set<LogicalVariable> getColumnSet() {
+ return columnSet;
+ }
+
+ public final void normalizeGroupingColumns(Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds) {
+ replaceGroupingColumnsByEqClasses(equivalenceClasses);
+ applyFDsToGroupingColumns(fds);
+ }
+
+ private void replaceGroupingColumnsByEqClasses(Map<LogicalVariable, EquivalenceClass> equivalenceClasses) {
+ if (equivalenceClasses == null || equivalenceClasses.isEmpty()) {
+ return;
+ }
+ HashSet<LogicalVariable> norm = new HashSet<LogicalVariable>();
+ for (LogicalVariable v : columnSet) {
+ EquivalenceClass ec = equivalenceClasses.get(v);
+ if (ec == null) {
+ norm.add(v);
+ } else {
+ if (ec.representativeIsConst()) {
+ // trivially satisfied, so the var. can be removed
+ } else {
+ norm.add(ec.getVariableRepresentative());
+ }
+ }
+ }
+ columnSet = norm;
+ }
+
+ private void applyFDsToGroupingColumns(List<FunctionalDependency> fds) {
+ // the set of vars. is unordered
+ // so we try all FDs on all variables (incomplete algo?)
+ if (fds == null || fds.isEmpty()) {
+ return;
+ }
+ HashSet<LogicalVariable> norm = new HashSet<LogicalVariable>();
+ for (LogicalVariable v : columnSet) {
+ boolean isImpliedByAnFD = false;
+ for (FunctionalDependency fdep : fds) {
+ if (columnSet.containsAll(fdep.getHead()) && fdep.getTail().contains(v)) {
+ isImpliedByAnFD = true;
+ norm.addAll(fdep.getHead());
+ break;
+ }
+
+ }
+ if (!isImpliedByAnFD) {
+ norm.add(v);
+ }
+ }
+ columnSet = norm;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/AsterixNodeGroupDomain.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/AsterixNodeGroupDomain.java
new file mode 100644
index 0000000..d27534a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/AsterixNodeGroupDomain.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+public class AsterixNodeGroupDomain implements INodeDomain {
+
+ private String groupName;
+
+ public AsterixNodeGroupDomain(String groupName) {
+ this.groupName = groupName;
+ }
+
+ @Override
+ public boolean sameAs(INodeDomain domain) {
+ if (!(domain instanceof AsterixNodeGroupDomain)) {
+ return false;
+ }
+ AsterixNodeGroupDomain dom2 = (AsterixNodeGroupDomain) domain;
+ return groupName.equals(dom2.groupName);
+ }
+
+ @Override
+ public String toString() {
+ return "AsterixDomain(" + groupName + ")";
+ }
+
+ @Override
+ public Integer cardinality() {
+ return null;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java
new file mode 100644
index 0000000..8a38cfb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public class BroadcastPartitioningProperty implements IPartitioningProperty {
+
+ private INodeDomain domain;
+
+ public BroadcastPartitioningProperty(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+ @Override
+ public PartitioningType getPartitioningType() {
+ return PartitioningType.BROADCAST;
+ }
+
+ @Override
+ public void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
+ // do nothing
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ }
+
+ @Override
+ public INodeDomain getNodeDomain() {
+ return domain;
+ }
+
+ @Override
+ public void setNodeDomain(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FileSplitDomain.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FileSplitDomain.java
new file mode 100644
index 0000000..876ee52
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FileSplitDomain.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
+public class FileSplitDomain implements INodeDomain {
+
+ private FileSplit[] splits;
+
+ public FileSplitDomain(FileSplit[] splits) {
+ this.splits = splits;
+ }
+
+ @Override
+ public Integer cardinality() {
+ return splits.length;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("FileSplitDomain[");
+ boolean fst = true;
+ for (FileSplit fs : splits) {
+ if (fst) {
+ fst = false;
+ } else {
+ sb.append(", ");
+ }
+ sb.append(fs.getNodeName() + ":" + fs.getLocalFile());
+ }
+ sb.append(']');
+ return sb.toString();
+ }
+
+ @Override
+ public boolean sameAs(INodeDomain domain) {
+ if (!(domain instanceof FileSplitDomain)) {
+ return false;
+ }
+ FileSplitDomain fsd = (FileSplitDomain) domain;
+ if (fsd.splits.length != splits.length) {
+ return false;
+ }
+ // conservative approach...
+ for (int i = 0; i < splits.length; i++) {
+ if (!ncEq(splits[i], fsd.splits[i])) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ private boolean ncEq(FileSplit fs1, FileSplit fs2) {
+ return fs1.getNodeName().equals(fs2.getNodeName());
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FilteredVariablePropagationPolicy.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FilteredVariablePropagationPolicy.java
new file mode 100644
index 0000000..40bbc36
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FilteredVariablePropagationPolicy.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+
+public class FilteredVariablePropagationPolicy extends VariablePropagationPolicy {
+
+ List<LogicalVariable> toPropagate;
+
+ public FilteredVariablePropagationPolicy(List<LogicalVariable> varList) {
+ toPropagate = varList;
+ }
+
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
+ for (LogicalVariable v : toPropagate) {
+ target.addVariable(v);
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FunctionalDependency.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FunctionalDependency.java
new file mode 100644
index 0000000..1c76171
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/FunctionalDependency.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public final class FunctionalDependency {
+ private List<LogicalVariable> head;
+ private List<LogicalVariable> tail;
+
+ public FunctionalDependency(List<LogicalVariable> head, List<LogicalVariable> tail) {
+ this.head = head;
+ this.tail = tail;
+ }
+
+ public List<LogicalVariable> getHead() {
+ return head;
+ }
+
+ public List<LogicalVariable> getTail() {
+ return tail;
+ }
+
+ @Override
+ public String toString() {
+ return head + "->" + tail;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof FunctionalDependency)) {
+ return false;
+ } else {
+ FunctionalDependency fd = (FunctionalDependency) obj;
+ return fd.getHead().equals(this.head) && fd.getTail().equals(this.tail);
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ILocalStructuralProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ILocalStructuralProperty.java
new file mode 100644
index 0000000..fbe30f5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ILocalStructuralProperty.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public interface ILocalStructuralProperty extends IStructuralProperty {
+ public enum PropertyType {
+ LOCAL_GROUPING_PROPERTY,
+ LOCAL_ORDER_PROPERTY
+ }
+
+ public void getVariables(Collection<LogicalVariable> variables);
+
+ public PropertyType getPropertyType();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ILogicalPropertiesVector.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ILogicalPropertiesVector.java
new file mode 100644
index 0000000..214a579
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ILogicalPropertiesVector.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+public interface ILogicalPropertiesVector {
+ public Integer getNumberOfTuples();
+
+ public Integer getMaxOutputFrames();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/INodeDomain.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/INodeDomain.java
new file mode 100644
index 0000000..76057f6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/INodeDomain.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+public interface INodeDomain {
+ public boolean sameAs(INodeDomain domain);
+
+ /** @return the number of nodes in that domain or null if unknown */
+ public Integer cardinality();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
new file mode 100644
index 0000000..8a0cab7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public interface IPartitioningProperty extends IStructuralProperty {
+ public enum PartitioningType {
+ UNPARTITIONED, RANDOM, BROADCAST, UNORDERED_PARTITIONED, ORDERED_PARTITIONED
+ }
+
+ static final INodeDomain DOMAIN_FOR_UNPARTITIONED_DATA = new INodeDomain() {
+ @Override
+ public boolean sameAs(INodeDomain domain) {
+ return domain == this;
+ }
+
+ @Override
+ public Integer cardinality() {
+ return null;
+ }
+ };
+
+ public static final IPartitioningProperty UNPARTITIONED = new IPartitioningProperty() {
+
+ @Override
+ public PartitioningType getPartitioningType() {
+ return PartitioningType.UNPARTITIONED;
+ }
+
+ @Override
+ public void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
+ // do nothing
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ }
+
+ @Override
+ public INodeDomain getNodeDomain() {
+ return DOMAIN_FOR_UNPARTITIONED_DATA;
+ }
+
+ @Override
+ public String toString() {
+ return getPartitioningType().toString();
+ }
+
+ @Override
+ public void setNodeDomain(INodeDomain domain) {
+ throw new IllegalStateException();
+ }
+ };
+
+ public abstract PartitioningType getPartitioningType();
+
+ public abstract void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds);
+
+ public abstract INodeDomain getNodeDomain();
+
+ public abstract void setNodeDomain(INodeDomain domain);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPartitioningRequirementsCoordinator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPartitioningRequirementsCoordinator.java
new file mode 100644
index 0000000..d884a17
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPartitioningRequirementsCoordinator.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+/**
+ *
+ * Implements constraints in between requirements for the children of the same
+ * operator.
+ *
+ */
+
+public interface IPartitioningRequirementsCoordinator {
+
+ public static IPartitioningRequirementsCoordinator NO_COORDINATION = new IPartitioningRequirementsCoordinator() {
+
+ @Override
+ public Pair<Boolean, IPartitioningProperty> coordinateRequirements(IPartitioningProperty requirements,
+ IPartitioningProperty firstDeliveredPartitioning, ILogicalOperator op, IOptimizationContext context) {
+ return new Pair<Boolean, IPartitioningProperty>(true, requirements);
+ }
+ };
+
+ public static IPartitioningRequirementsCoordinator EQCLASS_PARTITIONING_COORDINATOR = new IPartitioningRequirementsCoordinator() {
+
+ @Override
+ public Pair<Boolean, IPartitioningProperty> coordinateRequirements(IPartitioningProperty rqdpp,
+ IPartitioningProperty firstDeliveredPartitioning, ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ if (firstDeliveredPartitioning != null
+ && firstDeliveredPartitioning.getPartitioningType() == rqdpp.getPartitioningType()) {
+ switch (rqdpp.getPartitioningType()) {
+ case UNORDERED_PARTITIONED: {
+ UnorderedPartitionedProperty upp1 = (UnorderedPartitionedProperty) firstDeliveredPartitioning;
+ Set<LogicalVariable> set1 = upp1.getColumnSet();
+ UnorderedPartitionedProperty uppreq = (UnorderedPartitionedProperty) rqdpp;
+ Set<LogicalVariable> modifuppreq = new HashSet<LogicalVariable>();
+ Map<LogicalVariable, EquivalenceClass> eqmap = context.getEquivalenceClassMap(op);
+ Set<LogicalVariable> covered = new HashSet<LogicalVariable>();
+ for (LogicalVariable r : uppreq.getColumnSet()) {
+ EquivalenceClass ec = eqmap.get(r);
+ for (LogicalVariable v : set1) {
+ EquivalenceClass ecFirst = eqmap.get(v);
+ if (ecFirst == ec) {
+ covered.add(v);
+ modifuppreq.add(r);
+ break;
+ }
+ }
+ }
+ if (!covered.equals(set1)) {
+ throw new AlgebricksException("Could not modify " + rqdpp
+ + " to agree with partitioning property " + firstDeliveredPartitioning
+ + " delivered by previous input operator.");
+ }
+ UnorderedPartitionedProperty upp2 = new UnorderedPartitionedProperty(modifuppreq,
+ rqdpp.getNodeDomain());
+ return new Pair<Boolean, IPartitioningProperty>(false, upp2);
+ }
+ case ORDERED_PARTITIONED: {
+ throw new NotImplementedException();
+ }
+ }
+ }
+ return new Pair<Boolean, IPartitioningProperty>(true, rqdpp);
+ }
+
+ };
+
+ public Pair<Boolean, IPartitioningProperty> coordinateRequirements(IPartitioningProperty requirements,
+ IPartitioningProperty firstDeliveredPartitioning, ILogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPhysicalPropertiesVector.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPhysicalPropertiesVector.java
new file mode 100644
index 0000000..72f9683
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPhysicalPropertiesVector.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public interface IPhysicalPropertiesVector {
+
+ public IPartitioningProperty getPartitioningProperty();
+
+ public List<ILocalStructuralProperty> getLocalProperties();
+
+ /**
+ *
+ * @param reqd
+ * vector of required properties
+ * @param equivalenceClasses
+ * @param fds
+ * @return a vector of properties from pvector that are not delivered by the
+ * current vector or null if none
+ */
+ public IPhysicalPropertiesVector getUnsatisfiedPropertiesFrom(IPhysicalPropertiesVector reqd,
+ boolean mayExpandProperties, Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds);
+
+ public IPhysicalPropertiesVector clone();
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPropertiesComputer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPropertiesComputer.java
new file mode 100644
index 0000000..02afec1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IPropertiesComputer.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public interface IPropertiesComputer {
+
+ // can return null
+ public IPartitioningProperty computePartitioningProperty(ILogicalExpression expr);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IStructuralProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IStructuralProperty.java
new file mode 100644
index 0000000..aabf9e9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/IStructuralProperty.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public interface IStructuralProperty {
+ public void getColumns(Collection<LogicalVariable> columns);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LocalGroupingProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LocalGroupingProperty.java
new file mode 100644
index 0000000..5c82ee7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LocalGroupingProperty.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public class LocalGroupingProperty extends AbstractGroupingProperty implements ILocalStructuralProperty {
+
+ // preferredOrderEnforcer, if not null, is guaranteed to enforce grouping on
+ // columnSet
+ private List<LogicalVariable> preferredOrderEnforcer;
+
+ public LocalGroupingProperty(Set<LogicalVariable> columnSet) {
+ super(columnSet);
+ }
+
+ public LocalGroupingProperty(Set<LogicalVariable> columnSet, List<LogicalVariable> preferredOrderEnforcer) {
+ this(columnSet);
+ this.preferredOrderEnforcer = preferredOrderEnforcer;
+ }
+
+ @Override
+ public PropertyType getPropertyType() {
+ return PropertyType.LOCAL_GROUPING_PROPERTY;
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ columns.addAll(columnSet);
+ }
+
+ @Override
+ public String toString() {
+ return columnSet.toString();
+ }
+
+ @Override
+ public void getVariables(Collection<LogicalVariable> variables) {
+ variables.addAll(columnSet);
+ }
+
+ public List<LogicalVariable> getPreferredOrderEnforcer() {
+ return preferredOrderEnforcer;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
new file mode 100644
index 0000000..7810df7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+
+public final class LocalOrderProperty implements ILocalStructuralProperty {
+
+ private OrderColumn orderColumn;
+
+ public LocalOrderProperty(OrderColumn orderColumn) {
+ this.orderColumn = orderColumn;
+ }
+
+ public OrderColumn getOrderColumn() {
+ return orderColumn;
+ }
+
+ public void setOrderColumn(OrderColumn orderColumn) {
+ this.orderColumn = orderColumn;
+ }
+
+ public LogicalVariable getColumn() {
+ return orderColumn.getColumn();
+ }
+
+ public OrderKind getOrder() {
+ return orderColumn.getOrder();
+ }
+
+ @Override
+ public PropertyType getPropertyType() {
+ return PropertyType.LOCAL_ORDER_PROPERTY;
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ columns.add(getColumn());
+ }
+
+ @Override
+ public String toString() {
+ return orderColumn.toString();
+ }
+
+ @Override
+ public void getVariables(Collection<LogicalVariable> variables) {
+ variables.add(orderColumn.getColumn());
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LogicalPropertiesVectorImpl.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LogicalPropertiesVectorImpl.java
new file mode 100644
index 0000000..ae25f41
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/LogicalPropertiesVectorImpl.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+public class LogicalPropertiesVectorImpl implements ILogicalPropertiesVector {
+
+ private Integer numTuples, maxOutputFrames;
+
+ @Override
+ public String toString() {
+ return "LogicalPropertiesVector [ num.tuples: " + numTuples + ", maxOutputFrames: " + maxOutputFrames + " ]";
+ }
+
+ @Override
+ public Integer getNumberOfTuples() {
+ return numTuples;
+ }
+
+ public void setNumberOfTuples(Integer numTuples) {
+ this.numTuples = numTuples;
+ }
+
+ @Override
+ public Integer getMaxOutputFrames() {
+ return maxOutputFrames;
+ }
+
+ public void setMaxOutputFrames(Integer maxOutputFrames) {
+ this.maxOutputFrames = maxOutputFrames;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/OrderColumn.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/OrderColumn.java
new file mode 100644
index 0000000..5f22b95
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/OrderColumn.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+
+public final class OrderColumn {
+
+ private LogicalVariable column;
+ private OrderKind order;
+
+ public OrderColumn(LogicalVariable column, OrderKind order) {
+ this.column = column;
+ this.order = order;
+ }
+
+ public LogicalVariable getColumn() {
+ return column;
+ }
+
+ public OrderKind getOrder() {
+ return order;
+ }
+
+ public void setColumn(LogicalVariable column) {
+ this.column = column;
+ }
+
+ public void setOrder(OrderKind order) {
+ this.order = order;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof OrderColumn)) {
+ return false;
+ } else {
+ OrderColumn oc = (OrderColumn) obj;
+ return column.equals(oc.getColumn()) && order == oc.getOrder();
+ }
+ }
+
+ @Override
+ public String toString() {
+ return column.toString() + "(" + order + ")";
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
new file mode 100644
index 0000000..137c46d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public class OrderedPartitionedProperty implements IPartitioningProperty {
+
+ private ArrayList<OrderColumn> orderColumns;
+ private INodeDomain domain;
+
+ public OrderedPartitionedProperty(ArrayList<OrderColumn> orderColumns, INodeDomain domain) {
+ this.domain = domain;
+ this.orderColumns = orderColumns;
+ }
+
+ public ArrayList<OrderColumn> getOrderColumns() {
+ return orderColumns;
+ }
+
+ public ArrayList<LogicalVariable> getColumns() {
+ ArrayList<LogicalVariable> cols = new ArrayList<LogicalVariable>(orderColumns.size());
+ for (OrderColumn oc : orderColumns) {
+ cols.add(oc.getColumn());
+ }
+ return cols;
+ }
+
+ @Override
+ public PartitioningType getPartitioningType() {
+ return PartitioningType.ORDERED_PARTITIONED;
+ }
+
+ @Override
+ public String toString() {
+ return getPartitioningType().toString() + orderColumns;
+ }
+
+ @Override
+ public void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
+ orderColumns = PropertiesUtil.replaceOrderColumnsByEqClasses(orderColumns, equivalenceClasses);
+ orderColumns = PropertiesUtil.applyFDsToOrderColumns(orderColumns, fds);
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ for (OrderColumn oc : orderColumns) {
+ columns.add(oc.getColumn());
+ }
+ }
+
+ @Override
+ public INodeDomain getNodeDomain() {
+ return domain;
+ }
+
+ @Override
+ public void setNodeDomain(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/PhysicalRequirements.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/PhysicalRequirements.java
new file mode 100644
index 0000000..7503a3c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/PhysicalRequirements.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+public class PhysicalRequirements {
+ private final IPhysicalPropertiesVector[] requiredProperties;
+ private final IPartitioningRequirementsCoordinator partitioningCoordinator;
+
+ public PhysicalRequirements(IPhysicalPropertiesVector[] requiredProperties,
+ IPartitioningRequirementsCoordinator partitioningCoordinator) {
+ this.requiredProperties = requiredProperties;
+ this.partitioningCoordinator = partitioningCoordinator;
+ }
+
+ public IPhysicalPropertiesVector[] getRequiredProperties() {
+ return requiredProperties;
+ }
+
+ public IPartitioningRequirementsCoordinator getPartitioningCoordinator() {
+ return partitioningCoordinator;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
new file mode 100644
index 0000000..ad80e46
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
@@ -0,0 +1,323 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
+
+public class PropertiesUtil {
+
+ public Set<LogicalVariable> closureUnderFDs(Collection<LogicalVariable> vars, List<FunctionalDependency> fdList) {
+ Set<LogicalVariable> k = new HashSet<LogicalVariable>(vars);
+ boolean change;
+ do {
+ change = false;
+ for (FunctionalDependency fd : fdList) {
+ List<LogicalVariable> h = fd.getHead();
+ if (k.containsAll(h)) {
+ List<LogicalVariable> t = fd.getTail();
+ for (LogicalVariable v : t) {
+ if (!(k.contains(v))) {
+ k.add(v);
+ change = true;
+ }
+ }
+ }
+ }
+ } while (change);
+ return k;
+ }
+
+ public static boolean matchLocalProperties(List<ILocalStructuralProperty> reqd,
+ List<ILocalStructuralProperty> dlvd, Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds) {
+ if (reqd == null) {
+ return true;
+ }
+ if (dlvd == null) {
+ return false;
+ }
+ normalizeLocals(reqd, equivalenceClasses, fds);
+ normalizeLocals(dlvd, equivalenceClasses, fds);
+
+ ListIterator<ILocalStructuralProperty> dlvdIter = dlvd.listIterator();
+
+ Set<LogicalVariable> rqdCols = new HashSet<LogicalVariable>();
+ Set<LogicalVariable> dlvdCols = new HashSet<LogicalVariable>();
+ for (ILocalStructuralProperty r : reqd) {
+ if (r.getPropertyType() == PropertyType.LOCAL_GROUPING_PROPERTY) {
+ rqdCols.clear();
+ r.getVariables(rqdCols);
+ }
+ boolean implied = false;
+ while (!implied && dlvdIter.hasNext()) {
+ ILocalStructuralProperty d = dlvdIter.next();
+ switch (r.getPropertyType()) {
+ case LOCAL_ORDER_PROPERTY: {
+ if (d.getPropertyType() != PropertyType.LOCAL_ORDER_PROPERTY) {
+ return false;
+ }
+ LocalOrderProperty lop = (LocalOrderProperty) d;
+ if (lop.getColumn() == ((LocalOrderProperty) r).getColumn()
+ && lop.getOrder() == ((LocalOrderProperty) r).getOrder()) {
+ implied = true;
+ } else {
+ return false;
+ }
+ break;
+ }
+ case LOCAL_GROUPING_PROPERTY: {
+ dlvdCols.clear();
+ d.getColumns(dlvdCols);
+ for (LogicalVariable v : dlvdCols) {
+ if (rqdCols.contains(v)) {
+ rqdCols.remove(v);
+ } else {
+ return false;
+ }
+ }
+ if (rqdCols.isEmpty()) {
+ implied = true;
+ }
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ if (!implied) {
+ return false;
+ }
+ }
+ return true;
+
+ }
+
+ public static boolean matchPartitioningProps(IPartitioningProperty reqd, IPartitioningProperty dlvd,
+ boolean mayExpandProperties) {
+
+ INodeDomain dom1 = reqd.getNodeDomain();
+ INodeDomain dom2 = dlvd.getNodeDomain();
+ if (dom1 != null && dom2 != null && !dom1.sameAs(dom2)) {
+ return false;
+ }
+
+ switch (reqd.getPartitioningType()) {
+ case RANDOM: {
+ // anything matches RANDOM
+ return true;
+ }
+ case UNORDERED_PARTITIONED: {
+ switch (dlvd.getPartitioningType()) {
+ case UNORDERED_PARTITIONED: {
+ UnorderedPartitionedProperty ur = (UnorderedPartitionedProperty) reqd;
+ UnorderedPartitionedProperty ud = (UnorderedPartitionedProperty) dlvd;
+ if (mayExpandProperties) {
+ return ur.getColumnSet().containsAll(ud.getColumnSet());
+ } else {
+ return ur.getColumnSet().equals(ud.getColumnSet());
+ }
+ }
+ case ORDERED_PARTITIONED: {
+ UnorderedPartitionedProperty ur = (UnorderedPartitionedProperty) reqd;
+ OrderedPartitionedProperty od = (OrderedPartitionedProperty) dlvd;
+ if (mayExpandProperties) {
+ return ur.getColumnSet().containsAll(od.getOrderColumns());
+ } else {
+ return ur.getColumnSet().containsAll(od.getOrderColumns())
+ && od.getOrderColumns().containsAll(ur.getColumnSet());
+ }
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+ case ORDERED_PARTITIONED: {
+ switch (dlvd.getPartitioningType()) {
+ case ORDERED_PARTITIONED: {
+ OrderedPartitionedProperty or = (OrderedPartitionedProperty) reqd;
+ OrderedPartitionedProperty od = (OrderedPartitionedProperty) dlvd;
+ if (mayExpandProperties) {
+ return isPrefixOf(od.getOrderColumns(), or.getOrderColumns());
+ } else {
+ return od.getOrderColumns().equals(or.getOrderColumns());
+ }
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+ default: {
+ return (dlvd.getPartitioningType() == reqd.getPartitioningType());
+ }
+ }
+ }
+
+ /**
+ *
+ * @param pref
+ * @param target
+ * @return true iff pref is a prefix of target
+ */
+ private static boolean isPrefixOf(List<OrderColumn> pref, List<OrderColumn> target) {
+ Iterator<OrderColumn> iter = target.iterator();
+ for (OrderColumn v : pref) {
+ if (!iter.hasNext()) {
+ return false;
+ }
+ if (!v.equals(iter.next())) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public static ArrayList<OrderColumn> applyFDsToOrderColumns(ArrayList<OrderColumn> orderColumns,
+ List<FunctionalDependency> fds) {
+ // the set of vars. is ordered
+ // so we try the variables in order from last to first
+ if (fds == null || fds.isEmpty()) {
+ return orderColumns;
+ }
+
+ int deleted = 0;
+ for (int i = orderColumns.size() - 1; i >= 0; i--) {
+ for (FunctionalDependency fdep : fds) {
+ if (impliedByPrefix(orderColumns, i, fdep)) {
+ orderColumns.set(i, null);
+ deleted++;
+ break;
+ }
+ }
+ }
+ ArrayList<OrderColumn> norm = new ArrayList<OrderColumn>(orderColumns.size() - deleted);
+ for (OrderColumn oc : orderColumns) {
+ if (oc != null) {
+ norm.add(oc);
+ }
+ }
+ return norm;
+ }
+
+ public static ArrayList<OrderColumn> replaceOrderColumnsByEqClasses(ArrayList<OrderColumn> orderColumns,
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses) {
+ if (equivalenceClasses == null || equivalenceClasses.isEmpty()) {
+ return orderColumns;
+ }
+ ArrayList<OrderColumn> norm = new ArrayList<OrderColumn>();
+ for (OrderColumn v : orderColumns) {
+ EquivalenceClass ec = equivalenceClasses.get(v.getColumn());
+ if (ec == null) {
+ norm.add(v);
+ } else {
+ if (ec.representativeIsConst()) {
+ // trivially satisfied, so the var. can be removed
+ } else {
+ norm.add(new OrderColumn(ec.getVariableRepresentative(), v.getOrder()));
+ }
+ }
+ }
+ return norm;
+ }
+
+ private static boolean impliedByPrefix(ArrayList<OrderColumn> vars, int i, FunctionalDependency fdep) {
+ if (!fdep.getTail().contains(vars.get(i).getColumn())) {
+ return false;
+ }
+ boolean fdSat = true;
+ for (LogicalVariable pv : fdep.getHead()) {
+ boolean isInPrefix = false;
+ for (int j = 0; j < i; j++) {
+ if (vars.get(j).getColumn().equals(pv)) {
+ isInPrefix = true;
+ break;
+ }
+ }
+ if (!isInPrefix) {
+ fdSat = false;
+ break;
+ }
+ }
+ return fdSat;
+ }
+
+ private static void normalizeLocals(List<ILocalStructuralProperty> props,
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
+ ListIterator<ILocalStructuralProperty> propIter = props.listIterator();
+ int pos = -1;
+ while (propIter.hasNext()) {
+ ILocalStructuralProperty p = propIter.next();
+ if (p.getPropertyType() == PropertyType.LOCAL_GROUPING_PROPERTY) {
+ ((LocalGroupingProperty) p).normalizeGroupingColumns(equivalenceClasses, fds);
+ pos++;
+ } else {
+ LocalOrderProperty ord = (LocalOrderProperty) p;
+ EquivalenceClass ec = equivalenceClasses.get(ord.getColumn());
+ if (ec != null) {
+ if (ec.representativeIsConst()) {
+ propIter.remove();
+ } else {
+ ord.getOrderColumn().setColumn(ec.getVariableRepresentative());
+ pos++;
+ }
+ } else {
+ pos++;
+ }
+ }
+ }
+
+ if (pos < 1) {
+ return;
+ }
+
+ while (propIter.hasPrevious()) {
+ ILocalStructuralProperty p = propIter.previous();
+ ListIterator<ILocalStructuralProperty> secondIter = props.listIterator(pos);
+ pos--;
+ Set<LogicalVariable> cols = new HashSet<LogicalVariable>();
+ while (secondIter.hasPrevious()) {
+ secondIter.previous().getColumns(cols);
+ }
+ secondIter = null;
+ for (FunctionalDependency fdep : fds) {
+ LinkedList<LogicalVariable> columnsOfP = new LinkedList<LogicalVariable>();
+ p.getColumns(columnsOfP);
+ if (impliedByPrefix(columnsOfP, cols, fdep)) {
+ propIter.remove();
+ break;
+ }
+ }
+ }
+ }
+
+ private static boolean impliedByPrefix(List<LogicalVariable> colsOfProp, Set<LogicalVariable> colsOfPrefix,
+ FunctionalDependency fdep) {
+ return fdep.getTail().containsAll(colsOfProp) && colsOfPrefix.containsAll(fdep.getHead());
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/RandomPartitioningProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/RandomPartitioningProperty.java
new file mode 100644
index 0000000..66e900e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/RandomPartitioningProperty.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public class RandomPartitioningProperty implements IPartitioningProperty {
+
+ private INodeDomain domain;
+
+ public RandomPartitioningProperty(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+ @Override
+ public PartitioningType getPartitioningType() {
+ return PartitioningType.RANDOM;
+ }
+
+ @Override
+ public String toString() {
+ return getPartitioningType() + " domain:" + domain;
+ }
+
+ @Override
+ public void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
+ // do nothing
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ }
+
+ @Override
+ public INodeDomain getNodeDomain() {
+ return domain;
+ }
+
+ @Override
+ public void setNodeDomain(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
new file mode 100644
index 0000000..d8d8f35
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public class StructuralPropertiesVector implements IPhysicalPropertiesVector {
+ private List<ILocalStructuralProperty> propsLocal;
+ private IPartitioningProperty propPartitioning;
+
+ public static final StructuralPropertiesVector EMPTY_PROPERTIES_VECTOR = new StructuralPropertiesVector(null,
+ new ArrayList<ILocalStructuralProperty>());
+
+ public StructuralPropertiesVector(IPartitioningProperty propPartitioning, List<ILocalStructuralProperty> propsLocal) {
+ this.propPartitioning = propPartitioning;
+ this.propsLocal = propsLocal;
+ }
+
+ @Override
+ public String toString() {
+ return "propsLocal=" + propsLocal + "\tpropPartioning=" + propPartitioning;
+ }
+
+ @Override
+ public IPartitioningProperty getPartitioningProperty() {
+ return propPartitioning;
+ }
+
+ @Override
+ public List<ILocalStructuralProperty> getLocalProperties() {
+ return propsLocal;
+ }
+
+ @Override
+ public IPhysicalPropertiesVector clone() {
+ List<ILocalStructuralProperty> propsCopy = new LinkedList<ILocalStructuralProperty>();
+ if (propsLocal != null) {
+ propsCopy.addAll(propsLocal);
+ }
+ return new StructuralPropertiesVector(propPartitioning, propsCopy);
+ }
+
+ /**
+ *
+ * @param reqd
+ * vector of required properties
+ * @return a vector of properties from pvector that are not delivered by the
+ * current vector or null if none
+ */
+ @Override
+ public IPhysicalPropertiesVector getUnsatisfiedPropertiesFrom(IPhysicalPropertiesVector reqd,
+ boolean mayExpandProperties, Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds) {
+ List<ILocalStructuralProperty> plist = reqd.getLocalProperties();
+ List<ILocalStructuralProperty> diffLocals = null;
+ if (plist != null && !plist.isEmpty()) {
+ if (!PropertiesUtil.matchLocalProperties(plist, propsLocal, equivalenceClasses, fds)) {
+ diffLocals = plist;
+ }
+ }
+
+ IPartitioningProperty diffPart = null;
+ IPartitioningProperty reqdPart = reqd.getPartitioningProperty();
+ if (reqdPart != null) {
+ if (mayExpandProperties) {
+ reqdPart.normalize(equivalenceClasses, fds);
+ } else {
+ reqdPart.normalize(equivalenceClasses, null);
+ }
+ propPartitioning.normalize(equivalenceClasses, fds);
+ if (!PropertiesUtil.matchPartitioningProps(reqdPart, propPartitioning, mayExpandProperties)) {
+ diffPart = reqdPart;
+ }
+ }
+
+ if (diffLocals == null && diffPart == null) {
+ return null;
+ } else {
+ return new StructuralPropertiesVector(diffPart, diffLocals);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
new file mode 100644
index 0000000..6b14a9f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
@@ -0,0 +1,68 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class TypePropagationPolicy {
+ public static final TypePropagationPolicy ALL = new TypePropagationPolicy() {
+
+ @Override
+ public Object getVarType(LogicalVariable var, INullableTypeComputer ntc,
+ List<LogicalVariable> nonNullVariableList, ITypeEnvPointer... typeEnvs) throws AlgebricksException {
+ for (ITypeEnvPointer p : typeEnvs) {
+ IVariableTypeEnvironment env = p.getTypeEnv();
+ if (env == null) {
+ throw new AlgebricksException("Null environment for pointer " + p + " in getVarType for var=" + var);
+ }
+ Object t = env.getVarType(var, nonNullVariableList);
+ if (t != null) {
+ return t;
+ }
+ }
+ return null;
+ }
+ };
+
+ public static final TypePropagationPolicy LEFT_OUTER = new TypePropagationPolicy() {
+
+ @Override
+ public Object getVarType(LogicalVariable var, INullableTypeComputer ntc,
+ List<LogicalVariable> nonNullVariableList, ITypeEnvPointer... typeEnvs) throws AlgebricksException {
+ int n = typeEnvs.length;
+ for (int i = 0; i < n; i++) {
+ Object t = typeEnvs[i].getTypeEnv().getVarType(var, nonNullVariableList);
+ if (t != null) {
+ if (i == 0) { // inner branch
+ return t;
+ } else { // outer branch
+ boolean nonNullVarIsProduced = false;
+ for (LogicalVariable v : nonNullVariableList) {
+ if (v == var) {
+ nonNullVarIsProduced = true;
+ break;
+ }
+ if (typeEnvs[i].getTypeEnv().getVarType(v) != null) {
+ nonNullVarIsProduced = true;
+ break;
+ }
+ }
+ if (nonNullVarIsProduced) {
+ return t;
+ } else {
+ return ntc.makeNullableType(t);
+ }
+ }
+ }
+ }
+ return null;
+ }
+ };
+
+ public abstract Object getVarType(LogicalVariable var, INullableTypeComputer ntc,
+ List<LogicalVariable> nonNullVariableList, ITypeEnvPointer... typeEnvs) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
new file mode 100644
index 0000000..520e7db
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+public final class UnorderedPartitionedProperty extends AbstractGroupingProperty implements IPartitioningProperty {
+
+ private INodeDomain domain;
+
+ public UnorderedPartitionedProperty(Set<LogicalVariable> partitioningVariables, INodeDomain domain) {
+ super(partitioningVariables);
+ this.domain = domain;
+ }
+
+ @Override
+ public PartitioningType getPartitioningType() {
+ return PartitioningType.UNORDERED_PARTITIONED;
+ }
+
+ @Override
+ public void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
+ normalizeGroupingColumns(equivalenceClasses, fds);
+ }
+
+ @Override
+ public String toString() {
+ return getPartitioningType().toString() + columnSet;
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ columns.addAll(columnSet);
+ }
+
+ @Override
+ public INodeDomain getNodeDomain() {
+ return domain;
+ }
+
+ @Override
+ public void setNodeDomain(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/UnpartitionedPropertyComputer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/UnpartitionedPropertyComputer.java
new file mode 100644
index 0000000..acdecba
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/UnpartitionedPropertyComputer.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class UnpartitionedPropertyComputer implements IPropertiesComputer {
+
+ public static final UnpartitionedPropertyComputer INSTANCE = new UnpartitionedPropertyComputer();
+
+ private UnpartitionedPropertyComputer() {
+ }
+
+ @Override
+ public IPartitioningProperty computePartitioningProperty(ILogicalExpression expr) {
+ return IPartitioningProperty.UNPARTITIONED;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/VariablePropagationPolicy.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/VariablePropagationPolicy.java
new file mode 100644
index 0000000..6f936c0
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/VariablePropagationPolicy.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class VariablePropagationPolicy {
+ public static final VariablePropagationPolicy ALL = new VariablePropagationPolicy() {
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
+ int n = sources.length;
+ for (int i = 0; i < n; i++) {
+ target.addAllNewVariables(sources[i]);
+ }
+ }
+ };
+
+ public static final VariablePropagationPolicy NONE = new VariablePropagationPolicy() {
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
+ // do nothing
+ }
+ };
+
+ /**
+ * Adds, from each source, only variables that do not already appear in the
+ * target.
+ *
+ *
+ */
+ public static final VariablePropagationPolicy ADDNEWVARIABLES = new VariablePropagationPolicy() {
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
+ for (IOperatorSchema s : sources) {
+ for (LogicalVariable v : s) {
+ if (target.findVariable(v) < 0) {
+ target.addVariable(v);
+ }
+ }
+ }
+ }
+ };
+
+ public abstract void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException;
+
+ public static VariablePropagationPolicy concat(final VariablePropagationPolicy... policies) {
+ return new VariablePropagationPolicy() {
+ @Override
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+ throws AlgebricksException {
+ if (policies.length != sources.length) {
+ throw new IllegalArgumentException();
+ }
+ for (int i = 0; i < policies.length; ++i) {
+ VariablePropagationPolicy p = policies[i];
+ IOperatorSchema s = sources[i];
+ p.propagateVariables(target, s);
+ }
+ }
+ };
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/aggregators/TupleCountAggregateFunctionFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/aggregators/TupleCountAggregateFunctionFactory.java
new file mode 100644
index 0000000..acb2e0e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/aggregators/TupleCountAggregateFunctionFactory.java
@@ -0,0 +1,54 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class TupleCountAggregateFunctionFactory implements IAggregateFunctionFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateFunction createAggregateFunction(IDataOutputProvider provider) throws AlgebricksException {
+
+ final DataOutput out = provider.getDataOutput();
+ return new IAggregateFunction() {
+
+ int cnt;
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ ++cnt;
+ }
+
+ @Override
+ public void init() throws AlgebricksException {
+ cnt = 0;
+ }
+
+ @Override
+ public void finish() throws AlgebricksException {
+ try {
+ out.writeInt(cnt);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void finishPartial() throws AlgebricksException {
+ try {
+ out.writeInt(cnt);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/aggregators/TupleCountRunningAggregateFunctionFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/aggregators/TupleCountRunningAggregateFunctionFactory.java
new file mode 100644
index 0000000..02fe250
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/aggregators/TupleCountRunningAggregateFunctionFactory.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class TupleCountRunningAggregateFunctionFactory implements IRunningAggregateFunctionFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IRunningAggregateFunction createRunningAggregateFunction(IDataOutputProvider provider)
+ throws AlgebricksException {
+
+ final DataOutput out = provider.getDataOutput();
+
+ return new IRunningAggregateFunction() {
+
+ int cnt;
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws AlgebricksException {
+ ++cnt;
+ try {
+ out.writeInt(cnt);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void init() throws AlgebricksException {
+ cnt = 0;
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/AlgebricksPipeline.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/AlgebricksPipeline.java
new file mode 100644
index 0000000..ead6c37
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/AlgebricksPipeline.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class AlgebricksPipeline implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private final IPushRuntimeFactory[] runtimeFactories;
+ private final RecordDescriptor[] recordDescriptors;
+
+ public AlgebricksPipeline(IPushRuntimeFactory[] runtimeFactories, RecordDescriptor[] recordDescriptors) {
+ this.runtimeFactories = runtimeFactories;
+ this.recordDescriptors = recordDescriptors;
+ // this.projectedColumns = projectedColumns;
+ }
+
+ public IPushRuntimeFactory[] getRuntimeFactories() {
+ return runtimeFactories;
+ }
+
+ public RecordDescriptor[] getRecordDescriptors() {
+ return recordDescriptors;
+ }
+
+ public int getOutputWidth() {
+ return recordDescriptors[recordDescriptors.length - 1].getFields().length;
+ }
+
+ // public int[] getProjectedColumns() {
+ // return projectedColumns;
+ // }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IAggregateFunction.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IAggregateFunction.java
new file mode 100644
index 0000000..e68ae52
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IAggregateFunction.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public interface IAggregateFunction {
+ /** should be called each time a new aggregate value is computed */
+ public void init() throws AlgebricksException;
+
+ public void step(IFrameTupleReference tuple) throws AlgebricksException;
+
+ public void finish() throws AlgebricksException;
+
+ public void finishPartial() throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IAggregateFunctionFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IAggregateFunctionFactory.java
new file mode 100644
index 0000000..e70c052
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IAggregateFunctionFactory.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+public interface IAggregateFunctionFactory extends Serializable {
+ public IAggregateFunction createAggregateFunction(IDataOutputProvider provider) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IEvaluator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IEvaluator.java
new file mode 100644
index 0000000..566651d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IEvaluator.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public interface IEvaluator {
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException;
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IEvaluatorFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IEvaluatorFactory.java
new file mode 100644
index 0000000..46db040
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IEvaluatorFactory.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+public interface IEvaluatorFactory extends Serializable {
+ public IEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException;
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IPushRuntime.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IPushRuntime.java
new file mode 100644
index 0000000..01ae150
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IPushRuntime.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public interface IPushRuntime extends IFrameWriter {
+ public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc);
+
+ public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IPushRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IPushRuntimeFactory.java
new file mode 100644
index 0000000..4e2d113
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IPushRuntimeFactory.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IPushRuntimeFactory extends Serializable {
+ public IPushRuntime createPushRuntime(RuntimeContext context) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IRunningAggregateFunction.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IRunningAggregateFunction.java
new file mode 100644
index 0000000..ac381cf
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IRunningAggregateFunction.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public interface IRunningAggregateFunction {
+ public void init() throws AlgebricksException;
+
+ public void step(IFrameTupleReference tuple) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IRunningAggregateFunctionFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IRunningAggregateFunctionFactory.java
new file mode 100644
index 0000000..638f5f8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IRunningAggregateFunctionFactory.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+public interface IRunningAggregateFunctionFactory extends Serializable {
+ public IRunningAggregateFunction createRunningAggregateFunction(IDataOutputProvider provider)
+ throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/ISerializableAggregateFunction.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/ISerializableAggregateFunction.java
new file mode 100644
index 0000000..4e9c5fb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/ISerializableAggregateFunction.java
@@ -0,0 +1,44 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public interface ISerializableAggregateFunction {
+ /**
+ * initialize the space occupied by internal state
+ *
+ * @param state
+ * @throws AlgebricksException
+ * @return length of the intermediate state
+ */
+ public void init(DataOutput state) throws AlgebricksException;
+
+ /**
+ * update the internal state
+ *
+ * @param tuple
+ * @param state
+ * @throws AlgebricksException
+ */
+ public void step(IFrameTupleReference tuple, byte[] data, int start, int len) throws AlgebricksException;
+
+ /**
+ * output the state to result
+ *
+ * @param state
+ * @param result
+ * @throws AlgebricksException
+ */
+ public void finish(byte[] data, int start, int len, DataOutput result) throws AlgebricksException;
+
+ /**
+ * output the partial state to partial result
+ *
+ * @param state
+ * @param partialResult
+ * @throws AlgebricksException
+ */
+ public void finishPartial(byte[] data, int start, int len, DataOutput partialResult) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/ISerializableAggregateFunctionFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/ISerializableAggregateFunctionFactory.java
new file mode 100644
index 0000000..f9f871e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/ISerializableAggregateFunctionFactory.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface ISerializableAggregateFunctionFactory extends Serializable {
+ public ISerializableAggregateFunction createAggregateFunction() throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IUnnestingFunction.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IUnnestingFunction.java
new file mode 100644
index 0000000..5b891f2
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IUnnestingFunction.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public interface IUnnestingFunction {
+ public void init(IFrameTupleReference tuple) throws AlgebricksException;
+
+ public boolean step() throws AlgebricksException;
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IUnnestingFunctionFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IUnnestingFunctionFactory.java
new file mode 100644
index 0000000..2134fd6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/base/IUnnestingFunctionFactory.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+
+public interface IUnnestingFunctionFactory extends Serializable {
+ public IUnnestingFunction createUnnestingFunction(IDataOutputProvider provider) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/context/AsterixBTreeRegistry.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/context/AsterixBTreeRegistry.java
new file mode 100644
index 0000000..4ea64d1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/context/AsterixBTreeRegistry.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context;
+
+import java.util.HashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+
+public class AsterixBTreeRegistry {
+
+ private HashMap<Integer, BTree> map = new HashMap<Integer, BTree>();
+ private Lock registryLock = new ReentrantLock();
+
+ public BTree get(int fileId) {
+ return map.get(fileId);
+ }
+
+ // TODO: not very high concurrency, but good enough for now
+ public void lock() {
+ registryLock.lock();
+ }
+
+ public void unlock() {
+ registryLock.unlock();
+ }
+
+ public void register(int fileId, BTree btree) {
+ map.put(fileId, btree);
+ }
+
+ public void unregister(int fileId) {
+ map.remove(fileId);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/context/RuntimeContext.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/context/RuntimeContext.java
new file mode 100644
index 0000000..5afd17f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/context/RuntimeContext.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public class RuntimeContext {
+ private IHyracksTaskContext hyracksContext;
+
+ public RuntimeContext() {
+ }
+
+ public IHyracksTaskContext getHyracksContext() {
+ return hyracksContext;
+ }
+
+ public void setHyracksContext(IHyracksTaskContext hyracksContext) {
+ this.hyracksContext = hyracksContext;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/evaluators/ColumnAccessEvalFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/evaluators/ColumnAccessEvalFactory.java
new file mode 100644
index 0000000..4d8bf96
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/evaluators/ColumnAccessEvalFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.evaluators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ColumnAccessEvalFactory implements IEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int fieldIndex;
+
+ public ColumnAccessEvalFactory(int fieldIndex) {
+ this.fieldIndex = fieldIndex;
+ }
+
+ @Override
+ public String toString() {
+ return "ColumnAccess(" + fieldIndex + ")";
+ }
+
+ @Override
+ public IEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new IEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ byte[] buffer = tuple.getFieldData(fieldIndex);
+ int start = tuple.getFieldStart(fieldIndex);
+ int length = tuple.getFieldLength(fieldIndex);
+ try {
+ out.write(buffer, start, length);
+ } catch (IOException ioe) {
+ throw new AlgebricksException(ioe);
+ }
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/evaluators/ConstantEvalFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/evaluators/ConstantEvalFactory.java
new file mode 100644
index 0000000..f49f2ee
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/evaluators/ConstantEvalFactory.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.evaluators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ConstantEvalFactory implements IEvaluatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ private byte[] value;
+
+ public ConstantEvalFactory(byte[] value) {
+ this.value = value;
+ }
+
+ @Override
+ public String toString() {
+ return "Constant";
+ }
+
+ @Override
+ public IEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new IEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ try {
+ out.write(value, 0, value.length);
+ } catch (IOException ioe) {
+ throw new AlgebricksException(ioe);
+ }
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/BinaryBooleanInspectorImpl.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/BinaryBooleanInspectorImpl.java
new file mode 100644
index 0000000..fdb3663
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/BinaryBooleanInspectorImpl.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryBooleanInspector;
+
+public class BinaryBooleanInspectorImpl implements IBinaryBooleanInspector {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final BinaryBooleanInspectorImpl INSTANCE = new BinaryBooleanInspectorImpl();
+
+ private BinaryBooleanInspectorImpl() {
+ }
+
+ @Override
+ public boolean getBooleanValue(byte[] bytes, int offset, int length) {
+ return bytes[offset] == 1;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/BinaryIntegerInspectorImpl.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/BinaryIntegerInspectorImpl.java
new file mode 100644
index 0000000..9d0587d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/BinaryIntegerInspectorImpl.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class BinaryIntegerInspectorImpl implements IBinaryIntegerInspector {
+
+ private static final long serialVersionUID = 1L;
+ public static final BinaryIntegerInspectorImpl INSTANCE = new BinaryIntegerInspectorImpl();
+
+ @Override
+ public int getIntegerValue(byte[] bytes, int offset, int length) {
+ return IntegerSerializerDeserializer.getInt(bytes, offset);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/IntegerPrinterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/IntegerPrinterFactory.java
new file mode 100644
index 0000000..db47af6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/IntegerPrinterFactory.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.WriteValueTools;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class IntegerPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static final IntegerPrinterFactory INSTANCE = new IntegerPrinterFactory();
+
+ private IntegerPrinterFactory() {
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ return new IPrinter() {
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+ int d = IntegerSerializerDeserializer.getInt(b, s);
+ try {
+ WriteValueTools.writeInt(d, ps);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void init() throws AlgebricksException {
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/NoopNullWriterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/NoopNullWriterFactory.java
new file mode 100644
index 0000000..e064832
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/NoopNullWriterFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NoopNullWriterFactory implements INullWriterFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static final NoopNullWriterFactory INSTANCE = new NoopNullWriterFactory();
+
+ private NoopNullWriterFactory() {
+ }
+
+ @Override
+ public INullWriter createNullWriter() {
+
+ return new INullWriter() {
+
+ @Override
+ public void writeNull(DataOutput out) throws HyracksDataException {
+ // do nothing
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/UTF8StringPrinterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/UTF8StringPrinterFactory.java
new file mode 100644
index 0000000..95bf8a3
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/UTF8StringPrinterFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data;
+
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+
+public class UTF8StringPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final UTF8StringPrinterFactory INSTANCE = new UTF8StringPrinterFactory();
+
+ private UTF8StringPrinterFactory() {
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ return new IPrinter() {
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+ int strlen = StringUtils.getUTFLen(b, s);
+ int pos = s + 2;
+ int maxPos = pos + strlen;
+ ps.print("\"");
+ while (pos < maxPos) {
+ char c = StringUtils.charAt(b, pos);
+ switch (c) {
+ case '\\':
+ case '"':
+ ps.print('\\');
+ break;
+ }
+ ps.print(c);
+ pos += StringUtils.charSize(b, pos);
+ }
+ ps.print("\"");
+ }
+
+ @Override
+ public void init() throws AlgebricksException {
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobBuilder.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobBuilder.java
new file mode 100644
index 0000000..4463b43
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobBuilder.java
@@ -0,0 +1,324 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksCountPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraintHelper;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class JobBuilder implements IHyracksJobBuilder {
+
+ private JobSpecification jobSpec;
+ private AlgebricksPartitionConstraint clusterLocations;
+
+ private Map<ILogicalOperator, ArrayList<ILogicalOperator>> outEdges = new HashMap<ILogicalOperator, ArrayList<ILogicalOperator>>();
+ private Map<ILogicalOperator, ArrayList<ILogicalOperator>> inEdges = new HashMap<ILogicalOperator, ArrayList<ILogicalOperator>>();
+ private Map<ILogicalOperator, Pair<IConnectorDescriptor, TargetConstraint>> connectors = new HashMap<ILogicalOperator, Pair<IConnectorDescriptor, TargetConstraint>>();
+
+ private Map<ILogicalOperator, Pair<IPushRuntimeFactory, RecordDescriptor>> microOps = new HashMap<ILogicalOperator, Pair<IPushRuntimeFactory, RecordDescriptor>>();
+ private Map<IPushRuntimeFactory, ILogicalOperator> revMicroOpMap = new HashMap<IPushRuntimeFactory, ILogicalOperator>();
+ private Map<ILogicalOperator, IOperatorDescriptor> hyracksOps = new HashMap<ILogicalOperator, IOperatorDescriptor>();
+ private Map<ILogicalOperator, AlgebricksPartitionConstraint> pcForMicroOps = new HashMap<ILogicalOperator, AlgebricksPartitionConstraint>();
+
+ private int aodCounter = 0;
+ private Map<ILogicalOperator, Integer> algebraicOpBelongingToMetaAsterixOp = new HashMap<ILogicalOperator, Integer>();
+ private Map<Integer, List<Pair<IPushRuntimeFactory, RecordDescriptor>>> metaAsterixOpSkeletons = new HashMap<Integer, List<Pair<IPushRuntimeFactory, RecordDescriptor>>>();
+ private Map<Integer, AlgebricksMetaOperatorDescriptor> metaAsterixOps = new HashMap<Integer, AlgebricksMetaOperatorDescriptor>();
+ private final Map<IOperatorDescriptor, AlgebricksPartitionConstraint> partitionConstraintMap = new HashMap<IOperatorDescriptor, AlgebricksPartitionConstraint>();
+
+ public JobBuilder(JobSpecification jobSpec, AlgebricksPartitionConstraint clusterLocations) {
+ this.jobSpec = jobSpec;
+ this.clusterLocations = clusterLocations;
+ }
+
+ @Override
+ public void contributeMicroOperator(ILogicalOperator op, IPushRuntimeFactory runtime, RecordDescriptor recDesc) {
+ contributeMicroOperator(op, runtime, recDesc, null);
+ }
+
+ @Override
+ public void contributeMicroOperator(ILogicalOperator op, IPushRuntimeFactory runtime, RecordDescriptor recDesc,
+ AlgebricksPartitionConstraint pc) {
+ microOps.put(op, new Pair<IPushRuntimeFactory, RecordDescriptor>(runtime, recDesc));
+ revMicroOpMap.put(runtime, op);
+ if (pc != null) {
+ pcForMicroOps.put(op, pc);
+ }
+ }
+
+ @Override
+ public void contributeConnector(ILogicalOperator exchgOp, IConnectorDescriptor conn) {
+ connectors.put(exchgOp, new Pair<IConnectorDescriptor, TargetConstraint>(conn, null));
+ }
+
+ @Override
+ public void contributeConnectorWithTargetConstraint(ILogicalOperator exchgOp, IConnectorDescriptor conn,
+ TargetConstraint numberOfTargetPartitions) {
+ connectors.put(exchgOp, new Pair<IConnectorDescriptor, TargetConstraint>(conn, numberOfTargetPartitions));
+ }
+
+ @Override
+ public void contributeGraphEdge(ILogicalOperator src, int srcOutputIndex, ILogicalOperator dest, int destInputIndex) {
+ ArrayList<ILogicalOperator> outputs = outEdges.get(src);
+ if (outputs == null) {
+ outputs = new ArrayList<ILogicalOperator>();
+ outEdges.put(src, outputs);
+ }
+ addAtPos(outputs, dest, srcOutputIndex);
+
+ ArrayList<ILogicalOperator> inp = inEdges.get(dest);
+ if (inp == null) {
+ inp = new ArrayList<ILogicalOperator>();
+ inEdges.put(dest, inp);
+ }
+ addAtPos(inp, src, destInputIndex);
+ }
+
+ @Override
+ public void contributeHyracksOperator(ILogicalOperator op, IOperatorDescriptor opDesc) {
+ hyracksOps.put(op, opDesc);
+ }
+
+ @Override
+ public void contributeAlgebricksPartitionConstraint(IOperatorDescriptor opDesc, AlgebricksPartitionConstraint apc) {
+ partitionConstraintMap.put(opDesc, apc);
+ }
+
+ @Override
+ public JobSpecification getJobSpec() {
+ return jobSpec;
+ }
+
+ @Override
+ public void buildSpec(List<ILogicalOperator> roots) throws AlgebricksException {
+ buildAsterixComponents();
+ Map<IConnectorDescriptor, TargetConstraint> tgtConstraints = setupConnectors();
+ for (ILogicalOperator r : roots) {
+ IOperatorDescriptor opDesc = findOpDescForAlgebraicOp(r);
+ jobSpec.addRoot(opDesc);
+ }
+ setAllPartitionConstraints(tgtConstraints);
+ }
+
+ private void setAllPartitionConstraints(Map<IConnectorDescriptor, TargetConstraint> tgtConstraints) {
+ List<OperatorDescriptorId> roots = jobSpec.getRoots();
+ setSpecifiedPartitionConstraints();
+ for (OperatorDescriptorId rootId : roots) {
+ setPartitionConstraintsDFS(rootId, tgtConstraints, null);
+ }
+ }
+
+ private void setSpecifiedPartitionConstraints() {
+ for (ILogicalOperator op : pcForMicroOps.keySet()) {
+ AlgebricksPartitionConstraint pc = pcForMicroOps.get(op);
+ Integer k = algebraicOpBelongingToMetaAsterixOp.get(op);
+ AlgebricksMetaOperatorDescriptor amod = metaAsterixOps.get(k);
+ partitionConstraintMap.put(amod, pc);
+ }
+ for (IOperatorDescriptor opDesc : partitionConstraintMap.keySet()) {
+ AlgebricksPartitionConstraint pc = partitionConstraintMap.get(opDesc);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(jobSpec, opDesc, pc);
+ }
+ }
+
+ private void setPartitionConstraintsDFS(OperatorDescriptorId opId,
+ Map<IConnectorDescriptor, TargetConstraint> tgtConstraints, IOperatorDescriptor parentOp) {
+ List<IConnectorDescriptor> opInputs = jobSpec.getOperatorInputMap().get(opId);
+ AlgebricksPartitionConstraint opConstraint = null;
+ IOperatorDescriptor opDesc = jobSpec.getOperatorMap().get(opId);
+ if (opInputs != null) {
+ for (IConnectorDescriptor conn : opInputs) {
+ ConnectorDescriptorId cid = conn.getConnectorId();
+ edu.uci.ics.hyracks.api.util.Pair<edu.uci.ics.hyracks.api.util.Pair<IOperatorDescriptor, Integer>, edu.uci.ics.hyracks.api.util.Pair<IOperatorDescriptor, Integer>> p = jobSpec
+ .getConnectorOperatorMap().get(cid);
+ IOperatorDescriptor src = p.first.first;
+ // DFS
+ setPartitionConstraintsDFS(src.getOperatorId(), tgtConstraints, opDesc);
+
+ TargetConstraint constraint = tgtConstraints.get(conn);
+ if (constraint != null) {
+ switch (constraint) {
+ case ONE: {
+ opConstraint = new AlgebricksCountPartitionConstraint(1);
+ break;
+ }
+ case SAME_COUNT: {
+ opConstraint = partitionConstraintMap.get(src);
+ break;
+ }
+ }
+ }
+ }
+ }
+ if (partitionConstraintMap.get(opDesc) == null) {
+ if (opConstraint == null) {
+ if (parentOp != null) {
+ AlgebricksPartitionConstraint pc = partitionConstraintMap.get(parentOp);
+ if (pc != null) {
+ opConstraint = pc;
+ } else if (opInputs == null || opInputs.size() == 0) {
+ opConstraint = new AlgebricksCountPartitionConstraint(1);
+ }
+ }
+ if (opConstraint == null) {
+ opConstraint = clusterLocations;
+ }
+ }
+ partitionConstraintMap.put(opDesc, opConstraint);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(jobSpec, opDesc, opConstraint);
+ }
+ }
+
+ private Map<IConnectorDescriptor, TargetConstraint> setupConnectors() throws AlgebricksException {
+ Map<IConnectorDescriptor, TargetConstraint> tgtConstraints = new HashMap<IConnectorDescriptor, TargetConstraint>();
+ for (ILogicalOperator exchg : connectors.keySet()) {
+ ILogicalOperator inOp = inEdges.get(exchg).get(0);
+ ILogicalOperator outOp = outEdges.get(exchg).get(0);
+ IOperatorDescriptor inOpDesc = findOpDescForAlgebraicOp(inOp);
+ IOperatorDescriptor outOpDesc = findOpDescForAlgebraicOp(outOp);
+ Pair<IConnectorDescriptor, TargetConstraint> connPair = connectors.get(exchg);
+ IConnectorDescriptor conn = connPair.first;
+ int producerPort = outEdges.get(inOp).indexOf(exchg);
+ int consumerPort = inEdges.get(outOp).indexOf(exchg);
+ jobSpec.connect(conn, inOpDesc, producerPort, outOpDesc, consumerPort);
+ if (connPair.second != null) {
+ tgtConstraints.put(conn, connPair.second);
+ }
+ }
+ return tgtConstraints;
+ }
+
+ private IOperatorDescriptor findOpDescForAlgebraicOp(ILogicalOperator op) throws AlgebricksException {
+ IOperatorDescriptor hOpDesc = hyracksOps.get(op);
+ if (hOpDesc != null) {
+ return hOpDesc;
+ }
+ Integer metaOpKey = algebraicOpBelongingToMetaAsterixOp.get(op);
+ if (metaOpKey == null) {
+ throw new AlgebricksException("Could not generate operator descriptor for operator " + op);
+ }
+ return metaAsterixOps.get(metaOpKey);
+ }
+
+ private void buildAsterixComponents() {
+ for (ILogicalOperator aop : microOps.keySet()) {
+ addMicroOpToMetaRuntimeOp(aop);
+ }
+ for (Integer k : metaAsterixOpSkeletons.keySet()) {
+ List<Pair<IPushRuntimeFactory, RecordDescriptor>> opContents = metaAsterixOpSkeletons.get(k);
+ AlgebricksMetaOperatorDescriptor amod = buildMetaAsterixOpDesc(opContents);
+ metaAsterixOps.put(k, amod);
+ }
+ }
+
+ private AlgebricksMetaOperatorDescriptor buildMetaAsterixOpDesc(
+ List<Pair<IPushRuntimeFactory, RecordDescriptor>> opContents) {
+ // RecordDescriptor outputRecordDesc = null;
+ int n = opContents.size();
+ IPushRuntimeFactory[] runtimeFactories = new IPushRuntimeFactory[n];
+ RecordDescriptor[] internalRecordDescriptors = new RecordDescriptor[n];
+ int i = 0;
+ for (Pair<IPushRuntimeFactory, RecordDescriptor> p : opContents) {
+ runtimeFactories[i] = p.first;
+ internalRecordDescriptors[i] = p.second;
+ // if (i == n - 1) {
+ // outputRecordDesc = p.second;
+ // }
+ i++;
+ }
+ ILogicalOperator lastLogicalOp = revMicroOpMap.get(runtimeFactories[n - 1]);
+ ArrayList<ILogicalOperator> outOps = outEdges.get(lastLogicalOp);
+ int outArity = (outOps == null) ? 0 : outOps.size();
+ ILogicalOperator firstLogicalOp = revMicroOpMap.get(runtimeFactories[0]);
+ ArrayList<ILogicalOperator> inOps = inEdges.get(firstLogicalOp);
+ int inArity = (inOps == null) ? 0 : inOps.size();
+ // boolean isLeafOp = inEdges.get(firstLogicalOp) == null;
+ return new AlgebricksMetaOperatorDescriptor(jobSpec, inArity, outArity, runtimeFactories,
+ internalRecordDescriptors);
+ }
+
+ private void addMicroOpToMetaRuntimeOp(ILogicalOperator aop) {
+ Integer k = algebraicOpBelongingToMetaAsterixOp.get(aop);
+ if (k == null) {
+ k = createNewMetaOpInfo(aop);
+ }
+ ArrayList<ILogicalOperator> destList = outEdges.get(aop);
+ if (destList == null || destList.size() != 1) {
+ // for now, we only support linear plans inside meta-ops.
+ return;
+ }
+ ILogicalOperator dest = destList.get(0);
+ Integer j = algebraicOpBelongingToMetaAsterixOp.get(dest);
+ if (j == null && microOps.get(dest) != null) {
+ algebraicOpBelongingToMetaAsterixOp.put(dest, k);
+ List<Pair<IPushRuntimeFactory, RecordDescriptor>> aodContent1 = metaAsterixOpSkeletons.get(k);
+ aodContent1.add(microOps.get(dest));
+ } else if (j != null && j.intValue() != k.intValue()) {
+ // merge the j component into the k component
+ List<Pair<IPushRuntimeFactory, RecordDescriptor>> aodContent1 = metaAsterixOpSkeletons.get(k);
+ List<Pair<IPushRuntimeFactory, RecordDescriptor>> aodContent2 = metaAsterixOpSkeletons.get(j);
+ aodContent1.addAll(aodContent2);
+ metaAsterixOpSkeletons.remove(j);
+ for (ILogicalOperator m : algebraicOpBelongingToMetaAsterixOp.keySet()) {
+ Integer g = algebraicOpBelongingToMetaAsterixOp.get(m);
+ if (g.intValue() == j.intValue()) {
+ algebraicOpBelongingToMetaAsterixOp.put(m, k);
+ }
+ }
+ }
+
+ }
+
+ private int createNewMetaOpInfo(ILogicalOperator aop) {
+ int n = aodCounter;
+ aodCounter++;
+ List<Pair<IPushRuntimeFactory, RecordDescriptor>> metaOpContents = new ArrayList<Pair<IPushRuntimeFactory, RecordDescriptor>>();
+ metaOpContents.add(microOps.get(aop));
+ metaAsterixOpSkeletons.put(n, metaOpContents);
+ algebraicOpBelongingToMetaAsterixOp.put(aop, n);
+ return n;
+ }
+
+ private <E> void addAtPos(ArrayList<E> a, E elem, int pos) {
+ int n = a.size();
+ if (n > pos) {
+ a.set(pos, elem);
+ } else {
+ for (int k = n; k < pos; k++) {
+ a.add(null);
+ }
+ a.add(elem);
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenContext.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenContext.java
new file mode 100644
index 0000000..870ce91
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenContext.java
@@ -0,0 +1,191 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.INormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.ITypeTraitProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+
+public class JobGenContext {
+ private final IOperatorSchema outerFlowSchema;
+ private final Map<ILogicalOperator, IOperatorSchema> schemaMap = new HashMap<ILogicalOperator, IOperatorSchema>();
+ private final ISerializerDeserializerProvider serializerDeserializerProvider;
+ private final IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
+ private final IBinaryComparatorFactoryProvider comparatorFactoryProvider;
+ private final IPrinterFactoryProvider printerFactoryProvider;
+ private final ITypeTraitProvider typeTraitProvider;
+ private final IMetadataProvider<?, ?> metadataProvider;
+ private final INullWriterFactory nullWriterFactory;
+ private final INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
+ private final Object appContext;
+ private final IBinaryBooleanInspector booleanInspector;
+ private final IBinaryIntegerInspector integerInspector;
+ private final ILogicalExpressionJobGen exprJobGen;
+ private final IExpressionTypeComputer expressionTypeComputer;
+ private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
+ private final IPartialAggregationTypeComputer partialAggregationTypeComputer;
+ private final int frameSize;
+ private AlgebricksPartitionConstraint clusterLocations;
+ private int varCounter;
+ private final ITypingContext typingContext;
+
+ public JobGenContext(IOperatorSchema outerFlowSchema, IMetadataProvider<?, ?> metadataProvider, Object appContext,
+ ISerializerDeserializerProvider serializerDeserializerProvider,
+ IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider,
+ IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
+ IBinaryBooleanInspector booleanInspector, IBinaryIntegerInspector integerInspector,
+ IPrinterFactoryProvider printerFactoryProvider, INullWriterFactory nullWriterFactory,
+ INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
+ ILogicalExpressionJobGen exprJobGen, IExpressionTypeComputer expressionTypeComputer,
+ INullableTypeComputer nullableTypeComputer, ITypingContext typingContext,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ IPartialAggregationTypeComputer partialAggregationTypeComputer, int frameSize,
+ AlgebricksPartitionConstraint clusterLocations) {
+ this.outerFlowSchema = outerFlowSchema;
+ this.metadataProvider = metadataProvider;
+ this.appContext = appContext;
+ this.serializerDeserializerProvider = serializerDeserializerProvider;
+ this.hashFunctionFactoryProvider = hashFunctionFactoryProvider;
+ this.comparatorFactoryProvider = comparatorFactoryProvider;
+ this.typeTraitProvider = typeTraitProvider;
+ this.booleanInspector = booleanInspector;
+ this.integerInspector = integerInspector;
+ this.printerFactoryProvider = printerFactoryProvider;
+ this.clusterLocations = clusterLocations;
+ this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
+ this.nullWriterFactory = nullWriterFactory;
+ this.exprJobGen = exprJobGen;
+ this.expressionTypeComputer = expressionTypeComputer;
+ this.typingContext = typingContext;
+ this.expressionEvalSizeComputer = expressionEvalSizeComputer;
+ this.partialAggregationTypeComputer = partialAggregationTypeComputer;
+ this.frameSize = frameSize;
+ this.varCounter = 0;
+ }
+
+ public IOperatorSchema getOuterFlowSchema() {
+ return outerFlowSchema;
+ }
+
+ public AlgebricksPartitionConstraint getClusterLocations() {
+ return clusterLocations;
+ }
+
+ public IMetadataProvider<?, ?> getMetadataProvider() {
+ return metadataProvider;
+ }
+
+ public Object getAppContext() {
+ return appContext;
+ }
+
+ public ISerializerDeserializerProvider getSerializerDeserializerProvider() {
+ return serializerDeserializerProvider;
+ }
+
+ public IBinaryHashFunctionFactoryProvider getBinaryHashFunctionFactoryProvider() {
+ return hashFunctionFactoryProvider;
+ }
+
+ public IBinaryComparatorFactoryProvider getBinaryComparatorFactoryProvider() {
+ return comparatorFactoryProvider;
+ }
+
+ public ITypeTraitProvider getTypeTraitProvider() {
+ return typeTraitProvider;
+ }
+
+ public IBinaryBooleanInspector getBinaryBooleanInspector() {
+ return booleanInspector;
+ }
+
+ public IBinaryIntegerInspector getBinaryIntegerInspector() {
+ return integerInspector;
+ }
+
+ public IPrinterFactoryProvider getPrinterFactoryProvider() {
+ return printerFactoryProvider;
+ }
+
+ public ILogicalExpressionJobGen getExpressionJobGen() {
+ return exprJobGen;
+ }
+
+ public IOperatorSchema getSchema(ILogicalOperator op) {
+ return schemaMap.get(op);
+ }
+
+ public void putSchema(ILogicalOperator op, IOperatorSchema schema) {
+ schemaMap.put(op, schema);
+ }
+
+ public LogicalVariable createNewVar() {
+ varCounter++;
+ LogicalVariable var = new LogicalVariable(-varCounter);
+ return var;
+ }
+
+ public Object getType(ILogicalExpression expr, IVariableTypeEnvironment env) throws AlgebricksException {
+ return expressionTypeComputer.getType(expr, typingContext.getMetadataProvider(), env);
+ }
+
+ public INullWriterFactory getNullWriterFactory() {
+ return nullWriterFactory;
+ }
+
+ public INormalizedKeyComputerFactoryProvider getNormalizedKeyComputerFactoryProvider() {
+ return normalizedKeyComputerFactoryProvider;
+ }
+
+ public IExpressionEvalSizeComputer getExpressionEvalSizeComputer() {
+ return expressionEvalSizeComputer;
+ }
+
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ public IPartialAggregationTypeComputer getPartialAggregationTypeComputer() {
+ return partialAggregationTypeComputer;
+ }
+
+ public IVariableTypeEnvironment getTypeEnvironment(ILogicalOperator op) {
+ return typingContext.getOutputTypeEnvironment(op);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenHelper.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenHelper.java
new file mode 100644
index 0000000..8f03316
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenHelper.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl;
+
+import java.util.Collection;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.INormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.ITypeTraitProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+public final class JobGenHelper {
+
+ private static final Logger LOGGER = Logger.getLogger(JobGenHelper.class.getName());
+
+ public static ITreeIndexFrameFactory createBTreeNSMInteriorFrameFactory(ITypeTrait[] typeTraits) {
+ return new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
+ }
+
+ public static ITreeIndexFrameFactory createBTreeNSMLeafFrameFactory(ITypeTrait[] typeTraits) {
+ return new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
+ }
+
+ // TODO: VRB: Commented out for now. Need to be uncommented once this
+ // is made compatible with the RTree interfaces.
+ // public static ITreeIndexFrameFactory createRTreeNSMInteriorFrameFactory(ITypeTrait[] typeTraits, int keyFields) {
+ // return new RTreeNSMInteriorFrameFactory(new RTreeTypeAwareTupleWriterFactory(typeTraits), keyFields);
+ // }
+ //
+ // public static ITreeIndexFrameFactory createRTreeNSMLeafFrameFactory(ITypeTrait[] typeTraits, int keyFields) {
+ // return new RTreeNSMLeafFrameFactory(new RTreeTypeAwareTupleWriterFactory(typeTraits), keyFields);
+ // }
+
+ @SuppressWarnings("unchecked")
+ public static RecordDescriptor mkRecordDescriptor(ILogicalOperator op, IOperatorSchema opSchema,
+ JobGenContext context) throws AlgebricksException {
+ ISerializerDeserializer[] fields = new ISerializerDeserializer[opSchema.getSize()];
+ ISerializerDeserializerProvider sdp = context.getSerializerDeserializerProvider();
+ int i = 0;
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ for (LogicalVariable var : opSchema) {
+ Object t = env.getVarType(var);
+ if (t == null) {
+ LOGGER.warning("No type for variable " + var);
+ // throw new AlgebricksException("No type for variable " + var);
+ }
+ fields[i] = sdp.getSerializerDeserializer(t);
+ i++;
+ }
+ return new RecordDescriptor(fields);
+ }
+
+ public static IPrinterFactory[] mkPrinterFactories(IOperatorSchema opSchema, IVariableTypeEnvironment env,
+ JobGenContext context, int[] printColumns) throws AlgebricksException {
+ IPrinterFactory[] pf = new IPrinterFactory[printColumns.length];
+ IPrinterFactoryProvider pff = context.getPrinterFactoryProvider();
+ for (int i = 0; i < pf.length; i++) {
+ LogicalVariable v = opSchema.getVariable(printColumns[i]);
+ Object t = env.getVarType(v);
+ pf[i] = pff.getPrinterFactory(t);
+ }
+ return pf;
+ }
+
+ public static int[] variablesToFieldIndexes(Collection<LogicalVariable> varLogical, IOperatorSchema opSchema) {
+ int[] tuplePos = new int[varLogical.size()];
+ int i = 0;
+ for (LogicalVariable var : varLogical) {
+ tuplePos[i] = opSchema.findVariable(var);
+ i++;
+ }
+ return tuplePos;
+ }
+
+ public static IBinaryHashFunctionFactory[] variablesToBinaryHashFunctionFactories(
+ Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
+ throws AlgebricksException {
+ IBinaryHashFunctionFactory[] funFactories = new IBinaryHashFunctionFactory[varLogical.size()];
+ int i = 0;
+ IBinaryHashFunctionFactoryProvider bhffProvider = context.getBinaryHashFunctionFactoryProvider();
+ for (LogicalVariable var : varLogical) {
+ Object type = env.getVarType(var);
+ funFactories[i++] = bhffProvider.getBinaryHashFunctionFactory(type);
+ }
+ return funFactories;
+ }
+
+ public static IBinaryComparatorFactory[] variablesToAscBinaryComparatorFactories(
+ Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
+ throws AlgebricksException {
+ IBinaryComparatorFactory[] compFactories = new IBinaryComparatorFactory[varLogical.size()];
+ IBinaryComparatorFactoryProvider bcfProvider = context.getBinaryComparatorFactoryProvider();
+ int i = 0;
+ for (LogicalVariable v : varLogical) {
+ Object type = env.getVarType(v);
+ compFactories[i++] = bcfProvider.getBinaryComparatorFactory(type, OrderKind.ASC);
+ }
+ return compFactories;
+ }
+
+ public static INormalizedKeyComputerFactory variablesToAscNormalizedKeyComputerFactory(
+ Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
+ throws AlgebricksException {
+ INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+ if (nkcfProvider == null)
+ return null;
+ for (LogicalVariable v : varLogical) {
+ Object type = env.getVarType(v);
+ return nkcfProvider.getNormalizedKeyComputerFactory(type, OrderKind.ASC);
+ }
+ return null;
+ }
+
+ public static ITypeTrait[] variablesToTypeTraits(Collection<LogicalVariable> varLogical,
+ IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
+ ITypeTrait[] typeTraits = new ITypeTrait[varLogical.size()];
+ ITypeTraitProvider typeTraitProvider = context.getTypeTraitProvider();
+ int i = 0;
+ for (LogicalVariable v : varLogical) {
+ Object type = env.getVarType(v);
+ typeTraits[i++] = typeTraitProvider.getTypeTrait(type);
+ }
+ return typeTraits;
+ }
+
+ public static int[] projectAllVariables(IOperatorSchema opSchema) {
+ int[] projectionList = new int[opSchema.getSize()];
+ int k = 0;
+ for (LogicalVariable v : opSchema) {
+ projectionList[k++] = opSchema.findVariable(v);
+ }
+ return projectionList;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/OperatorSchemaImpl.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/OperatorSchemaImpl.java
new file mode 100644
index 0000000..1d874f3
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/OperatorSchemaImpl.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+
+public class OperatorSchemaImpl implements IOperatorSchema {
+ private final Map<LogicalVariable, Integer> varMap;
+
+ private final List<LogicalVariable> varList;
+
+ public OperatorSchemaImpl() {
+ varMap = new HashMap<LogicalVariable, Integer>();
+ varList = new ArrayList<LogicalVariable>();
+ }
+
+ @Override
+ public void addAllVariables(IOperatorSchema source) {
+ for (LogicalVariable v : source) {
+ varMap.put(v, varList.size());
+ varList.add(v);
+ }
+ }
+
+ @Override
+ public void addAllNewVariables(IOperatorSchema source) {
+ for (LogicalVariable v : source) {
+ if (varMap.get(v) == null) {
+ varMap.put(v, varList.size());
+ varList.add(v);
+ }
+ }
+ }
+
+ @Override
+ public int addVariable(LogicalVariable var) {
+ int idx = varList.size();
+ varMap.put(var, idx);
+ varList.add(var);
+ return idx;
+ }
+
+ @Override
+ public void clear() {
+ varMap.clear();
+ varList.clear();
+ }
+
+ @Override
+ public int findVariable(LogicalVariable var) {
+ Integer i = varMap.get(var);
+ if (i == null) {
+ return -1;
+ }
+ return i;
+ }
+
+ @Override
+ public int getSize() {
+ return varList.size();
+ }
+
+ @Override
+ public LogicalVariable getVariable(int index) {
+ return varList.get(index);
+ }
+
+ @Override
+ public Iterator<LogicalVariable> iterator() {
+ return varList.iterator();
+ }
+
+ @Override
+ public String toString() {
+ return varMap.toString();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/PlanCompiler.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/PlanCompiler.java
new file mode 100644
index 0000000..afab295
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/PlanCompiler.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class PlanCompiler {
+ private JobGenContext context;
+ private Map<LogicalOperatorReference, List<LogicalOperatorReference>> operatorVisitedToParents = new HashMap<LogicalOperatorReference, List<LogicalOperatorReference>>();
+
+ public PlanCompiler(JobGenContext context) {
+ this.context = context;
+ }
+
+ public JobGenContext getContext() {
+ return context;
+ }
+
+ public JobSpecification compilePlan(ILogicalPlan plan, IOperatorSchema outerPlanSchema) throws AlgebricksException {
+ JobSpecification spec = new JobSpecification();
+ List<ILogicalOperator> rootOps = new ArrayList<ILogicalOperator>();
+ IHyracksJobBuilder builder = new JobBuilder(spec, context.getClusterLocations());
+ for (LogicalOperatorReference opRef : plan.getRoots()) {
+ compileOpRef(opRef, spec, builder, outerPlanSchema);
+ rootOps.add(opRef.getOperator());
+ }
+ reviseEdges(builder);
+ operatorVisitedToParents.clear();
+ builder.buildSpec(rootOps);
+ return spec;
+ }
+
+ private void compileOpRef(LogicalOperatorReference opRef, JobSpecification spec, IHyracksJobBuilder builder,
+ IOperatorSchema outerPlanSchema) throws AlgebricksException {
+ ILogicalOperator op = opRef.getOperator();
+ int n = op.getInputs().size();
+ IOperatorSchema[] schemas = new IOperatorSchema[n];
+ int i = 0;
+ for (LogicalOperatorReference opRef2 : op.getInputs()) {
+ List<LogicalOperatorReference> parents = operatorVisitedToParents.get(opRef2);
+ if (parents == null) {
+ parents = new ArrayList<LogicalOperatorReference>();
+ operatorVisitedToParents.put(opRef2, parents);
+ parents.add(opRef);
+ compileOpRef(opRef2, spec, builder, outerPlanSchema);
+ schemas[i++] = context.getSchema(opRef2.getOperator());
+ } else {
+ if (!parents.contains(opRef))
+ parents.add(opRef);
+ schemas[i++] = context.getSchema(opRef2.getOperator());
+ continue;
+ }
+ }
+
+ IOperatorSchema opSchema = new OperatorSchemaImpl();
+ context.putSchema(op, opSchema);
+ op.getVariablePropagationPolicy().propagateVariables(opSchema, schemas);
+ op.contributeRuntimeOperator(builder, context, opSchema, schemas, outerPlanSchema);
+ }
+
+ private void reviseEdges(IHyracksJobBuilder builder) {
+ /**
+ * revise the edges for the case of replicate operator
+ */
+ for (Entry<LogicalOperatorReference, List<LogicalOperatorReference>> entry : operatorVisitedToParents
+ .entrySet()) {
+ LogicalOperatorReference child = entry.getKey();
+ List<LogicalOperatorReference> parents = entry.getValue();
+ if (parents.size() > 1) {
+ int i = 0;
+ for (LogicalOperatorReference parent : parents) {
+ builder.contributeGraphEdge(child.getOperator(), i, parent.getOperator(), 0);
+ i++;
+ }
+ }
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/AggregateRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/AggregateRuntimeFactory.java
new file mode 100644
index 0000000..5184e12
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/AggregateRuntimeFactory.java
@@ -0,0 +1,135 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+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.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public class AggregateRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ // private int[] outColumns;
+ private IAggregateFunctionFactory[] aggregFactories;
+
+ public AggregateRuntimeFactory(IAggregateFunctionFactory[] aggregFactories) {
+ super(null);
+ // this.outColumns = outColumns;
+ this.aggregFactories = aggregFactories;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("assign [");
+ for (int i = 0; i < aggregFactories.length; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ sb.append(aggregFactories[i]);
+ }
+ sb.append("]");
+ return sb.toString();
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private IAggregateFunction[] aggregs = new IAggregateFunction[aggregFactories.length];
+ private ArrayBackedValueStorage evalOutput = new ArrayBackedValueStorage();
+ private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(aggregs.length);
+
+ private boolean first = true;
+
+ @Override
+ public void open() throws HyracksDataException {
+ try {
+ if (first) {
+ first = false;
+ initAccessAppendRef(context);
+ for (int i = 0; i < aggregFactories.length; i++) {
+ aggregs[i] = aggregFactories[i].createAggregateFunction(evalOutput);
+ }
+ }
+ for (int i = 0; i < aggregFactories.length; i++) {
+ aggregs[i].init();
+ }
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tRef.reset(tAccess, t);
+ processTuple(tRef);
+ }
+
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ computeAggregate();
+ appendToFrameFromTupleBuilder(tupleBuilder);
+ super.close();
+ }
+
+ private void computeAggregate() throws HyracksDataException {
+ tupleBuilder.reset();
+ for (int f = 0; f < aggregs.length; f++) {
+ evalOutput.reset();
+ try {
+ aggregs[f].finish();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ tupleBuilder.addField(evalOutput.getBytes(), evalOutput.getStartIndex(), evalOutput.getLength());
+ }
+ }
+
+ private void processTuple(FrameTupleReference tupleRef) throws HyracksDataException {
+ for (int f = 0; f < aggregs.length; f++) {
+ try {
+ aggregs[f].step(tupleRef);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
new file mode 100644
index 0000000..4b232a8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -0,0 +1,212 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+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.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregator;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+
+public class NestedPlansAccumulatingAggregatorFactory implements IAccumulatingAggregatorFactory {
+
+ private static final long serialVersionUID = 1L;
+ private AlgebricksPipeline[] subplans;
+ private int[] keyFieldIdx;
+ private int[] decorFieldIdx;
+
+ public NestedPlansAccumulatingAggregatorFactory(AlgebricksPipeline[] subplans, int[] keyFieldIdx,
+ int[] decorFieldIdx) {
+ this.subplans = subplans;
+ this.keyFieldIdx = keyFieldIdx;
+ this.decorFieldIdx = decorFieldIdx;
+ }
+
+ @Override
+ public IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
+ RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+ final RuntimeContext rc = new RuntimeContext();
+ rc.setHyracksContext(ctx);
+ final AggregatorOutput outputWriter = new AggregatorOutput(ctx.getFrameSize(), subplans, keyFieldIdx.length,
+ decorFieldIdx.length);
+ final NestedTupleSourceRuntime[] pipelines = new NestedTupleSourceRuntime[subplans.length];
+ for (int i = 0; i < subplans.length; i++) {
+ try {
+ pipelines[i] = (NestedTupleSourceRuntime) assemblePipeline(subplans[i], outputWriter, rc);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ return new IAccumulatingAggregator() {
+
+ private boolean pending;
+
+ @Override
+ public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ ArrayTupleBuilder tb = outputWriter.getTupleBuilder();
+ tb.reset();
+ for (int i = 0; i < keyFieldIdx.length; ++i) {
+ tb.addField(accessor, tIndex, keyFieldIdx[i]);
+ }
+ for (int i = 0; i < decorFieldIdx.length; ++i) {
+ tb.addField(accessor, tIndex, decorFieldIdx[i]);
+ }
+ for (int i = 0; i < pipelines.length; ++i) {
+ pipelines[i].open();
+ }
+ pending = false;
+ }
+
+ @Override
+ public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ // it only works if the output of the aggregator fits in one
+ // frame
+ for (int i = 0; i < pipelines.length; i++) {
+ pipelines[i].writeTuple(accessor.getBuffer(), tIndex);
+ }
+ }
+
+ @Override
+ public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex,
+ int[] keyFieldIndexes) throws HyracksDataException {
+ if (!pending) {
+ for (int i = 0; i < pipelines.length; i++) {
+ outputWriter.setInputIdx(i);
+ pipelines[i].close();
+ }
+ }
+ if (!outputWriter.writeTuple(appender)) {
+ pending = true;
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ };
+ }
+
+ private IFrameWriter assemblePipeline(AlgebricksPipeline subplan, IFrameWriter writer, RuntimeContext rc)
+ throws AlgebricksException {
+ // plug the operators
+ IFrameWriter start = writer;
+ IPushRuntimeFactory[] runtimeFactories = subplan.getRuntimeFactories();
+ RecordDescriptor[] recordDescriptors = subplan.getRecordDescriptors();
+ for (int i = runtimeFactories.length - 1; i >= 0; i--) {
+ IPushRuntime newRuntime = runtimeFactories[i].createPushRuntime(rc);
+ newRuntime.setFrameWriter(0, start, recordDescriptors[i]);
+ if (i > 0) {
+ newRuntime.setInputRecordDescriptor(0, recordDescriptors[i - 1]);
+ } else {
+ // the nts has the same input and output rec. desc.
+ newRuntime.setInputRecordDescriptor(0, recordDescriptors[0]);
+ }
+ start = newRuntime;
+ }
+ return start;
+ }
+
+ /**
+ *
+ *
+ * We suppose for now, that each subplan only produces one tuple.
+ *
+ */
+ private static class AggregatorOutput implements IFrameWriter {
+
+ // private ByteBuffer frame;
+ private FrameTupleAccessor[] tAccess;
+ private RecordDescriptor[] inputRecDesc;
+ private int inputIdx;
+ private ArrayTupleBuilder tb;
+ private AlgebricksPipeline[] subplans;
+
+ public AggregatorOutput(int frameSize, AlgebricksPipeline[] subplans, int numKeys, int numDecors) {
+ this.subplans = subplans;
+ // this.keyFieldIndexes = keyFieldIndexes;
+ int totalAggFields = 0;
+ this.inputRecDesc = new RecordDescriptor[subplans.length];
+ for (int i = 0; i < subplans.length; i++) {
+ RecordDescriptor[] rd = subplans[i].getRecordDescriptors();
+ this.inputRecDesc[i] = rd[rd.length - 1];
+ totalAggFields += subplans[i].getOutputWidth();
+ }
+ tb = new ArrayTupleBuilder(numKeys + numDecors + totalAggFields);
+
+ this.tAccess = new FrameTupleAccessor[inputRecDesc.length];
+ for (int i = 0; i < inputRecDesc.length; i++) {
+ tAccess[i] = new FrameTupleAccessor(frameSize, inputRecDesc[i]);
+ }
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ }
+
+ /**
+ *
+ * Since each pipeline only produces one tuple, this method is only
+ * called by the close method of the pipelines.
+ *
+ */
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ int tIndex = 0;
+ int w = subplans[inputIdx].getOutputWidth();
+ IFrameTupleAccessor accessor = tAccess[inputIdx];
+ accessor.reset(buffer);
+ for (int f = 0; f < w; f++) {
+ tb.addField(accessor, tIndex, f);
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ // clearFrame();
+ }
+
+ public void setInputIdx(int inputIdx) {
+ this.inputIdx = inputIdx;
+ }
+
+ public boolean writeTuple(FrameTupleAppender appender) {
+ return appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+ }
+
+ public ArrayTupleBuilder getTupleBuilder() {
+ return tb;
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
new file mode 100644
index 0000000..9082ec1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
@@ -0,0 +1,135 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.ISerializableAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.ISerializableAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+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.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
+
+public class SerializableAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+ private static final long serialVersionUID = 1L;
+
+ private ISerializableAggregateFunctionFactory[] aggFactories;
+
+ public SerializableAggregatorDescriptorFactory(ISerializableAggregateFunctionFactory[] aggFactories) {
+ this.aggFactories = aggFactories;
+ }
+
+ @Override
+ public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+ RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException {
+ final int[] keys = keyFields;
+ final int OFFSET_INT_LENGTH = 4;
+
+ /**
+ * one IAggregatorDescriptor instance per Gby operator
+ */
+ return new IAggregatorDescriptor() {
+ private FrameTupleReference ftr = new FrameTupleReference();
+ private ISerializableAggregateFunction[] aggs = new ISerializableAggregateFunction[aggFactories.length];
+ private int offsetFieldIndex = keys.length;
+ private int stateFieldLength[] = new int[aggFactories.length];
+
+ @Override
+ public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tb)
+ throws HyracksDataException {
+ DataOutput output = tb.getDataOutput();
+ ftr.reset(accessor, tIndex);
+ int startSize = tb.getSize();
+ for (int i = 0; i < aggs.length; i++) {
+ try {
+ int begin = tb.getSize();
+ if (aggs[i] == null) {
+ aggs[i] = aggFactories[i].createAggregateFunction();
+ }
+ aggs[i].init(output);
+ tb.addFieldEndOffset();
+ stateFieldLength[i] = tb.getSize() - begin;
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ int startOffset = 0;
+ if (offsetFieldIndex > 0)
+ startOffset = tb.getFieldEndOffsets()[offsetFieldIndex - 1];
+ else
+ startOffset = 0;
+ int endSize = tb.getSize();
+ int len = endSize - startSize;
+ aggregate(accessor, tIndex, tb.getByteArray(), startOffset, len);
+ }
+
+ @Override
+ public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
+ throws HyracksDataException {
+ ftr.reset(accessor, tIndex);
+ int start = offset;
+ for (int i = 0; i < aggs.length; i++) {
+ try {
+ aggs[i].step(ftr, data, start, stateFieldLength[i]);
+ start += stateFieldLength[i];
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ return OFFSET_INT_LENGTH;
+ }
+
+ @Override
+ public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tb)
+ throws HyracksDataException {
+ byte[] data = accessor.getBuffer().array();
+ int startOffset = accessor.getTupleStartOffset(tIndex);
+ int aggFieldOffset = accessor.getFieldStartOffset(tIndex, offsetFieldIndex);
+ int refOffset = startOffset + accessor.getFieldSlotsLength() + aggFieldOffset;
+ int start = refOffset;
+ for (int i = 0; i < aggs.length; i++) {
+ try {
+ aggs[i].finishPartial(data, start, stateFieldLength[i], tb.getDataOutput());
+ start += stateFieldLength[i];
+ tb.addFieldEndOffset();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ @Override
+ public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tb)
+ throws HyracksDataException {
+ byte[] data = accessor.getBuffer().array();
+ int startOffset = accessor.getTupleStartOffset(tIndex);
+ int aggFieldOffset = accessor.getFieldStartOffset(tIndex, offsetFieldIndex);
+ int refOffset = startOffset + accessor.getFieldSlotsLength() + aggFieldOffset;
+ int start = refOffset;
+ for (int i = 0; i < aggs.length; i++) {
+ try {
+ aggs[i].finish(data, start, stateFieldLength[i], tb.getDataOutput());
+ start += stateFieldLength[i];
+ tb.addFieldEndOffset();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ @Override
+ public void reset() {
+ }
+
+ @Override
+ public void close() {
+ reset();
+ }
+
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
new file mode 100644
index 0000000..986f21c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+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.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregator;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+
+public class SimpleAlgebricksAccumulatingAggregatorFactory implements IAccumulatingAggregatorFactory {
+
+ private static final long serialVersionUID = 1L;
+ private IAggregateFunctionFactory[] aggFactories;
+ private int[] keys;
+ private int[] fdColumns;
+
+ public SimpleAlgebricksAccumulatingAggregatorFactory(IAggregateFunctionFactory[] aggFactories, int[] keys,
+ int[] fdColumns) {
+ this.aggFactories = aggFactories;
+ this.keys = keys;
+ this.fdColumns = fdColumns;
+ }
+
+ @Override
+ public IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
+ RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+ final IAggregateFunction[] agg = new IAggregateFunction[aggFactories.length];
+ final ArrayBackedValueStorage[] aggOutput = new ArrayBackedValueStorage[aggFactories.length];
+ for (int i = 0; i < agg.length; i++) {
+ aggOutput[i] = new ArrayBackedValueStorage();
+ try {
+ agg[i] = aggFactories[i].createAggregateFunction(aggOutput[i]);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ return new IAccumulatingAggregator() {
+
+ private FrameTupleReference ftr = new FrameTupleReference();
+ private ArrayTupleBuilder tb = new ArrayTupleBuilder(keys.length + fdColumns.length + agg.length);
+ private boolean pending;
+
+ @Override
+ public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ pending = false;
+ for (int i = 0; i < aggOutput.length; i++) {
+ aggOutput[i].reset();
+ try {
+ agg[i].init();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ tb.reset();
+ for (int i = 0; i < keys.length; ++i) {
+ tb.addField(accessor, tIndex, keys[i]);
+ }
+ for (int i = 0; i < fdColumns.length; i++) {
+ tb.addField(accessor, tIndex, fdColumns[i]);
+ }
+
+ }
+
+ @Override
+ public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ ftr.reset(accessor, tIndex);
+ for (int i = 0; i < agg.length; i++) {
+ try {
+ agg[i].step(ftr);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ @Override
+ public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex,
+ int[] keyFieldIndexes) throws HyracksDataException {
+ if (!pending) {
+ for (int i = 0; i < agg.length; i++) {
+ try {
+ agg[i].finish();
+ tb.addField(aggOutput[i].getBytes(), aggOutput[i].getStartIndex(), aggOutput[i].getLength());
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ pending = true;
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
new file mode 100644
index 0000000..5d5e7fb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+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.FrameTupleAccessor;
+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.common.data.accessors.FrameTupleReference;
+
+public abstract class AbstractOneInputOneOutputOneFramePushRuntime extends AbstractOneInputOneOutputPushRuntime {
+
+ protected FrameTupleAppender appender;
+ protected ByteBuffer frame;
+ protected FrameTupleAccessor tAccess;
+ protected FrameTupleReference tRef;
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ writer.close();
+ appender.reset(frame, true);
+ }
+
+ protected void appendToFrameFromTupleBuilder(ArrayTupleBuilder tb) throws HyracksDataException {
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException(
+ "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime.appendToFrameFromTupleBuilder).");
+ }
+ }
+ }
+
+ protected void appendProjectionToFrame(int tIndex, int[] projectionList) throws HyracksDataException {
+ if (!appender.appendProjection(tAccess, tIndex, projectionList)) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.appendProjection(tAccess, tIndex, projectionList)) {
+ throw new IllegalStateException(
+ "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime.appendProjectionToFrame).");
+ }
+ }
+ }
+
+ protected void appendTupleToFrame(int tIndex) throws HyracksDataException {
+ if (!appender.append(tAccess, tIndex)) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tAccess, tIndex)) {
+ throw new IllegalStateException(
+ "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime.appendTupleToFrame).");
+ }
+ }
+ }
+
+ protected final void initAccessAppend(RuntimeContext context) {
+ IHyracksTaskContext hCtx = context.getHyracksContext();
+ // if (allocFrame) {
+ frame = hCtx.allocateFrame();
+ appender = new FrameTupleAppender(hCtx.getFrameSize());
+ appender.reset(frame, true);
+ // }
+ tAccess = new FrameTupleAccessor(hCtx.getFrameSize(), inputRecordDesc);
+ }
+
+ protected final void initAccessAppendRef(RuntimeContext context) {
+ initAccessAppend(context);
+ tRef = new FrameTupleReference();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputPushRuntime.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputPushRuntime.java
new file mode 100644
index 0000000..1034232
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputPushRuntime.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public abstract class AbstractOneInputOneOutputPushRuntime extends AbstractOneInputPushRuntime {
+
+ protected RecordDescriptor inputRecordDesc;
+
+ @Override
+ public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+ this.inputRecordDesc = recordDescriptor;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
new file mode 100644
index 0000000..23f4f6f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class AbstractOneInputOneOutputRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ protected int[] projectionList;
+
+ public AbstractOneInputOneOutputRuntimeFactory(int[] projectionList) {
+ this.projectionList = projectionList;
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(RuntimeContext context) throws AlgebricksException {
+ return createOneOutputPushRuntime(context);
+ }
+
+ public abstract AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(RuntimeContext context)
+ throws AlgebricksException;
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputPushRuntime.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputPushRuntime.java
new file mode 100644
index 0000000..5660b2c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputPushRuntime.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractOneInputPushRuntime implements IPushRuntime {
+ protected IFrameWriter writer;
+ protected RecordDescriptor outputRecordDesc;
+
+ @Override
+ public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ this.writer = writer;
+ this.outputRecordDesc = recordDesc;
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputSinkPushRuntime.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputSinkPushRuntime.java
new file mode 100644
index 0000000..82cf050
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputSinkPushRuntime.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public abstract class AbstractOneInputSinkPushRuntime implements IPushRuntime {
+ protected RecordDescriptor inputRecordDesc;
+
+ @Override
+ public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+ this.inputRecordDesc = recordDescriptor;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputSourcePushRuntime.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputSourcePushRuntime.java
new file mode 100644
index 0000000..3f1c0f3
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/AbstractOneInputSourcePushRuntime.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractOneInputSourcePushRuntime extends AbstractOneInputPushRuntime {
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/SinkRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/SinkRuntimeFactory.java
new file mode 100644
index 0000000..791d9a1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/base/SinkRuntimeFactory.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class SinkRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public SinkRuntimeFactory() {
+ }
+
+ @Override
+ public String toString() {
+ return "sink";
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(RuntimeContext context) throws AlgebricksException {
+ return new AbstractOneInputSinkPushRuntime() {
+
+ @Override
+ public void open() throws HyracksDataException {
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
new file mode 100644
index 0000000..237a903
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
@@ -0,0 +1,93 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.group;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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.std.group.IAccumulatingAggregator;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupWriter;
+
+public class MicroPreClusteredGroupRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final int[] groupFields;
+ private final IBinaryComparatorFactory[] comparatorFactories;
+ private final IAccumulatingAggregatorFactory aggregatorFactory;
+ private final RecordDescriptor inRecordDesc;
+ private final RecordDescriptor outRecordDesc;
+
+ public MicroPreClusteredGroupRuntimeFactory(int[] groupFields, IBinaryComparatorFactory[] comparatorFactories,
+ IAccumulatingAggregatorFactory aggregatorFactory, RecordDescriptor inRecordDesc,
+ RecordDescriptor outRecordDesc, int[] projectionList) {
+ super(projectionList);
+ // Obs: the projection list is currently ignored.
+ if (projectionList != null) {
+ throw new NotImplementedException("Cannot push projection into InMemorySortRuntime.");
+ }
+ this.groupFields = groupFields;
+ this.comparatorFactories = comparatorFactories;
+ this.aggregatorFactory = aggregatorFactory;
+ this.inRecordDesc = inRecordDesc;
+ this.outRecordDesc = outRecordDesc;
+ }
+
+ @Override
+ public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(RuntimeContext context)
+ throws AlgebricksException {
+ try {
+ final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ final IHyracksTaskContext ctx = context.getHyracksContext();
+ final IAccumulatingAggregator aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
+ outRecordDesc);
+ final ByteBuffer copyFrame = ctx.allocateFrame();
+ final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
+ copyFrameAccessor.reset(copyFrame);
+ ByteBuffer outFrame = ctx.allocateFrame();
+ final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender.reset(outFrame, true);
+
+ return new AbstractOneInputOneOutputPushRuntime() {
+
+ private PreclusteredGroupWriter pgw;
+
+ @Override
+ public void open() throws HyracksDataException {
+ pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregator, inRecordDesc, writer);
+ pgw.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ pgw.nextFrame(buffer);
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ pgw.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ pgw.close();
+ }
+ };
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
new file mode 100644
index 0000000..2fdeb23
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta;
+
+import java.nio.ByteBuffer;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+public class AlgebricksMetaOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ // array of factories for building the local runtime pipeline
+ private final AlgebricksPipeline pipeline;
+
+ public AlgebricksMetaOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
+ IPushRuntimeFactory[] runtimeFactories, RecordDescriptor[] internalRecordDescriptors) {
+ super(spec, inputArity, outputArity);
+ if (outputArity == 1) {
+ this.recordDescriptors[0] = internalRecordDescriptors[internalRecordDescriptors.length - 1];
+ }
+ this.pipeline = new AlgebricksPipeline(runtimeFactories, internalRecordDescriptors);
+ }
+
+ public AlgebricksPipeline getPipeline() {
+ return pipeline;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = super.toJSON();
+ json.put("micro-operators", pipeline.getRuntimeFactories());
+ return json;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Asterix { \n");
+ for (IPushRuntimeFactory f : pipeline.getRuntimeFactories()) {
+ sb.append(" " + f.toString() + ";\n");
+ }
+ sb.append("}");
+ // sb.append(super.getInputArity());
+ // sb.append(";");
+ // sb.append(super.getOutputArity());
+ // sb.append(";");
+ return sb.toString();
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ if (inputArity == 0) {
+ return createSourceInputPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ } else {
+ return createOneInputOneOutputPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ }
+ }
+
+ private IOperatorNodePushable createSourceInputPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ return new AbstractUnaryOutputSourceOperatorNodePushable() {
+
+ public void initialize() throws HyracksDataException {
+ IFrameWriter startOfPipeline;
+ RecordDescriptor pipelineOutputRecordDescriptor = outputArity > 0 ? AlgebricksMetaOperatorDescriptor.this.recordDescriptors[0]
+ : null;
+
+ PipelineAssembler pa = new PipelineAssembler(pipeline, inputArity, outputArity, null,
+ pipelineOutputRecordDescriptor);
+ try {
+ RuntimeContext rc = new RuntimeContext();
+ rc.setHyracksContext(ctx);
+ startOfPipeline = pa.assemblePipeline(writer, rc);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ startOfPipeline.open();
+ startOfPipeline.close();
+ }
+ };
+ }
+
+ private IOperatorNodePushable createOneInputOneOutputPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+
+ private IFrameWriter startOfPipeline;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (startOfPipeline == null) {
+ RecordDescriptor pipelineOutputRecordDescriptor = outputArity > 0 ? AlgebricksMetaOperatorDescriptor.this.recordDescriptors[0]
+ : null;
+ RecordDescriptor pipelineInputRecordDescriptor = recordDescProvider.getInputRecordDescriptor(
+ AlgebricksMetaOperatorDescriptor.this.getOperatorId(), 0);
+ PipelineAssembler pa = new PipelineAssembler(pipeline, inputArity, outputArity,
+ pipelineInputRecordDescriptor, pipelineOutputRecordDescriptor);
+ try {
+ RuntimeContext rc = new RuntimeContext();
+ rc.setHyracksContext(ctx);
+ startOfPipeline = pa.assemblePipeline(writer, rc);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ startOfPipeline.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ startOfPipeline.nextFrame(buffer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ startOfPipeline.close();
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+ };
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/PipelineAssembler.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/PipelineAssembler.java
new file mode 100644
index 0000000..f0f5029
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/PipelineAssembler.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class PipelineAssembler {
+
+ // array of factories for building the local runtime pipeline
+ private final RecordDescriptor pipelineInputRecordDescriptor;
+ private final RecordDescriptor pipelineOutputRecordDescriptor;
+
+ private final int inputArity;
+ private final int outputArity;
+ private final AlgebricksPipeline pipeline;
+
+ public PipelineAssembler(AlgebricksPipeline pipeline, int inputArity, int outputArity,
+ RecordDescriptor pipelineInputRecordDescriptor, RecordDescriptor pipelineOutputRecordDescriptor) {
+ this.pipeline = pipeline;
+ this.pipelineInputRecordDescriptor = pipelineInputRecordDescriptor;
+ this.pipelineOutputRecordDescriptor = pipelineOutputRecordDescriptor;
+ this.inputArity = inputArity;
+ this.outputArity = outputArity;
+ }
+
+ public IFrameWriter assemblePipeline(IFrameWriter writer, RuntimeContext rc) throws AlgebricksException {
+ // plug the operators
+ IFrameWriter start = writer;// this.writer;
+ for (int i = pipeline.getRuntimeFactories().length - 1; i >= 0; i--) {
+ IPushRuntime newRuntime = pipeline.getRuntimeFactories()[i].createPushRuntime(rc);
+ if (i == pipeline.getRuntimeFactories().length - 1) {
+ if (outputArity == 1) {
+ newRuntime.setFrameWriter(0, start, pipelineOutputRecordDescriptor);
+ }
+ } else {
+ newRuntime.setFrameWriter(0, start, pipeline.getRecordDescriptors()[i]);
+ }
+ if (i > 0) {
+ newRuntime.setInputRecordDescriptor(0, pipeline.getRecordDescriptors()[i - 1]);
+ } else if (inputArity > 0) {
+ newRuntime.setInputRecordDescriptor(0, pipelineInputRecordDescriptor);
+ }
+ start = newRuntime;
+ }
+ return start;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/SubplanRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/SubplanRuntimeFactory.java
new file mode 100644
index 0000000..53808e1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -0,0 +1,177 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta;
+
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+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;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+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.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class SubplanRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final AlgebricksPipeline pipeline;
+ private final RecordDescriptor inputRecordDesc;
+ private final INullWriterFactory[] nullWriterFactories;
+
+ public SubplanRuntimeFactory(AlgebricksPipeline pipeline, INullWriterFactory[] nullWriterFactories,
+ RecordDescriptor inputRecordDesc, int[] projectionList) {
+ super(projectionList);
+ this.pipeline = pipeline;
+ this.nullWriterFactories = nullWriterFactories;
+ this.inputRecordDesc = inputRecordDesc;
+ if (projectionList != null) {
+ throw new NotImplementedException();
+ }
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Subplan { \n");
+ for (IPushRuntimeFactory f : pipeline.getRuntimeFactories()) {
+ sb.append(" " + f.toString() + ";\n");
+ }
+ sb.append("}");
+ return sb.toString();
+ }
+
+ @Override
+ public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+
+ RecordDescriptor pipelineOutputRecordDescriptor = null;
+
+ final PipelineAssembler pa = new PipelineAssembler(pipeline, 1, 1, inputRecordDesc,
+ pipelineOutputRecordDescriptor);
+ final INullWriter[] nullWriters = new INullWriter[nullWriterFactories.length];
+ for (int i = 0; i < nullWriterFactories.length; i++) {
+ nullWriters[i] = nullWriterFactories[i].createNullWriter();
+ }
+
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ /**
+ *
+ * Computes the outer product between a given tuple and the frames
+ * passed.
+ *
+ */
+ class TupleOuterProduct implements IFrameWriter {
+
+ private boolean smthWasWritten = false;
+ private IHyracksTaskContext hCtx = context.getHyracksContext();
+ private int frameSize = hCtx.getFrameSize();
+ private FrameTupleAccessor ta = new FrameTupleAccessor(frameSize,
+ pipeline.getRecordDescriptors()[pipeline.getRecordDescriptors().length - 1]);
+ private ArrayTupleBuilder tb = new ArrayTupleBuilder(nullWriters.length);
+
+ @Override
+ public void open() throws HyracksDataException {
+ smthWasWritten = false;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ ta.reset(buffer);
+ int nTuple = ta.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ if (!appender.appendConcat(tRef.getFrameTupleAccessor(), tRef.getTupleIndex(), ta, t)) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.appendConcat(tRef.getFrameTupleAccessor(), tRef.getTupleIndex(), ta, t)) {
+ throw new IllegalStateException("Could not write frame.");
+ }
+ }
+ }
+ smthWasWritten = true;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (!smthWasWritten) {
+ // the case when we need to write nulls
+ appendNullsToTuple();
+ appendToFrameFromTupleBuilder(tb);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ private void appendNullsToTuple() throws HyracksDataException {
+ tb.reset();
+ int n0 = tRef.getFieldCount();
+ for (int f = 0; f < n0; f++) {
+ tb.addField(tRef.getFrameTupleAccessor(), tRef.getTupleIndex(), f);
+ }
+ DataOutput dos = tb.getDataOutput();
+ for (int i = 0; i < nullWriters.length; i++) {
+ nullWriters[i].writeNull(dos);
+ tb.addFieldEndOffset();
+ }
+ }
+
+ }
+
+ IFrameWriter endPipe = new TupleOuterProduct();
+
+ NestedTupleSourceRuntime startOfPipeline = (NestedTupleSourceRuntime) pa.assemblePipeline(endPipe, context);
+
+ boolean first = true;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (first) {
+ first = false;
+ initAccessAppendRef(context);
+ }
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tRef.reset(tAccess, t);
+ startOfPipeline.writeTuple(buffer, t);
+ startOfPipeline.open();
+ startOfPipeline.close();
+ }
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/sort/InMemorySortRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/sort/InMemorySortRuntimeFactory.java
new file mode 100644
index 0000000..2a6c196
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.sort;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorter;
+
+public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int[] sortFields;
+ private INormalizedKeyComputerFactory firstKeyNormalizerFactory;
+ private IBinaryComparatorFactory[] comparatorFactories;
+
+ public InMemorySortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+ IBinaryComparatorFactory[] comparatorFactories, int[] projectionList) {
+ super(projectionList);
+ // Obs: the projection list is currently ignored.
+ if (projectionList != null) {
+ throw new NotImplementedException("Cannot push projection into InMemorySortRuntime.");
+ }
+ this.sortFields = sortFields;
+ this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
+ this.comparatorFactories = comparatorFactories;
+ }
+
+ @Override
+ public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+
+ return new AbstractOneInputOneOutputPushRuntime() {
+
+ FrameSorter frameSorter = null;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (frameSorter == null) {
+ frameSorter = new FrameSorter(context.getHyracksContext(), sortFields, firstKeyNormalizerFactory,
+ comparatorFactories, outputRecordDesc);
+ }
+ frameSorter.reset();
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ frameSorter.insertFrame(buffer);
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ frameSorter.sortFrames();
+ frameSorter.flushFrames(writer);
+ writer.close();
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/AssignRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/AssignRuntimeFactory.java
new file mode 100644
index 0000000..04dc7d6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/AssignRuntimeFactory.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+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.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public class AssignRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private int[] outColumns;
+ private IEvaluatorFactory[] evalFactories;
+
+ /**
+ *
+ * @param outColumns
+ * a sorted array of columns into which the result is written to
+ * @param evalFactories
+ * @param projectionList
+ * an array of columns to be projected
+ */
+
+ public AssignRuntimeFactory(int[] outColumns, IEvaluatorFactory[] evalFactories, int[] projectionList) {
+ super(projectionList);
+ this.outColumns = outColumns;
+ this.evalFactories = evalFactories;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("assign [");
+ for (int i = 0; i < outColumns.length; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ sb.append(outColumns[i]);
+ }
+ sb.append("] := [");
+ for (int i = 0; i < evalFactories.length; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ sb.append(evalFactories[i]);
+ }
+ sb.append("]");
+ return sb.toString();
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+ final int[] projectionToOutColumns = new int[projectionList.length];
+ for (int j = 0; j < projectionList.length; j++) {
+ projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
+ }
+
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private ArrayBackedValueStorage evalOutput = new ArrayBackedValueStorage();
+ private IEvaluator[] eval = new IEvaluator[evalFactories.length];
+ private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(projectionList.length);
+ private boolean first = true;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (first) {
+ initAccessAppendRef(context);
+ first = false;
+ int n = evalFactories.length;
+ for (int i = 0; i < n; i++) {
+ try {
+ eval[i] = evalFactories[i].createEvaluator(evalOutput);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ }
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tRef.reset(tAccess, t);
+ produceTuple(tupleBuilder, tAccess, t, tRef);
+ appendToFrameFromTupleBuilder(tupleBuilder);
+ }
+ }
+
+ private void produceTuple(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+ FrameTupleReference tupleRef) throws HyracksDataException {
+ tb.reset();
+ for (int f = 0; f < projectionList.length; f++) {
+ int k = projectionToOutColumns[f];
+ if (k >= 0) {
+ evalOutput.reset();
+ try {
+ eval[k].evaluate(tupleRef);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ tb.addField(evalOutput.getBytes(), evalOutput.getStartIndex(), evalOutput.getLength());
+ } else {
+ tb.addField(accessor, tIndex, projectionList[f]);
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
new file mode 100644
index 0000000..2d0a038
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputSourcePushRuntime;
+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;
+
+public class EmptyTupleSourceRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public EmptyTupleSourceRuntimeFactory() {
+ }
+
+ @Override
+ public String toString() {
+ return "ets";
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(final RuntimeContext context) {
+ return new AbstractOneInputSourcePushRuntime() {
+
+ private IHyracksTaskContext hCtx = context.getHyracksContext();
+ private ByteBuffer frame = hCtx.allocateFrame();
+ private ArrayTupleBuilder tb = new ArrayTupleBuilder(0);
+ private FrameTupleAppender appender = new FrameTupleAppender(hCtx.getFrameSize());
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ FrameUtils.flushFrame(frame, writer);
+ writer.close();
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/NestedTupleSourceRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
new file mode 100644
index 0000000..8c650b5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NestedTupleSourceRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public NestedTupleSourceRuntimeFactory() {
+ }
+
+ @Override
+ public String toString() {
+ return "nts";
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(RuntimeContext context) {
+ return new NestedTupleSourceRuntime(context);
+ }
+
+ public static class NestedTupleSourceRuntime extends AbstractOneInputOneOutputOneFramePushRuntime {
+
+ public NestedTupleSourceRuntime(RuntimeContext rc) {
+ initAccessAppend(rc);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ }
+
+ public void writeTuple(ByteBuffer inputBuffer, int tIndex) throws HyracksDataException {
+ tAccess.reset(inputBuffer);
+ appendTupleToFrame(tIndex);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/PrinterRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/PrinterRuntimeFactory.java
new file mode 100644
index 0000000..787e001
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/PrinterRuntimeFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.writers.PrinterBasedWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class PrinterRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int[] printColumns;
+ private final IPrinterFactory[] printerFactories;
+ private final RecordDescriptor inputRecordDesc;
+
+ public PrinterRuntimeFactory(int[] printColumns, IPrinterFactory[] printerFactories,
+ RecordDescriptor inputRecordDesc) {
+ this.printColumns = printColumns;
+ this.printerFactories = printerFactories;
+ this.inputRecordDesc = inputRecordDesc;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder buf = new StringBuilder();
+ buf.append("print [");
+ for (int i = 0; i < printColumns.length; i++) {
+ if (i > 0) {
+ buf.append("; ");
+ }
+ buf.append(printColumns[i]);
+ }
+ buf.append("]");
+ return buf.toString();
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(final RuntimeContext context) {
+ IAWriter w = PrinterBasedWriterFactory.INSTANCE.createWriter(printColumns, System.out, printerFactories,
+ inputRecordDesc);
+ return new SinkWriterRuntime(w, context, System.out, inputRecordDesc);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/RunningAggregateRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/RunningAggregateRuntimeFactory.java
new file mode 100644
index 0000000..bca8f2f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/RunningAggregateRuntimeFactory.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+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.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public class RunningAggregateRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private int[] outColumns;
+ private IRunningAggregateFunctionFactory[] runningAggregates;
+
+ /**
+ *
+ * @param outColumns
+ * a sorted array of columns into which the result is written to
+ * @param runningAggregates
+ * @param projectionList
+ * an array of columns to be projected
+ */
+
+ public RunningAggregateRuntimeFactory(int[] outColumns, IRunningAggregateFunctionFactory[] runningAggregates,
+ int[] projectionList) {
+ super(projectionList);
+ this.outColumns = outColumns;
+ this.runningAggregates = runningAggregates;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("running-aggregate [");
+ for (int i = 0; i < outColumns.length; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ sb.append(outColumns[i]);
+ }
+ sb.append("] := [");
+ for (int i = 0; i < runningAggregates.length; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ sb.append(runningAggregates[i]);
+ }
+ sb.append("]");
+ return sb.toString();
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+ final int[] projectionToOutColumns = new int[projectionList.length];
+ for (int j = 0; j < projectionList.length; j++) {
+ projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
+ }
+
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private ArrayBackedValueStorage evalOutput = new ArrayBackedValueStorage();
+ private IRunningAggregateFunction[] raggs = new IRunningAggregateFunction[runningAggregates.length];
+ private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(projectionList.length);
+ private boolean first = true;
+
+ @Override
+ public void open() throws HyracksDataException {
+ initAccessAppendRef(context);
+ if (first) {
+ first = false;
+ int n = runningAggregates.length;
+ for (int i = 0; i < n; i++) {
+ try {
+ raggs[i] = runningAggregates[i].createRunningAggregateFunction(evalOutput);
+ raggs[i].init();
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ }
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tRef.reset(tAccess, t);
+ produceTuple(tupleBuilder, tAccess, t, tRef);
+ appendToFrameFromTupleBuilder(tupleBuilder);
+ }
+ }
+
+ private void produceTuple(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+ FrameTupleReference tupleRef) throws HyracksDataException {
+ tb.reset();
+ for (int f = 0; f < projectionList.length; f++) {
+ int k = projectionToOutColumns[f];
+ if (k >= 0) {
+ evalOutput.reset();
+ try {
+ raggs[k].step(tupleRef);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ tb.addField(evalOutput.getBytes(), evalOutput.getStartIndex(), evalOutput.getLength());
+ } else {
+ tb.addField(accessor, tIndex, projectionList[f]);
+ }
+ }
+ }
+
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/SinkWriterRuntime.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/SinkWriterRuntime.java
new file mode 100644
index 0000000..8101a8a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/SinkWriterRuntime.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputSinkPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class SinkWriterRuntime extends AbstractOneInputSinkPushRuntime {
+
+ private final RuntimeContext context;
+ private final PrintStream printStream;
+ private final IAWriter writer;
+ private RecordDescriptor inputRecordDesc;
+ private FrameTupleAccessor tAccess;
+ private boolean autoClose = false;
+ private boolean first = true;
+
+ public SinkWriterRuntime(IAWriter writer, RuntimeContext context, PrintStream printStream,
+ RecordDescriptor inputRecordDesc) {
+ this.writer = writer;
+ this.context = context;
+ this.printStream = printStream;
+ this.inputRecordDesc = inputRecordDesc;
+ this.tAccess = new FrameTupleAccessor(context.getHyracksContext().getFrameSize(), inputRecordDesc);
+ }
+
+ public SinkWriterRuntime(IAWriter writer, RuntimeContext context, PrintStream printStream,
+ RecordDescriptor inputRecordDesc, boolean autoClose) {
+ this(writer, context, printStream, inputRecordDesc);
+ this.autoClose = autoClose;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (first) {
+ first = false;
+ tAccess = new FrameTupleAccessor(context.getHyracksContext().getFrameSize(), inputRecordDesc);
+ try {
+ writer.init();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ try {
+ writer.printTuple(tAccess, t);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (autoClose) {
+ printStream.close();
+ }
+ }
+
+ @Override
+ public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+ this.inputRecordDesc = recordDescriptor;
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/SinkWriterRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/SinkWriterRuntimeFactory.java
new file mode 100644
index 0000000..4ac1d16
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/SinkWriterRuntimeFactory.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class SinkWriterRuntimeFactory implements IPushRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int[] fields;
+ private final IPrinterFactory[] printerFactories;
+ private final File outputFile;
+ private final RecordDescriptor inputRecordDesc;
+ private final IAWriterFactory writerFactory;
+
+ public SinkWriterRuntimeFactory(int[] fields, IPrinterFactory[] printerFactories, File outputFile,
+ IAWriterFactory writerFactory, RecordDescriptor inputRecordDesc) {
+ this.fields = fields;
+ this.printerFactories = printerFactories;
+ this.outputFile = outputFile;
+ this.writerFactory = writerFactory;
+ this.inputRecordDesc = inputRecordDesc;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder buf = new StringBuilder();
+ buf.append("sink-write " + "[");
+ for (int i = 0; i < fields.length; i++) {
+ if (i > 0) {
+ buf.append("; ");
+ }
+ buf.append(fields[i]);
+ }
+ buf.append("] outputFile");
+ return buf.toString();
+ }
+
+ @Override
+ public IPushRuntime createPushRuntime(RuntimeContext context) throws AlgebricksException {
+ PrintStream filePrintStream = null;
+ try {
+ filePrintStream = new PrintStream(new BufferedOutputStream(new FileOutputStream(outputFile)));
+ } catch (FileNotFoundException e) {
+ throw new AlgebricksException(e);
+ }
+ IAWriter w = writerFactory.createWriter(fields, filePrintStream, printerFactories, inputRecordDesc);
+ return new SinkWriterRuntime(w, context, filePrintStream, inputRecordDesc, true);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamDieRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamDieRuntimeFactory.java
new file mode 100644
index 0000000..86b9b3b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamDieRuntimeFactory.java
@@ -0,0 +1,98 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class StreamDieRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private IEvaluatorFactory aftterObjectsEvalFactory;
+ private IBinaryIntegerInspector binaryIntegerInspector;
+
+ public StreamDieRuntimeFactory(IEvaluatorFactory maxObjectsEvalFactory, int[] projectionList,
+ IBinaryIntegerInspector binaryIntegerInspector) {
+ super(projectionList);
+ this.aftterObjectsEvalFactory = maxObjectsEvalFactory;
+ this.binaryIntegerInspector = binaryIntegerInspector;
+ }
+
+ @Override
+ public String toString() {
+ String s = "stream-die " + aftterObjectsEvalFactory.toString();
+ return s;
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context) {
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private IEvaluator evalAfterObjects;
+ private ArrayBackedValueStorage evalOutput;
+ private int toWrite = -1;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (evalAfterObjects == null) {
+ initAccessAppendRef(context);
+ evalOutput = new ArrayBackedValueStorage();
+ try {
+ evalAfterObjects = aftterObjectsEvalFactory.createEvaluator(evalOutput);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ if (toWrite < 0) {
+ toWrite = evaluateInteger(evalAfterObjects, 0);
+ }
+ for (int t = 0; t < nTuple; t++) {
+ if (toWrite > 0) {
+ toWrite--;
+ if (projectionList != null) {
+ appendProjectionToFrame(t, projectionList);
+ } else {
+ appendTupleToFrame(t);
+ }
+ } else {
+ throw new HyracksDataException("injected failure");
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ super.close();
+ }
+
+ private int evaluateInteger(IEvaluator eval, int tIdx) throws HyracksDataException {
+ tRef.reset(tAccess, tIdx);
+ evalOutput.reset();
+ try {
+ eval.evaluate(tRef);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ int lim = binaryIntegerInspector.getIntegerValue(evalOutput.getBytes(), 0, evalOutput.getLength());
+ return lim;
+ }
+
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamLimitRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamLimitRuntimeFactory.java
new file mode 100644
index 0000000..5caea6e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamLimitRuntimeFactory.java
@@ -0,0 +1,135 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryIntegerInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class StreamLimitRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private IEvaluatorFactory maxObjectsEvalFactory;
+ private IEvaluatorFactory offsetEvalFactory;
+ private IBinaryIntegerInspector binaryIntegerInspector;
+
+ public StreamLimitRuntimeFactory(IEvaluatorFactory maxObjectsEvalFactory, IEvaluatorFactory offsetEvalFactory,
+ int[] projectionList, IBinaryIntegerInspector binaryIntegerInspector) {
+ super(projectionList);
+ this.maxObjectsEvalFactory = maxObjectsEvalFactory;
+ this.offsetEvalFactory = offsetEvalFactory;
+ this.binaryIntegerInspector = binaryIntegerInspector;
+ }
+
+ @Override
+ public String toString() {
+ String s = "stream-limit " + maxObjectsEvalFactory.toString();
+ if (offsetEvalFactory != null) {
+ return s + ", " + offsetEvalFactory.toString();
+ } else {
+ return s;
+ }
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context) {
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private IEvaluator evalMaxObjects;
+ private ArrayBackedValueStorage evalOutput;
+ private IEvaluator evalOffset = null;
+ private int toWrite = 0; // how many tuples still to write
+ private int toSkip = 0; // how many tuples still to skip
+ private boolean firstTuple = true;
+ private boolean afterLastTuple = false;
+
+ @Override
+ public void open() throws HyracksDataException {
+ // if (first) {
+ if (evalMaxObjects == null) {
+ initAccessAppendRef(context);
+ evalOutput = new ArrayBackedValueStorage();
+ try {
+ evalMaxObjects = maxObjectsEvalFactory.createEvaluator(evalOutput);
+ if (offsetEvalFactory != null) {
+ evalOffset = offsetEvalFactory.createEvaluator(evalOutput);
+ }
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ writer.open();
+ afterLastTuple = false;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ if (afterLastTuple) {
+ // ignore the data
+ return;
+ }
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ int start = 0;
+ if (nTuple <= toSkip) {
+ toSkip -= nTuple;
+ return;
+ } else if (toSkip > 0) {
+ start = toSkip;
+ toSkip = 0;
+ }
+ for (int t = start; t < nTuple; t++) {
+ if (firstTuple) {
+ firstTuple = false;
+ toWrite = evaluateInteger(evalMaxObjects, t);
+ if (evalOffset != null) {
+ toSkip = evaluateInteger(evalOffset, t);
+ }
+ }
+ if (toSkip > 0) {
+ toSkip--;
+ } else if (toWrite > 0) {
+ toWrite--;
+ if (projectionList != null) {
+ appendProjectionToFrame(t, projectionList);
+ } else {
+ appendTupleToFrame(t);
+ }
+ } else {
+ // close();
+ afterLastTuple = true;
+ break;
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ // if (!afterLastTuple) {
+ super.close();
+ // }
+ }
+
+ private int evaluateInteger(IEvaluator eval, int tIdx) throws HyracksDataException {
+ tRef.reset(tAccess, tIdx);
+ evalOutput.reset();
+ try {
+ eval.evaluate(tRef);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ int lim = binaryIntegerInspector.getIntegerValue(evalOutput.getBytes(), 0, evalOutput.getLength());
+ return lim;
+ }
+
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamProjectRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamProjectRuntimeFactory.java
new file mode 100644
index 0000000..fe30701
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamProjectRuntimeFactory.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class StreamProjectRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public StreamProjectRuntimeFactory(int[] projectionList) {
+ super(projectionList);
+ }
+
+ @Override
+ public String toString() {
+ return "stream-project " + Arrays.toString(projectionList);
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private boolean first = true;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (first) {
+ first = false;
+ initAccessAppend(context);
+ }
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ appendProjectionToFrame(t, projectionList);
+ }
+
+ }
+
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamSelectRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamSelectRuntimeFactory.java
new file mode 100644
index 0000000..be579e5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StreamSelectRuntimeFactory.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+
+public class StreamSelectRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private IEvaluatorFactory cond;
+
+ private IBinaryBooleanInspector binaryBooleanInspector;
+
+ /**
+ *
+ * @param cond
+ * @param projectionList
+ * if projectionList is null, then no projection is performed
+ */
+ public StreamSelectRuntimeFactory(IEvaluatorFactory cond, int[] projectionList,
+ IBinaryBooleanInspector binaryBooleanInspector) {
+ super(projectionList);
+ this.cond = cond;
+ this.binaryBooleanInspector = binaryBooleanInspector;
+ }
+
+ @Override
+ public String toString() {
+ return "stream-select " + cond.toString();
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context) {
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private IEvaluator eval;
+ private ArrayBackedValueStorage evalOutput;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (eval == null) {
+ initAccessAppendRef(context);
+ evalOutput = new ArrayBackedValueStorage();
+ try {
+ eval = cond.createEvaluator(evalOutput);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tRef.reset(tAccess, t);
+ evalOutput.reset();
+ try {
+ eval.evaluate(tRef);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ if (binaryBooleanInspector.getBooleanValue(evalOutput.getBytes(), 0, evalOutput.getLength())) {
+ if (projectionList != null) {
+ appendProjectionToFrame(t, projectionList);
+ } else {
+ appendTupleToFrame(t);
+ }
+ }
+ }
+ }
+
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StringStreamingRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StringStreamingRuntimeFactory.java
new file mode 100644
index 0000000..f3d07b1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/StringStreamingRuntimeFactory.java
@@ -0,0 +1,189 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class StringStreamingRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private String command;
+ private IPrinterFactory[] printerFactories;
+ private char fieldDelimiter;
+ private ITupleParserFactory parserFactory;
+
+ public StringStreamingRuntimeFactory(String command, IPrinterFactory[] printerFactories, char fieldDelimiter,
+ ITupleParserFactory parserFactory) {
+ super(null);
+ this.command = command;
+ this.printerFactories = printerFactories;
+ this.fieldDelimiter = fieldDelimiter;
+ this.parserFactory = parserFactory;
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+ final IPrinter[] printers = new IPrinter[printerFactories.length];
+ for (int i = 0; i < printerFactories.length; i++) {
+ printers[i] = printerFactories[i].createPrinter();
+ }
+
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ final class ForwardScriptOutput implements Runnable {
+
+ private InputStream inStream;
+ private ITupleParser parser;
+
+ public ForwardScriptOutput(ITupleParser parser, InputStream inStream) {
+ this.parser = parser;
+ this.inStream = inStream;
+ }
+
+ @Override
+ public void run() {
+ try {
+ parser.parse(inStream, writer);
+ } catch (HyracksDataException e) {
+ throw new RuntimeException(e);
+ } finally {
+ try {
+ inStream.close();
+ } catch (Exception e) {
+ }
+ }
+ }
+ }
+
+ final class DumpInStreamToPrintStream implements Runnable {
+
+ private BufferedReader reader;
+ private PrintStream printStream;
+
+ public DumpInStreamToPrintStream(InputStream inStream, PrintStream printStream) {
+ this.reader = new BufferedReader(new InputStreamReader(inStream));
+ this.printStream = printStream;
+ }
+
+ @Override
+ public void run() {
+ String s;
+ try {
+ while ((s = reader.readLine()) != null) {
+ printStream.println(s);
+ }
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ } finally {
+ try {
+ reader.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ printStream.close();
+ }
+ }
+
+ }
+
+ private Process process;
+ private PrintStream ps;
+ private boolean first = true;
+ private Thread outputPipe;
+ private Thread dumpStderr;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (first) {
+ first = false;
+ initAccessAppendRef(context);
+ }
+
+ try {
+ ITupleParser parser = parserFactory.createTupleParser(context.getHyracksContext());
+ process = Runtime.getRuntime().exec(command);
+ ps = new PrintStream(process.getOutputStream());
+ ForwardScriptOutput fso = new ForwardScriptOutput(parser, process.getInputStream());
+ outputPipe = new Thread(fso);
+ outputPipe.start();
+ DumpInStreamToPrintStream disps = new DumpInStreamToPrintStream(process.getErrorStream(),
+ System.err);
+ dumpStderr = new Thread(disps);
+ dumpStderr.start();
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tRef.reset(tAccess, t);
+ for (int i = 0; i < printers.length; i++) {
+ try {
+ printers[i].print(buffer.array(), tRef.getFieldStart(i), tRef.getFieldLength(i), ps);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ ps.print(fieldDelimiter);
+ if (i == printers.length - 1) {
+ ps.print('\n');
+ }
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ // first close the printer printing to the process
+ ps.close();
+ int ret = 0;
+ // then wait for the process to finish
+
+ try {
+ ret = process.waitFor();
+ outputPipe.join();
+ dumpStderr.join();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ if (ret != 0) {
+ throw new HyracksDataException("Process exit value: " + ret);
+ }
+ // close the following operator in the chain
+ super.close();
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/UnnestRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/UnnestRuntimeFactory.java
new file mode 100644
index 0000000..bc55b6a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/std/UnnestRuntimeFactory.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.context.RuntimeContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+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.data.accessors.ArrayBackedValueStorage;
+
+public class UnnestRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int outCol;
+ private final IUnnestingFunctionFactory unnestingFactory;
+ private int outColPos;
+ private final boolean outColIsProjected;
+
+ // Each time step() is called on the aggregate, a new value is written in
+ // its output. One byte is written before that value and is neglected.
+ // By convention, if the aggregate function writes nothing, it means it
+ // produced the last value.
+
+ public UnnestRuntimeFactory(int outCol, IUnnestingFunctionFactory unnestingFactory, int[] projectionList) {
+ super(projectionList);
+ this.outCol = outCol;
+ this.unnestingFactory = unnestingFactory;
+ outColPos = -1;
+ for (int f = 0; f < projectionList.length; f++) {
+ if (projectionList[f] == outCol) {
+ outColPos = f;
+ }
+ }
+ outColIsProjected = outColPos >= 0;
+ }
+
+ @Override
+ public String toString() {
+ return "unnest " + outCol + " <- " + unnestingFactory;
+ }
+
+ @Override
+ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final RuntimeContext context)
+ throws AlgebricksException {
+
+ return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+ private ArrayBackedValueStorage evalOutput;
+ private IUnnestingFunction agg;
+ private ArrayTupleBuilder tupleBuilder;
+
+ @Override
+ public void open() throws HyracksDataException {
+ initAccessAppendRef(context);
+ evalOutput = new ArrayBackedValueStorage();
+ try {
+ agg = unnestingFactory.createUnnestingFunction(evalOutput);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ tupleBuilder = new ArrayTupleBuilder(projectionList.length);
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ int nTuple = tAccess.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tRef.reset(tAccess, t);
+ try {
+ agg.init(tRef);
+ boolean goon = true;
+ do {
+ tupleBuilder.reset();
+ evalOutput.reset();
+ if (!agg.step()) {
+ goon = false;
+ } else {
+ if (!outColIsProjected) {
+ appendProjectionToFrame(t, projectionList);
+ } else {
+ for (int f = 0; f < outColPos; f++) {
+ tupleBuilder.addField(tAccess, t, f);
+ }
+ tupleBuilder.addField(evalOutput.getBytes(), evalOutput.getStartIndex(), evalOutput
+ .getLength());
+ for (int f = outColPos + 1; f < projectionList.length; f++) {
+ tupleBuilder.addField(tAccess, t, f);
+ }
+ }
+ appendToFrameFromTupleBuilder(tupleBuilder);
+ }
+ } while (goon);
+ } catch (AlgebricksException ae) {
+ throw new HyracksDataException(ae);
+ }
+ }
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/writers/PrinterBasedWriterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/writers/PrinterBasedWriterFactory.java
new file mode 100644
index 0000000..36ef827
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/writers/PrinterBasedWriterFactory.java
@@ -0,0 +1,54 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.writers;
+
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class PrinterBasedWriterFactory implements IAWriterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final PrinterBasedWriterFactory INSTANCE = new PrinterBasedWriterFactory();
+
+ public PrinterBasedWriterFactory() {
+ }
+
+ @Override
+ public IAWriter createWriter(final int[] fields, final PrintStream printStream,
+ final IPrinterFactory[] printerFactories, RecordDescriptor inputRecordDescriptor) {
+ final IPrinter[] printers = new IPrinter[printerFactories.length];
+ for (int i = 0; i < printerFactories.length; i++) {
+ printers[i] = printerFactories[i].createPrinter();
+ }
+
+ return new IAWriter() {
+
+ @Override
+ public void init() throws AlgebricksException {
+ for (int i = 0; i < printers.length; i++) {
+ printers[i].init();
+ }
+ }
+
+ @Override
+ public void printTuple(FrameTupleAccessor tAccess, int tIdx) throws AlgebricksException {
+ for (int i = 0; i < fields.length; i++) {
+ int fldStart = tAccess.getTupleStartOffset(tIdx) + tAccess.getFieldSlotsLength()
+ + tAccess.getFieldStartOffset(tIdx, fields[i]);
+ int fldLen = tAccess.getFieldLength(tIdx, fields[i]);
+ if (i > 0) {
+ printStream.print("; ");
+ }
+ printers[i].print(tAccess.getBuffer().array(), fldStart, fldLen, printStream);
+ }
+ printStream.println();
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/writers/SerializedDataWriterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/writers/SerializedDataWriterFactory.java
new file mode 100644
index 0000000..79d7a99
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/writers/SerializedDataWriterFactory.java
@@ -0,0 +1,49 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.runtime.writers;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IAWriterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class SerializedDataWriterFactory implements IAWriterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAWriter createWriter(final int[] fields, final PrintStream ps, IPrinterFactory[] printerFactories,
+ final RecordDescriptor inputRecordDescriptor) {
+ return new IAWriter() {
+
+ @Override
+ public void init() throws AlgebricksException {
+ // dump the SerializerDeserializers to disk
+ try {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ ObjectOutputStream oos = new ObjectOutputStream(baos);
+ oos.writeObject(inputRecordDescriptor);
+ baos.writeTo(ps);
+ oos.close();
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void printTuple(FrameTupleAccessor tAccess, int tIdx) throws AlgebricksException {
+ for (int i = 0; i < fields.length; i++) {
+ int fldStart = tAccess.getTupleStartOffset(tIdx) + tAccess.getFieldSlotsLength()
+ + tAccess.getFieldStartOffset(tIdx, fields[i]);
+ int fldLen = tAccess.getFieldLength(tIdx, fields[i]);
+ ps.write(tAccess.getBuffer().array(), fldStart, fldLen);
+ }
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/scripting/IScriptDescription.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/scripting/IScriptDescription.java
new file mode 100644
index 0000000..f292dd7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/scripting/IScriptDescription.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.scripting;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public interface IScriptDescription {
+ public enum ScriptKind {
+ STRING_STREAMING
+ }
+
+ public ScriptKind getKind();
+
+ public List<Pair<LogicalVariable, Object>> getVarTypePairs();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/scripting/StringStreamingScriptDescription.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/scripting/StringStreamingScriptDescription.java
new file mode 100644
index 0000000..64cea03
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/scripting/StringStreamingScriptDescription.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.scripting;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class StringStreamingScriptDescription implements IScriptDescription {
+
+ private final String command;
+ private final IPrinterFactory[] printerFactories;
+ private final char fieldDelimiter;
+ private final ITupleParserFactory parserFactory;
+ private final List<Pair<LogicalVariable, Object>> varTypePairs;
+
+ public StringStreamingScriptDescription(String command, IPrinterFactory[] printerFactories, char fieldDelimiter,
+ ITupleParserFactory parserFactory, List<Pair<LogicalVariable, Object>> varTypePairs) {
+ this.command = command;
+ this.printerFactories = printerFactories;
+ this.fieldDelimiter = fieldDelimiter;
+ this.parserFactory = parserFactory;
+ this.varTypePairs = varTypePairs;
+ }
+
+ @Override
+ public ScriptKind getKind() {
+ return ScriptKind.STRING_STREAMING;
+ }
+
+ public String getCommand() {
+ return command;
+ }
+
+ public IPrinterFactory[] getPrinterFactories() {
+ return printerFactories;
+ }
+
+ public char getFieldDelimiter() {
+ return fieldDelimiter;
+ }
+
+ public ITupleParserFactory getParserFactory() {
+ return parserFactory;
+ }
+
+ @Override
+ public List<Pair<LogicalVariable, Object>> getVarTypePairs() {
+ return varTypePairs;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/AbstractTypeEnvironment.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/AbstractTypeEnvironment.java
new file mode 100644
index 0000000..d1ebbab
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/AbstractTypeEnvironment.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.typing;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class AbstractTypeEnvironment implements IVariableTypeEnvironment {
+
+ protected final Map<LogicalVariable, Object> varTypeMap = new HashMap<LogicalVariable, Object>();
+ protected final IExpressionTypeComputer expressionTypeComputer;
+ protected final IMetadataProvider<?, ?> metadataProvider;
+
+ public AbstractTypeEnvironment(IExpressionTypeComputer expressionTypeComputer,
+ IMetadataProvider<?, ?> metadataProvider) {
+ this.expressionTypeComputer = expressionTypeComputer;
+ this.metadataProvider = metadataProvider;
+ }
+
+ @Override
+ public Object getType(ILogicalExpression expr) throws AlgebricksException {
+ return expressionTypeComputer.getType(expr, metadataProvider, this);
+ }
+
+ @Override
+ public void setVarType(LogicalVariable var, Object type) {
+ varTypeMap.put(var, type);
+ }
+
+ @Override
+ public boolean substituteProducedVariable(LogicalVariable v1, LogicalVariable v2) throws AlgebricksException {
+ Object t = varTypeMap.get(v1);
+ if (t == null) {
+ return false;
+ }
+ varTypeMap.put(v1, null);
+ varTypeMap.put(v2, t);
+ return true;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/ITypeEnvPointer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/ITypeEnvPointer.java
new file mode 100644
index 0000000..3c8d674
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/ITypeEnvPointer.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.typing;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public interface ITypeEnvPointer {
+ public IVariableTypeEnvironment getTypeEnv();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/ITypingContext.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/ITypingContext.java
new file mode 100644
index 0000000..4996b48
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/ITypingContext.java
@@ -0,0 +1,19 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.typing;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public interface ITypingContext {
+ public abstract IVariableTypeEnvironment getOutputTypeEnvironment(ILogicalOperator op);
+
+ public abstract void setOutputTypeEnvironment(ILogicalOperator op, IVariableTypeEnvironment env);
+
+ public abstract IExpressionTypeComputer getExpressionTypeComputer();
+
+ public abstract INullableTypeComputer getNullableTypeComputer();
+
+ public abstract IMetadataProvider<?, ?> getMetadataProvider();
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
new file mode 100644
index 0000000..e0f4e16
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.typing;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class NonPropagatingTypeEnvironment extends AbstractTypeEnvironment {
+
+ public NonPropagatingTypeEnvironment(IExpressionTypeComputer expressionTypeComputer,
+ IMetadataProvider<?, ?> metadataProvider) {
+ super(expressionTypeComputer, metadataProvider);
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var) throws AlgebricksException {
+ return varTypeMap.get(var);
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables) throws AlgebricksException {
+ return getVarType(var);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/OpRefTypeEnvPointer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/OpRefTypeEnvPointer.java
new file mode 100644
index 0000000..7c4aaa1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/OpRefTypeEnvPointer.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.typing;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class OpRefTypeEnvPointer implements ITypeEnvPointer {
+
+ private final LogicalOperatorReference op;
+ private final ITypingContext ctx;
+
+ public OpRefTypeEnvPointer(LogicalOperatorReference op, ITypingContext ctx) {
+ this.op = op;
+ this.ctx = ctx;
+ }
+
+ @Override
+ public IVariableTypeEnvironment getTypeEnv() {
+ return ctx.getOutputTypeEnvironment(op.getOperator());
+ }
+
+ @Override
+ public String toString() {
+ return this.getClass().getName() + ":" + op;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
new file mode 100644
index 0000000..26109c6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
@@ -0,0 +1,59 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.typing;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class PropagateOperatorInputsTypeEnvironment extends AbstractTypeEnvironment {
+
+ private final List<LogicalVariable> nonNullVariables = new ArrayList<LogicalVariable>();
+ private final ILogicalOperator op;
+ private final ITypingContext ctx;
+
+ public PropagateOperatorInputsTypeEnvironment(ILogicalOperator op, ITypingContext ctx,
+ IExpressionTypeComputer expressionTypeComputer, IMetadataProvider<?, ?> metadataProvider) {
+ super(expressionTypeComputer, metadataProvider);
+ this.op = op;
+ this.ctx = ctx;
+ }
+
+ public List<LogicalVariable> getNonNullVariables() {
+ return nonNullVariables;
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariableList) throws AlgebricksException {
+ nonNullVariableList.addAll(nonNullVariables);
+ return getVarTypeFullList(var, nonNullVariableList);
+ }
+
+ private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonNullVariableList)
+ throws AlgebricksException {
+ Object t = varTypeMap.get(var);
+ if (t != null) {
+ return t;
+ }
+ for (LogicalOperatorReference r : op.getInputs()) {
+ ILogicalOperator c = r.getOperator();
+ IVariableTypeEnvironment env = ctx.getOutputTypeEnvironment(c);
+ Object t2 = env.getVarType(var, nonNullVariableList);
+ if (t2 != null) {
+ return t2;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var) throws AlgebricksException {
+ return getVarTypeFullList(var, nonNullVariables);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
new file mode 100644
index 0000000..bb2db54
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.hyracks.algebricks.core.algebra.typing;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class PropagatingTypeEnvironment extends AbstractTypeEnvironment {
+
+ private final TypePropagationPolicy policy;
+
+ private final INullableTypeComputer nullableTypeComputer;
+
+ private final ITypeEnvPointer[] envPointers;
+
+ private final List<LogicalVariable> nonNullVariables = new ArrayList<LogicalVariable>();
+
+ public PropagatingTypeEnvironment(IExpressionTypeComputer expressionTypeComputer,
+ INullableTypeComputer nullableTypeComputer, IMetadataProvider<?, ?> metadataProvider,
+ TypePropagationPolicy policy, ITypeEnvPointer[] envPointers) {
+ super(expressionTypeComputer, metadataProvider);
+ this.nullableTypeComputer = nullableTypeComputer;
+ this.policy = policy;
+ this.envPointers = envPointers;
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var) throws AlgebricksException {
+ return getVarTypeFullList(var, nonNullVariables);
+ }
+
+ public List<LogicalVariable> getNonNullVariables() {
+ return nonNullVariables;
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariableList) throws AlgebricksException {
+ nonNullVariableList.addAll(nonNullVariables);
+ return getVarTypeFullList(var, nonNullVariableList);
+ }
+
+ private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonNullVariableList)
+ throws AlgebricksException {
+ Object t = varTypeMap.get(var);
+ if (t != null) {
+ return t;
+ }
+ return policy.getVarType(var, nullableTypeComputer, nonNullVariableList, envPointers);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
new file mode 100644
index 0000000..d1299ff
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -0,0 +1,159 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.util;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class OperatorManipulationUtil {
+
+ // Transforms all NestedTupleSource operators to EmptyTupleSource operators
+ public static void ntsToEts(LogicalOperatorReference opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ EmptyTupleSourceOperator ets = new EmptyTupleSourceOperator();
+ context.computeAndSetTypeEnvironmentForOperator(ets);
+ opRef.setOperator(ets);
+ } else {
+ for (LogicalOperatorReference i : opRef.getOperator().getInputs()) {
+ ntsToEts(i, context);
+ }
+ }
+ }
+
+ public static ILogicalOperator eliminateSingleSubplanOverEts(SubplanOperator subplan) {
+ if (subplan.getNestedPlans().size() > 1) {
+ // not a single subplan
+ List<LogicalOperatorReference> subInpList = subplan.getInputs();
+ subInpList.clear();
+ subInpList.add(new LogicalOperatorReference(new EmptyTupleSourceOperator()));
+ return subplan;
+ }
+ ILogicalPlan plan = subplan.getNestedPlans().get(0);
+ if (plan.getRoots().size() > 1) {
+ // not a single subplan
+ List<LogicalOperatorReference> subInpList = subplan.getInputs();
+ subInpList.clear();
+ subInpList.add(new LogicalOperatorReference(new EmptyTupleSourceOperator()));
+ return subplan;
+ }
+ return plan.getRoots().get(0).getOperator();
+ }
+
+ public static boolean setOperatorMode(AbstractLogicalOperator op) {
+ boolean change = false;
+ switch (op.getOperatorTag()) {
+ case DATASOURCESCAN: {
+ // ILogicalExpression e = ((UnnestOperator) op).getExpression();
+ // if (AnalysisUtil.isDataSetCall(e)) {
+ op.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ AbstractLogicalOperator child = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ if (child.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ child.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ }
+ change = true;
+ // }
+ break;
+ }
+ case NESTEDTUPLESOURCE: {
+ NestedTupleSourceOperator nts = (NestedTupleSourceOperator) op;
+ AbstractLogicalOperator prevOp = (AbstractLogicalOperator) nts.getDataSourceReference().getOperator()
+ .getInputs().get(0).getOperator();
+ if (prevOp.getExecutionMode() != AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) {
+ nts.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+ change = true;
+ }
+ break;
+ }
+ default: {
+ if (op.getOperatorTag() == LogicalOperatorTag.LIMIT) {
+ LimitOperator opLim = (LimitOperator) op;
+ if (opLim.isTopmostLimitOp()) {
+ if (opLim.getExecutionMode() != AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) {
+ opLim.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+ change = true;
+ }
+ break;
+ }
+ }
+
+ for (LogicalOperatorReference i : op.getInputs()) {
+ boolean exit = false;
+ AbstractLogicalOperator inputOp = (AbstractLogicalOperator) i.getOperator();
+ switch (inputOp.getExecutionMode()) {
+ case PARTITIONED: {
+ op.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ change = true;
+ exit = true;
+ break;
+ }
+ case LOCAL: {
+ op.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+ change = true;
+ break;
+ }
+ }
+ if (exit) {
+ break;
+ }
+ }
+ break;
+ }
+ }
+ return change;
+ }
+
+ public static void substituteVarRec(AbstractLogicalOperator op, LogicalVariable v1, LogicalVariable v2,
+ boolean goThroughNts, ITypingContext ctx) throws AlgebricksException {
+ VariableUtilities.substituteVariables(op, v1, v2, goThroughNts, ctx);
+ for (LogicalOperatorReference opRef2 : op.getInputs()) {
+ substituteVarRec((AbstractLogicalOperator) opRef2.getOperator(), v1, v2, goThroughNts, ctx);
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE && goThroughNts) {
+ NestedTupleSourceOperator nts = (NestedTupleSourceOperator) op;
+ if (nts.getDataSourceReference() != null) {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) nts.getDataSourceReference().getOperator()
+ .getInputs().get(0).getOperator();
+ substituteVarRec(op2, v1, v2, goThroughNts, ctx);
+ }
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans aonp = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : aonp.getNestedPlans()) {
+ for (LogicalOperatorReference ref : p.getRoots()) {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) ref.getOperator();
+ substituteVarRec(aop, v1, v2, goThroughNts, ctx);
+ }
+ }
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
new file mode 100644
index 0000000..2223991
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -0,0 +1,210 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.util;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class OperatorPropertiesUtil {
+
+ public static <T> boolean disjoint(Collection<T> c1, Collection<T> c2) {
+ for (T m : c1) {
+ if (c2.contains(m)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ // Obs: doesn't return expected result for op. with nested plans.
+ private static void getFreeVariablesInOp(ILogicalOperator op, Set<LogicalVariable> freeVars)
+ throws AlgebricksException {
+ VariableUtilities.getUsedVariables(op, freeVars);
+ HashSet<LogicalVariable> produced = new HashSet<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op, produced);
+ for (LogicalVariable v : produced) {
+ freeVars.remove(v);
+ }
+ }
+
+ /**
+ * Adds the free variables of the plan rooted at that operator to the
+ * collection provided.
+ *
+ * @param op
+ * @param vars
+ * - The collection to which the free variables will be added.
+ */
+ public static void getFreeVariablesInSelfOrDesc(AbstractLogicalOperator op, Set<LogicalVariable> freeVars)
+ throws AlgebricksException {
+ HashSet<LogicalVariable> produced = new HashSet<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op, produced);
+ for (LogicalVariable v : produced) {
+ freeVars.remove(v);
+ }
+
+ HashSet<LogicalVariable> used = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, used);
+ for (LogicalVariable v : used) {
+ if (!freeVars.contains(v)) {
+ freeVars.add(v);
+ }
+ }
+
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans s = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : s.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ getFreeVariablesInSelfOrDesc((AbstractLogicalOperator) r.getOperator(), freeVars);
+ }
+ }
+ s.getUsedVariablesExceptNestedPlans(freeVars);
+ HashSet<LogicalVariable> produced2 = new HashSet<LogicalVariable>();
+ s.getProducedVariablesExceptNestedPlans(produced2);
+ freeVars.removeAll(produced);
+ }
+ for (LogicalOperatorReference i : op.getInputs()) {
+ getFreeVariablesInSelfOrDesc((AbstractLogicalOperator) i.getOperator(), freeVars);
+ }
+ }
+
+ public static void getFreeVariablesInSubplans(AbstractOperatorWithNestedPlans op, Set<LogicalVariable> freeVars)
+ throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ getFreeVariablesInSelfOrDesc((AbstractLogicalOperator) r.getOperator(), freeVars);
+ }
+ }
+ }
+
+ public static boolean hasFreeVariablesInSelfOrDesc(AbstractLogicalOperator op) throws AlgebricksException {
+ HashSet<LogicalVariable> free = new HashSet<LogicalVariable>();
+ getFreeVariablesInSelfOrDesc(op, free);
+ return !free.isEmpty();
+ }
+
+ public static boolean hasFreeVariables(ILogicalOperator op) throws AlgebricksException {
+ HashSet<LogicalVariable> free = new HashSet<LogicalVariable>();
+ getFreeVariablesInOp(op, free);
+ return !free.isEmpty();
+ }
+
+ public static void computeSchemaAndPropertiesRecIfNull(AbstractLogicalOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ if (op.getSchema() == null) {
+ for (LogicalOperatorReference i : op.getInputs()) {
+ computeSchemaAndPropertiesRecIfNull((AbstractLogicalOperator) i.getOperator(), context);
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans a = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : a.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ computeSchemaAndPropertiesRecIfNull((AbstractLogicalOperator) r.getOperator(), context);
+ }
+ }
+ }
+ op.recomputeSchema();
+ op.computeDeliveredPhysicalProperties(context);
+ }
+ }
+
+ public static void computeSchemaRecIfNull(AbstractLogicalOperator op) throws AlgebricksException {
+ if (op.getSchema() == null) {
+ for (LogicalOperatorReference i : op.getInputs()) {
+ computeSchemaRecIfNull((AbstractLogicalOperator) i.getOperator());
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans a = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : a.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ computeSchemaRecIfNull((AbstractLogicalOperator) r.getOperator());
+ }
+ }
+ }
+ op.recomputeSchema();
+ }
+ }
+
+ public static boolean isNullTest(AbstractLogicalOperator op) {
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return false;
+ }
+ ILogicalExpression eu = ((SelectOperator) op).getCondition().getExpression();
+ if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
+ if (f1.getFunctionIdentifier() != AlgebricksBuiltinFunctions.NOT) {
+ return false;
+ }
+ ILogicalExpression a1 = f1.getArguments().get(0).getExpression();
+ if (a1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+ if (f2.getFunctionIdentifier() != AlgebricksBuiltinFunctions.IS_NULL) {
+ return false;
+ }
+ return true;
+ }
+
+ public static void typePlan(ILogicalPlan p, IOptimizationContext context) throws AlgebricksException {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ typeOpRec(r, context);
+ }
+ }
+
+ public static void typeOpRec(LogicalOperatorReference r, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) r.getOperator();
+ for (LogicalOperatorReference i : op.getInputs()) {
+ typeOpRec(i, context);
+ }
+ if (op.hasNestedPlans()) {
+ for (ILogicalPlan p : ((AbstractOperatorWithNestedPlans) op).getNestedPlans()) {
+ typePlan(p, context);
+ }
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+
+ public static boolean isAlwaysTrueCond(ILogicalExpression cond) {
+ if (cond.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ return ((ConstantExpression) cond).getValue().isTrue();
+ }
+ return false;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/AbstractConstVarFunVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/AbstractConstVarFunVisitor.java
new file mode 100644
index 0000000..d4c5530
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/AbstractConstVarFunVisitor.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.visitors;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public abstract class AbstractConstVarFunVisitor<R, T> implements ILogicalExpressionVisitor<R, T> {
+
+ public abstract R visitFunctionCallExpression(AbstractFunctionCallExpression expr, T arg)
+ throws AlgebricksException;
+
+ public R visitAggregateFunctionCallExpression(AggregateFunctionCallExpression expr, T arg)
+ throws AlgebricksException {
+ return visitFunctionCallExpression(expr, arg);
+ }
+
+ public R visitScalarFunctionCallExpression(ScalarFunctionCallExpression expr, T arg) throws AlgebricksException {
+ return visitFunctionCallExpression(expr, arg);
+ }
+
+ public R visitStatefulFunctionCallExpression(StatefulFunctionCallExpression expr, T arg) throws AlgebricksException {
+ return visitFunctionCallExpression(expr, arg);
+ }
+
+ public R visitUnnestingFunctionCallExpression(UnnestingFunctionCallExpression expr, T arg)
+ throws AlgebricksException {
+ return visitFunctionCallExpression(expr, arg);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalExpressionReferenceTransform.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalExpressionReferenceTransform.java
new file mode 100644
index 0000000..e8f2d06
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalExpressionReferenceTransform.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.visitors;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface ILogicalExpressionReferenceTransform {
+ // returns true if any change is made to the expression
+ public boolean transform(LogicalExpressionReference expression) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalExpressionVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalExpressionVisitor.java
new file mode 100644
index 0000000..a299574
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalExpressionVisitor.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.visitors;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface ILogicalExpressionVisitor<R, T> {
+ public R visitConstantExpression(ConstantExpression expr, T arg) throws AlgebricksException;
+
+ public R visitVariableReferenceExpression(VariableReferenceExpression expr, T arg) throws AlgebricksException;
+
+ public R visitAggregateFunctionCallExpression(AggregateFunctionCallExpression expr, T arg)
+ throws AlgebricksException;
+
+ public R visitScalarFunctionCallExpression(ScalarFunctionCallExpression expr, T arg) throws AlgebricksException;
+
+ public R visitStatefulFunctionCallExpression(StatefulFunctionCallExpression expr, T arg) throws AlgebricksException;
+
+ public R visitUnnestingFunctionCallExpression(UnnestingFunctionCallExpression expr, T arg)
+ throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
new file mode 100644
index 0000000..bfc7acc
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.visitors;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface ILogicalOperatorVisitor<R, T> {
+
+ public R visitAggregateOperator(AggregateOperator op, T arg) throws AlgebricksException;
+
+ public R visitRunningAggregateOperator(RunningAggregateOperator op, T arg) throws AlgebricksException;
+
+ public R visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, T arg) throws AlgebricksException;
+
+ public R visitGroupByOperator(GroupByOperator op, T arg) throws AlgebricksException;
+
+ public R visitLimitOperator(LimitOperator op, T arg) throws AlgebricksException;
+
+ public R visitInnerJoinOperator(InnerJoinOperator op, T arg) throws AlgebricksException;
+
+ public R visitLeftOuterJoinOperator(LeftOuterJoinOperator op, T arg) throws AlgebricksException;
+
+ public R visitNestedTupleSourceOperator(NestedTupleSourceOperator op, T arg) throws AlgebricksException;
+
+ public R visitOrderOperator(OrderOperator op, T arg) throws AlgebricksException;
+
+ public R visitAssignOperator(AssignOperator op, T arg) throws AlgebricksException;
+
+ public R visitSelectOperator(SelectOperator op, T arg) throws AlgebricksException;
+
+ public R visitProjectOperator(ProjectOperator op, T arg) throws AlgebricksException;
+
+ public R visitPartitioningSplitOperator(PartitioningSplitOperator op, T arg) throws AlgebricksException;
+
+ public R visitReplicateOperator(ReplicateOperator op, T arg) throws AlgebricksException;
+
+ public R visitScriptOperator(ScriptOperator op, T arg) throws AlgebricksException;
+
+ public R visitSubplanOperator(SubplanOperator op, T arg) throws AlgebricksException;
+
+ public R visitSinkOperator(SinkOperator op, T arg) throws AlgebricksException;
+
+ public R visitUnionOperator(UnionAllOperator op, T arg) throws AlgebricksException;
+
+ public R visitUnnestOperator(UnnestOperator op, T arg) throws AlgebricksException;
+
+ public R visitUnnestMapOperator(UnnestMapOperator op, T arg) throws AlgebricksException;
+
+ public R visitDataScanOperator(DataSourceScanOperator op, T arg) throws AlgebricksException;
+
+ public R visitDistinctOperator(DistinctOperator op, T arg) throws AlgebricksException;
+
+ public R visitExchangeOperator(ExchangeOperator op, T arg) throws AlgebricksException;
+
+ public R visitWriteOperator(WriteOperator op, T arg) throws AlgebricksException;
+
+ public R visitWriteResultOperator(WriteResultOperator op, T arg) throws AlgebricksException;
+
+ public R visitInsertDeleteOperator(InsertDeleteOperator op, T tag) throws AlgebricksException;
+
+ public R visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, T tag) throws AlgebricksException;
+
+ public R visitDieOperator(DieOperator op, T arg) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksAbsolutePartitionConstraint.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksAbsolutePartitionConstraint.java
new file mode 100644
index 0000000..de5f778
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksAbsolutePartitionConstraint.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.api.constraints;
+
+public class AlgebricksAbsolutePartitionConstraint extends AlgebricksPartitionConstraint {
+ private final String[] locations;
+
+ public AlgebricksAbsolutePartitionConstraint(String[] locations) {
+ this.locations = locations;
+ }
+
+ @Override
+ public PartitionConstraintType getPartitionConstraintType() {
+ return PartitionConstraintType.ABSOLUTE;
+ }
+
+ public String[] getLocations() {
+ return locations;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksCountPartitionConstraint.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksCountPartitionConstraint.java
new file mode 100644
index 0000000..9cd8079
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksCountPartitionConstraint.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.api.constraints;
+
+public class AlgebricksCountPartitionConstraint extends AlgebricksPartitionConstraint {
+
+ private final int count;
+
+ public AlgebricksCountPartitionConstraint(int count) {
+ this.count = count;
+ }
+
+ @Override
+ public PartitionConstraintType getPartitionConstraintType() {
+ return PartitionConstraintType.COUNT;
+ }
+
+ public int getCount() {
+ return count;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksPartitionConstraint.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksPartitionConstraint.java
new file mode 100644
index 0000000..0a63b80
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksPartitionConstraint.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.api.constraints;
+
+public abstract class AlgebricksPartitionConstraint {
+ public enum PartitionConstraintType {
+ ABSOLUTE,
+ COUNT
+ }
+
+ public abstract PartitionConstraintType getPartitionConstraintType();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksPartitionConstraintHelper.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksPartitionConstraintHelper.java
new file mode 100644
index 0000000..e64602b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/constraints/AlgebricksPartitionConstraintHelper.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.api.constraints;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class AlgebricksPartitionConstraintHelper {
+
+ public static void setPartitionConstraintInJobSpec(JobSpecification jobSpec, IOperatorDescriptor opDesc,
+ AlgebricksPartitionConstraint apc) {
+ switch (apc.getPartitionConstraintType()) {
+ case ABSOLUTE: {
+ AlgebricksAbsolutePartitionConstraint absPc = (AlgebricksAbsolutePartitionConstraint) apc;
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, opDesc, absPc.getLocations());
+ break;
+ }
+ case COUNT: {
+ AlgebricksCountPartitionConstraint cntPc = (AlgebricksCountPartitionConstraint) apc;
+ PartitionConstraintHelper.addPartitionCountConstraint(jobSpec, opDesc, cntPc.getCount());
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ public static int getPartitionCount(AlgebricksPartitionConstraint partitionConstraint) {
+ switch (partitionConstraint.getPartitionConstraintType()) {
+ case COUNT: {
+ AlgebricksCountPartitionConstraint pcc = (AlgebricksCountPartitionConstraint) partitionConstraint;
+ return pcc.getCount();
+ }
+ case ABSOLUTE: {
+ AlgebricksAbsolutePartitionConstraint epc = (AlgebricksAbsolutePartitionConstraint) partitionConstraint;
+ return epc.getLocations().length;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/exceptions/AlgebricksException.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/exceptions/AlgebricksException.java
new file mode 100644
index 0000000..c852ecd
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/exceptions/AlgebricksException.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.api.exceptions;
+
+public class AlgebricksException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public AlgebricksException() {
+ }
+
+ public AlgebricksException(String message) {
+ super(message);
+ }
+
+ public AlgebricksException(Throwable cause) {
+ super(cause);
+ }
+
+ public AlgebricksException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/exceptions/NotImplementedException.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/exceptions/NotImplementedException.java
new file mode 100644
index 0000000..f5343a8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/api/exceptions/NotImplementedException.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.api.exceptions;
+
+public class NotImplementedException extends RuntimeException {
+ private static final long serialVersionUID = 2L;
+
+ public NotImplementedException() {
+ System.err.println("Not implemented.");
+ }
+
+ public NotImplementedException(String message) {
+ super(message);
+ }
+
+ public NotImplementedException(Throwable cause) {
+ super(cause);
+ }
+
+ public NotImplementedException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/AlgebricksConfig.java
new file mode 100644
index 0000000..397f51b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.config;
+
+import java.util.logging.Logger;
+
+public class AlgebricksConfig {
+ public static final boolean DEBUG = true;
+ public static final String ALGEBRICKS_LOGGER_NAME = "edu.uci.ics.algebricks";
+ public static final Logger ALGEBRICKS_LOGGER = Logger.getLogger(ALGEBRICKS_LOGGER_NAME);
+ public static final String HYRACKS_APP_NAME = "algebricks";
+
+ // public static final Level ALGEBRICKS_LOG_LEVEL = Level.FINEST;
+ //
+ // static {
+ // Handler h;
+ // try {
+ // h = new ConsoleHandler();
+ // h.setFormatter(new SysoutFormatter());
+ // } catch (Exception e) {
+ // h = new ConsoleHandler();
+ // }
+ // h.setLevel(ALGEBRICKS_LOG_LEVEL);
+ // ALGEBRICKS_LOGGER.addHandler(h);
+ // ALGEBRICKS_LOGGER.setLevel(ALGEBRICKS_LOG_LEVEL);
+ // }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/SysoutFormatter.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/SysoutFormatter.java
new file mode 100644
index 0000000..872676b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/SysoutFormatter.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.config;
+
+import java.util.logging.Formatter;
+import java.util.logging.Handler;
+import java.util.logging.LogRecord;
+
+public class SysoutFormatter extends Formatter {
+ public String format(LogRecord rec) {
+ StringBuffer buf = new StringBuffer(5000);
+ buf.append(formatMessage(rec));
+ return buf.toString();
+ }
+
+ public String getHead(Handler h) {
+ return "";
+ }
+
+ public String getTail(Handler h) {
+ return "";
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
new file mode 100644
index 0000000..25441a9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.rewriter.base;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+
+public abstract class AbstractRuleController {
+
+ protected IOptimizationContext context;
+ private LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
+
+ public AbstractRuleController() {
+ }
+
+ public void setContext(IOptimizationContext context) {
+ this.context = context;
+ }
+
+ /**
+ * Each rewriting strategy may differ in the
+ *
+ * @param root
+ * @param ruleClasses
+ * @return true iff one of the rules in the collection fired
+ */
+ public abstract boolean rewriteWithRuleCollection(LogicalOperatorReference root,
+ Collection<IAlgebraicRewriteRule> rules) throws AlgebricksException;
+
+ /**
+ * @param opRef
+ * @param rule
+ * @return true if any rewrite was fired, either on opRef or any operator
+ * under it.
+ */
+ protected boolean rewriteOperatorRef(LogicalOperatorReference opRef, IAlgebraicRewriteRule rule)
+ throws AlgebricksException {
+ return rewriteOperatorRef(opRef, rule, true, false);
+ }
+
+ private void printRuleApplication(IAlgebraicRewriteRule rule, LogicalOperatorReference opRef)
+ throws AlgebricksException {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Rule " + rule.getClass() + " fired.\n");
+ StringBuilder sb = new StringBuilder();
+ PlanPrettyPrinter.printOperator((AbstractLogicalOperator) opRef.getOperator(), sb, pvisitor, 0);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(sb.toString());
+ }
+
+ protected boolean rewriteOperatorRef(LogicalOperatorReference opRef, IAlgebraicRewriteRule rule,
+ boolean enterNestedPlans, boolean fullDFS) throws AlgebricksException {
+
+ if (rule.rewritePre(opRef, context)) {
+ printRuleApplication(rule, opRef);
+ return true;
+ }
+ boolean rewritten = false;
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+
+ for (LogicalOperatorReference inp : op.getInputs()) {
+ if (rewriteOperatorRef(inp, rule, enterNestedPlans, fullDFS)) {
+ rewritten = true;
+ if (!fullDFS) {
+ break;
+ }
+ }
+ }
+
+ if (op.hasNestedPlans() && enterNestedPlans) {
+ AbstractOperatorWithNestedPlans o2 = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : o2.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ if (rewriteOperatorRef(r, rule, enterNestedPlans, fullDFS)) {
+ rewritten = true;
+ if (!fullDFS) {
+ break;
+ }
+ }
+ }
+ if (rewritten && !fullDFS) {
+ break;
+ }
+ }
+ }
+
+ if (rule.rewritePost(opRef, context)) {
+ printRuleApplication(rule, opRef);
+ return true;
+ }
+
+ return rewritten;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
new file mode 100644
index 0000000..ab436f4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
@@ -0,0 +1,285 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.rewriter.base;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableEvalSizeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class AlgebricksOptimizationContext implements IOptimizationContext {
+
+ private int varCounter;
+ private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
+ private final IMergeAggregationExpressionFactory mergeAggregationExpressionFactory;
+ private final PhysicalOptimizationConfig physicalOptimizationConfig;
+ private final IVariableEvalSizeEnvironment varEvalSizeEnv = new IVariableEvalSizeEnvironment() {
+
+ Map<LogicalVariable, Integer> varSizeMap = new HashMap<LogicalVariable, Integer>();
+
+ @Override
+ public void setVariableEvalSize(LogicalVariable var, int size) {
+ varSizeMap.put(var, size);
+ }
+
+ @Override
+ public int getVariableEvalSize(LogicalVariable var) {
+ return varSizeMap.get(var);
+ }
+ };
+
+ private Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<ILogicalOperator, IVariableTypeEnvironment>();
+
+ private Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<ILogicalOperator, HashSet<ILogicalOperator>>();
+ private Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<IAlgebraicRewriteRule, HashSet<ILogicalOperator>>();
+ private Map<LogicalVariable, FunctionalDependency> recordToPrimaryKey = new HashMap<LogicalVariable, FunctionalDependency>();
+
+ @SuppressWarnings("unchecked")
+ private IMetadataProvider metadataProvider;
+ private HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<LogicalVariable>();
+
+ protected final Map<ILogicalOperator, List<FunctionalDependency>> fdGlobalMap = new HashMap<ILogicalOperator, List<FunctionalDependency>>();
+ protected final Map<ILogicalOperator, Map<LogicalVariable, EquivalenceClass>> eqClassGlobalMap = new HashMap<ILogicalOperator, Map<LogicalVariable, EquivalenceClass>>();
+
+ protected final Map<ILogicalOperator, ILogicalPropertiesVector> logicalProps = new HashMap<ILogicalOperator, ILogicalPropertiesVector>();
+ private final int frameSize;
+ private final IExpressionTypeComputer expressionTypeComputer;
+ private final INullableTypeComputer nullableTypeComputer;
+
+ public AlgebricksOptimizationContext(int varCounter, int frameSize,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
+ IExpressionTypeComputer expressionTypeComputer, INullableTypeComputer nullableTypeComputer,
+ PhysicalOptimizationConfig physicalOptimizationConfig) {
+ this.varCounter = varCounter;
+ this.frameSize = frameSize;
+ this.expressionEvalSizeComputer = expressionEvalSizeComputer;
+ this.mergeAggregationExpressionFactory = mergeAggregationExpressionFactory;
+ this.expressionTypeComputer = expressionTypeComputer;
+ this.nullableTypeComputer = nullableTypeComputer;
+ this.physicalOptimizationConfig = physicalOptimizationConfig;
+ }
+
+ public int getVarCounter() {
+ return varCounter;
+ }
+
+ public void setVarCounter(int varCounter) {
+ this.varCounter = varCounter;
+ }
+
+ public LogicalVariable newVar() {
+ varCounter++;
+ LogicalVariable var = new LogicalVariable(varCounter);
+ return var;
+ }
+
+ @SuppressWarnings("unchecked")
+ public IMetadataProvider getMetadataProvider() {
+ return metadataProvider;
+ }
+
+ public void setMetadataDeclarations(IMetadataProvider<?, ?> metadataProvider) {
+ this.metadataProvider = metadataProvider;
+ }
+
+ public boolean checkIfInDontApplySet(IAlgebraicRewriteRule rule, ILogicalOperator op) {
+ HashSet<ILogicalOperator> operators = dontApply.get(rule);
+ if (operators == null) {
+ return false;
+ } else {
+ return operators.contains(op);
+ }
+ }
+
+ public void addToDontApplySet(IAlgebraicRewriteRule rule, ILogicalOperator op) {
+ HashSet<ILogicalOperator> operators = dontApply.get(rule);
+ if (operators == null) {
+ HashSet<ILogicalOperator> os = new HashSet<ILogicalOperator>();
+ os.add(op);
+ dontApply.put(rule, os);
+ } else {
+ operators.add(op);
+ }
+
+ }
+
+ /*
+ * returns true if op1 and op2 have already been compared
+ */
+ public boolean checkAndAddToAlreadyCompared(ILogicalOperator op1, ILogicalOperator op2) {
+ HashSet<ILogicalOperator> ops = alreadyCompared.get(op1);
+ if (ops == null) {
+ HashSet<ILogicalOperator> newEntry = new HashSet<ILogicalOperator>();
+ newEntry.add(op2);
+ alreadyCompared.put(op1, newEntry);
+ return false;
+ } else {
+ if (ops.contains(op2)) {
+ return true;
+ } else {
+ ops.add(op2);
+ return false;
+ }
+ }
+ }
+
+ public void addNotToBeInlinedVar(LogicalVariable var) {
+ notToBeInlinedVars.add(var);
+ }
+
+ public boolean shouldNotBeInlined(LogicalVariable var) {
+ return notToBeInlinedVars.contains(var);
+ }
+
+ public void addPrimaryKey(FunctionalDependency pk) {
+ assert (pk.getTail().size() == 1);
+ LogicalVariable recordVar = pk.getTail().get(0);
+ recordToPrimaryKey.put(recordVar, pk);
+ }
+
+ public List<LogicalVariable> findPrimaryKey(LogicalVariable recordVar) {
+ FunctionalDependency fd = recordToPrimaryKey.get(recordVar);
+ if (fd == null) {
+ return null;
+ }
+ return fd.getHead();
+ }
+
+ @Override
+ public Map<LogicalVariable, EquivalenceClass> getEquivalenceClassMap(ILogicalOperator op) {
+ return eqClassGlobalMap.get(op);
+ }
+
+ @Override
+ public List<FunctionalDependency> getFDList(ILogicalOperator op) {
+ return fdGlobalMap.get(op);
+ }
+
+ @Override
+ public void putEquivalenceClassMap(ILogicalOperator op, Map<LogicalVariable, EquivalenceClass> eqClassMap) {
+ this.eqClassGlobalMap.put(op, eqClassMap);
+ }
+
+ @Override
+ public void putFDList(ILogicalOperator op, List<FunctionalDependency> fdList) {
+ this.fdGlobalMap.put(op, fdList);
+ }
+
+ @Override
+ public ILogicalPropertiesVector getLogicalPropertiesVector(ILogicalOperator op) {
+ return logicalProps.get(op);
+ }
+
+ @Override
+ public void putLogicalPropertiesVector(ILogicalOperator op, ILogicalPropertiesVector v) {
+ logicalProps.put(op, v);
+ }
+
+ @Override
+ public IExpressionEvalSizeComputer getExpressionEvalSizeComputer() {
+ return expressionEvalSizeComputer;
+ }
+
+ @Override
+ public IVariableEvalSizeEnvironment getVariableEvalSizeEnvironment() {
+ return varEvalSizeEnv;
+ }
+
+ @Override
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ public IMergeAggregationExpressionFactory getMergeAggregationExpressionFactory() {
+ return mergeAggregationExpressionFactory;
+ }
+
+ public PhysicalOptimizationConfig getPhysicalOptimizationConfig() {
+ return physicalOptimizationConfig;
+ }
+
+ @Override
+ public IVariableTypeEnvironment getOutputTypeEnvironment(ILogicalOperator op) {
+ return typeEnvMap.get(op);
+ }
+
+ @Override
+ public void setOutputTypeEnvironment(ILogicalOperator op, IVariableTypeEnvironment env) {
+ typeEnvMap.put(op, env);
+ }
+
+ @Override
+ public IExpressionTypeComputer getExpressionTypeComputer() {
+ return expressionTypeComputer;
+ }
+
+ @Override
+ public INullableTypeComputer getNullableTypeComputer() {
+ return nullableTypeComputer;
+ }
+
+ @Override
+ public void invalidateTypeEnvironmentForOperator(ILogicalOperator op) {
+ typeEnvMap.put(op, null);
+ }
+
+ @Override
+ public void computeAndSetTypeEnvironmentForOperator(ILogicalOperator op) throws AlgebricksException {
+ setOutputTypeEnvironment(op, op.computeOutputTypeEnvironment(this));
+ }
+
+ @Override
+ public void updatePrimaryKeys(Map<LogicalVariable, LogicalVariable> mappedVars) {
+ for (Map.Entry<LogicalVariable, FunctionalDependency> me : recordToPrimaryKey.entrySet()) {
+ FunctionalDependency fd = me.getValue();
+ List<LogicalVariable> hd = new ArrayList<LogicalVariable>();
+ for (LogicalVariable v : fd.getHead()) {
+ LogicalVariable v2 = mappedVars.get(v);
+ if (v2 == null) {
+ hd.add(v);
+ } else {
+ hd.add(v2);
+ }
+ }
+ List<LogicalVariable> tl = new ArrayList<LogicalVariable>();
+ for (LogicalVariable v : fd.getTail()) {
+ LogicalVariable v2 = mappedVars.get(v);
+ if (v2 == null) {
+ tl.add(v);
+ } else {
+ tl.add(v2);
+ }
+ }
+ me.setValue(new FunctionalDependency(hd, tl));
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
new file mode 100644
index 0000000..b8b1066
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
@@ -0,0 +1,111 @@
+package edu.uci.ics.hyracks.algebricks.core.rewriter.base;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class HeuristicOptimizer {
+
+ public static PhysicalOperatorTag[] hyraxOperators = new PhysicalOperatorTag[] {
+ PhysicalOperatorTag.DATASOURCE_SCAN, PhysicalOperatorTag.BTREE_SEARCH,
+ PhysicalOperatorTag.EXTERNAL_GROUP_BY, PhysicalOperatorTag.HASH_GROUP_BY, PhysicalOperatorTag.HDFS_READER,
+ PhysicalOperatorTag.HYBRID_HASH_JOIN, PhysicalOperatorTag.IN_MEMORY_HASH_JOIN,
+ PhysicalOperatorTag.NESTED_LOOP, PhysicalOperatorTag.PRE_SORTED_DISTINCT_BY,
+ PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY, PhysicalOperatorTag.SPLIT, PhysicalOperatorTag.STABLE_SORT,
+ PhysicalOperatorTag.UNION_ALL };
+ public static PhysicalOperatorTag[] hyraxOperatorsBelowWhichJobGenIsDisabled = new PhysicalOperatorTag[] {};
+
+ public static boolean isHyraxOp(PhysicalOperatorTag opTag) {
+ for (PhysicalOperatorTag t : hyraxOperators) {
+ if (t == opTag) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private IOptimizationContext context;
+ private List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
+ private List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+ private ILogicalPlan plan;
+ private LogicalOperatorPrettyPrintVisitor ppvisitor = new LogicalOperatorPrettyPrintVisitor();
+
+ public HeuristicOptimizer(ILogicalPlan plan,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites,
+ IOptimizationContext context) {
+ this.plan = plan;
+ this.context = context;
+ this.logicalRewrites = logicalRewrites;
+ this.physicalRewrites = physicalRewrites;
+ }
+
+ public void optimize() throws AlgebricksException {
+ if (plan == null) {
+ return;
+ }
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Starting logical optimizations.\n");
+ }
+
+ StringBuilder sb = new StringBuilder();
+ PlanPrettyPrinter.printPlan(plan, sb, ppvisitor, 0);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Logical Plan:\n" + sb.toString());
+ runOptimizationSets(plan, logicalRewrites);
+ computeSchemaBottomUpForPlan(plan);
+ runPhysicalOptimizations(plan, physicalRewrites);
+ StringBuilder sb2 = new StringBuilder();
+ PlanPrettyPrinter.printPlan(plan, sb2, ppvisitor, 0);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Optimized Plan:\n" + sb2.toString());
+ }
+
+ private void runOptimizationSets(ILogicalPlan plan,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> optimSet) throws AlgebricksException {
+ for (Pair<AbstractRuleController, List<IAlgebraicRewriteRule>> ruleList : optimSet) {
+ for (LogicalOperatorReference r : plan.getRoots()) {
+ ruleList.first.setContext(context);
+ ruleList.first.rewriteWithRuleCollection(r, ruleList.second);
+ }
+ }
+ }
+
+ private static void computeSchemaBottomUpForPlan(ILogicalPlan p) throws AlgebricksException {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ computeSchemaBottomUpForOp((AbstractLogicalOperator) r.getOperator());
+ }
+ }
+
+ private static void computeSchemaBottomUpForOp(AbstractLogicalOperator op) throws AlgebricksException {
+ for (LogicalOperatorReference i : op.getInputs()) {
+ computeSchemaBottomUpForOp((AbstractLogicalOperator) i.getOperator());
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans a = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : a.getNestedPlans()) {
+ computeSchemaBottomUpForPlan(p);
+ }
+ }
+ op.recomputeSchema();
+ }
+
+ private void runPhysicalOptimizations(ILogicalPlan plan,
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites)
+ throws AlgebricksException {
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Starting physical optimizations.\n");
+ }
+ // PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(plan);
+ runOptimizationSets(plan, physicalRewrites);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/IAlgebraicRewriteRule.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/IAlgebraicRewriteRule.java
new file mode 100644
index 0000000..c9aeaaa
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/IAlgebraicRewriteRule.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.rewriter.base;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public interface IAlgebraicRewriteRule {
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException;
+
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException;
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
new file mode 100644
index 0000000..1373fa7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.rewriter.base;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionEvalSizeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.INullableTypeComputer;
+
+public interface IOptimizationContextFactory {
+ public IOptimizationContext createOptimizationContext(int varCounter, int frameSize,
+ IExpressionEvalSizeComputer expressionEvalSizeComputer,
+ IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
+ IExpressionTypeComputer expressionTypeComputer, INullableTypeComputer nullableTypeComputer,
+ PhysicalOptimizationConfig physicalOptimizationConfig);
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
new file mode 100644
index 0000000..9ce910b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -0,0 +1,91 @@
+package edu.uci.ics.hyracks.algebricks.core.rewriter.base;
+
+import java.util.Properties;
+
+public class PhysicalOptimizationConfig {
+ private static final int MB = 1048576;
+ private static final String FRAMESIZE = "FRAMESIZE";
+ private static final String MAX_FRAMES_EXTERNAL_SORT = "MAX_FRAMES_EXTERNAL_SORT";
+ private static final String MAX_FRAMES_EXTERNAL_GROUP_BY = "MAX_FRAMES_EXTERNAL_GROUP_BY";
+
+ private static final String DEFAULT_HASH_GROUP_TABLE_SIZE = "DEFAULT_HASH_GROUP_TABLE_SIZE";
+ private static final String DEFAULT_EXTERNAL_GROUP_TABLE_SIZE = "DEFAULT_EXTERNAL_GROUP_TABLE_SIZE";
+ private static final String DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE = "DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE";
+
+ private Properties properties = new Properties();
+
+ public PhysicalOptimizationConfig() {
+ int frameSize = 32768;
+ setInt(FRAMESIZE, frameSize);
+ setInt(MAX_FRAMES_EXTERNAL_SORT, (int) (((long) 512 * MB) / frameSize));
+ setInt(MAX_FRAMES_EXTERNAL_GROUP_BY, (int) (((long) 256 * MB) / frameSize));
+
+ // use http://www.rsok.com/~jrm/printprimes.html to find prime numbers
+ setInt(DEFAULT_HASH_GROUP_TABLE_SIZE, 10485767);
+ setInt(DEFAULT_EXTERNAL_GROUP_TABLE_SIZE, 10485767);
+ setInt(DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE, 10485767);
+ }
+
+ public int getFrameSize() {
+ return getInt(FRAMESIZE, 32768);
+ }
+
+ public void setFrameSize(int frameSize) {
+ setInt(FRAMESIZE, frameSize);
+ }
+
+ public int getMaxFramesExternalSort() {
+ int frameSize = getFrameSize();
+ return getInt(MAX_FRAMES_EXTERNAL_SORT, (int) (((long) 512 * MB) / frameSize));
+ }
+
+ public void setMaxFramesExternalSort(int frameLimit) {
+ setInt(MAX_FRAMES_EXTERNAL_SORT, frameLimit);
+ }
+
+ public int getMaxFramesExternalGroupBy() {
+ int frameSize = getFrameSize();
+ return getInt(MAX_FRAMES_EXTERNAL_GROUP_BY, (int) (((long) 256 * MB) / frameSize));
+ }
+
+ public void setMaxFramesExternalGroupBy(int frameLimit) {
+ setInt(MAX_FRAMES_EXTERNAL_GROUP_BY, frameLimit);
+ }
+
+ public int getHashGroupByTableSize() {
+ return getInt(DEFAULT_HASH_GROUP_TABLE_SIZE, 10485767);
+ }
+
+ public void setHashGroupByTableSize(int tableSize) {
+ setInt(DEFAULT_HASH_GROUP_TABLE_SIZE, tableSize);
+ }
+
+ public int getExternalGroupByTableSize() {
+ return getInt(DEFAULT_EXTERNAL_GROUP_TABLE_SIZE, 10485767);
+ }
+
+ public void setExternalGroupByTableSize(int tableSize) {
+ setInt(DEFAULT_EXTERNAL_GROUP_TABLE_SIZE, tableSize);
+ }
+
+ public int getInMemHashJoinTableSize() {
+ return getInt(DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE, 10485767);
+ }
+
+ public void setInMemHashJoinTableSize(int tableSize) {
+ setInt(DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE, tableSize);
+ }
+
+ private void setInt(String property, int value) {
+ properties.setProperty(property, Integer.toString(value));
+ }
+
+ private int getInt(String property, int defaultValue) {
+ String value = properties.getProperty(property);
+ if (value == null)
+ return defaultValue;
+ else
+ return Integer.parseInt(value);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Pair.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Pair.java
new file mode 100644
index 0000000..d3aff9f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Pair.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.utils;
+
+import java.io.Serializable;
+
+public class Pair<T1, T2> implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ public T1 first;
+ public T2 second;
+
+ public Pair(T1 first, T2 second) {
+ this.first = first;
+ this.second = second;
+ }
+
+ @Override
+ public String toString() {
+ return first + "," + second;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof Pair<?, ?>)) {
+ return false;
+ } else {
+ Pair<?, ?> p = (Pair<?, ?>) obj;
+ return this.first.equals(p.first) && this.second.equals(p.second);
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return first.hashCode() * 31 + second.hashCode();
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Substitution.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Substitution.java
new file mode 100644
index 0000000..c52cdff
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Substitution.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.utils;
+
+public class Substitution<T> {
+ public T substituted;
+ public T substitutedWith;
+
+ public Substitution(T substituted, T substitutedWith) {
+ this.substituted = substituted;
+ this.substitutedWith = substitutedWith;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Triple.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Triple.java
new file mode 100644
index 0000000..ad76ef1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/Triple.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.core.utils;
+
+public class Triple<T1, T2, T3> {
+ public T1 first;
+ public T2 second;
+ public T3 third;
+
+ public Triple(T1 first, T2 second, T3 third) {
+ this.first = first;
+ this.second = second;
+ this.third = third;
+ }
+
+ @Override
+ public String toString() {
+ return first + "," + second + ", " + third;
+ }
+
+ @Override
+ public int hashCode() {
+ return first.hashCode() * 31 + second.hashCode() * 15 + third.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof Triple<?, ?, ?>))
+ return false;
+ Triple<?, ?, ?> triple = (Triple<?, ?, ?>) o;
+ return first.equals(triple.first) && second.equals(triple.second) && third.equals(triple.third);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/WriteValueTools.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/WriteValueTools.java
new file mode 100644
index 0000000..9780ca6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/WriteValueTools.java
@@ -0,0 +1,106 @@
+package edu.uci.ics.hyracks.algebricks.core.utils;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+
+public final class WriteValueTools {
+
+ private final static int[] INT_INTERVALS = { 9, 99, 999, 9999, 99999, 999999, 9999999, 99999999, 999999999,
+ Integer.MAX_VALUE };
+ private final static int[] INT_DIVIDERS = { 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000 };
+ private final static int[] DIGITS = { '0', '1', '2', '3', '4', '5', '6', '7', '8', '9' };
+
+ public static void writeInt(int i, OutputStream os) throws IOException {
+ if (i < 0) {
+ if (i == Integer.MIN_VALUE) {
+ os.write("-2147483648".getBytes());
+ return;
+ }
+ os.write('-');
+ i = -i;
+ }
+ int k = 0;
+ for (; k < INT_INTERVALS.length; k++) {
+ if (i <= INT_INTERVALS[k]) {
+ break;
+ }
+ }
+ while (k > 0) {
+ int q = i / INT_DIVIDERS[k - 1];
+ os.write(DIGITS[q % 10]);
+ k--;
+ }
+ // now, print the units
+ os.write(DIGITS[i % 10]);
+ }
+
+ public static void writeLong(long d, OutputStream os) throws IOException {
+ // now, print the units
+ if (d < 0) {
+ if (d == Long.MIN_VALUE) {
+ os.write("-9223372036854775808".getBytes());
+ return;
+ }
+ os.write('-');
+ d = -d;
+ }
+ long divisor = 1000000000000000000L;
+ while (divisor > d) {
+ divisor = divisor / 10;
+ }
+ while (divisor > 1) {
+ os.write(DIGITS[(int) ((d / divisor) % 10)]);
+ divisor = divisor / 10;
+ }
+ os.write(DIGITS[(int) (d % 10)]);
+ }
+
+ public static void writeUTF8String(byte[] b, int s, int l, OutputStream os) throws IOException {
+ int stringLength = StringUtils.getUTFLen(b, s);
+ int position = s + 2;
+ int maxPosition = position + stringLength;
+ os.write('\"');
+ while (position < maxPosition) {
+ char c = StringUtils.charAt(b, position);
+ switch (c) {
+ // escape
+ case '\\':
+ case '"':
+ os.write('\\');
+ break;
+ }
+ int sz = StringUtils.charSize(b, position);
+ while (sz > 0) {
+ os.write(b[position]);
+ position++;
+ sz--;
+ }
+ }
+ os.write('\"');
+ }
+
+ public static void writeUTF8StringNoQuotes(byte[] b, int s, int l, OutputStream os) throws IOException {
+ int stringLength = StringUtils.getUTFLen(b, s);
+ int position = s + 2;
+ int maxPosition = position + stringLength;
+ while (position < maxPosition) {
+ char c = StringUtils.charAt(b, position);
+ switch (c) {
+ // escape
+ case '\\':
+ case '"':
+ os.write('\\');
+ break;
+ }
+ int sz = StringUtils.charSize(b, position);
+ while (sz > 0) {
+ os.write(b[position]);
+ position++;
+ sz--;
+ }
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/pom.xml b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/pom.xml
new file mode 100644
index 0000000..92c4f5f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/pom.xml
@@ -0,0 +1,65 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>piglet-example</artifactId>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks-examples</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>javacc-maven-plugin</artifactId>
+ <version>2.6</version>
+ <executions>
+ <execution>
+ <id>javacc</id>
+ <goals>
+ <goal>javacc</goal>
+ </goals>
+ <configuration>
+ <isStatic>false</isStatic>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.7.2</version>
+ <configuration>
+ <includes>
+ <include>**/*TestSuite.java</include>
+ <include>**/*Test.java</include>
+ </includes>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks-compiler</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>4.8.2</version>
+ <type>jar</type>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ASTNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ASTNode.java
new file mode 100644
index 0000000..d94fb9e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ASTNode.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public abstract class ASTNode {
+ public enum Tag {
+ ASSIGNMENT,
+ DUMP,
+ LOAD,
+ FILTER,
+
+ SCALAR_FUNCTION,
+ LITERAL,
+ FIELD_ACCESS,
+ }
+
+ public abstract Tag getTag();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/AssignmentNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/AssignmentNode.java
new file mode 100644
index 0000000..7fa8c6c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/AssignmentNode.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public class AssignmentNode extends ASTNode {
+ private String alias;
+
+ private RelationNode relation;
+
+ public AssignmentNode(String alias, RelationNode relation) {
+ this.alias = alias;
+ this.relation = relation;
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.ASSIGNMENT;
+ }
+
+ public String getAlias() {
+ return alias;
+ }
+
+ public RelationNode getRelation() {
+ return relation;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/DumpNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/DumpNode.java
new file mode 100644
index 0000000..5e8bc76
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/DumpNode.java
@@ -0,0 +1,24 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public class DumpNode extends RelationNode {
+ private final String file;
+ private final String alias;
+
+ public DumpNode(String file, String alias) {
+ this.file = file;
+ this.alias = alias;
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.DUMP;
+ }
+
+ public String getFile() {
+ return file;
+ }
+
+ public String getAlias() {
+ return alias;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ExpressionNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ExpressionNode.java
new file mode 100644
index 0000000..3f68ae6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ExpressionNode.java
@@ -0,0 +1,4 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public abstract class ExpressionNode extends ASTNode {
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FieldAccessExpressionNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FieldAccessExpressionNode.java
new file mode 100644
index 0000000..69c9262
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FieldAccessExpressionNode.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public class FieldAccessExpressionNode extends ExpressionNode {
+ private String relationName;
+
+ private String fieldName;
+
+ public FieldAccessExpressionNode(String relationName, String fieldName) {
+ this.relationName = relationName;
+ this.fieldName = fieldName;
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.FIELD_ACCESS;
+ }
+
+ public String getRelationName() {
+ return relationName;
+ }
+
+ public String getFieldName() {
+ return fieldName;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FilterNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FilterNode.java
new file mode 100644
index 0000000..bc90043
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FilterNode.java
@@ -0,0 +1,25 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public class FilterNode extends RelationNode {
+ private String alias;
+
+ private ExpressionNode expression;
+
+ public FilterNode(String alias, ExpressionNode expression) {
+ this.alias = alias;
+ this.expression = expression;
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.FILTER;
+ }
+
+ public String getAlias() {
+ return alias;
+ }
+
+ public ExpressionNode getExpression() {
+ return expression;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FunctionTag.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FunctionTag.java
new file mode 100644
index 0000000..2ab397a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/FunctionTag.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public enum FunctionTag {
+ BOOLEAN_AND,
+ BOOLEAN_OR,
+ BOOLEAN_NOT,
+
+ EQ,
+ NEQ,
+ LT,
+ LTE,
+ GT,
+ GTE,
+
+ ADD,
+ SUBTRACT,
+ MULTIPLY,
+ DIVIDE,
+ MOD,
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/LiteralExpressionNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/LiteralExpressionNode.java
new file mode 100644
index 0000000..547d487
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/LiteralExpressionNode.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Type;
+
+public class LiteralExpressionNode extends ExpressionNode {
+ private String image;
+
+ private Type type;
+
+ public LiteralExpressionNode(String image, Type type) {
+ this.image = image;
+ this.type = type;
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.LITERAL;
+ }
+
+ public String getImage() {
+ return image;
+ }
+
+ public Type getType() {
+ return type;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/LoadNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/LoadNode.java
new file mode 100644
index 0000000..e5da760
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/LoadNode.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Schema;
+
+public class LoadNode extends RelationNode {
+ private String dataFile;
+
+ private Schema schema;
+
+ public LoadNode(String dataFile, Schema schema) {
+ this.dataFile = dataFile;
+ this.schema = schema;
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.LOAD;
+ }
+
+ public String getDataFile() {
+ return dataFile;
+ }
+
+ public Schema getSchema() {
+ return schema;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/RelationNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/RelationNode.java
new file mode 100644
index 0000000..51e275f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/RelationNode.java
@@ -0,0 +1,4 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+public abstract class RelationNode extends ASTNode {
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ScalarFunctionExpressionNode.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ScalarFunctionExpressionNode.java
new file mode 100644
index 0000000..9b55ae9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/ast/ScalarFunctionExpressionNode.java
@@ -0,0 +1,34 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.ast;
+
+import java.util.List;
+
+public class ScalarFunctionExpressionNode extends ExpressionNode {
+ private FunctionTag fTag;
+
+ private String fName;
+
+ private List<ASTNode> arguments;
+
+ public ScalarFunctionExpressionNode(FunctionTag fTag, String fName, List<ASTNode> arguments) {
+ this.fTag = fTag;
+ this.fName = fName;
+ this.arguments = arguments;
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.SCALAR_FUNCTION;
+ }
+
+ public FunctionTag getFunctionTag() {
+ return fTag;
+ }
+
+ public String getFunctionName() {
+ return fName;
+ }
+
+ public List<ASTNode> getArguments() {
+ return arguments;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/ConstantValue.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/ConstantValue.java
new file mode 100644
index 0000000..e6226f4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/ConstantValue.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.compiler;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Type;
+
+public final class ConstantValue implements IAlgebricksConstantValue {
+ private final Type type;
+
+ private final String image;
+
+ public ConstantValue(Type type, String image) {
+ this.type = type;
+ this.image = image;
+ }
+
+ public Type getType() {
+ return type;
+ }
+
+ public String getImage() {
+ return image;
+ }
+
+ @Override
+ public boolean isFalse() {
+ return false;
+ }
+
+ @Override
+ public boolean isNull() {
+ return false;
+ }
+
+ @Override
+ public boolean isTrue() {
+ return false;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
new file mode 100644
index 0000000..cc5f27b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
@@ -0,0 +1,351 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.compiler;
+
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.algebricks.compiler.api.HeuristicCompilerFactoryBuilder;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompiler;
+import edu.uci.ics.hyracks.algebricks.compiler.api.ICompilerFactory;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
+import edu.uci.ics.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.ASTNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.AssignmentNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.DumpNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.ExpressionNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.FieldAccessExpressionNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.FilterNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.FunctionTag;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.LiteralExpressionNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.LoadNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.RelationNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.ScalarFunctionExpressionNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.exceptions.PigletException;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.metadata.PigletFileDataSink;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.metadata.PigletFileDataSource;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.metadata.PigletMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.parser.ParseException;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.parser.PigletParser;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.rewriter.PigletRewriteRuleset;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.runtime.PigletExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Schema;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Type;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class PigletCompiler {
+ private static final Logger LOGGER = Logger.getLogger(PigletCompiler.class.getName());
+
+ private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildDefaultLogicalRewrites() {
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultLogicalRewrites = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+ SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+ SequentialFixpointRuleController seqCtrlFullDfs = new SequentialFixpointRuleController(true);
+ SequentialOnceRuleController seqOnceCtrl = new SequentialOnceRuleController(true);
+ defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ PigletRewriteRuleset.buildTypeInferenceRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlFullDfs,
+ PigletRewriteRuleset.buildNormalizationRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ PigletRewriteRuleset.buildCondPushDownRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ PigletRewriteRuleset.buildJoinInferenceRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ PigletRewriteRuleset.buildOpPushDownRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrl,
+ PigletRewriteRuleset.buildDataExchangeRuleCollection()));
+ defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
+ PigletRewriteRuleset.buildConsolidationRuleCollection()));
+ return defaultLogicalRewrites;
+ }
+
+ private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildDefaultPhysicalRewrites() {
+ List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultPhysicalRewrites = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();
+ SequentialOnceRuleController seqOnceCtrlAllLevels = new SequentialOnceRuleController(true);
+ SequentialOnceRuleController seqOnceCtrlTopLevel = new SequentialOnceRuleController(false);
+ defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrlAllLevels,
+ PigletRewriteRuleset.buildPhysicalRewritesAllLevelsRuleCollection()));
+ defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrlTopLevel,
+ PigletRewriteRuleset.buildPhysicalRewritesTopLevelRuleCollection()));
+ defaultPhysicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqOnceCtrlAllLevels,
+ PigletRewriteRuleset.prepareForJobGenRuleCollection()));
+ return defaultPhysicalRewrites;
+ }
+
+ private final ICompilerFactory cFactory;
+
+ private final PigletMetadataProvider metadataProvider;
+
+ private int varCounter;
+
+ private ILogicalOperator previousOp;
+
+ public PigletCompiler() {
+ HeuristicCompilerFactoryBuilder builder = new HeuristicCompilerFactoryBuilder();
+ builder.setLogicalRewrites(buildDefaultLogicalRewrites());
+ builder.setPhysicalRewrites(buildDefaultPhysicalRewrites());
+ builder.setSerializerDeserializerProvider(new ISerializerDeserializerProvider() {
+ @SuppressWarnings("unchecked")
+ @Override
+ public ISerializerDeserializer getSerializerDeserializer(Object type) throws AlgebricksException {
+ return null;
+ }
+ });
+ builder.setPrinterProvider(PigletPrinterFactoryProvider.INSTANCE);
+ builder.setExprJobGen(new PigletExpressionJobGen());
+ builder.setExpressionTypeComputer(new IExpressionTypeComputer() {
+ @Override
+ public Object getType(ILogicalExpression expr, IMetadataProvider<?, ?> metadataProvider,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+ return null;
+ }
+ });
+ cFactory = builder.create();
+ metadataProvider = new PigletMetadataProvider();
+ }
+
+ public List<ASTNode> parse(Reader in) throws ParseException {
+ PigletParser parser = new PigletParser(in);
+ List<ASTNode> statements = parser.Statements();
+ return statements;
+ }
+
+ public JobSpecification compile(List<ASTNode> ast) throws AlgebricksException, PigletException {
+ ILogicalPlan plan = translate(ast);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Translated Plan:");
+ LOGGER.info(getPrettyPrintedPlan(plan));
+ }
+ ICompiler compiler = cFactory.createCompiler(plan, metadataProvider, varCounter);
+ compiler.optimize();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Optimized Plan:");
+ LOGGER.info(getPrettyPrintedPlan(plan));
+ }
+ return compiler.createJob(null);
+ }
+
+ private ILogicalPlan translate(List<ASTNode> ast) throws PigletException {
+ Map<String, Relation> symMap = new HashMap<String, Relation>();
+ List<LogicalOperatorReference> roots = new ArrayList<LogicalOperatorReference>();
+ previousOp = null;
+ for (ASTNode an : ast) {
+ switch (an.getTag()) {
+ case DUMP: {
+ DumpNode dn = (DumpNode) an;
+ Relation input = symMap.get(dn.getAlias());
+ List<LogicalExpressionReference> expressions = new ArrayList<LogicalExpressionReference>();
+ for (LogicalVariable v : input.schema.values()) {
+ expressions.add(new LogicalExpressionReference(new VariableReferenceExpression(v)));
+ }
+ PigletFileDataSink dataSink = new PigletFileDataSink(dn.getFile());
+ ILogicalOperator op = new WriteOperator(expressions, dataSink);
+ op.getInputs().add(new LogicalOperatorReference(input.op));
+ roots.add(new LogicalOperatorReference(op));
+ }
+ break;
+
+ case ASSIGNMENT: {
+ AssignmentNode asn = (AssignmentNode) an;
+ String alias = asn.getAlias();
+ RelationNode rn = asn.getRelation();
+ Relation rel = translate(rn, symMap);
+ previousOp = rel.op;
+ rel.alias = alias;
+ symMap.put(alias, rel);
+ }
+ break;
+ }
+ }
+ return new ALogicalPlanImpl(roots);
+ }
+
+ private Relation translate(RelationNode rn, Map<String, Relation> symMap) throws PigletException {
+ switch (rn.getTag()) {
+ case LOAD: {
+ LoadNode ln = (LoadNode) rn;
+ String file = ln.getDataFile();
+ Schema schema = ln.getSchema();
+ List<Pair<String, Type>> fieldsSchema = schema.getSchema();
+ List<LogicalVariable> variables = new ArrayList<LogicalVariable>();
+ List<Object> types = new ArrayList<Object>();
+ Relation rel = new Relation();
+ for (Pair<String, Type> p : fieldsSchema) {
+ LogicalVariable v = newVariable();
+ rel.schema.put(p.first, v);
+ variables.add(v);
+ types.add(p.second);
+ }
+ PigletFileDataSource ds = new PigletFileDataSource(file, types.toArray());
+ rel.op = new DataSourceScanOperator(variables, ds);
+ rel.op.getInputs().add(
+ new LogicalOperatorReference(previousOp == null ? new EmptyTupleSourceOperator() : previousOp));
+ return rel;
+ }
+
+ case FILTER: {
+ FilterNode fn = (FilterNode) rn;
+ String alias = fn.getAlias();
+ ExpressionNode conditionNode = fn.getExpression();
+ Relation inputRel = findInputRelation(alias, symMap);
+ Pair<Relation, LogicalVariable> tempInput = translateScalarExpression(inputRel, conditionNode);
+ Relation rel = new Relation();
+ rel.op = new SelectOperator(new LogicalExpressionReference(new VariableReferenceExpression(
+ tempInput.second)));
+ rel.op.getInputs().add(new LogicalOperatorReference(tempInput.first.op));
+ rel.schema.putAll(tempInput.first.schema);
+ return rel;
+ }
+ }
+ throw new IllegalArgumentException("Unknown node: " + rn.getTag() + " encountered");
+ }
+
+ private Pair<Relation, LogicalVariable> translateScalarExpression(Relation inputRel, ExpressionNode expressionNode)
+ throws PigletException {
+ switch (expressionNode.getTag()) {
+ case FIELD_ACCESS: {
+ FieldAccessExpressionNode faen = (FieldAccessExpressionNode) expressionNode;
+ String fieldName = faen.getFieldName();
+ LogicalVariable lVar = findField(fieldName, inputRel.schema);
+ return new Pair<Relation, LogicalVariable>(inputRel, lVar);
+ }
+
+ case LITERAL: {
+ LiteralExpressionNode len = (LiteralExpressionNode) expressionNode;
+ String image = len.getImage();
+ Type type = len.getType();
+ ConstantExpression ce = new ConstantExpression(new ConstantValue(type, image));
+ Relation rel = new Relation();
+ LogicalVariable var = newVariable();
+ List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+ vars.add(var);
+
+ List<LogicalExpressionReference> exprs = new ArrayList<LogicalExpressionReference>();
+ exprs.add(new LogicalExpressionReference(ce));
+
+ rel.op = new AssignOperator(vars, exprs);
+ rel.op.getInputs().add(new LogicalOperatorReference(inputRel.op));
+ rel.schema.putAll(inputRel.schema);
+
+ return new Pair<Relation, LogicalVariable>(rel, var);
+ }
+
+ case SCALAR_FUNCTION: {
+ ScalarFunctionExpressionNode sfen = (ScalarFunctionExpressionNode) expressionNode;
+ List<LogicalExpressionReference> argExprs = new ArrayList<LogicalExpressionReference>();
+ List<ASTNode> arguments = sfen.getArguments();
+ Relation rel = inputRel;
+ for (ASTNode a : arguments) {
+ Pair<Relation, LogicalVariable> argPair = translateScalarExpression(rel, (ExpressionNode) a);
+ rel = argPair.first;
+ argExprs.add(new LogicalExpressionReference(new VariableReferenceExpression(argPair.second)));
+ }
+ Relation outRel = new Relation();
+ outRel.schema.putAll(rel.schema);
+ LogicalVariable var = newVariable();
+ List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+ vars.add(var);
+
+ IFunctionInfo fInfo = lookupFunction(sfen.getFunctionTag(), sfen.getFunctionName());
+
+ List<LogicalExpressionReference> exprs = new ArrayList<LogicalExpressionReference>();
+ exprs.add(new LogicalExpressionReference(new ScalarFunctionCallExpression(fInfo, argExprs)));
+ outRel.op = new AssignOperator(vars, exprs);
+ outRel.op.getInputs().add(new LogicalOperatorReference(rel.op));
+ return new Pair<Relation, LogicalVariable>(outRel, var);
+ }
+ }
+ return null;
+ }
+
+ private IFunctionInfo lookupFunction(FunctionTag functionTag, String functionName) throws PigletException {
+ switch (functionTag) {
+ case EQ:
+ return AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.EQ);
+
+ case NEQ:
+ return AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.NEQ);
+
+ case LT:
+ return AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.LT);
+
+ case LTE:
+ return AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.LE);
+
+ case GT:
+ return AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.GT);
+
+ case GTE:
+ return AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.GE);
+ }
+ throw new PigletException("Unsupported function: " + functionTag);
+ }
+
+ private LogicalVariable newVariable() {
+ return new LogicalVariable(varCounter++);
+ }
+
+ private LogicalVariable findField(String fieldName, Map<String, LogicalVariable> schema) throws PigletException {
+ LogicalVariable var = schema.get(fieldName);
+ if (var == null) {
+ throw new PigletException("Unable to find field named: " + fieldName);
+ }
+ return var;
+ }
+
+ private Relation findInputRelation(String alias, Map<String, Relation> symMap) throws PigletException {
+ Relation rel = symMap.get(alias);
+ if (rel == null) {
+ throw new PigletException("Unknown alias " + alias + "referenced");
+ }
+ return rel;
+ }
+
+ private static class Relation {
+ String alias;
+ ILogicalOperator op;
+ final Map<String, LogicalVariable> schema;
+
+ public Relation() {
+ schema = new LinkedHashMap<String, LogicalVariable>();
+ }
+ }
+
+ private String getPrettyPrintedPlan(ILogicalPlan plan) throws AlgebricksException {
+ LogicalOperatorPrettyPrintVisitor v = new LogicalOperatorPrettyPrintVisitor();
+ StringBuilder buffer = new StringBuilder();
+ PlanPrettyPrinter.printPlan(plan, buffer, v, 0);
+ return buffer.toString();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletPrinterFactoryProvider.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletPrinterFactoryProvider.java
new file mode 100644
index 0000000..cb12b7d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletPrinterFactoryProvider.java
@@ -0,0 +1,90 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.compiler;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data.IntegerPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.WriteValueTools;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Type;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
+
+public class PigletPrinterFactoryProvider implements IPrinterFactoryProvider {
+
+ public static final PigletPrinterFactoryProvider INSTANCE = new PigletPrinterFactoryProvider();
+
+ private PigletPrinterFactoryProvider() {
+ }
+
+ @Override
+ public IPrinterFactory getPrinterFactory(Object type) throws AlgebricksException {
+ Type t = (Type) type;
+ switch (t.getTag()) {
+ case INTEGER:
+ return IntegerPrinterFactory.INSTANCE;
+ case CHAR_ARRAY:
+ return CharArrayPrinterFactory.INSTANCE;
+ case FLOAT:
+ return FloatPrinterFactory.INSTANCE;
+ default:
+ throw new UnsupportedOperationException();
+
+ }
+ }
+
+ public static class CharArrayPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final CharArrayPrinterFactory INSTANCE = new CharArrayPrinterFactory();
+
+ private CharArrayPrinterFactory() {
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ return new IPrinter() {
+ @Override
+ public void init() throws AlgebricksException {
+ }
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+ try {
+ WriteValueTools.writeUTF8String(b, s, l, ps);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+ };
+ }
+ }
+
+ public static class FloatPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final FloatPrinterFactory INSTANCE = new FloatPrinterFactory();
+
+ private FloatPrinterFactory() {
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ return new IPrinter() {
+ @Override
+ public void init() throws AlgebricksException {
+ }
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+ ps.print(FloatSerializerDeserializer.getFloat(b, s));
+ }
+ };
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/exceptions/PigletException.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/exceptions/PigletException.java
new file mode 100644
index 0000000..d196b70
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/exceptions/PigletException.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.exceptions;
+
+public class PigletException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public PigletException(String message) {
+ super(message);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/FileSplitUtils.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/FileSplitUtils.java
new file mode 100644
index 0000000..d0f9b44
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/FileSplitUtils.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.metadata;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
+public class FileSplitUtils {
+ public static FileSplit[] parseFileSplits(String fileSplits) {
+ String[] splits = fileSplits.split(",");
+ FileSplit[] fSplits = new FileSplit[splits.length];
+ for (int i = 0; i < splits.length; ++i) {
+ String s = splits[i].trim();
+ int idx = s.indexOf(':');
+ if (idx < 0) {
+ throw new IllegalArgumentException("File split " + s + " not well formed");
+ }
+ fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
+ }
+ return fSplits;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletFileDataSink.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletFileDataSink.java
new file mode 100644
index 0000000..93f9cae
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletFileDataSink.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.metadata;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FileSplitDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
+public class PigletFileDataSink implements IDataSink {
+ private String file;
+
+ private FileSplit[] fileSplits;
+
+ private IPartitioningProperty partProp;
+
+ public PigletFileDataSink(String file) {
+ this.file = file;
+ fileSplits = FileSplitUtils.parseFileSplits(file);
+ partProp = new RandomPartitioningProperty(new FileSplitDomain(fileSplits));
+ }
+
+ @Override
+ public Object getId() {
+ return file;
+ }
+
+ public FileSplit[] getFileSplits() {
+ return fileSplits;
+ }
+
+ @Override
+ public Object[] getSchemaTypes() {
+ return null;
+ }
+
+ @Override
+ public IPartitioningProperty getPartitioningProperty() {
+ return partProp;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletFileDataSource.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletFileDataSource.java
new file mode 100644
index 0000000..6763d45
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletFileDataSource.java
@@ -0,0 +1,62 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.metadata;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FileSplitDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
+public class PigletFileDataSource implements IDataSource<String> {
+ private final String file;
+
+ private final Object[] types;
+
+ private final FileSplit[] fileSplits;
+
+ private IDataSourcePropertiesProvider propProvider;
+
+ public PigletFileDataSource(String file, Object[] types) {
+ this.file = file;
+ this.types = types;
+ fileSplits = FileSplitUtils.parseFileSplits(file);
+ final IPhysicalPropertiesVector vec = new StructuralPropertiesVector(new RandomPartitioningProperty(
+ new FileSplitDomain(fileSplits)), new ArrayList<ILocalStructuralProperty>());
+ propProvider = new IDataSourcePropertiesProvider() {
+ @Override
+ public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables) {
+ return vec;
+ }
+ };
+ }
+
+ @Override
+ public String getId() {
+ return file;
+ }
+
+ @Override
+ public Object[] getSchemaTypes() {
+ return types;
+ }
+
+ public FileSplit[] getFileSplits() {
+ return fileSplits;
+ }
+
+ @Override
+ public IDataSourcePropertiesProvider getPropertiesProvider() {
+ return propProvider;
+ }
+
+ @Override
+ public void computeFDs(List<LogicalVariable> scanVariables, List<FunctionalDependency> fdList) {
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
new file mode 100644
index 0000000..668d7a2
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
@@ -0,0 +1,167 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.metadata;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.SinkWriterRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.writers.PrinterBasedWriterFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksAbsolutePartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Type;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class PigletMetadataProvider implements IMetadataProvider<String, String> {
+ @Override
+ public IDataSource<String> findDataSource(String id) throws AlgebricksException {
+ return null;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<String> dataSource,
+ List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+ JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
+ PigletFileDataSource ds = (PigletFileDataSource) dataSource;
+
+ FileSplit[] fileSplits = ds.getFileSplits();
+ String[] locations = new String[fileSplits.length];
+ for (int i = 0; i < fileSplits.length; ++i) {
+ locations[i] = fileSplits[i].getNodeName();
+ }
+ IFileSplitProvider fsp = new ConstantFileSplitProvider(fileSplits);
+
+ Object[] colTypes = ds.getSchemaTypes();
+ IValueParserFactory[] vpfs = new IValueParserFactory[colTypes.length];
+ ISerializerDeserializer[] serDesers = new ISerializerDeserializer[colTypes.length];
+
+ for (int i = 0; i < colTypes.length; ++i) {
+ Type colType = (Type) colTypes[i];
+ IValueParserFactory vpf;
+ ISerializerDeserializer serDeser;
+ switch (colType.getTag()) {
+ case INTEGER:
+ vpf = IntegerParserFactory.INSTANCE;
+ serDeser = IntegerSerializerDeserializer.INSTANCE;
+ break;
+
+ case CHAR_ARRAY:
+ vpf = UTF8StringParserFactory.INSTANCE;
+ serDeser = UTF8StringSerializerDeserializer.INSTANCE;
+ break;
+
+ case FLOAT:
+ vpf = FloatParserFactory.INSTANCE;
+ serDeser = FloatSerializerDeserializer.INSTANCE;
+ break;
+
+ default:
+ throw new UnsupportedOperationException();
+ }
+ vpfs[i] = vpf;
+ serDesers[i] = serDeser;
+ }
+
+ ITupleParserFactory tpf = new DelimitedDataTupleParserFactory(vpfs, ',');
+ RecordDescriptor rDesc = new RecordDescriptor(serDesers);
+
+ IOperatorDescriptor scanner = new FileScanOperatorDescriptor(jobSpec, fsp, tpf, rDesc);
+ AlgebricksAbsolutePartitionConstraint constraint = new AlgebricksAbsolutePartitionConstraint(locations);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(scanner, constraint);
+ }
+
+ @Override
+ public boolean scannerOperatorIsLeaf(IDataSource<String> dataSource) {
+ return true;
+ }
+
+ @Override
+ public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
+ int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
+ throws AlgebricksException {
+ PigletFileDataSink ds = (PigletFileDataSink) sink;
+ FileSplit[] fileSplits = ds.getFileSplits();
+ String[] locations = new String[fileSplits.length];
+ for (int i = 0; i < fileSplits.length; ++i) {
+ locations[i] = fileSplits[i].getNodeName();
+ }
+ IPushRuntimeFactory prf = new SinkWriterRuntimeFactory(printColumns, printerFactories, fileSplits[0]
+ .getLocalFile().getFile(), PrinterBasedWriterFactory.INSTANCE, inputDesc);
+ AlgebricksAbsolutePartitionConstraint constraint = new AlgebricksAbsolutePartitionConstraint(locations);
+ return new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(prf, constraint);
+ }
+
+ @Override
+ public IDataSourceIndex<String, String> findDataSourceIndex(String indexId, String dataSourceId)
+ throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(
+ IDataSource<String> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
+ LogicalVariable payLoadVar, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<String> dataSource,
+ IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<String> dataSource,
+ IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(
+ IDataSourceIndex<String, String> dataSource, IOperatorSchema propagatedSchema,
+ List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys, RecordDescriptor recordDesc,
+ JobGenContext context, JobSpecification spec) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(
+ IDataSourceIndex<String, String> dataSource, IOperatorSchema propagatedSchema,
+ List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys, RecordDescriptor recordDesc,
+ JobGenContext context, JobSpecification spec) throws AlgebricksException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/rewriter/PigletRewriteRuleset.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/rewriter/PigletRewriteRuleset.java
new file mode 100644
index 0000000..90ee5bb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/rewriter/PigletRewriteRuleset.java
@@ -0,0 +1,116 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.rewriter;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.BreakSelectIntoConjunctsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexJoinInferenceRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractGbyExpressionsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.FactorRedundantGroupAndDecorVarsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InferTypesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineVariablesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByForStandaloneAggregRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushLimitDownRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectDownRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectIntoDataSourceScanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectDownRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectIntoJoinRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ReinferAllTypesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveUnusedAssignAndAggregateRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetAlgebricksPhysicalOperatorsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetExecutionModeRule;
+
+public class PigletRewriteRuleset {
+
+ public final static List<IAlgebraicRewriteRule> buildTypeInferenceRuleCollection() {
+ List<IAlgebraicRewriteRule> typeInfer = new LinkedList<IAlgebraicRewriteRule>();
+ typeInfer.add(new InferTypesRule());
+ return typeInfer;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildNormalizationRuleCollection() {
+ List<IAlgebraicRewriteRule> normalization = new LinkedList<IAlgebraicRewriteRule>();
+ normalization.add(new EliminateSubplanRule());
+ normalization.add(new IntroduceGroupByForStandaloneAggregRule());
+ normalization.add(new BreakSelectIntoConjunctsRule());
+ normalization.add(new PushSelectIntoJoinRule());
+ normalization.add(new ExtractGbyExpressionsRule());
+ return normalization;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildCondPushDownRuleCollection() {
+ List<IAlgebraicRewriteRule> condPushDown = new LinkedList<IAlgebraicRewriteRule>();
+ condPushDown.add(new PushSelectDownRule());
+ condPushDown.add(new InlineVariablesRule());
+ condPushDown.add(new FactorRedundantGroupAndDecorVarsRule());
+ condPushDown.add(new EliminateSubplanRule());
+ return condPushDown;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildJoinInferenceRuleCollection() {
+ List<IAlgebraicRewriteRule> joinInference = new LinkedList<IAlgebraicRewriteRule>();
+ joinInference.add(new InlineVariablesRule());
+ joinInference.add(new ComplexJoinInferenceRule());
+ return joinInference;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildOpPushDownRuleCollection() {
+ List<IAlgebraicRewriteRule> opPushDown = new LinkedList<IAlgebraicRewriteRule>();
+ opPushDown.add(new PushProjectDownRule());
+ opPushDown.add(new PushSelectDownRule());
+ return opPushDown;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildDataExchangeRuleCollection() {
+ List<IAlgebraicRewriteRule> dataExchange = new LinkedList<IAlgebraicRewriteRule>();
+ dataExchange.add(new SetExecutionModeRule());
+ return dataExchange;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildConsolidationRuleCollection() {
+ List<IAlgebraicRewriteRule> consolidation = new LinkedList<IAlgebraicRewriteRule>();
+ consolidation.add(new ConsolidateSelectsRule());
+ consolidation.add(new ConsolidateAssignsRule());
+ consolidation.add(new RemoveUnusedAssignAndAggregateRule());
+ return consolidation;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildPhysicalRewritesAllLevelsRuleCollection() {
+ List<IAlgebraicRewriteRule> physicalPlanRewrites = new LinkedList<IAlgebraicRewriteRule>();
+ physicalPlanRewrites.add(new PullSelectOutOfEqJoin());
+ physicalPlanRewrites.add(new SetAlgebricksPhysicalOperatorsRule());
+ physicalPlanRewrites.add(new EnforceStructuralPropertiesRule());
+ physicalPlanRewrites.add(new PushProjectDownRule());
+ physicalPlanRewrites.add(new PushLimitDownRule());
+ return physicalPlanRewrites;
+ }
+
+ public final static List<IAlgebraicRewriteRule> buildPhysicalRewritesTopLevelRuleCollection() {
+ List<IAlgebraicRewriteRule> physicalPlanRewrites = new LinkedList<IAlgebraicRewriteRule>();
+ physicalPlanRewrites.add(new PushLimitDownRule());
+ return physicalPlanRewrites;
+ }
+
+
+ public final static List<IAlgebraicRewriteRule> prepareForJobGenRuleCollection() {
+ List<IAlgebraicRewriteRule> prepareForJobGenRewrites = new LinkedList<IAlgebraicRewriteRule>();
+ prepareForJobGenRewrites.add(new IsolateHyracksOperatorsRule(
+ HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
+ prepareForJobGenRewrites.add(new ExtractCommonOperatorsRule());
+ // Re-infer all types, so that, e.g., the effect of not-is-null is
+ // propagated.
+ prepareForJobGenRewrites.add(new PushProjectIntoDataSourceScanRule());
+ prepareForJobGenRewrites.add(new ReinferAllTypesRule());
+ return prepareForJobGenRewrites;
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/PigletExpressionJobGen.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/PigletExpressionJobGen.java
new file mode 100644
index 0000000..d06d9a8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/PigletExpressionJobGen.java
@@ -0,0 +1,125 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.runtime;
+
+import java.io.DataOutput;
+import java.util.Arrays;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.ISerializableAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.evaluators.ColumnAccessEvalFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.evaluators.ConstantEvalFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.compiler.ConstantValue;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.exceptions.PigletException;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.runtime.functions.PigletFunctionRegistry;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Type;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+
+public class PigletExpressionJobGen implements ILogicalExpressionJobGen {
+ @Override
+ public IEvaluatorFactory createEvaluatorFactory(ILogicalExpression expr, IVariableTypeEnvironment env,
+ IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
+ switch (expr.getExpressionTag()) {
+ case CONSTANT: {
+ ConstantValue cv = (ConstantValue) ((ConstantExpression) expr).getValue();
+ Type type = cv.getType();
+ String image = cv.getImage();
+ ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+ DataOutput dos = abvs.getDataOutput();
+ switch (type.getTag()) {
+ case INTEGER:
+ try {
+ IntegerSerializerDeserializer.INSTANCE.serialize(Integer.valueOf(image), dos);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ break;
+
+ case CHAR_ARRAY:
+ try {
+ UTF8StringSerializerDeserializer.INSTANCE.serialize(image, dos);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ break;
+
+ default:
+ throw new UnsupportedOperationException("Unsupported constant type: " + type.getTag());
+ }
+ return new ConstantEvalFactory(Arrays.copyOf(abvs.getBytes(), abvs.getLength()));
+ }
+
+ case FUNCTION_CALL: {
+ ScalarFunctionCallExpression sfce = (ScalarFunctionCallExpression) expr;
+
+ List<LogicalExpressionReference> argExprs = sfce.getArguments();
+ IEvaluatorFactory argEvalFactories[] = new IEvaluatorFactory[argExprs.size()];
+ for (int i = 0; i < argEvalFactories.length; ++i) {
+ LogicalExpressionReference er = argExprs.get(i);
+ argEvalFactories[i] = createEvaluatorFactory(er.getExpression(), env, inputSchemas, context);
+ }
+ IEvaluatorFactory funcEvalFactory;
+ try {
+ funcEvalFactory = PigletFunctionRegistry.createFunctionEvaluatorFactory(sfce
+ .getFunctionIdentifier(), argEvalFactories);
+ } catch (PigletException e) {
+ throw new AlgebricksException(e);
+ }
+ return funcEvalFactory;
+ }
+
+ case VARIABLE: {
+ LogicalVariable var = ((VariableReferenceExpression) expr).getVariableReference();
+ int index = inputSchemas[0].findVariable(var);
+ return new ColumnAccessEvalFactory(index);
+ }
+ }
+ throw new IllegalArgumentException("Unknown expression type: " + expr.getExpressionTag());
+ }
+
+ @Override
+ public IAggregateFunctionFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ISerializableAggregateFunctionFactory createSerializableAggregateFunctionFactory(
+ AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
+ JobGenContext context) throws AlgebricksException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public IRunningAggregateFunctionFactory createRunningAggregateFunctionFactory(StatefulFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public IUnnestingFunctionFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,
+ IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
+ throws AlgebricksException {
+ throw new UnsupportedOperationException();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/IPigletFunctionEvaluatorFactoryBuilder.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/IPigletFunctionEvaluatorFactoryBuilder.java
new file mode 100644
index 0000000..38e7615
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/IPigletFunctionEvaluatorFactoryBuilder.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.runtime.functions;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+
+public interface IPigletFunctionEvaluatorFactoryBuilder {
+ public IEvaluatorFactory buildEvaluatorFactory(FunctionIdentifier fid, IEvaluatorFactory[] arguments);
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/IntegerEqFunctionEvaluatorFactory.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/IntegerEqFunctionEvaluatorFactory.java
new file mode 100644
index 0000000..d654d77
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/IntegerEqFunctionEvaluatorFactory.java
@@ -0,0 +1,52 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.runtime.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class IntegerEqFunctionEvaluatorFactory implements IEvaluatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ private final IEvaluatorFactory arg1Factory;
+
+ private final IEvaluatorFactory arg2Factory;
+
+ public IntegerEqFunctionEvaluatorFactory(IEvaluatorFactory arg1Factory, IEvaluatorFactory arg2Factory) {
+ this.arg1Factory = arg1Factory;
+ this.arg2Factory = arg2Factory;
+ }
+
+ @Override
+ public IEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new IEvaluator() {
+ private DataOutput dataout = output.getDataOutput();
+ private ArrayBackedValueStorage out1 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage out2 = new ArrayBackedValueStorage();
+ private IEvaluator eval1 = arg1Factory.createEvaluator(out1);
+ private IEvaluator eval2 = arg2Factory.createEvaluator(out2);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ out1.reset();
+ eval1.evaluate(tuple);
+ out2.reset();
+ eval2.evaluate(tuple);
+ int v1 = IntegerSerializerDeserializer.getInt(out1.getBytes(), 0);
+ int v2 = IntegerSerializerDeserializer.getInt(out2.getBytes(), 0);
+ boolean r = v1 == v2;
+ try {
+ dataout.writeBoolean(r);
+ } catch (IOException ioe) {
+ throw new AlgebricksException(ioe);
+ }
+ }
+ };
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/PigletFunctionRegistry.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/PigletFunctionRegistry.java
new file mode 100644
index 0000000..5acded6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/runtime/functions/PigletFunctionRegistry.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.runtime.functions;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.exceptions.PigletException;
+
+public class PigletFunctionRegistry {
+ private static final Map<FunctionIdentifier, IPigletFunctionEvaluatorFactoryBuilder> builderMap;
+
+ static {
+ Map<FunctionIdentifier, IPigletFunctionEvaluatorFactoryBuilder> temp = new HashMap<FunctionIdentifier, IPigletFunctionEvaluatorFactoryBuilder>();
+
+ temp.put(AlgebricksBuiltinFunctions.EQ, new IPigletFunctionEvaluatorFactoryBuilder() {
+ @Override
+ public IEvaluatorFactory buildEvaluatorFactory(FunctionIdentifier fid, IEvaluatorFactory[] arguments) {
+ return new IntegerEqFunctionEvaluatorFactory(arguments[0], arguments[1]);
+ }
+ });
+
+ builderMap = Collections.unmodifiableMap(temp);
+ }
+
+ public static IEvaluatorFactory createFunctionEvaluatorFactory(FunctionIdentifier fid, IEvaluatorFactory[] args)
+ throws PigletException {
+ IPigletFunctionEvaluatorFactoryBuilder builder = builderMap.get(fid);
+ if (builder == null) {
+ throw new PigletException("Unknown function: " + fid);
+ }
+ return builder.buildEvaluatorFactory(fid, args);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/BagType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/BagType.java
new file mode 100644
index 0000000..0d964b0
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/BagType.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class BagType extends Type {
+ @Override
+ public Tag getTag() {
+ return Tag.BAG;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/CharArrayType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/CharArrayType.java
new file mode 100644
index 0000000..ddb8e2b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/CharArrayType.java
@@ -0,0 +1,13 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class CharArrayType extends Type {
+ public static final Type INSTANCE = new CharArrayType();
+
+ private CharArrayType() {
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.CHAR_ARRAY;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/DoubleType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/DoubleType.java
new file mode 100644
index 0000000..f3a3a1b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/DoubleType.java
@@ -0,0 +1,13 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class DoubleType extends Type {
+ public static final Type INSTANCE = new DoubleType();
+
+ private DoubleType() {
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.DOUBLE;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/FloatType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/FloatType.java
new file mode 100644
index 0000000..4e6a9d9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/FloatType.java
@@ -0,0 +1,13 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class FloatType extends Type {
+ public static final Type INSTANCE = new FloatType();
+
+ private FloatType() {
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.FLOAT;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/IntegerType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/IntegerType.java
new file mode 100644
index 0000000..3e755c6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/IntegerType.java
@@ -0,0 +1,13 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class IntegerType extends Type {
+ public static final Type INSTANCE = new IntegerType();
+
+ private IntegerType() {
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.INTEGER;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/LongType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/LongType.java
new file mode 100644
index 0000000..e3af89a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/LongType.java
@@ -0,0 +1,13 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class LongType extends Type {
+ public static final Type INSTANCE = new LongType();
+
+ private LongType() {
+ }
+
+ @Override
+ public Tag getTag() {
+ return Tag.LONG;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/MapType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/MapType.java
new file mode 100644
index 0000000..74cf717
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/MapType.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class MapType extends Type {
+ @Override
+ public Tag getTag() {
+ return Tag.MAP;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/Schema.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/Schema.java
new file mode 100644
index 0000000..ba2401c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/Schema.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class Schema {
+ private List<Pair<String, Type>> schema;
+
+ public Schema(List<Pair<String, Type>> schema) {
+ this.schema = schema;
+ }
+
+ public List<Pair<String, Type>> getSchema() {
+ return schema;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/TupleType.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/TupleType.java
new file mode 100644
index 0000000..4c68c2d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/TupleType.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public class TupleType extends Type {
+ @Override
+ public Tag getTag() {
+ return Tag.TUPLE;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/Type.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/Type.java
new file mode 100644
index 0000000..b8286e5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/types/Type.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hyracks.algebricks.examples.piglet.types;
+
+public abstract class Type {
+ public enum Tag {
+ INTEGER,
+ LONG,
+ FLOAT,
+ DOUBLE,
+ CHAR_ARRAY,
+ TUPLE,
+ BAG,
+ MAP
+ }
+
+ public abstract Tag getTag();
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/javacc/PigletParser.jj b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/javacc/PigletParser.jj
new file mode 100644
index 0000000..9e339e4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/javacc/PigletParser.jj
@@ -0,0 +1,386 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+options {
+ STATIC = false;
+ IGNORE_CASE = true;
+ LOOKAHEAD = 2;
+}
+
+PARSER_BEGIN(PigletParser)
+
+package edu.uci.ics.hyracks.algebricks.examples.piglet.parser;
+
+import java.util.*;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.*;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.types.*;
+
+public class PigletParser {
+ private ExpressionNode createFunction(FunctionTag fTag, String fName, ExpressionNode... arguments) {
+ List<ASTNode> args = new ArrayList<ASTNode>();
+ for(ExpressionNode e : arguments) {
+ args.add(e);
+ }
+ return new ScalarFunctionExpressionNode(fTag, fName, args);
+ }
+
+ private String stripQuotes(String s) {
+ s = s.substring(1);
+ s = s.substring(0, s.length() - 1);
+ return s;
+ }
+}
+
+PARSER_END(PigletParser)
+
+List<ASTNode> Statements(): {
+ List<ASTNode> statememts = new ArrayList<ASTNode>();
+ ASTNode s;
+} {
+ (
+ (
+ s = AssignmentStatement() {
+ statememts.add(s);
+ }
+ | s = DumpStatement() {
+ statememts.add(s);
+ }
+ ) ";"
+ )* <EOF> {
+ return statememts;
+ }
+}
+
+ASTNode AssignmentStatement(): {
+ String a;
+ RelationNode r;
+} {
+ a = Alias() "=" r = RelationalStatement() {
+ return new AssignmentNode(a, r);
+ }
+}
+
+String Alias(): {
+ Token t;
+} {
+ t = <IDENTIFIER> {
+ return t.image;
+ }
+}
+
+String ColumnName(): {
+ Token t;
+} {
+ t = <IDENTIFIER> {
+ return t.image;
+ }
+}
+
+Type TypeName(): {
+ Token t;
+} {
+ "int" {
+ return IntegerType.INSTANCE;
+ }
+ | "long" {
+ return LongType.INSTANCE;
+ }
+ | "float" {
+ return FloatType.INSTANCE;
+ }
+ | "double" {
+ return DoubleType.INSTANCE;
+ }
+ | "chararray" {
+ return CharArrayType.INSTANCE;
+ }
+ | "tuple" {
+ return new TupleType();
+ }
+ | "bag" {
+ return new BagType();
+ }
+ | "map" {
+ return new MapType();
+ }
+}
+
+Pair<String, Type> ColumnSchema(): {
+ String c;
+ Type t;
+} {
+ c = ColumnName() ":" t = TypeName() {
+ return new Pair<String, Type>(c, t);
+ }
+}
+
+Schema Schema(): {
+ List<Pair<String, Type>> schema = new ArrayList<Pair<String, Type>>();
+ Pair<String, Type> cSchema;
+} {
+ "(" (
+ cSchema = ColumnSchema() {
+ schema.add(cSchema);
+ } (
+ "," cSchema = ColumnSchema() {
+ schema.add(cSchema);
+ }
+ )*
+ )? ")" {
+ return new Schema(schema);
+ }
+}
+
+RelationNode RelationalStatement(): {
+ RelationNode r;
+} {
+ r = LoadStatement() {
+ return r;
+ }
+ | r = FilterStatement() {
+ return r;
+ }
+}
+
+RelationNode LoadStatement(): {
+ Token t;
+ Schema s;
+} {
+ "load" t = <STRING_LITERAL> "as" s = Schema() {
+ return new LoadNode(t.image, s);
+ }
+}
+
+RelationNode FilterStatement(): {
+ String a;
+ ExpressionNode e;
+} {
+ "filter" a = Alias() "by" e = Expression() {
+ return new FilterNode(a, e);
+ }
+}
+
+ASTNode DumpStatement(): {
+ String a;
+ Token t;
+} {
+ "dump" a = Alias() "into" t = <STRING_LITERAL> {
+ return new DumpNode(t.image, a);
+ }
+}
+
+ExpressionNode Expression(): {
+ ExpressionNode e;
+} {
+ e = OrExpression() {
+ return e;
+ }
+}
+
+ExpressionNode OrExpression(): {
+ ExpressionNode e1;
+ ExpressionNode e2;
+} {
+ e1 = AndExpression() (
+ "or" e2 = AndExpression() {
+ e1 = createFunction(FunctionTag.BOOLEAN_OR, null, e1, e2);
+ }
+ )* {
+ return e1;
+ }
+}
+
+ExpressionNode AndExpression(): {
+ ExpressionNode e1;
+ ExpressionNode e2;
+} {
+ e1 = ComparisonExpression() (
+ "and" e2 = ComparisonExpression() {
+ e1 = createFunction(FunctionTag.BOOLEAN_AND, null, e1, e2);
+ }
+ )* {
+ return e1;
+ }
+}
+
+ExpressionNode ComparisonExpression(): {
+ ExpressionNode e1;
+ ExpressionNode e2;
+ FunctionTag fTag;
+} {
+ e1 = AdditiveExpression() (
+ fTag = ComparisonOperator() e2 = AdditiveExpression() {
+ e1 = createFunction(fTag, null, e1, e2);
+ }
+ )? {
+ return e1;
+ }
+}
+
+FunctionTag ComparisonOperator(): {
+} {
+ "==" {
+ return FunctionTag.EQ;
+ }
+ | "!=" {
+ return FunctionTag.NEQ;
+ }
+ | "<" {
+ return FunctionTag.LT;
+ }
+ | "<=" {
+ return FunctionTag.LTE;
+ }
+ | ">" {
+ return FunctionTag.GT;
+ }
+ | ">=" {
+ return FunctionTag.GTE;
+ }
+}
+
+ExpressionNode AdditiveExpression(): {
+ ExpressionNode e1;
+ ExpressionNode e2;
+ FunctionTag fTag;
+} {
+ e1 = MultiplicativeExpression() (
+ fTag = AdditiveOperator() e2 = MultiplicativeExpression() {
+ e1 = createFunction(fTag, null, e1, e2);
+ }
+ )* {
+ return e1;
+ }
+}
+
+FunctionTag AdditiveOperator(): {
+} {
+ "+" {
+ return FunctionTag.ADD;
+ }
+ | "-" {
+ return FunctionTag.SUBTRACT;
+ }
+}
+
+ExpressionNode MultiplicativeExpression(): {
+ ExpressionNode e1;
+ ExpressionNode e2;
+ FunctionTag fTag;
+} {
+ e1 = PrimaryExpression() (
+ fTag = MultiplicativeOperator() e2 = PrimaryExpression() {
+ e1 = createFunction(fTag, null, e1, e2);
+ }
+ )* {
+ return e1;
+ }
+}
+
+FunctionTag MultiplicativeOperator(): {
+} {
+ "*" {
+ return FunctionTag.MULTIPLY;
+ }
+ | "/" {
+ return FunctionTag.DIVIDE;
+ }
+ | "%" {
+ return FunctionTag.MOD;
+ }
+}
+
+ExpressionNode PrimaryExpression(): {
+ ExpressionNode e;
+} {
+ e = Literal() {
+ return e;
+ }
+ | e = FieldAccess() {
+ return e;
+ }
+}
+
+ExpressionNode Literal(): {
+ Token t;
+} {
+ t = <STRING_LITERAL> {
+ return new LiteralExpressionNode(stripQuotes(t.image), CharArrayType.INSTANCE);
+ }
+ | t = <INTEGER_LITERAL> {
+ return new LiteralExpressionNode(t.image, IntegerType.INSTANCE);
+ }
+ | t = <DOUBLE_LITERAL> {
+ return new LiteralExpressionNode(t.image, DoubleType.INSTANCE);
+ }
+}
+
+ExpressionNode FieldAccess(): {
+ String relName = null;
+ Token fieldName;
+} {
+ (relName = Alias() ".")? fieldName = <IDENTIFIER> {
+ return new FieldAccessExpressionNode(relName, fieldName.image);
+ }
+}
+
+<DEFAULT>
+TOKEN : {
+ <STRING_LITERAL: (("\"" (~["\"", "\n"])* "\"") | ("'" (~["'", "\n"])* "'"))>
+ | <IDENTIFIER: <Letter> (<Letter> | <Digit> | <Extender>)*>
+ | <INTEGER_LITERAL: (<Digit>)+>
+ | <DOUBLE_LITERAL: (((<Digit>)* "." (<Digit>)+ (<Exponent>)?) | (<INTEGER_LITERAL> <Exponent>))>
+ | <INDEXED_FIELD: ("$" <INTEGER_LITERAL>)>
+}
+
+TOKEN :
+{
+ < #Exponent : ((["+", "-"])? ["E", "e"] <INTEGER_LITERAL>)>
+}
+
+SPECIAL_TOKEN :
+{
+ < WhitespaceChar : ["\t", "\r", "\n", " "] >
+}
+
+TOKEN :
+{
+ < #Letter : (<BaseChar> | <Ideographic>) >
+}
+
+TOKEN :
+{
+ < #BaseChar : ["\u0041" - "\u005a", "\u0061" - "\u007a", "\u00c0" - "\u00d6", "\u00d8" - "\u00f6", "\u00f8" - "\u00ff", "\u0100" - "\u0131", "\u0134" - "\u013e", "\u0141" - "\u0148", "\u014a" - "\u017e", "\u0180" - "\u01c3", "\u01cd" - "\u01f0", "\u01f4" - "\u01f5", "\u01fa" - "\u0217", "\u0250" - "\u02a8", "\u02bb" - "\u02c1", "\u0386", "\u0388" - "\u038a", "\u038c", "\u038e" - "\u03a1", "\u03a3" - "\u03ce", "\u03d0" - "\u03d6", "\u03da", "\u03dc", "\u03de", "\u03e0", "\u03e2" - "\u03f3", "\u0401" - "\u040c", "\u040e" - "\u044f", "\u0451" - "\u045c", "\u045e" - "\u0481", "\u0490" - "\u04c4", "\u04c7" - "\u04c8", "\u04cb" - "\u04cc", "\u04d0" - "\u04eb", "\u04ee" - "\u04f5", "\u04f8" - "\u04f9", "\u0531" - "\u0556", "\u0559", "\u0561" - "\u0586", "\u05d0" - "\u05ea", "\u05f0" - "\u05f2", "\u0621" - "\u063a", "\u0641" - "\u064a", "\u0671" - "\u06b7", "\u06ba" - "\u06be", "\u06c0" - "\u06ce", "\u06d0" - "\u06d3", "\u06d5", "\u06e5" - "\u06e6", "\u0905" - "\u0939", "\u093d", "\u0958" - "\u0961", "\u0985" - "\u098c", "\u098f" - "\u0990", "\u0993" - "\u09a8", "\u09aa" - "\u09b0", "\u09b2", "\u09b6" - "\u09b9", "\u09dc" - "\u09dd", "\u09df" - "\u09e1", "\u09f0" - "\u09f1", "\u0a05" - "\u0a0a", "\u0a0f" - "\u0a10", "\u0a13" - "\u0a28", "\u0a2a" - "\u0a30", "\u0a32" - "\u0a33", "\u0a35" - "\u0a36", "\u0a38" - "\u0a39", "\u0a59" - "\u0a5c", "\u0a5e", "\u0a72" - "\u0a74", "\u0a85" - "\u0a8b", "\u0a8d", "\u0a8f" - "\u0a91", "\u0a93" - "\u0aa8", "\u0aaa" - "\u0ab0", "\u0ab2" - "\u0ab3", "\u0ab5" - "\u0ab9", "\u0abd", "\u0ae0", "\u0b05" - "\u0b0c", "\u0b0f" - "\u0b10", "\u0b13" - "\u0b28", "\u0b2a" - "\u0b30", "\u0b32" - "\u0b33", "\u0b36" - "\u0b39", "\u0b3d", "\u0b5c" - "\u0b5d", "\u0b5f" - "\u0b61", "\u0b85" - "\u0b8a", "\u0b8e" - "\u0b90", "\u0b92" - "\u0b95", "\u0b99" - "\u0b9a", "\u0b9c", "\u0b9e" - "\u0b9f", "\u0ba3" - "\u0ba4", "\u0ba8" - "\u0baa", "\u0bae" - "\u0bb5", "\u0bb7" - "\u0bb9", "\u0c05" - "\u0c0c", "\u0c0e" - "\u0c10", "\u0c12" - "\u0c28", "\u0c2a" - "\u0c33", "\u0c35" - "\u0c39", "\u0c60" - "\u0c61", "\u0c85" - "\u0c8c", "\u0c8e" - "\u0c90", "\u0c92" - "\u0ca8", "\u0caa" - "\u0cb3", "\u0cb5" - "\u0cb9", "\u0cde", "\u0ce0" - "\u0ce1", "\u0d05" - "\u0d0c", "\u0d0e" - "\u0d10", "\u0d12" - "\u0d28", "\u0d2a" - "\u0d39", "\u0d60" - "\u0d61", "\u0e01" - "\u0e2e", "\u0e30", "\u0e32" - "\u0e33", "\u0e40" - "\u0e45", "\u0e81" - "\u0e82", "\u0e84", "\u0e87" - "\u0e88", "\u0e8a", "\u0e8d", "\u0e94" - "\u0e97", "\u0e99" - "\u0e9f", "\u0ea1" - "\u0ea3", "\u0ea5", "\u0ea7", "\u0eaa" - "\u0eab", "\u0ead" - "\u0eae", "\u0eb0", "\u0eb2" - "\u0eb3", "\u0ebd", "\u0ec0" - "\u0ec4", "\u0f40" - "\u0f47", "\u0f49" - "\u0f69", "\u10a0" - "\u10c5", "\u10d0" - "\u10f6", "\u1100", "\u1102" - "\u1103", "\u1105" - "\u1107", "\u1109", "\u110b" - "\u110c", "\u110e" - "\u1112", "\u113c", "\u113e", "\u1140", "\u114c", "\u114e", "\u1150", "\u1154" - "\u1155", "\u1159", "\u115f" - "\u1161", "\u1163", "\u1165", "\u1167", "\u1169", "\u116d" - "\u116e", "\u1172" - "\u1173", "\u1175", "\u119e", "\u11a8", "\u11ab", "\u11ae" - "\u11af", "\u11b7" - "\u11b8", "\u11ba", "\u11bc" - "\u11c2", "\u11eb", "\u11f0", "\u11f9", "\u1e00" - "\u1e9b", "\u1ea0" - "\u1ef9", "\u1f00" - "\u1f15", "\u1f18" - "\u1f1d", "\u1f20" - "\u1f45", "\u1f48" - "\u1f4d", "\u1f50" - "\u1f57", "\u1f59", "\u1f5b", "\u1f5d", "\u1f5f" - "\u1f7d", "\u1f80" - "\u1fb4", "\u1fb6" - "\u1fbc", "\u1fbe", "\u1fc2" - "\u1fc4", "\u1fc6" - "\u1fcc", "\u1fd0" - "\u1fd3", "\u1fd6" - "\u1fdb", "\u1fe0" - "\u1fec", "\u1ff2" - "\u1ff4", "\u1ff6" - "\u1ffc", "\u2126", "\u212a" - "\u212b", "\u212e", "\u2180" - "\u2182", "\u3041" - "\u3094", "\u30a1" - "\u30fa", "\u3105" - "\u312c", "\uac00" - "\ud7a3"] >
+}
+
+TOKEN :
+{
+ < #Ideographic : ["\u4e00" - "\u9fa5", "\u3007", "\u3021" - "\u3029"] >
+}
+
+TOKEN :
+{
+ < #CombiningChar : ["\u0300" - "\u0345", "\u0360" - "\u0361", "\u0483" - "\u0486", "\u0591" - "\u05a1", "\u05a3" - "\u05b9", "\u05bb" - "\u05bd", "\u05bf", "\u05c1" - "\u05c2", "\u05c4", "\u064b" - "\u0652", "\u0670", "\u06d6" - "\u06dc", "\u06dd" - "\u06df", "\u06e0" - "\u06e4", "\u06e7" - "\u06e8", "\u06ea" - "\u06ed", "\u0901" - "\u0903", "\u093c", "\u093e" - "\u094c", "\u094d", "\u0951" - "\u0954", "\u0962" - "\u0963", "\u0981" - "\u0983", "\u09bc", "\u09be", "\u09bf", "\u09c0" - "\u09c4", "\u09c7" - "\u09c8", "\u09cb" - "\u09cd", "\u09d7", "\u09e2" - "\u09e3", "\u0a02", "\u0a3c", "\u0a3e", "\u0a3f", "\u0a40" - "\u0a42", "\u0a47" - "\u0a48", "\u0a4b" - "\u0a4d", "\u0a70" - "\u0a71", "\u0a81" - "\u0a83", "\u0abc", "\u0abe" - "\u0ac5", "\u0ac7" - "\u0ac9", "\u0acb" - "\u0acd", "\u0b01" - "\u0b03", "\u0b3c", "\u0b3e" - "\u0b43", "\u0b47" - "\u0b48", "\u0b4b" - "\u0b4d", "\u0b56" - "\u0b57", "\u0b82" - "\u0b83", "\u0bbe" - "\u0bc2", "\u0bc6" - "\u0bc8", "\u0bca" - "\u0bcd", "\u0bd7", "\u0c01" - "\u0c03", "\u0c3e" - "\u0c44", "\u0c46" - "\u0c48", "\u0c4a" - "\u0c4d", "\u0c55" - "\u0c56", "\u0c82" - "\u0c83", "\u0cbe" - "\u0cc4", "\u0cc6" - "\u0cc8", "\u0cca" - "\u0ccd", "\u0cd5" - "\u0cd6", "\u0d02" - "\u0d03", "\u0d3e" - "\u0d43", "\u0d46" - "\u0d48", "\u0d4a" - "\u0d4d", "\u0d57", "\u0e31", "\u0e34" - "\u0e3a", "\u0e47" - "\u0e4e", "\u0eb1", "\u0eb4" - "\u0eb9", "\u0ebb" - "\u0ebc", "\u0ec8" - "\u0ecd", "\u0f18" - "\u0f19", "\u0f35", "\u0f37", "\u0f39", "\u0f3e", "\u0f3f", "\u0f71" - "\u0f84", "\u0f86" - "\u0f8b", "\u0f90" - "\u0f95", "\u0f97", "\u0f99" - "\u0fad", "\u0fb1" - "\u0fb7", "\u0fb9", "\u20d0" - "\u20dc", "\u20e1", "\u302a" - "\u302f", "\u3099", "\u309a"] >
+}
+
+TOKEN :
+{
+ < #Digit : ["\u0030" - "\u0039", "\u0660" - "\u0669", "\u06f0" - "\u06f9", "\u0966" - "\u096f", "\u09e6" - "\u09ef", "\u0a66" - "\u0a6f", "\u0ae6" - "\u0aef", "\u0b66" - "\u0b6f", "\u0be7" - "\u0bef", "\u0c66" - "\u0c6f", "\u0ce6" - "\u0cef", "\u0d66" - "\u0d6f", "\u0e50" - "\u0e59", "\u0ed0" - "\u0ed9", "\u0f20" - "\u0f29"] >
+}
+
+TOKEN :
+{
+ < #Extender : ["\u00b7", "\u02d0", "\u02d1", "\u0387", "\u0640", "\u0e46", "\u0ec6", "\u3005", "\u3031" - "\u3035", "\u309d" - "\u309e", "\u30fc" - "\u30fe"] >
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/test/java/edu/uci/ics/algebricks/examples/piglet/test/PigletTest.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/test/java/edu/uci/ics/algebricks/examples/piglet/test/PigletTest.java
new file mode 100644
index 0000000..7e027fa
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/test/java/edu/uci/ics/algebricks/examples/piglet/test/PigletTest.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.algebricks.examples.piglet.test;
+
+import java.io.File;
+
+import junit.framework.Test;
+import junit.framework.TestSuite;
+
+public class PigletTest {
+ public static Test suite() {
+ TestSuite suite = new TestSuite();
+ File dir = new File("testcases");
+ findAndAddTests(suite, dir);
+
+ return suite;
+ }
+
+ private static void findAndAddTests(TestSuite suite, File dir) {
+ for (final File f : dir.listFiles()) {
+ if (f.getName().startsWith(".")) {
+ continue;
+ }
+ if (f.isDirectory()) {
+ findAndAddTests(suite, f);
+ } else if (f.getName().endsWith(".piglet")) {
+ suite.addTest(new PigletTestCase(f));
+ }
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/test/java/edu/uci/ics/algebricks/examples/piglet/test/PigletTestCase.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/test/java/edu/uci/ics/algebricks/examples/piglet/test/PigletTestCase.java
new file mode 100644
index 0000000..1a329f6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/test/java/edu/uci/ics/algebricks/examples/piglet/test/PigletTestCase.java
@@ -0,0 +1,42 @@
+package edu.uci.ics.algebricks.examples.piglet.test;
+
+import java.io.File;
+import java.io.FileReader;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.ASTNode;
+import edu.uci.ics.hyracks.algebricks.examples.piglet.compiler.PigletCompiler;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class PigletTestCase extends TestCase {
+
+ private final File file;
+
+ PigletTestCase(File file) {
+ super("testPiglet");
+ this.file = file;
+ }
+
+ @Test
+ public void testPiglet() {
+ try {
+ FileReader in = new FileReader(file);
+ try {
+ PigletCompiler c = new PigletCompiler();
+
+ List<ASTNode> ast = c.parse(in);
+ JobSpecification jobSpec = c.compile(ast);
+
+ System.err.println(jobSpec.toJSON());
+ } finally {
+ in.close();
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/testcases/q1.piglet b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/testcases/q1.piglet
new file mode 100644
index 0000000..88111ff
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/testcases/q1.piglet
@@ -0,0 +1,2 @@
+R = load "nc1:data/file1.txt,nc2:data/file2.txt" as (id : int, name : chararray);
+dump R into "nc1:output";
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/testcases/q2.piglet b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/testcases/q2.piglet
new file mode 100644
index 0000000..5c45ac1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/testcases/q2.piglet
@@ -0,0 +1,3 @@
+R = load "nc1:data/file1.txt,nc2:data/file2.txt" as (id : int, name : chararray);
+S = filter R by id == 5;
+dump S into "nc1:output";
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/pom.xml b/hyracks-algebricks/hyracks-algebricks-examples/pom.xml
new file mode 100644
index 0000000..05dd6df
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-examples/pom.xml
@@ -0,0 +1,15 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-algebricks-examples</artifactId>
+ <packaging>pom</packaging>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <modules>
+ <module>piglet-example</module>
+ </modules>
+</project>
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/pom.xml b/hyracks-algebricks/hyracks-algebricks-rewriter/pom.xml
new file mode 100644
index 0000000..fd3b2d1
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/pom.xml
@@ -0,0 +1,31 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-algebricks-rewriter</artifactId>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks-core</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
new file mode 100644
index 0000000..62c4559
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+
+public abstract class AbstractDecorrelationRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ protected boolean descOrSelfIsScanOrJoin(AbstractLogicalOperator op2) {
+ LogicalOperatorTag t = op2.getOperatorTag();
+ if (t == LogicalOperatorTag.DATASOURCESCAN || t == LogicalOperatorTag.INNERJOIN
+ || t == LogicalOperatorTag.LEFTOUTERJOIN) {
+ return true;
+ }
+ if (op2.getInputs().size() != 1) {
+ return false;
+ }
+ AbstractLogicalOperator alo = (AbstractLogicalOperator) op2.getInputs().get(0).getOperator();
+ if (descOrSelfIsScanOrJoin(alo)) {
+ return true;
+ }
+ return false;
+ }
+
+ protected Set<LogicalVariable> computeGbyVarsUsingPksOnly(Set<LogicalVariable> varSet, AbstractLogicalOperator op,
+ IOptimizationContext context) throws AlgebricksException {
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
+ List<FunctionalDependency> fdList = context.getFDList(op);
+ if (fdList == null) {
+ return null;
+ }
+ // check if any of the FDs is a key
+ for (FunctionalDependency fd : fdList) {
+ if (fd.getTail().containsAll(varSet)) {
+ return new HashSet<LogicalVariable>(fd.getHead());
+ }
+ }
+ return null;
+ }
+
+ protected void buildVarExprList(Collection<LogicalVariable> vars, IOptimizationContext context, GroupByOperator g,
+ List<Pair<LogicalVariable, LogicalExpressionReference>> outVeList) throws AlgebricksException {
+ for (LogicalVariable ov : vars) {
+ LogicalVariable newVar = context.newVar();
+ ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+ outVeList.add(new Pair<LogicalVariable, LogicalExpressionReference>(ov, new LogicalExpressionReference(
+ varExpr)));
+ for (ILogicalPlan p : g.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getOperator(), ov, newVar,
+ true, context);
+ }
+ }
+ // g.substituteVarInNestedPlans(ov, newVar);
+ // OperatorManipulationUtil.substituteVarRec(lojoin, ov, newVar);
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
new file mode 100644
index 0000000..f247492
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public abstract class AbstractExtractExprRule implements IAlgebraicRewriteRule {
+
+ protected LogicalVariable extractExprIntoAssignOpRef(ILogicalExpression gExpr, LogicalOperatorReference opRef2,
+ IOptimizationContext context) throws AlgebricksException {
+ LogicalVariable v = context.newVar();
+ AssignOperator a = new AssignOperator(v, new LogicalExpressionReference(gExpr));
+ a.getInputs().add(new LogicalOperatorReference(opRef2.getOperator()));
+ opRef2.setOperator(a);
+ if (gExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ context.addNotToBeInlinedVar(v);
+ }
+ context.computeAndSetTypeEnvironmentForOperator(a);
+ return v;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
new file mode 100644
index 0000000..7902e2c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class BreakSelectIntoConjunctsRule implements IAlgebraicRewriteRule {
+
+ private List<LogicalExpressionReference> conjs = new ArrayList<LogicalExpressionReference>();
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator select = (SelectOperator) op;
+
+ ILogicalExpression cond = select.getCondition().getExpression();
+
+ conjs.clear();
+ if (!cond.splitIntoConjuncts(conjs)) {
+ return false;
+ }
+
+ LogicalOperatorReference childOfSelect = select.getInputs().get(0);
+ boolean fst = true;
+ ILogicalOperator botOp = select;
+ ILogicalExpression firstExpr = null;
+ for (LogicalExpressionReference eRef : conjs) {
+ ILogicalExpression e = eRef.getExpression();
+ if (fst) {
+ fst = false;
+ firstExpr = e;
+ } else {
+ SelectOperator newSelect = new SelectOperator(new LogicalExpressionReference(e));
+ List<LogicalOperatorReference> botInpList = botOp.getInputs();
+ botInpList.clear();
+ botInpList.add(new LogicalOperatorReference(newSelect));
+ context.computeAndSetTypeEnvironmentForOperator(botOp);
+ botOp = newSelect;
+ }
+ }
+ botOp.getInputs().add(childOfSelect);
+ select.getCondition().setExpression(firstExpr);
+ context.computeAndSetTypeEnvironmentForOperator(botOp);
+ context.computeAndSetTypeEnvironmentForOperator(select);
+
+ return true;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
new file mode 100644
index 0000000..5e02124
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ComplexJoinInferenceRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator op = opRef.getOperator();
+ if (!(op instanceof AbstractScanOperator)) {
+ return false;
+ }
+
+ LogicalOperatorReference opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op2;
+
+ LogicalOperatorReference opRef3 = subplan.getInputs().get(0);
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getOperator();
+
+ if (op3.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE
+ || op3.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return false;
+ }
+
+ if (subplanHasFreeVariables(subplan)) {
+ return false;
+ }
+
+ HashSet<LogicalVariable> varsUsedInUnnest = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, varsUsedInUnnest);
+
+ HashSet<LogicalVariable> producedInSubplan = new HashSet<LogicalVariable>();
+ VariableUtilities.getLiveVariables(subplan, producedInSubplan);
+
+ if (!producedInSubplan.containsAll(varsUsedInUnnest)) {
+ return false;
+ }
+
+ ntsToEtsInSubplan(subplan, context);
+ InnerJoinOperator join = new InnerJoinOperator(new LogicalExpressionReference(ConstantExpression.TRUE));
+ join.getInputs().add(opRef3);
+ opRef2.setOperator(OperatorManipulationUtil.eliminateSingleSubplanOverEts(subplan));
+ join.getInputs().add(new LogicalOperatorReference(op));
+ opRef.setOperator(join);
+ context.computeAndSetTypeEnvironmentForOperator(join);
+ return true;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ private static void ntsToEtsInSubplan(SubplanOperator s, IOptimizationContext context) throws AlgebricksException {
+ for (ILogicalPlan p : s.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ OperatorManipulationUtil.ntsToEts(r, context);
+ }
+ }
+ }
+
+ private static boolean subplanHasFreeVariables(SubplanOperator s) throws AlgebricksException {
+ for (ILogicalPlan p : s.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ if (OperatorPropertiesUtil.hasFreeVariablesInSelfOrDesc((AbstractLogicalOperator) r.getOperator())) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java
new file mode 100644
index 0000000..0812576
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ConsolidateAssignsRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ AssignOperator assign1 = (AssignOperator) op;
+
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) assign1.getInputs().get(0).getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+
+ AssignOperator assign2 = (AssignOperator) op2;
+
+ HashSet<LogicalVariable> used1 = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(assign1, used1);
+ for (LogicalVariable v2 : assign2.getVariables()) {
+ if (used1.contains(v2)) {
+ return false;
+ }
+ }
+
+ assign1.getVariables().addAll(assign2.getVariables());
+ assign1.getExpressions().addAll(assign2.getExpressions());
+
+ LogicalOperatorReference botOpRef = assign2.getInputs().get(0);
+ List<LogicalOperatorReference> asgnInpList = assign1.getInputs();
+ asgnInpList.clear();
+ asgnInpList.add(botOpRef);
+ context.computeAndSetTypeEnvironmentForOperator(assign1);
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
new file mode 100644
index 0000000..6484cff
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ConsolidateSelectsRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator select = (SelectOperator) op;
+
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) select.getInputs().get(0).getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression conj = new ScalarFunctionCallExpression(AlgebricksBuiltinFunctions
+ .getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+ conj.getArguments().add(new LogicalExpressionReference(select.getCondition().getExpression()));
+ conj.getArguments().add(((SelectOperator) op2).getCondition());
+
+ LogicalOperatorReference botOpRef = select.getInputs().get(0);
+ boolean more = true;
+ while (more) {
+ botOpRef = botOpRef.getOperator().getInputs().get(0);
+ AbstractLogicalOperator botOp = (AbstractLogicalOperator) botOpRef.getOperator();
+ if (botOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ conj.getArguments().add(((SelectOperator) botOp).getCondition());
+ } else {
+ more = false;
+ }
+ }
+ select.getCondition().setExpression(conj);
+ List<LogicalOperatorReference> selInptList = select.getInputs();
+ selInptList.clear();
+ selInptList.add(botOpRef);
+ context.computeAndSetTypeEnvironmentForOperator(select);
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EliminateSubplanRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EliminateSubplanRule.java
new file mode 100644
index 0000000..6a866ae
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EliminateSubplanRule.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.LinkedList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class EliminateSubplanRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ /**
+ * Eliminate Subplan above ETS
+ *
+ * and Subplan that has only ops. with one input and no free vars. (could we
+ * modify it to consider free vars which are sources of Unnest or Assign, if
+ * there are no aggregates?)
+ */
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op;
+
+ LogicalOperatorReference outerRef = subplan.getInputs().get(0);
+ AbstractLogicalOperator outerRefOp = (AbstractLogicalOperator) outerRef.getOperator();
+ if (outerRefOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ elimSubplanOverEts(opRef, context);
+ return true;
+ }
+ if (subplan.getNestedPlans().size() == 1 && subplan.getNestedPlans().get(0).getRoots().size() == 1
+ && !OperatorPropertiesUtil.hasFreeVariables(subplan)) {
+ if (elimOneSubplanWithNoFreeVars(opRef)) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ private boolean elimOneSubplanWithNoFreeVars(LogicalOperatorReference opRef) {
+ SubplanOperator subplan = (SubplanOperator) opRef.getOperator();
+ AbstractLogicalOperator rootOp = (AbstractLogicalOperator) subplan.getNestedPlans().get(0).getRoots().get(0)
+ .getOperator();
+ if (rootOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ opRef.setOperator(subplan.getInputs().get(0).getOperator());
+ return true;
+ } else {
+ AbstractLogicalOperator botOp = rootOp;
+ if (botOp.getInputs().size() != 1) {
+ return false;
+ }
+ do {
+ LogicalOperatorReference botRef = botOp.getInputs().get(0);
+ botOp = (AbstractLogicalOperator) botRef.getOperator();
+ if (botOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ botRef.setOperator(subplan.getInputs().get(0).getOperator());
+ opRef.setOperator(rootOp);
+ return true;
+ }
+ } while (botOp.getInputs().size() == 1);
+ return false;
+ }
+ }
+
+ private void elimSubplanOverEts(LogicalOperatorReference opRef, IOptimizationContext ctx)
+ throws AlgebricksException {
+ SubplanOperator subplan = (SubplanOperator) opRef.getOperator();
+ for (ILogicalPlan p : subplan.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ OperatorManipulationUtil.ntsToEts(r, ctx);
+ }
+ }
+ LinkedList<LogicalOperatorReference> allRoots = subplan.allRootsInReverseOrder();
+ if (allRoots.size() == 1) {
+ opRef.setOperator(allRoots.get(0).getOperator());
+ } else {
+ ILogicalOperator topOp = null;
+ for (LogicalOperatorReference r : allRoots) {
+ if (topOp == null) {
+ topOp = r.getOperator();
+ } else {
+ LeftOuterJoinOperator j = new LeftOuterJoinOperator(new LogicalExpressionReference(
+ ConstantExpression.TRUE));
+ j.getInputs().add(new LogicalOperatorReference(topOp));
+ j.getInputs().add(r);
+ ctx.setOutputTypeEnvironment(j, j.computeOutputTypeEnvironment(ctx));
+ topOp = j;
+ }
+ }
+ opRef.setOperator(topOp);
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
new file mode 100644
index 0000000..eaaf799
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -0,0 +1,579 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractStableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.BroadcastPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreSortedDistinctByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RandomMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RangePartitionPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.AsterixNodeGroupDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty.PartitioningType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+
+public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
+
+ private static final INodeDomain DEFAULT_DOMAIN = new AsterixNodeGroupDomain("__DEFAULT");
+
+ private PhysicalOptimizationConfig physicalOptimizationConfig;
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ // wait for the physical operators to be set first
+ if (op.getPhysicalOperator() == null) {
+ return false;
+ }
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+
+ List<FunctionalDependency> fds = context.getFDList(op);
+ if (fds != null && !fds.isEmpty()) {
+ return false;
+ }
+ // These are actually logical constraints, so they could be pre-computed
+ // somewhere else, too.
+
+ physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Optimizing operator " + op.getPhysicalOperator() + ".\n");
+
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
+
+ StructuralPropertiesVector pvector = new StructuralPropertiesVector(new RandomPartitioningProperty(null),
+ new LinkedList<ILocalStructuralProperty>());
+ boolean changed = physOptimizeOp(opRef, pvector, false, context);
+ op.computeDeliveredPhysicalProperties(context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + op.getPhysicalOperator() + ": "
+ + op.getDeliveredPhysicalProperties() + "\n");
+
+ context.addToDontApplySet(this, opRef.getOperator());
+
+ return changed;
+ }
+
+ private boolean physOptimizePlan(ILogicalPlan plan, IPhysicalPropertiesVector pvector, boolean nestedPlan,
+ IOptimizationContext context) throws AlgebricksException {
+ boolean changed = false;
+ for (LogicalOperatorReference root : plan.getRoots()) {
+ if (physOptimizeOp(root, pvector, nestedPlan, context)) {
+ changed = true;
+ }
+ AbstractLogicalOperator op = (AbstractLogicalOperator) root.getOperator();
+ op.computeDeliveredPhysicalProperties(context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + op.getPhysicalOperator()
+ + ": " + op.getDeliveredPhysicalProperties() + "\n");
+ }
+ return changed;
+ }
+
+ private boolean physOptimizeOp(LogicalOperatorReference opRef, IPhysicalPropertiesVector required,
+ boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
+
+ boolean changed = false;
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ optimizeUsingConstraintsAndEquivClasses(op);
+ PhysicalRequirements pr = op.getRequiredPhysicalPropertiesForChildren(required);
+ IPhysicalPropertiesVector[] reqdProperties = null;
+ if (pr != null) {
+ reqdProperties = pr.getRequiredProperties();
+ }
+ boolean opIsRedundantSort = false;
+
+ // compute properties and figure out the domain
+ INodeDomain childrenDomain = null;
+ {
+ int j = 0;
+ for (LogicalOperatorReference childRef : op.getInputs()) {
+ AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getOperator();
+ // recursive call
+ if (physOptimizeOp(childRef, reqdProperties[j], nestedPlan, context)) {
+ changed = true;
+ }
+ child.computeDeliveredPhysicalProperties(context);
+ IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
+ if (childrenDomain == null) {
+ childrenDomain = delivered.getPartitioningProperty().getNodeDomain();
+ } else {
+ INodeDomain dom2 = delivered.getPartitioningProperty().getNodeDomain();
+ if (!childrenDomain.sameAs(dom2)) {
+ childrenDomain = DEFAULT_DOMAIN;
+ }
+ }
+ j++;
+ }
+ }
+
+ if (reqdProperties != null) {
+ for (int k = 0; k < reqdProperties.length; k++) {
+ IPhysicalPropertiesVector pv = reqdProperties[k];
+ IPartitioningProperty pp = pv.getPartitioningProperty();
+ if (pp != null && pp.getNodeDomain() == null) {
+ pp.setNodeDomain(childrenDomain);
+ }
+ }
+ }
+
+ IPartitioningProperty firstDeliveredPartitioning = null;
+ int i = 0;
+ for (LogicalOperatorReference childRef : op.getInputs()) {
+ AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getOperator();
+ IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
+
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Properties delivered by " + child.getPhysicalOperator()
+ + ": " + delivered + "\n");
+ IPartitioningRequirementsCoordinator prc = pr.getPartitioningCoordinator();
+ Pair<Boolean, IPartitioningProperty> pbpp = prc.coordinateRequirements(reqdProperties[i]
+ .getPartitioningProperty(), firstDeliveredPartitioning, op, context);
+ boolean mayExpandPartitioningProperties = pbpp.first;
+ IPhysicalPropertiesVector rqd = new StructuralPropertiesVector(pbpp.second, reqdProperties[i]
+ .getLocalProperties());
+
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Required properties for " + child.getPhysicalOperator()
+ + ": " + rqd + "\n");
+ IPhysicalPropertiesVector diff = delivered.getUnsatisfiedPropertiesFrom(rqd,
+ mayExpandPartitioningProperties, context.getEquivalenceClassMap(child), context.getFDList(child));
+
+ if (isRedundantSort(opRef, delivered, diff, context)) {
+ opIsRedundantSort = true;
+ }
+
+ if (diff != null) {
+ changed = true;
+ addEnforcers(op, i, diff, rqd, delivered, childrenDomain, nestedPlan, context);
+
+ AbstractLogicalOperator newChild = ((AbstractLogicalOperator) op.getInputs().get(i).getOperator());
+
+ if (newChild != child) {
+ delivered = newChild.getDeliveredPhysicalProperties();
+ IPhysicalPropertiesVector newDiff = newPropertiesDiff(newChild, rqd,
+ mayExpandPartitioningProperties, context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> New properties diff: " + newDiff + "\n");
+
+ if (isRedundantSort(opRef, delivered, newDiff, context)) {
+ opIsRedundantSort = true;
+ break;
+ }
+ }
+
+ }
+ if (firstDeliveredPartitioning == null) {
+ IPartitioningProperty dpp = delivered.getPartitioningProperty();
+ if (dpp.getPartitioningType() == PartitioningType.ORDERED_PARTITIONED
+ || dpp.getPartitioningType() == PartitioningType.UNORDERED_PARTITIONED) {
+ firstDeliveredPartitioning = dpp;
+ }
+ }
+
+ i++;
+ }
+
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans nested = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : nested.getNestedPlans()) {
+ if (physOptimizePlan(p, required, true, context)) {
+ changed = true;
+ }
+ }
+ }
+
+ if (opIsRedundantSort) {
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Removing redundant SORT operator "
+ + op.getPhysicalOperator() + "\n");
+ printOp((AbstractLogicalOperator) op);
+ }
+ changed = true;
+ AbstractLogicalOperator nextOp = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ if (nextOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ nextOp = (AbstractLogicalOperator) nextOp.getInputs().get(0).getOperator();
+ }
+ opRef.setOperator(nextOp);
+ // Now, transfer annotations from the original sort op. to this one.
+ AbstractLogicalOperator transferTo = nextOp;
+ if (transferTo.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ transferTo = (AbstractLogicalOperator) transferTo.getInputs().get(0).getOperator();
+ }
+ transferTo.getAnnotations().putAll(op.getAnnotations());
+ physOptimizeOp(opRef, required, nestedPlan, context);
+ }
+ return changed;
+ }
+
+ private IPhysicalPropertiesVector newPropertiesDiff(AbstractLogicalOperator newChild,
+ IPhysicalPropertiesVector required, boolean mayExpandPartitioningProperties, IOptimizationContext context)
+ throws AlgebricksException {
+ IPhysicalPropertiesVector newDelivered = newChild.getDeliveredPhysicalProperties();
+
+ Map<LogicalVariable, EquivalenceClass> newChildEqClasses = context.getEquivalenceClassMap(newChild);
+ List<FunctionalDependency> newChildFDs = context.getFDList(newChild);
+ if (newChildEqClasses == null || newChildFDs == null) {
+ FDsAndEquivClassesVisitor fdsVisitor = new FDsAndEquivClassesVisitor();
+ newChild.accept(fdsVisitor, context);
+ newChildEqClasses = context.getEquivalenceClassMap(newChild);
+ newChildFDs = context.getFDList(newChild);
+ }
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Required properties for new op. "
+ + newChild.getPhysicalOperator() + ": " + required + "\n");
+
+ return newDelivered.getUnsatisfiedPropertiesFrom(required, mayExpandPartitioningProperties, newChildEqClasses,
+ newChildFDs);
+ }
+
+ private void optimizeUsingConstraintsAndEquivClasses(AbstractLogicalOperator op) {
+ IPhysicalOperator pOp = op.getPhysicalOperator();
+ switch (pOp.getOperatorTag()) {
+ case HASH_GROUP_BY:
+ case EXTERNAL_GROUP_BY: {
+ GroupByOperator gby = (GroupByOperator) op;
+ ExternalGroupByPOperator hgbyOp = (ExternalGroupByPOperator) pOp;
+ hgbyOp.computeColumnSet(gby.getGroupByList());
+ break;
+ }
+ case PRE_CLUSTERED_GROUP_BY: {
+ GroupByOperator gby = (GroupByOperator) op;
+ PreclusteredGroupByPOperator preSortedGby = (PreclusteredGroupByPOperator) pOp;
+ preSortedGby.setGbyColumns(gby.getGbyVarList());
+ break;
+ }
+ case PRE_SORTED_DISTINCT_BY: {
+ DistinctOperator d = (DistinctOperator) op;
+ PreSortedDistinctByPOperator preSortedDistinct = (PreSortedDistinctByPOperator) pOp;
+ preSortedDistinct.setDistinctByColumns(d.getDistinctByVarList());
+ break;
+ }
+ }
+ }
+
+ private List<OrderColumn> getOrderColumnsFromGroupingProperties(List<ILocalStructuralProperty> reqd,
+ List<ILocalStructuralProperty> dlvd) {
+ List<OrderColumn> returnedProperties = new ArrayList<OrderColumn>();
+ List<LogicalVariable> rqdCols = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> dlvdCols = new ArrayList<LogicalVariable>();
+ for (ILocalStructuralProperty r : reqd) {
+ r.getVariables(rqdCols);
+ }
+ for (ILocalStructuralProperty d : dlvd) {
+ d.getVariables(dlvdCols);
+ }
+
+ int prefix = dlvdCols.size() - 1;
+ for (; prefix >= 0;)
+ if (!rqdCols.contains(dlvdCols.get(prefix)))
+ prefix--;
+ else
+ break;
+ for (int j = 0; j <= prefix; j++) {
+ LocalOrderProperty orderProp = (LocalOrderProperty) dlvd.get(j);
+ returnedProperties.add(new OrderColumn(orderProp.getColumn(), orderProp.getOrder()));
+ }
+ return returnedProperties;
+ }
+
+ /*
+ * We assume delivered to be already normalized.
+ */
+ private boolean isRedundantSort(LogicalOperatorReference opRef, IPhysicalPropertiesVector delivered,
+ IPhysicalPropertiesVector diffOfProperties, IOptimizationContext context) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.ORDER
+ || (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT && op
+ .getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT)
+ || delivered.getLocalProperties() == null) {
+ return false;
+ }
+ AbstractStableSortPOperator sortOp = (AbstractStableSortPOperator) op.getPhysicalOperator();
+ sortOp.computeLocalProperties(op);
+ List<ILocalStructuralProperty> orderProps = sortOp.getOrderProperties();
+ return PropertiesUtil.matchLocalProperties(orderProps, delivered.getLocalProperties(), context
+ .getEquivalenceClassMap(op), context.getFDList(op));
+ }
+
+ private void addEnforcers(AbstractLogicalOperator op, int childIndex,
+ IPhysicalPropertiesVector diffPropertiesVector, IPhysicalPropertiesVector required,
+ IPhysicalPropertiesVector deliveredByChild, INodeDomain domain, boolean nestedPlan,
+ IOptimizationContext context) throws AlgebricksException {
+
+ IPartitioningProperty pp = diffPropertiesVector.getPartitioningProperty();
+ if (pp == null || pp.getPartitioningType() == PartitioningType.UNPARTITIONED) {
+ addLocalEnforcers(op, childIndex, diffPropertiesVector.getLocalProperties(), nestedPlan, context);
+ IPhysicalPropertiesVector deliveredByNewChild = ((AbstractLogicalOperator) op.getInputs().get(0)
+ .getOperator()).getDeliveredPhysicalProperties();
+ addPartitioningEnforcers(op, childIndex, pp, required, deliveredByNewChild, domain, context);
+ } else {
+ addPartitioningEnforcers(op, childIndex, pp, required, deliveredByChild, domain, context);
+ AbstractLogicalOperator newChild = (AbstractLogicalOperator) op.getInputs().get(childIndex).getOperator();
+ IPhysicalPropertiesVector newDiff = newPropertiesDiff(newChild, required, true, context);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> New properties diff: " + newDiff + "\n");
+ if (newDiff != null) {
+ addLocalEnforcers(op, childIndex, newDiff.getLocalProperties(), nestedPlan, context);
+ }
+ }
+ }
+
+ private void addLocalEnforcers(AbstractLogicalOperator op, int i, List<ILocalStructuralProperty> localProperties,
+ boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Adding local enforcers for local props = " + localProperties
+ + "\n");
+ }
+
+ if (localProperties == null || localProperties.isEmpty()) {
+ return;
+ }
+
+ LogicalOperatorReference topOp = new LogicalOperatorReference();
+ topOp.setOperator(op.getInputs().get(i).getOperator());
+ LinkedList<LocalOrderProperty> oList = new LinkedList<LocalOrderProperty>();
+
+ for (ILocalStructuralProperty prop : localProperties) {
+ switch (prop.getPropertyType()) {
+ case LOCAL_ORDER_PROPERTY: {
+ oList.add((LocalOrderProperty) prop);
+ break;
+ }
+ case LOCAL_GROUPING_PROPERTY: {
+ LocalGroupingProperty g = (LocalGroupingProperty) prop;
+ Collection<LogicalVariable> vars = (g.getPreferredOrderEnforcer() != null) ? g
+ .getPreferredOrderEnforcer() : g.getColumnSet();
+ for (LogicalVariable v : vars) {
+ OrderColumn oc = new OrderColumn(v, OrderKind.ASC);
+ LocalOrderProperty lop = new LocalOrderProperty(oc);
+ oList.add(lop);
+ }
+ break;
+ }
+ default: {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ if (!oList.isEmpty()) {
+ topOp = enforceOrderProperties(oList, topOp, nestedPlan, context);
+ }
+
+ op.getInputs().set(i, topOp);
+ OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull((AbstractLogicalOperator) topOp.getOperator(), context);
+ printOp((AbstractLogicalOperator) topOp.getOperator());
+ }
+
+ private LogicalOperatorReference enforceOrderProperties(List<LocalOrderProperty> oList,
+ LogicalOperatorReference topOp, boolean isMicroOp, IOptimizationContext context) throws AlgebricksException {
+ List<Pair<IOrder, LogicalExpressionReference>> oe = new LinkedList<Pair<IOrder, LogicalExpressionReference>>();
+ for (LocalOrderProperty o : oList) {
+ IOrder ordType = (o.getOrder() == OrderKind.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
+ Pair<IOrder, LogicalExpressionReference> pair = new Pair<IOrder, LogicalExpressionReference>(ordType,
+ new LogicalExpressionReference(new VariableReferenceExpression(o.getColumn())));
+ oe.add(pair);
+ }
+ OrderOperator oo = new OrderOperator(oe);
+ oo.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+ if (isMicroOp) {
+ oo.setPhysicalOperator(new InMemoryStableSortPOperator());
+ } else {
+ oo.setPhysicalOperator(new StableSortPOperator(physicalOptimizationConfig.getMaxFramesExternalSort()));
+ }
+ oo.getInputs().add(topOp);
+ context.computeAndSetTypeEnvironmentForOperator(oo);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Added sort enforcer " + oo.getPhysicalOperator() + ".\n");
+ }
+ return new LogicalOperatorReference(oo);
+ }
+
+ private void addPartitioningEnforcers(ILogicalOperator op, int i, IPartitioningProperty pp,
+ IPhysicalPropertiesVector required, IPhysicalPropertiesVector deliveredByChild, INodeDomain domain,
+ IOptimizationContext context) throws AlgebricksException {
+ if (pp != null) {
+ IPhysicalOperator pop;
+ switch (pp.getPartitioningType()) {
+ case UNPARTITIONED: {
+ List<OrderColumn> ordCols = computeOrderColumns(deliveredByChild);
+ if (ordCols == null || ordCols.size() == 0) {
+ pop = new RandomMergeExchangePOperator();
+ } else {
+ OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
+ sortColumns = ordCols.toArray(sortColumns);
+ pop = new SortMergeExchangePOperator(sortColumns);
+ }
+ break;
+ }
+ case UNORDERED_PARTITIONED: {
+ List<LogicalVariable> varList = new ArrayList<LogicalVariable>(((UnorderedPartitionedProperty) pp)
+ .getColumnSet());
+ List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
+ List<ILocalStructuralProperty> reqdLocals = required.getLocalProperties();
+ boolean propWasSet = false;
+ pop = null;
+ if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
+ AbstractLogicalOperator c = (AbstractLogicalOperator) op.getInputs().get(i).getOperator();
+ Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(c);
+ List<FunctionalDependency> fds = context.getFDList(c);
+ if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
+ List<OrderColumn> orderColumns = getOrderColumnsFromGroupingProperties(reqdLocals,
+ cldLocals);
+ pop = new HashPartitionMergeExchangePOperator(orderColumns, varList, domain);
+ propWasSet = true;
+ }
+ }
+ if (!propWasSet) {
+ pop = new HashPartitionExchangePOperator(varList, domain);
+ }
+ break;
+ }
+ case ORDERED_PARTITIONED: {
+ pop = new RangePartitionPOperator(((OrderedPartitionedProperty) pp).getOrderColumns(), domain);
+ break;
+ }
+ case BROADCAST: {
+ pop = new BroadcastPOperator(domain);
+ break;
+ }
+ case RANDOM: {
+ RandomPartitioningProperty rpp = (RandomPartitioningProperty) pp;
+ INodeDomain nd = rpp.getNodeDomain();
+ if (nd == null) {
+ throw new AlgebricksException("Unknown node domain for " + rpp);
+ }
+ if (nd.cardinality() == null) {
+ throw new AlgebricksException("Unknown cardinality for node domain " + nd);
+ }
+ if (nd.cardinality() != 1) {
+ throw new NotImplementedException(
+ "Random repartitioning is only implemented for target domains of"
+ + "cardinality equal to 1.");
+ }
+ pop = new BroadcastPOperator(nd);
+ break;
+ }
+ default: {
+ throw new NotImplementedException("Enforcer for " + pp.getPartitioningType()
+ + " partitioning type has not been implemented.");
+ }
+ }
+ LogicalOperatorReference ci = op.getInputs().get(i);
+ ExchangeOperator exchg = new ExchangeOperator();
+ exchg.setPhysicalOperator(pop);
+ setNewOp(ci, exchg, context);
+ exchg.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(exchg, context);
+ context.computeAndSetTypeEnvironmentForOperator(exchg);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Added partitioning enforcer "
+ + exchg.getPhysicalOperator() + ".\n");
+ printOp((AbstractLogicalOperator) op);
+ }
+ }
+ }
+
+ private boolean allAreOrderProps(List<ILocalStructuralProperty> cldLocals) {
+ for (ILocalStructuralProperty lsp : cldLocals) {
+ if (lsp.getPropertyType() != PropertyType.LOCAL_ORDER_PROPERTY) {
+ return false;
+ }
+ }
+ return !cldLocals.isEmpty();
+ }
+
+ private void printOp(AbstractLogicalOperator op) throws AlgebricksException {
+ StringBuilder sb = new StringBuilder();
+ LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
+ PlanPrettyPrinter.printOperator(op, sb, pvisitor, 0);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine(sb.toString());
+ }
+
+ private List<OrderColumn> computeOrderColumns(IPhysicalPropertiesVector pv) {
+ List<OrderColumn> ordCols = new ArrayList<OrderColumn>();
+ List<ILocalStructuralProperty> localProps = pv.getLocalProperties();
+ if (localProps == null || localProps.size() == 0) {
+ return null;
+ } else {
+ for (ILocalStructuralProperty p : localProps) {
+ if (p.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
+ LocalOrderProperty lop = (LocalOrderProperty) p;
+ ordCols.add(lop.getOrderColumn());
+ } else {
+ return null;
+ }
+ }
+ return ordCols;
+ }
+
+ }
+
+ private void setNewOp(LogicalOperatorReference opRef, AbstractLogicalOperator newOp, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator oldOp = opRef.getOperator();
+ opRef.setOperator(newOp);
+ newOp.getInputs().add(new LogicalOperatorReference(oldOp));
+ newOp.recomputeSchema();
+ newOp.computeDeliveredPhysicalProperties(context);
+ context.computeAndSetTypeEnvironmentForOperator(newOp);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + newOp.getPhysicalOperator()
+ + ": " + newOp.getDeliveredPhysicalProperties() + "\n");
+
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(newOp, context);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
new file mode 100644
index 0000000..11f94fe
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -0,0 +1,389 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.IsomorphismUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
+
+ private HashMap<LogicalOperatorReference, List<LogicalOperatorReference>> childrenToParents = new HashMap<LogicalOperatorReference, List<LogicalOperatorReference>>();
+ private List<LogicalOperatorReference> roots = new ArrayList<LogicalOperatorReference>();
+ private List<LogicalOperatorReference> joins = new ArrayList<LogicalOperatorReference>();
+ private List<List<LogicalOperatorReference>> equivalenceClasses = new ArrayList<List<LogicalOperatorReference>>();
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT) {
+ return false;
+ }
+ if (!roots.contains(op))
+ roots.add(new LogicalOperatorReference(op));
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT) {
+ return false;
+ }
+ boolean rewritten = false;
+ boolean changed = false;
+ if (roots.size() > 0) {
+ do {
+ changed = false;
+ // applying the rewriting until fixpoint
+ topDownMaterialization(roots);
+ removeNonJoinBuildBranchCandidates();
+ genCandidates(context);
+ removeTrivialShare();
+ removeNonJoinBuildBranchCandidates();
+ if (equivalenceClasses.size() > 0)
+ changed = rewrite(context);
+ if (!rewritten)
+ rewritten = changed;
+ equivalenceClasses.clear();
+ childrenToParents.clear();
+ joins.clear();
+ } while (changed);
+ roots.clear();
+ }
+ return rewritten;
+ }
+
+ private void removeTrivialShare() {
+ for (List<LogicalOperatorReference> candidates : equivalenceClasses) {
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ LogicalOperatorReference opRef = candidates.get(i);
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) opRef.getOperator();
+ if (aop.getOperatorTag() == LogicalOperatorTag.EXCHANGE)
+ aop = (AbstractLogicalOperator) aop.getInputs().get(0).getOperator();
+ if (aop.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE)
+ candidates.remove(i);
+ }
+ }
+ for (int i = equivalenceClasses.size() - 1; i >= 0; i--)
+ if (equivalenceClasses.get(i).size() < 2)
+ equivalenceClasses.remove(i);
+ }
+
+ private void removeNonJoinBuildBranchCandidates() {
+ for (List<LogicalOperatorReference> candidates : equivalenceClasses) {
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ LogicalOperatorReference opRef = candidates.get(i);
+ boolean reserve = false;
+ for (LogicalOperatorReference join : joins)
+ if (isInJoinBuildBranch(join, opRef)) {
+ reserve = true;
+ }
+ if (!reserve)
+ candidates.remove(i);
+ }
+ }
+ for (int i = equivalenceClasses.size() - 1; i >= 0; i--)
+ if (equivalenceClasses.get(i).size() < 2)
+ equivalenceClasses.remove(i);
+ }
+
+ private boolean isInJoinBuildBranch(LogicalOperatorReference joinRef, LogicalOperatorReference opRef) {
+ LogicalOperatorReference buildBranch = joinRef.getOperator().getInputs().get(1);
+ do {
+ if (buildBranch.equals(opRef)) {
+ return true;
+ } else {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) buildBranch.getOperator();
+ if (aop.getOperatorTag() == LogicalOperatorTag.INNERJOIN
+ || aop.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN
+ || buildBranch.getOperator().getInputs().size() == 0)
+ return false;
+ else
+ buildBranch = buildBranch.getOperator().getInputs().get(0);
+ }
+ } while (true);
+ }
+
+ private boolean rewrite(IOptimizationContext context) throws AlgebricksException {
+ boolean changed = false;
+ for (List<LogicalOperatorReference> members : equivalenceClasses) {
+ if (rewriteForOneEquivalentClass(members, context))
+ changed = true;
+ }
+ return changed;
+ }
+
+ private boolean rewriteForOneEquivalentClass(List<LogicalOperatorReference> members, IOptimizationContext context)
+ throws AlgebricksException {
+ List<LogicalOperatorReference> group = new ArrayList<LogicalOperatorReference>();
+ boolean rewritten = false;
+ while (members.size() > 0) {
+ group.clear();
+ LogicalOperatorReference candidate = members.remove(members.size() - 1);
+ group.add(candidate);
+ for (int i = members.size() - 1; i >= 0; i--) {
+ LogicalOperatorReference peer = members.get(i);
+ if (IsomorphismUtilities.isOperatorIsomorphic(candidate.getOperator(), peer.getOperator())) {
+ group.add(peer);
+ members.remove(i);
+ }
+ }
+ AbstractLogicalOperator rop = new ReplicateOperator(group.size());
+ rop.setPhysicalOperator(new ReplicatePOperator());
+ rop.setExecutionMode(ExecutionMode.PARTITIONED);
+ LogicalOperatorReference ropRef = new LogicalOperatorReference(rop);
+ AbstractLogicalOperator aopCandidate = (AbstractLogicalOperator) candidate.getOperator();
+
+ if (aopCandidate.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ rop.getInputs().add(candidate);
+ } else {
+ AbstractLogicalOperator beforeExchange = new ExchangeOperator();
+ beforeExchange.setPhysicalOperator(new OneToOneExchangePOperator());
+ beforeExchange.getInputs().add(candidate);
+ context.computeAndSetTypeEnvironmentForOperator(beforeExchange);
+ rop.getInputs().add(new LogicalOperatorReference(beforeExchange));
+ }
+ context.computeAndSetTypeEnvironmentForOperator(rop);
+
+ List<LogicalOperatorReference> parents = childrenToParents.get(candidate);
+ for (LogicalOperatorReference parentRef : parents) {
+ AbstractLogicalOperator parent = (AbstractLogicalOperator) parentRef.getOperator();
+ int index = parent.getInputs().indexOf(candidate);
+ if (parent.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ parent.getInputs().set(index, ropRef);
+ } else {
+ AbstractLogicalOperator exchange = new ExchangeOperator();
+ exchange.setPhysicalOperator(new OneToOneExchangePOperator());
+ exchange.getInputs().add(ropRef);
+ context.computeAndSetTypeEnvironmentForOperator(exchange);
+ // parent.getInputs().get(index).setOperator(exchange);
+ parent.getInputs().set(index, new LogicalOperatorReference(exchange));
+ context.computeAndSetTypeEnvironmentForOperator(parent);
+ }
+ }
+
+ List<LogicalVariable> liveVarsNew = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(candidate.getOperator(), liveVarsNew);
+ ArrayList<LogicalExpressionReference> assignExprs = new ArrayList<LogicalExpressionReference>();
+ for (LogicalVariable liveVar : liveVarsNew)
+ assignExprs.add(new LogicalExpressionReference(new VariableReferenceExpression(liveVar)));
+ for (LogicalOperatorReference ref : group) {
+ if (ref.equals(candidate))
+ continue;
+ ArrayList<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
+ Map<LogicalVariable, LogicalVariable> variableMappingBack = new HashMap<LogicalVariable, LogicalVariable>();
+ IsomorphismUtilities.mapVariablesTopDown(ref.getOperator(), candidate.getOperator(),
+ variableMappingBack);
+ for (int i = 0; i < liveVarsNew.size(); i++) {
+ liveVars.add(variableMappingBack.get(liveVarsNew.get(i)));
+ }
+
+ AbstractLogicalOperator assignOperator = new AssignOperator(liveVars, assignExprs);
+ assignOperator.setPhysicalOperator(new AssignPOperator());
+ AbstractLogicalOperator projectOperator = new ProjectOperator(liveVars);
+ projectOperator.setPhysicalOperator(new StreamProjectPOperator());
+ AbstractLogicalOperator exchOp = new ExchangeOperator();
+ exchOp.setPhysicalOperator(new OneToOneExchangePOperator());
+ exchOp.getInputs().add(ropRef);
+
+ assignOperator.getInputs().add(new LogicalOperatorReference(exchOp));
+ projectOperator.getInputs().add(new LogicalOperatorReference(assignOperator));
+ // set the types
+ context.computeAndSetTypeEnvironmentForOperator(exchOp);
+ context.computeAndSetTypeEnvironmentForOperator(assignOperator);
+ context.computeAndSetTypeEnvironmentForOperator(projectOperator);
+
+ List<LogicalOperatorReference> parentOpList = childrenToParents.get(ref);
+ for (LogicalOperatorReference parentOpRef : parentOpList) {
+ AbstractLogicalOperator parentOp = (AbstractLogicalOperator) parentOpRef.getOperator();
+ int index = parentOp.getInputs().indexOf(ref);
+ if (parentOp.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ AbstractLogicalOperator parentOpNext = (AbstractLogicalOperator) childrenToParents
+ .get(parentOpRef).get(0).getOperator();
+ if (parentOpNext.isMap()) {
+ index = parentOpNext.getInputs().indexOf(parentOpRef);
+ parentOp = parentOpNext;
+ }
+ }
+
+ AbstractLogicalOperator exchg = new ExchangeOperator();
+ exchg.setPhysicalOperator(new OneToOneExchangePOperator());
+
+ ILogicalOperator childOp = parentOp.getOperatorTag() == LogicalOperatorTag.PROJECT ? assignOperator
+ : projectOperator;
+ if (parentOp.isMap()) {
+ parentOp.getInputs().set(index, new LogicalOperatorReference(childOp));
+ } else {
+ exchg.getInputs().add(new LogicalOperatorReference(childOp));
+ parentOp.getInputs().set(index, new LogicalOperatorReference(exchg));
+ }
+ context.computeAndSetTypeEnvironmentForOperator(exchg);
+ }
+ }
+ rewritten = true;
+ }
+ return rewritten;
+ }
+
+ private void genCandidates(IOptimizationContext context) throws AlgebricksException {
+ List<List<LogicalOperatorReference>> previousEquivalenceClasses = new ArrayList<List<LogicalOperatorReference>>();
+ while (equivalenceClasses.size() > 0) {
+ previousEquivalenceClasses.clear();
+ for (List<LogicalOperatorReference> candidates : equivalenceClasses) {
+ List<LogicalOperatorReference> candidatesCopy = new ArrayList<LogicalOperatorReference>();
+ candidatesCopy.addAll(candidates);
+ previousEquivalenceClasses.add(candidatesCopy);
+ }
+ List<LogicalOperatorReference> currentLevelOpRefs = new ArrayList<LogicalOperatorReference>();
+ for (List<LogicalOperatorReference> candidates : equivalenceClasses) {
+ if (candidates.size() > 0) {
+ for (LogicalOperatorReference opRef : candidates) {
+ List<LogicalOperatorReference> refs = childrenToParents.get(opRef);
+ if (refs != null)
+ currentLevelOpRefs.addAll(refs);
+ }
+ }
+ if (currentLevelOpRefs.size() == 0)
+ continue;
+ candidatesGrow(currentLevelOpRefs, candidates);
+ }
+ if (currentLevelOpRefs.size() == 0)
+ break;
+ prune(context);
+ }
+ if (equivalenceClasses.size() < 1 && previousEquivalenceClasses.size() > 0) {
+ equivalenceClasses.addAll(previousEquivalenceClasses);
+ prune(context);
+ }
+ }
+
+ private void topDownMaterialization(List<LogicalOperatorReference> tops) {
+ List<LogicalOperatorReference> candidates = new ArrayList<LogicalOperatorReference>();
+ List<LogicalOperatorReference> nextLevel = new ArrayList<LogicalOperatorReference>();
+ for (LogicalOperatorReference op : tops) {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) op.getOperator();
+ if ((aop.getOperatorTag() == LogicalOperatorTag.INNERJOIN || aop.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN)
+ && !joins.contains(op)) {
+ joins.add(op);
+ }
+ for (LogicalOperatorReference opRef : op.getOperator().getInputs()) {
+ List<LogicalOperatorReference> opRefList = childrenToParents.get(opRef);
+ if (opRefList == null) {
+ opRefList = new ArrayList<LogicalOperatorReference>();
+ childrenToParents.put(opRef, opRefList);
+ nextLevel.add(opRef);
+ }
+ opRefList.add(op);
+ }
+ if (op.getOperator().getInputs().size() == 0)
+ candidates.add(op);
+ }
+ if (equivalenceClasses.size() > 0) {
+ equivalenceClasses.get(0).addAll(candidates);
+ } else {
+ equivalenceClasses.add(candidates);
+ }
+ if (nextLevel.size() > 0) {
+ topDownMaterialization(nextLevel);
+ }
+ }
+
+ private void candidatesGrow(List<LogicalOperatorReference> opList, List<LogicalOperatorReference> candidates) {
+ List<LogicalOperatorReference> previousCandidates = new ArrayList<LogicalOperatorReference>();
+ previousCandidates.addAll(candidates);
+ candidates.clear();
+ boolean validCandidate = false;
+ for (LogicalOperatorReference op : opList) {
+ for (LogicalOperatorReference inputRef : op.getOperator().getInputs()) {
+ validCandidate = false;
+ // if current input is in candidates
+ for (LogicalOperatorReference candidate : previousCandidates)
+ if (inputRef.getOperator().equals(candidate.getOperator()))
+ validCandidate = true;
+ // if one input is not in candidates
+ if (!validCandidate)
+ break;
+ }
+ if (!validCandidate)
+ continue;
+ candidates.add(op);
+ }
+ }
+
+ private void prune(IOptimizationContext context) throws AlgebricksException {
+ List<List<LogicalOperatorReference>> previousEquivalenceClasses = new ArrayList<List<LogicalOperatorReference>>();
+ for (List<LogicalOperatorReference> candidates : equivalenceClasses) {
+ List<LogicalOperatorReference> candidatesCopy = new ArrayList<LogicalOperatorReference>();
+ candidatesCopy.addAll(candidates);
+ previousEquivalenceClasses.add(candidatesCopy);
+ }
+ equivalenceClasses.clear();
+ for (List<LogicalOperatorReference> candidates : previousEquivalenceClasses) {
+ boolean[] reserved = new boolean[candidates.size()];
+ for (int i = 0; i < reserved.length; i++)
+ reserved[i] = false;
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ if (reserved[i] == false) {
+ List<LogicalOperatorReference> equivalentClass = new ArrayList<LogicalOperatorReference>();
+ ILogicalOperator candidate = candidates.get(i).getOperator();
+ equivalentClass.add(candidates.get(i));
+ for (int j = i - 1; j >= 0; j--) {
+ ILogicalOperator peer = candidates.get(j).getOperator();
+ if (IsomorphismUtilities.isOperatorIsomorphic(candidate, peer)) {
+ reserved[i] = true;
+ reserved[j] = true;
+ equivalentClass.add(candidates.get(j));
+ }
+ }
+ if (equivalentClass.size() > 1) {
+ equivalenceClasses.add(equivalentClass);
+ Collections.reverse(equivalentClass);
+ }
+ }
+ }
+ for (int i = candidates.size() - 1; i >= 0; i--) {
+ if (!reserved[i]) {
+ candidates.remove(i);
+ }
+ }
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
new file mode 100644
index 0000000..e9b895e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+/**
+ * Needed only bc. current Hyrax operators require keys to be fields.
+ *
+ */
+public class ExtractGbyExpressionsRule extends AbstractExtractExprRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op1.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+
+ if (context.checkIfInDontApplySet(this, op1)) {
+ return false;
+ }
+ context.addToDontApplySet(this, op1);
+ GroupByOperator g = (GroupByOperator) op1;
+ boolean r1 = gbyExprWasRewritten(g, context);
+ boolean r2 = decorExprWasRewritten(g, context);
+ boolean fired = r1 || r2;
+ if (fired) {
+ context.computeAndSetTypeEnvironmentForOperator(g);
+ }
+ return fired;
+ }
+
+ private boolean gbyExprWasRewritten(GroupByOperator g, IOptimizationContext context) throws AlgebricksException {
+ if (!gbyHasComplexExpr(g)) {
+ return false;
+ }
+ LogicalOperatorReference opRef2 = g.getInputs().get(0);
+ for (Pair<LogicalVariable, LogicalExpressionReference> gbyPair : g.getGroupByList()) {
+ ILogicalExpression expr = gbyPair.second.getExpression();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
+ gbyPair.second.setExpression(new VariableReferenceExpression(v));
+ }
+ }
+ return true;
+ }
+
+ private boolean decorExprWasRewritten(GroupByOperator g, IOptimizationContext context) throws AlgebricksException {
+ if (!decorHasComplexExpr(g)) {
+ return false;
+ }
+ LogicalOperatorReference opRef2 = g.getInputs().get(0);
+ for (Pair<LogicalVariable, LogicalExpressionReference> decorPair : g.getDecorList()) {
+ ILogicalExpression expr = decorPair.second.getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
+ decorPair.second.setExpression(new VariableReferenceExpression(v));
+ }
+ }
+ return true;
+ }
+
+ private boolean gbyHasComplexExpr(GroupByOperator g) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> gbyPair : g.getGroupByList()) {
+ if (gbyPair.second.getExpression().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean decorHasComplexExpr(GroupByOperator g) {
+ for (Pair<LogicalVariable, LogicalExpressionReference> gbyPair : g.getDecorList()) {
+ if (gbyPair.second.getExpression().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
new file mode 100644
index 0000000..91ebde7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class FactorRedundantGroupAndDecorVarsRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator gby = (GroupByOperator) op;
+ Map<LogicalVariable, LogicalVariable> varRhsToLhs = new HashMap<LogicalVariable, LogicalVariable>();
+ boolean gvChanged = factorRedundantRhsVars(gby.getGroupByList(), opRef, varRhsToLhs, context);
+ boolean dvChanged = factorRedundantRhsVars(gby.getDecorList(), opRef, varRhsToLhs, context);
+
+ return gvChanged || dvChanged;
+ }
+
+ private boolean factorRedundantRhsVars(List<Pair<LogicalVariable, LogicalExpressionReference>> veList,
+ LogicalOperatorReference opRef, Map<LogicalVariable, LogicalVariable> varRhsToLhs,
+ IOptimizationContext context) throws AlgebricksException {
+ varRhsToLhs.clear();
+ ListIterator<Pair<LogicalVariable, LogicalExpressionReference>> iter = veList.listIterator();
+ boolean changed = false;
+ while (iter.hasNext()) {
+ Pair<LogicalVariable, LogicalExpressionReference> p = iter.next();
+ if (p.second.getExpression().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ continue;
+ }
+ LogicalVariable v = GroupByOperator.getDecorVariable(p);
+ LogicalVariable lhs = varRhsToLhs.get(v);
+ if (lhs != null) {
+ if (p.first != null) {
+ AssignOperator assign = new AssignOperator(p.first, new LogicalExpressionReference(
+ new VariableReferenceExpression(lhs)));
+ ILogicalOperator op = opRef.getOperator();
+ assign.getInputs().add(new LogicalOperatorReference(op));
+ opRef.setOperator(assign);
+ context.computeAndSetTypeEnvironmentForOperator(assign);
+ }
+ iter.remove();
+ changed = true;
+ } else {
+ varRhsToLhs.put(v, p.first);
+ }
+ }
+ return changed;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InferTypesRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InferTypesRule.java
new file mode 100644
index 0000000..a69db2a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InferTypesRule.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class InferTypesRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator op = opRef.getOperator();
+ if (context.getOutputTypeEnvironment(op) != null) {
+ return false;
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
new file mode 100644
index 0000000..3a093d7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -0,0 +1,343 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class InlineVariablesRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ /**
+ *
+ * Does one big DFS sweep over the plan.
+ *
+ */
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getOperator())) {
+ return false;
+ }
+ VariableSubstitutionVisitor substVisitor = new VariableSubstitutionVisitor(false);
+ VariableSubstitutionVisitor substVisitorForWrites = new VariableSubstitutionVisitor(true);
+ substVisitor.setContext(context);
+ substVisitorForWrites.setContext(context);
+ Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(opRef, context, true,
+ new LinkedList<EquivalenceClass>(), substVisitor, substVisitorForWrites);
+ return bb.first;
+ }
+
+ private Pair<Boolean, Boolean> collectEqClassesAndRemoveRedundantOps(LogicalOperatorReference opRef,
+ IOptimizationContext context, boolean first, List<EquivalenceClass> equivClasses,
+ VariableSubstitutionVisitor substVisitor, VariableSubstitutionVisitor substVisitorForWrites)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ // if (context.checkIfInDontApplySet(this, opRef.getOperator())) {
+ // return false;
+ // }
+ if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+ return new Pair<Boolean, Boolean>(false, false);
+ }
+ boolean modified = false;
+ boolean ecChange = false;
+ int cnt = 0;
+ for (LogicalOperatorReference i : op.getInputs()) {
+ boolean isOuterInputBranch = op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN && cnt == 1;
+ List<EquivalenceClass> eqc = isOuterInputBranch ? new LinkedList<EquivalenceClass>() : equivClasses;
+
+ Pair<Boolean, Boolean> bb = (collectEqClassesAndRemoveRedundantOps(i, context, false, eqc, substVisitor,
+ substVisitorForWrites));
+
+ if (bb.first) {
+ modified = true;
+ }
+ if (bb.second) {
+ ecChange = true;
+ }
+
+ if (isOuterInputBranch) {
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- Equivalence classes for inner branch of outer op.: "
+ + eqc + "\n");
+ }
+ for (EquivalenceClass ec : eqc) {
+ if (!ec.representativeIsConst()) {
+ equivClasses.add(ec);
+ }
+ }
+ }
+
+ ++cnt;
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans n = (AbstractOperatorWithNestedPlans) op;
+ List<EquivalenceClass> eqc = equivClasses;
+ if (n.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ eqc = new LinkedList<EquivalenceClass>();
+ } else {
+ eqc = equivClasses;
+ }
+ for (ILogicalPlan p : n.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(r, context, false, eqc,
+ substVisitor, substVisitorForWrites);
+ if (bb.first) {
+ modified = true;
+ }
+ if (bb.second) {
+ ecChange = true;
+ }
+ }
+ }
+ }
+ // we assume a variable is assigned a value only once
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator a = (AssignOperator) op;
+ ILogicalExpression rhs = a.getExpressions().get(0).getExpression();
+ if (rhs.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable varLeft = a.getVariables().get(0);
+ VariableReferenceExpression varRef = (VariableReferenceExpression) rhs;
+ LogicalVariable varRight = varRef.getVariableReference();
+
+ EquivalenceClass ecRight = findEquivClass(varRight, equivClasses);
+ if (ecRight != null) {
+ ecRight.addMember(varLeft);
+ } else {
+ List<LogicalVariable> m = new LinkedList<LogicalVariable>();
+ m.add(varRight);
+ m.add(varLeft);
+ EquivalenceClass ec = new EquivalenceClass(m, varRight);
+ equivClasses.add(ec);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- New equivalence class: " + ec + "\n");
+ }
+ }
+ ecChange = true;
+ } else if (((AbstractLogicalExpression) rhs).getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ LogicalVariable varLeft = a.getVariables().get(0);
+ List<LogicalVariable> m = new LinkedList<LogicalVariable>();
+ m.add(varLeft);
+ EquivalenceClass ec = new EquivalenceClass(m, (ConstantExpression) rhs);
+ // equivClassesForParent.add(ec);
+ equivClasses.add(ec);
+ ecChange = true;
+ }
+ } else if (op.getOperatorTag() == LogicalOperatorTag.GROUP && !(context.checkIfInDontApplySet(this, op))) {
+ GroupByOperator group = (GroupByOperator) op;
+ Pair<Boolean, Boolean> r1 = processVarExprPairs(group.getGroupByList(), equivClasses);
+ Pair<Boolean, Boolean> r2 = processVarExprPairs(group.getDecorList(), equivClasses);
+ modified = modified || r1.first || r2.first;
+ ecChange = r1.second || r2.second;
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ assignVarsNeededByProject((ProjectOperator) op, equivClasses, context);
+ } else {
+ if (op.getOperatorTag() == LogicalOperatorTag.WRITE) {
+ substVisitorForWrites.setEquivalenceClasses(equivClasses);
+ if (op.acceptExpressionTransform(substVisitorForWrites)) {
+ modified = true;
+ }
+ } else {
+ substVisitor.setEquivalenceClasses(equivClasses);
+ if (op.acceptExpressionTransform(substVisitor)) {
+ modified = true;
+ if (op.getOperatorTag() == LogicalOperatorTag.GROUP) {
+ GroupByOperator group = (GroupByOperator) op;
+ for (Pair<LogicalVariable, LogicalExpressionReference> gp : group.getGroupByList()) {
+ if (gp.first != null
+ && gp.second.getExpression().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable gv = ((VariableReferenceExpression) gp.second.getExpression())
+ .getVariableReference();
+ Iterator<Pair<LogicalVariable, LogicalExpressionReference>> iter = group.getDecorList()
+ .iterator();
+ while (iter.hasNext()) {
+ Pair<LogicalVariable, LogicalExpressionReference> dp = iter.next();
+ if (dp.first == null
+ && dp.second.getExpression().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable dv = ((VariableReferenceExpression) dp.second.getExpression())
+ .getVariableReference();
+ if (dv == gv) {
+ // The decor variable is redundant,
+ // since it is
+ // propagated as a grouping
+ // variable.
+ EquivalenceClass ec1 = findEquivClass(gv, equivClasses);
+ if (ec1 != null) {
+ ec1.addMember(gp.first);
+ ec1.setVariableRepresentative(gp.first);
+ } else {
+ List<LogicalVariable> varList = new ArrayList<LogicalVariable>();
+ varList.add(gp.first);
+ varList.add(gv);
+ ec1 = new EquivalenceClass(varList, gp.first);
+ }
+ iter.remove();
+ break;
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ return new Pair<Boolean, Boolean>(modified, ecChange);
+ }
+
+ private Pair<Boolean, Boolean> processVarExprPairs(List<Pair<LogicalVariable, LogicalExpressionReference>> vePairs,
+ List<EquivalenceClass> equivClasses) {
+ boolean ecFromGroup = false;
+ boolean modified = false;
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : vePairs) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (p.first != null && expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ LogicalVariable rhsVar = varRef.getVariableReference();
+ ecFromGroup = true;
+ EquivalenceClass ecRight = findEquivClass(rhsVar, equivClasses);
+ if (ecRight != null) {
+ LogicalVariable replacingVar = ecRight.getVariableRepresentative();
+ if (replacingVar != null && replacingVar != rhsVar) {
+ varRef.setVariable(replacingVar);
+ modified = true;
+ }
+ }
+ }
+ }
+ return new Pair<Boolean, Boolean>(modified, ecFromGroup);
+ }
+
+ // Instead of doing this, we could make Projection to be more expressive and
+ // also take constants (or even expression), at the expense of a more
+ // complex project push down.
+ private void assignVarsNeededByProject(ProjectOperator op, List<EquivalenceClass> equivClasses,
+ IOptimizationContext context) throws AlgebricksException {
+ List<LogicalVariable> prVars = op.getVariables();
+ int sz = prVars.size();
+ for (int i = 0; i < sz; i++) {
+ EquivalenceClass ec = findEquivClass(prVars.get(i), equivClasses);
+ if (ec != null) {
+ if (!ec.representativeIsConst()) {
+ prVars.set(i, ec.getVariableRepresentative());
+ }
+ }
+ }
+ }
+
+ private final static EquivalenceClass findEquivClass(LogicalVariable var, List<EquivalenceClass> equivClasses) {
+ for (EquivalenceClass ec : equivClasses) {
+ if (ec.contains(var)) {
+ return ec;
+ }
+ }
+ return null;
+ }
+
+ private class VariableSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
+ private List<EquivalenceClass> equivClasses;
+ private IOptimizationContext context;
+ private final boolean doNotSubstWithConst;
+
+ public VariableSubstitutionVisitor(boolean doNotSubstWithConst) {
+ this.doNotSubstWithConst = doNotSubstWithConst;
+ }
+
+ public void setContext(IOptimizationContext context) {
+ this.context = context;
+ }
+
+ public void setEquivalenceClasses(List<EquivalenceClass> equivClasses) {
+ this.equivClasses = equivClasses;
+ }
+
+ @Override
+ public boolean transform(LogicalExpressionReference exprRef) {
+ ILogicalExpression e = exprRef.getExpression();
+ switch (((AbstractLogicalExpression) e).getExpressionTag()) {
+ case VARIABLE: {
+ // look for a required substitution
+ LogicalVariable var = ((VariableReferenceExpression) e).getVariableReference();
+ if (context.shouldNotBeInlined(var)) {
+ return false;
+ }
+ EquivalenceClass ec = findEquivClass(var, equivClasses);
+ if (ec == null) {
+ return false;
+ }
+ if (ec.representativeIsConst()) {
+ if (doNotSubstWithConst) {
+ return false;
+ }
+ exprRef.setExpression(ec.getConstRepresentative());
+ return true;
+ } else {
+ LogicalVariable r = ec.getVariableRepresentative();
+ if (!r.equals(var)) {
+ exprRef.setExpression(new VariableReferenceExpression(r));
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+ case FUNCTION_CALL: {
+ AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) e;
+ boolean m = false;
+ for (LogicalExpressionReference arg : fce.getArguments()) {
+ if (transform(arg)) {
+ m = true;
+ }
+ }
+ return m;
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertOuterJoinRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertOuterJoinRule.java
new file mode 100644
index 0000000..d15af57
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertOuterJoinRule.java
@@ -0,0 +1,118 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.Iterator;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class InsertOuterJoinRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op0;
+
+ Iterator<ILogicalPlan> plansIter = subplan.getNestedPlans().iterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ LogicalOperatorReference subplanRoot = p.getRoots().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) subplanRoot.getOperator();
+ LogicalOperatorReference opUnder = subplan.getInputs().get(0);
+
+ if (OperatorPropertiesUtil.isNullTest((AbstractLogicalOperator) opUnder.getOperator())) {
+ return false;
+ }
+
+ switch (op1.getOperatorTag()) {
+ case INNERJOIN: {
+ InnerJoinOperator join = (InnerJoinOperator) op1;
+ LogicalOperatorReference leftRef = join.getInputs().get(0);
+ LogicalOperatorReference rightRef = join.getInputs().get(1);
+ LogicalOperatorReference ntsRef = getNtsAtEndOfPipeline(leftRef);
+ if (ntsRef == null) {
+ ntsRef = getNtsAtEndOfPipeline(rightRef);
+ if (ntsRef == null) {
+ return false;
+ } else {
+ LogicalOperatorReference t = leftRef;
+ leftRef = rightRef;
+ rightRef = t;
+ }
+ }
+ ntsRef.setOperator(opUnder.getOperator());
+ LeftOuterJoinOperator loj = new LeftOuterJoinOperator(join.getCondition());
+ loj.getInputs().add(leftRef);
+ loj.getInputs().add(rightRef);
+ opRef.setOperator(loj);
+ context.computeAndSetTypeEnvironmentForOperator(loj);
+ return true;
+ }
+ case LEFTOUTERJOIN: {
+ LeftOuterJoinOperator join = (LeftOuterJoinOperator) op1;
+ LogicalOperatorReference leftRef = join.getInputs().get(0);
+ LogicalOperatorReference ntsRef = getNtsAtEndOfPipeline(leftRef);
+ if (ntsRef == null) {
+ return false;
+ }
+ ntsRef.setOperator(opUnder.getOperator());
+ opRef.setOperator(join);
+ context.computeAndSetTypeEnvironmentForOperator(join);
+ return true;
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+
+ private LogicalOperatorReference getNtsAtEndOfPipeline(LogicalOperatorReference opRef) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return opRef;
+ }
+ if (op.getInputs().size() != 1) {
+ return null;
+ }
+ return getNtsAtEndOfPipeline(op.getInputs().get(0));
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
new file mode 100644
index 0000000..d87f00b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Triple;
+
+public class InsertProjectBeforeUnionRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ /**
+ * When the input schema to WriteOperator is different from the output
+ * schema in terms of variable order, add a project operator to get the
+ * write order
+ */
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
+ return false;
+ }
+ UnionAllOperator opUnion = (UnionAllOperator) op;
+ List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = opUnion.getVariableMappings();
+ ArrayList<LogicalVariable> usedVariablesFromOne = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> usedVariablesFromTwo = new ArrayList<LogicalVariable>();
+
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple : varMap) {
+ usedVariablesFromOne.add(triple.first);
+ usedVariablesFromTwo.add(triple.second);
+ }
+
+ ArrayList<LogicalVariable> inputSchemaOne = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(opUnion.getInputs().get(0).getOperator(), inputSchemaOne);
+
+ ArrayList<LogicalVariable> inputSchemaTwo = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(opUnion.getInputs().get(1).getOperator(), inputSchemaTwo);
+
+ boolean rewritten = false;
+ if (!isIdentical(usedVariablesFromOne, inputSchemaOne)) {
+ insertProjectOperator(opUnion, 0, usedVariablesFromOne, context);
+ rewritten = true;
+ }
+ if (!isIdentical(usedVariablesFromTwo, inputSchemaTwo)) {
+ insertProjectOperator(opUnion, 1, usedVariablesFromTwo, context);
+ rewritten = true;
+ }
+ return rewritten;
+ }
+
+ private void insertProjectOperator(UnionAllOperator opUnion, int branch, ArrayList<LogicalVariable> usedVariables,
+ IOptimizationContext context) throws AlgebricksException {
+ ProjectOperator projectOp = new ProjectOperator(usedVariables);
+ ILogicalOperator parentOp = opUnion.getInputs().get(branch).getOperator();
+ projectOp.getInputs().add(new LogicalOperatorReference(parentOp));
+ opUnion.getInputs().get(branch).setOperator(projectOp);
+ projectOp.setPhysicalOperator(new StreamProjectPOperator());
+ context.computeAndSetTypeEnvironmentForOperator(parentOp);
+ }
+
+ private boolean isIdentical(List<LogicalVariable> finalSchema, List<LogicalVariable> inputSchema)
+ throws AlgebricksException {
+ int finalSchemaSize = finalSchema.size();
+ int inputSchemaSize = inputSchema.size();
+ if (finalSchemaSize != inputSchemaSize) {
+ return false;
+ }
+ for (int i = 0; i < finalSchemaSize; i++) {
+ LogicalVariable var1 = finalSchema.get(i);
+ LogicalVariable var2 = inputSchema.get(i);
+ if (!var1.equals(var2))
+ return false;
+ }
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java
new file mode 100644
index 0000000..494a7ba
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java
@@ -0,0 +1,49 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroHashPartitionMergeExchange implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op1.getPhysicalOperator() == null
+ || op1.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_EXCHANGE) {
+ return false;
+ }
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getOperator();
+ if (op2.getPhysicalOperator() == null
+ || op2.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.SORT_MERGE_EXCHANGE) {
+ return false;
+ }
+ HashPartitionExchangePOperator hpe = (HashPartitionExchangePOperator) op1.getPhysicalOperator();
+ SortMergeExchangePOperator sme = (SortMergeExchangePOperator) op2.getPhysicalOperator();
+ List<OrderColumn> ocList = new ArrayList<OrderColumn>();
+ for (OrderColumn oc : sme.getSortColumns()) {
+ ocList.add(oc);
+ }
+ HashPartitionMergeExchangePOperator hpme = new HashPartitionMergeExchangePOperator(ocList, hpe.getHashFields(),
+ hpe.getDomain());
+ op1.setPhysicalOperator(hpme);
+ op1.getInputs().get(0).setOperator(op2.getInputs().get(0).getOperator());
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroJoinInsideSubplanRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroJoinInsideSubplanRule.java
new file mode 100644
index 0000000..31bb3b7
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroJoinInsideSubplanRule.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.ListIterator;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+
+public class IntroJoinInsideSubplanRule extends AbstractDecorrelationRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op0;
+
+ LogicalOperatorReference leftRef = subplan.getInputs().get(0);
+ if (((AbstractLogicalOperator) leftRef.getOperator()).getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ return false;
+ }
+
+ ListIterator<ILogicalPlan> plansIter = subplan.getNestedPlans().listIterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ LogicalOperatorReference opRef1 = p.getRoots().get(0);
+
+ while (true) {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef1.getOperator();
+ if (op1.getInputs().size() != 1) {
+ return false;
+ }
+ if (op1.getOperatorTag() == LogicalOperatorTag.SELECT) {
+ LogicalOperatorReference op2Ref = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.SELECT && descOrSelfIsScanOrJoin(op2)) {
+ Set<LogicalVariable> free2 = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(op2, free2);
+ if (free2.isEmpty()) {
+ Set<LogicalVariable> free1 = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(op1, free1);
+ if (!free1.isEmpty()) {
+ OperatorManipulationUtil.ntsToEts(op2Ref, context);
+ NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new LogicalOperatorReference(
+ subplan));
+ LogicalOperatorReference ntsRef = new LogicalOperatorReference(nts);
+ LogicalOperatorReference innerRef = new LogicalOperatorReference(op2);
+ InnerJoinOperator join = new InnerJoinOperator(new LogicalExpressionReference(
+ ConstantExpression.TRUE), ntsRef, innerRef);
+ op2Ref.setOperator(join);
+ context.computeAndSetTypeEnvironmentForOperator(nts);
+ context.computeAndSetTypeEnvironmentForOperator(join);
+ return true;
+ }
+ }
+ }
+ }
+ opRef1 = op1.getInputs().get(0);
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceCombinerRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceCombinerRule.java
new file mode 100644
index 0000000..d90504c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceCombinerRule.java
@@ -0,0 +1,321 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class IntroduceCombinerRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+ context.addToDontApplySet(this, op);
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator gbyOp = (GroupByOperator) op;
+ if (gbyOp.getExecutionMode() != ExecutionMode.PARTITIONED) {
+ return false;
+ }
+
+ Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap = new HashMap<AggregateFunctionCallExpression, SimilarAggregatesInfo>();
+ BookkeepingInfo bi = new BookkeepingInfo();
+ bi.toReplaceMap = toReplaceMap;
+ bi.modifGbyMap = new HashMap<GroupByOperator, List<LogicalVariable>>();
+
+ GroupByOperator newGbyOp = opToPush(gbyOp, bi, context);
+ if (newGbyOp == null) {
+ return false;
+ }
+
+ for (Map.Entry<AggregateFunctionCallExpression, SimilarAggregatesInfo> entry : toReplaceMap.entrySet()) {
+ SimilarAggregatesInfo sai = entry.getValue();
+ for (AggregateExprInfo aei : sai.simAggs) {
+ AbstractFunctionCallExpression afce = (AbstractFunctionCallExpression) aei.aggExprRef.getExpression();
+ afce.setFunctionInfo(aei.newFunInfo);
+ afce.getArguments().clear();
+ afce.getArguments().add(new LogicalExpressionReference(sai.stepOneResult));
+ }
+ }
+
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gbyOp.getDecorList()) {
+ LogicalVariable newDecorVar = context.newVar();
+ newGbyOp.addDecorExpression(newDecorVar, p.second.getExpression());
+ p.second.setExpression(new VariableReferenceExpression(newDecorVar));
+ }
+ newGbyOp.setExecutionMode(ExecutionMode.LOCAL);
+ Object v = gbyOp.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY);
+ newGbyOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, v);
+
+ Object v2 = gbyOp.getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY);
+ newGbyOp.getAnnotations().put(OperatorAnnotations.USE_EXTERNAL_GROUP_BY, v2);
+
+ List<LogicalVariable> propagatedVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(newGbyOp, propagatedVars);
+
+ Set<LogicalVariable> freeVars = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSubplans(gbyOp, freeVars);
+
+ for (LogicalVariable var : freeVars) {
+ if (!propagatedVars.contains(var)) {
+ LogicalVariable newDecorVar = context.newVar();
+ newGbyOp.addDecorExpression(newDecorVar, new VariableReferenceExpression(var));
+ VariableUtilities.substituteVariables(gbyOp.getNestedPlans().get(0).getRoots().get(0).getOperator(),
+ var, newDecorVar, context);
+ }
+ }
+
+ LogicalOperatorReference opRef3 = gbyOp.getInputs().get(0);
+ opRef3.setOperator(newGbyOp);
+ typeGby(newGbyOp, context);
+ typeGby(gbyOp, context);
+ return true;
+ }
+
+ private void typeGby(AbstractOperatorWithNestedPlans op, IOptimizationContext context) throws AlgebricksException {
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ OperatorPropertiesUtil.typePlan(p, context);
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+
+ private GroupByOperator opToPush(GroupByOperator gbyOp, BookkeepingInfo bi, IOptimizationContext context)
+ throws AlgebricksException {
+
+ LogicalOperatorReference opRef3 = gbyOp.getInputs().get(0);
+ ILogicalOperator op3 = opRef3.getOperator();
+ GroupByOperator newGbyOp = new GroupByOperator();
+ newGbyOp.getInputs().add(new LogicalOperatorReference(op3));
+ // copy annotations
+ Map<String, Object> annotations = newGbyOp.getAnnotations();
+ for (Entry<String, Object> a : gbyOp.getAnnotations().entrySet())
+ annotations.put(a.getKey(), a.getValue());
+
+ List<LogicalVariable> gbyVars = gbyOp.getGbyVarList();
+
+ for (ILogicalPlan p : gbyOp.getNestedPlans()) {
+ Pair<Boolean, ILogicalPlan> bip = tryToPushSubplan(p, gbyOp, newGbyOp, bi, gbyVars, context);
+ if (!bip.first) {
+ // for now, if we cannot push everything, give up
+ return null;
+ }
+ ILogicalPlan pushedSubplan = bip.second;
+ if (pushedSubplan != null) {
+ newGbyOp.getNestedPlans().add(pushedSubplan);
+ }
+ }
+
+ ArrayList<LogicalVariable> newOpGbyList = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalVariable> replGbyList = new ArrayList<LogicalVariable>();
+ // find maximal sequence of variable
+ for (Map.Entry<GroupByOperator, List<LogicalVariable>> e : bi.modifGbyMap.entrySet()) {
+ List<LogicalVariable> varList = e.getValue();
+ boolean see1 = true;
+ int sz1 = newOpGbyList.size();
+ int i = 0;
+ for (LogicalVariable v : varList) {
+ if (see1) {
+ if (i < sz1) {
+ LogicalVariable v2 = newOpGbyList.get(i);
+ if (v != v2) {
+ // cannot linearize
+ return null;
+ }
+ } else {
+ see1 = false;
+ newOpGbyList.add(v);
+ replGbyList.add(context.newVar());
+ }
+ i++;
+ } else {
+ newOpGbyList.add(v);
+ replGbyList.add(context.newVar());
+ }
+ }
+ }
+ // set the vars in the new op
+ int n = newOpGbyList.size();
+ for (int i = 0; i < n; i++) {
+ newGbyOp.addGbyExpression(replGbyList.get(i), new VariableReferenceExpression(newOpGbyList.get(i)));
+ VariableUtilities.substituteVariables(gbyOp, newOpGbyList.get(i), replGbyList.get(i), false, context);
+ }
+ return newGbyOp;
+ }
+
+ private Pair<Boolean, ILogicalPlan> tryToPushSubplan(ILogicalPlan p, GroupByOperator oldGbyOp,
+ GroupByOperator newGbyOp, BookkeepingInfo bi, List<LogicalVariable> gbyVars, IOptimizationContext context) {
+ List<LogicalOperatorReference> pushedRoots = new ArrayList<LogicalOperatorReference>();
+ List<LogicalOperatorReference> toPushR = new ArrayList<LogicalOperatorReference>();
+ for (LogicalOperatorReference r : p.getRoots()) {
+ if (!tryToPushRoot(r, oldGbyOp, newGbyOp, bi, gbyVars, context, toPushR)) {
+ // for now, if we cannot push everything, give up
+ return new Pair<Boolean, ILogicalPlan>(false, null);
+ }
+ }
+ for (LogicalOperatorReference root : toPushR) {
+ pushedRoots.add(root);
+ }
+ if (pushedRoots.isEmpty()) {
+ return new Pair<Boolean, ILogicalPlan>(true, null);
+ } else {
+ return new Pair<Boolean, ILogicalPlan>(true, new ALogicalPlanImpl(pushedRoots));
+ }
+ }
+
+ private boolean tryToPushRoot(LogicalOperatorReference r, GroupByOperator oldGbyOp, GroupByOperator newGbyOp,
+ BookkeepingInfo bi, List<LogicalVariable> gbyVars, IOptimizationContext context,
+ List<LogicalOperatorReference> toPushAccumulate) {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) r.getOperator();
+ if (op1.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getOperator();
+ if (op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ AggregateOperator initAgg = (AggregateOperator) op1;
+ Pair<Boolean, LogicalOperatorReference> pOpRef = tryToPushAgg(initAgg, newGbyOp, bi.toReplaceMap, context);
+ if (!pOpRef.first) {
+ return false;
+ }
+ LogicalOperatorReference opRef = pOpRef.second;
+ if (opRef != null) {
+ toPushAccumulate.add(opRef);
+ }
+ bi.modifGbyMap.put(oldGbyOp, gbyVars);
+ return true;
+ } else {
+ while (op2.getOperatorTag() != LogicalOperatorTag.GROUP && op2.getInputs().size() == 1) {
+ op2 = (AbstractLogicalOperator) op2.getInputs().get(0).getOperator();
+ }
+ if (op2.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator nestedGby = (GroupByOperator) op2;
+ List<LogicalVariable> gbyVars2 = nestedGby.getGbyVarList();
+ List<LogicalVariable> concatGbyVars = new ArrayList<LogicalVariable>(gbyVars);
+ concatGbyVars.addAll(gbyVars2);
+ for (ILogicalPlan p : nestedGby.getNestedPlans()) {
+ for (LogicalOperatorReference r2 : p.getRoots()) {
+ if (!tryToPushRoot(r2, nestedGby, newGbyOp, bi, concatGbyVars, context, toPushAccumulate)) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+ }
+
+ private Pair<Boolean, LogicalOperatorReference> tryToPushAgg(AggregateOperator initAgg, GroupByOperator newGbyOp,
+ Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap, IOptimizationContext context) {
+
+ ArrayList<LogicalVariable> pushedVars = new ArrayList<LogicalVariable>();
+ ArrayList<LogicalExpressionReference> pushedExprs = new ArrayList<LogicalExpressionReference>();
+
+ List<LogicalVariable> initVars = initAgg.getVariables();
+ List<LogicalExpressionReference> initExprs = initAgg.getExpressions();
+ int sz = initVars.size();
+ for (int i = 0; i < sz; i++) {
+ AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) initExprs.get(i).getExpression();
+ if (!aggFun.isTwoStep()) {
+ return new Pair<Boolean, LogicalOperatorReference>(false, null);
+ }
+ }
+
+ boolean haveAggToReplace = false;
+ for (int i = 0; i < sz; i++) {
+ LogicalExpressionReference expRef = initExprs.get(i);
+ AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) expRef.getExpression();
+ IFunctionInfo fi1 = aggFun.getStepOneAggregate();
+ List<LogicalExpressionReference> newArgs = new ArrayList<LogicalExpressionReference>(aggFun.getArguments()
+ .size());
+ for (LogicalExpressionReference er : aggFun.getArguments()) {
+ newArgs.add(new LogicalExpressionReference(er.getExpression().cloneExpression()));
+ }
+// AggregateFunctionCallExpression aggLocal = new AggregateFunctionCallExpression(fi1, false, newArgs);
+// pushedExprs.add(new LogicalExpressionReference(aggLocal));
+
+ IFunctionInfo fi2 = aggFun.getStepTwoAggregate();
+
+ SimilarAggregatesInfo inf = toReplaceMap.get(aggFun);
+ if (inf == null) {
+ inf = new SimilarAggregatesInfo();
+ LogicalVariable newAggVar = context.newVar();
+ pushedVars.add(newAggVar);
+ inf.stepOneResult = new VariableReferenceExpression(newAggVar);
+ inf.simAggs = new ArrayList<AggregateExprInfo>();
+ toReplaceMap.put(aggFun, inf);
+ AggregateFunctionCallExpression aggLocal = new AggregateFunctionCallExpression(fi1, false, newArgs);
+ pushedExprs.add(new LogicalExpressionReference(aggLocal));
+ }
+ AggregateExprInfo aei = new AggregateExprInfo();
+ aei.aggExprRef = expRef;
+ aei.newFunInfo = fi2;
+ inf.simAggs.add(aei);
+ haveAggToReplace = true;
+ }
+
+ if (!pushedVars.isEmpty()) {
+ AggregateOperator pushedAgg = new AggregateOperator(pushedVars, pushedExprs);
+ pushedAgg.setExecutionMode(ExecutionMode.LOCAL);
+ NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new LogicalOperatorReference(newGbyOp));
+ nts.setExecutionMode(ExecutionMode.LOCAL);
+ pushedAgg.getInputs().add(new LogicalOperatorReference(nts));
+ return new Pair<Boolean, LogicalOperatorReference>(true, new LogicalOperatorReference(pushedAgg));
+ } else {
+ return new Pair<Boolean, LogicalOperatorReference>(haveAggToReplace, null);
+ }
+ }
+
+ private class SimilarAggregatesInfo {
+ ILogicalExpression stepOneResult;
+ List<AggregateExprInfo> simAggs;
+ }
+
+ private class AggregateExprInfo {
+ LogicalExpressionReference aggExprRef;
+ IFunctionInfo newFunInfo;
+ }
+
+ private class BookkeepingInfo {
+ Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap;
+ Map<GroupByOperator, List<LogicalVariable>> modifGbyMap;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForStandaloneAggregRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForStandaloneAggregRule.java
new file mode 100644
index 0000000..eeaac23
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForStandaloneAggregRule.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+/**
+ *
+ * When aggregates appear w/o group-by, a default group by a constant is
+ * introduced.
+ *
+ */
+
+public class IntroduceGroupByForStandaloneAggregRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ LogicalOperatorReference opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+
+ AssignOperator assign = (AssignOperator) op;
+ AggregateOperator agg = (AggregateOperator) op2;
+ if (agg.getVariables().size() != 1) {
+ return false;
+ }
+ LogicalVariable aggVar = agg.getVariables().get(0);
+ List<LogicalVariable> used = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(assign, used);
+ if (used.contains(aggVar)) {
+ LogicalOperatorReference opRef3 = op2.getInputs().get(0);
+ List<Pair<LogicalVariable, LogicalExpressionReference>> groupByList = new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>();
+ LogicalVariable gbyVar = context.newVar();
+ // ILogicalExpression constOne = new ConstantExpression(new
+ // IntegerLiteral(new Integer(1)));
+ groupByList.add(new Pair<LogicalVariable, LogicalExpressionReference>(gbyVar,
+ new LogicalExpressionReference(ConstantExpression.TRUE)));
+ NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new LogicalOperatorReference());
+ List<LogicalOperatorReference> aggInpList = agg.getInputs();
+ aggInpList.clear();
+ aggInpList.add(new LogicalOperatorReference(nts));
+ ILogicalPlan np1 = new ALogicalPlanImpl(opRef2);
+ ArrayList<ILogicalPlan> nestedPlans = new ArrayList<ILogicalPlan>();
+ nestedPlans.add(np1);
+ GroupByOperator gbyOp = new GroupByOperator(groupByList,
+ new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>(), nestedPlans);
+ LogicalOperatorReference opRefGby = new LogicalOperatorReference(gbyOp);
+ nts.getDataSourceReference().setOperator(gbyOp);
+ gbyOp.getInputs().add(opRef3);
+ List<LogicalOperatorReference> asgnInpList = assign.getInputs();
+ context.computeAndSetTypeEnvironmentForOperator(nts);
+ context.computeAndSetTypeEnvironmentForOperator(agg);
+ context.computeAndSetTypeEnvironmentForOperator(gbyOp);
+ asgnInpList.clear();
+ asgnInpList.add(opRefGby);
+ return true;
+ }
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
new file mode 100644
index 0000000..4f9b2ea
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
@@ -0,0 +1,267 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+
+public class IntroduceGroupByForSubplanRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op0.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op0;
+
+ Iterator<ILogicalPlan> plansIter = subplan.getNestedPlans().iterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ LogicalOperatorReference subplanRoot = p.getRoots().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) subplanRoot.getOperator();
+
+ LogicalOperatorReference botRef = subplanRoot;
+ AbstractLogicalOperator op2;
+ // Project is optional
+ if (op1.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ op2 = op1;
+ } else {
+ ProjectOperator project = (ProjectOperator) op1;
+ botRef = project.getInputs().get(0);
+ op2 = (AbstractLogicalOperator) botRef.getOperator();
+ }
+ if (op2.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ AggregateOperator aggregate = (AggregateOperator) op2;
+
+ Set<LogicalVariable> free = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(aggregate, free);
+
+ LogicalOperatorReference op3Ref = aggregate.getInputs().get(0);
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) op3Ref.getOperator();
+
+ while (op3.getInputs().size() == 1) {
+ Set<LogicalVariable> prod = new HashSet<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op3, prod);
+ free.removeAll(prod);
+ VariableUtilities.getUsedVariables(op3, free);
+ botRef = op3Ref;
+ op3Ref = op3.getInputs().get(0);
+ op3 = (AbstractLogicalOperator) op3Ref.getOperator();
+ }
+
+ if (op3.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && op3.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op3;
+ if (join.getCondition().getExpression() == ConstantExpression.TRUE) {
+ return false;
+ }
+ VariableUtilities.getUsedVariables(join, free);
+
+ AbstractLogicalOperator b0 = (AbstractLogicalOperator) join.getInputs().get(0).getOperator();
+ // see if there's an NTS at the end of the pipeline
+ NestedTupleSourceOperator outerNts = getNts(b0);
+ if (outerNts == null) {
+ AbstractLogicalOperator b1 = (AbstractLogicalOperator) join.getInputs().get(1).getOperator();
+ outerNts = getNts(b1);
+ if (outerNts == null) {
+ return false;
+ }
+ }
+
+ Set<LogicalVariable> pkVars = computeGbyVars(outerNts, free, context);
+ if (pkVars == null || pkVars.size() < 1) {
+ // could not group only by primary keys
+ return false;
+ }
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Found FD for introducing group-by: " + pkVars);
+
+ LogicalOperatorReference rightRef = join.getInputs().get(1);
+ LogicalVariable testForNull = null;
+ AbstractLogicalOperator right = (AbstractLogicalOperator) rightRef.getOperator();
+ switch (right.getOperatorTag()) {
+ case UNNEST: {
+ UnnestOperator innerUnnest = (UnnestOperator) right;
+ // Select [ $y != null ]
+ testForNull = innerUnnest.getVariable();
+ break;
+ }
+ case DATASOURCESCAN: {
+ DataSourceScanOperator innerScan = (DataSourceScanOperator) right;
+ // Select [ $y != null ]
+ if (innerScan.getVariables().size() == 1) {
+ testForNull = innerScan.getVariables().get(0);
+ }
+ break;
+ }
+ }
+ if (testForNull == null) {
+ testForNull = context.newVar();
+ AssignOperator tmpAsgn = new AssignOperator(testForNull, new LogicalExpressionReference(
+ ConstantExpression.TRUE));
+ tmpAsgn.getInputs().add(new LogicalOperatorReference(rightRef.getOperator()));
+ rightRef.setOperator(tmpAsgn);
+ context.computeAndSetTypeEnvironmentForOperator(tmpAsgn);
+ }
+
+ IFunctionInfo finfoEq = AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.IS_NULL);
+ ILogicalExpression isNullTest = new ScalarFunctionCallExpression(finfoEq, new LogicalExpressionReference(
+ new VariableReferenceExpression(testForNull)));
+ IFunctionInfo finfoNot = AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.NOT);
+ ScalarFunctionCallExpression nonNullTest = new ScalarFunctionCallExpression(finfoNot,
+ new LogicalExpressionReference(isNullTest));
+ SelectOperator selectNonNull = new SelectOperator(new LogicalExpressionReference(nonNullTest));
+ GroupByOperator g = new GroupByOperator();
+ LogicalOperatorReference newSubplanRef = new LogicalOperatorReference(subplan);
+ NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new LogicalOperatorReference(g));
+ opRef.setOperator(g);
+ selectNonNull.getInputs().add(new LogicalOperatorReference(nts));
+
+ List<LogicalOperatorReference> prodInpList = botRef.getOperator().getInputs();
+ prodInpList.clear();
+ prodInpList.add(new LogicalOperatorReference(selectNonNull));
+
+ ILogicalPlan gPlan = new ALogicalPlanImpl(new LogicalOperatorReference(subplanRoot.getOperator()));
+ g.getNestedPlans().add(gPlan);
+ subplanRoot.setOperator(op3Ref.getOperator());
+ g.getInputs().add(newSubplanRef);
+
+ HashSet<LogicalVariable> underVars = new HashSet<LogicalVariable>();
+ VariableUtilities.getLiveVariables(subplan.getInputs().get(0).getOperator(), underVars);
+ underVars.removeAll(pkVars);
+ Map<LogicalVariable, LogicalVariable> mappedVars = buildVarExprList(pkVars, context, g, g.getGroupByList());
+ context.updatePrimaryKeys(mappedVars);
+ for (LogicalVariable uv : underVars) {
+ g.getDecorList().add(
+ new Pair<LogicalVariable, LogicalExpressionReference>(null, new LogicalExpressionReference(
+ new VariableReferenceExpression(uv))));
+ }
+ OperatorPropertiesUtil.typeOpRec(subplanRoot, context);
+ OperatorPropertiesUtil.typeOpRec(gPlan.getRoots().get(0), context);
+ context.computeAndSetTypeEnvironmentForOperator(g);
+ return true;
+ }
+
+ private NestedTupleSourceOperator getNts(AbstractLogicalOperator op) {
+ AbstractLogicalOperator alo = op;
+ do {
+ if (alo.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return (NestedTupleSourceOperator) alo;
+ }
+ if (alo.getInputs().size() != 1) {
+ return null;
+ }
+ alo = (AbstractLogicalOperator) alo.getInputs().get(0).getOperator();
+ } while (true);
+ }
+
+ protected Set<LogicalVariable> computeGbyVars(AbstractLogicalOperator op, Set<LogicalVariable> freeVars,
+ IOptimizationContext context) throws AlgebricksException {
+ PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
+ List<FunctionalDependency> fdList = context.getFDList(op);
+ if (fdList == null) {
+ return null;
+ }
+ // check if any of the FDs is a key
+ List<LogicalVariable> all = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(op, all);
+ all.retainAll(freeVars);
+ for (FunctionalDependency fd : fdList) {
+ if (fd.getTail().containsAll(all)) {
+ return new HashSet<LogicalVariable>(fd.getHead());
+ }
+ }
+ return null;
+ }
+
+ private Map<LogicalVariable, LogicalVariable> buildVarExprList(Collection<LogicalVariable> vars,
+ IOptimizationContext context, GroupByOperator g,
+ List<Pair<LogicalVariable, LogicalExpressionReference>> outVeList) throws AlgebricksException {
+ Map<LogicalVariable, LogicalVariable> m = new HashMap<LogicalVariable, LogicalVariable>();
+ for (LogicalVariable ov : vars) {
+ LogicalVariable newVar = context.newVar();
+ ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+ outVeList.add(new Pair<LogicalVariable, LogicalExpressionReference>(ov, new LogicalExpressionReference(
+ varExpr)));
+ for (ILogicalPlan p : g.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getOperator(), ov, newVar,
+ true, context);
+ }
+ }
+ AbstractLogicalOperator opUnder = (AbstractLogicalOperator) g.getInputs().get(0).getOperator();
+ OperatorManipulationUtil.substituteVarRec(opUnder, ov, newVar, true, context);
+ m.put(ov, newVar);
+ }
+ return m;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
new file mode 100644
index 0000000..40b5c85
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
@@ -0,0 +1,137 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IsolateHyracksOperatorsRule implements IAlgebraicRewriteRule {
+
+ private final PhysicalOperatorTag[] operatorsBelowWhichJobGenIsDisabled;
+
+ public IsolateHyracksOperatorsRule(PhysicalOperatorTag[] operatorsBelowWhichJobGenIsDisabled) {
+ this.operatorsBelowWhichJobGenIsDisabled = operatorsBelowWhichJobGenIsDisabled;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ IPhysicalOperator pt = op.getPhysicalOperator();
+
+ if (pt == null || op.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ return false;
+ }
+ if (!pt.isMicroOperator()) {
+ return testIfExchangeBelow(opRef, context);
+ } else {
+ return testIfExchangeAbove(opRef, context);
+ }
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ private boolean testIfExchangeBelow(LogicalOperatorReference opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ boolean exchInserted = false;
+
+ for (LogicalOperatorReference i : op.getInputs()) {
+ AbstractLogicalOperator c = (AbstractLogicalOperator) i.getOperator();
+ if (c.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+ if (c.getPhysicalOperator() == null) {
+ return false;
+ }
+ insertOneToOneExchange(i, context);
+ exchInserted = true;
+ }
+ }
+ IPhysicalOperator pt = op.getPhysicalOperator();
+ if (pt.isJobGenDisabledBelowMe() || arrayContains(operatorsBelowWhichJobGenIsDisabled, pt.getOperatorTag())) {
+ for (LogicalOperatorReference i : op.getInputs()) {
+ disableJobGenRec(i.getOperator());
+ }
+ }
+ return exchInserted;
+ }
+
+ private void disableJobGenRec(ILogicalOperator operator) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) operator;
+ op.disableJobGen();
+ for (LogicalOperatorReference i : op.getInputs()) {
+ disableJobGenRec(i.getOperator());
+ }
+ }
+
+ private boolean testIfExchangeAbove(LogicalOperatorReference opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ return false;
+ }
+ boolean exchInserted = false;
+ for (LogicalOperatorReference i : op.getInputs()) {
+ AbstractLogicalOperator c = (AbstractLogicalOperator) i.getOperator();
+ IPhysicalOperator cpop = c.getPhysicalOperator();
+ if (c.getOperatorTag() == LogicalOperatorTag.EXCHANGE || cpop == null) {
+ continue;
+ }
+ if (!cpop.isMicroOperator()) {
+ insertOneToOneExchange(i, context);
+ exchInserted = true;
+ }
+ }
+ return exchInserted;
+ }
+
+ private final static <T> boolean arrayContains(T[] array, T tag) {
+ for (int i = 0; i < array.length; i++) {
+ if (array[i] == tag) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private final static void insertOneToOneExchange(LogicalOperatorReference i, IOptimizationContext context)
+ throws AlgebricksException {
+ ExchangeOperator e = new ExchangeOperator();
+ e.setPhysicalOperator(new OneToOneExchangePOperator());
+ ILogicalOperator inOp = i.getOperator();
+
+ e.getInputs().add(new LogicalOperatorReference(inOp));
+ i.setOperator(e);
+ // e.recomputeSchema();
+ OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(e, context);
+ ExecutionMode em = ((AbstractLogicalOperator) inOp).getExecutionMode();
+ e.setExecutionMode(em);
+ e.computeDeliveredPhysicalProperties(context);
+ context.computeAndSetTypeEnvironmentForOperator(e);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
new file mode 100644
index 0000000..6755e59
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PullSelectOutOfEqJoin implements IAlgebraicRewriteRule {
+
+ private List<LogicalExpressionReference> eqVarVarComps = new ArrayList<LogicalExpressionReference>();
+ private List<LogicalExpressionReference> otherPredicates = new ArrayList<LogicalExpressionReference>();
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+
+ if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
+
+ ILogicalExpression expr = join.getCondition().getExpression();
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier fi = fexp.getFunctionIdentifier();
+ if (fi != AlgebricksBuiltinFunctions.AND) {
+ return false;
+ }
+ eqVarVarComps.clear();
+ otherPredicates.clear();
+ for (LogicalExpressionReference arg : fexp.getArguments()) {
+ if (isEqVarVar(arg.getExpression())) {
+ eqVarVarComps.add(arg);
+ } else {
+ otherPredicates.add(arg);
+ }
+ }
+ if (eqVarVarComps.isEmpty() || otherPredicates.isEmpty()) {
+ return false;
+ }
+ // pull up
+ ILogicalExpression pulledCond = makeCondition(otherPredicates);
+ SelectOperator select = new SelectOperator(new LogicalExpressionReference(pulledCond));
+ ILogicalExpression newJoinCond = makeCondition(eqVarVarComps);
+ join.getCondition().setExpression(newJoinCond);
+ select.getInputs().add(new LogicalOperatorReference(join));
+ opRef.setOperator(select);
+ context.computeAndSetTypeEnvironmentForOperator(select);
+ return true;
+ }
+
+ private ILogicalExpression makeCondition(List<LogicalExpressionReference> predList) {
+ if (predList.size() > 1) {
+ IFunctionInfo finfo = AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND);
+ return new ScalarFunctionCallExpression(finfo, predList);
+ } else {
+ return predList.get(0).getExpression();
+ }
+ }
+
+ private boolean isEqVarVar(ILogicalExpression expr) {
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+ if (f.getFunctionIdentifier() != AlgebricksBuiltinFunctions.EQ) {
+ return false;
+ }
+ ILogicalExpression e1 = f.getArguments().get(0).getExpression();
+ if (e1.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return false;
+ } else {
+ ILogicalExpression e2 = f.getArguments().get(1).getExpression();
+ return e2.getExpressionTag() == LogicalExpressionTag.VARIABLE;
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignDownThroughProductRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignDownThroughProductRule.java
new file mode 100644
index 0000000..b7e650c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushAssignDownThroughProductRule.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushAssignDownThroughProductRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op1.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return false;
+ }
+ LogicalOperatorReference op2Ref = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op2;
+ if (join.getCondition().getExpression() != ConstantExpression.TRUE) {
+ return false;
+ }
+
+ List<LogicalVariable> used = new ArrayList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op1, used);
+
+ LogicalOperatorReference b0Ref = op2.getInputs().get(0);
+ ILogicalOperator b0 = b0Ref.getOperator();
+ List<LogicalVariable> b0Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b0, b0Scm);
+ if (b0Scm.containsAll(used)) {
+ // push assign on left branch
+ op2Ref.setOperator(b0);
+ b0Ref.setOperator(op1);
+ opRef.setOperator(op2);
+ return true;
+ } else {
+ LogicalOperatorReference b1Ref = op2.getInputs().get(1);
+ ILogicalOperator b1 = b1Ref.getOperator();
+ List<LogicalVariable> b1Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b1, b1Scm);
+ if (b1Scm.containsAll(used)) {
+ // push assign on right branch
+ op2Ref.setOperator(b1);
+ b1Ref.setOperator(op1);
+ opRef.setOperator(op2);
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushDieUpRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushDieUpRule.java
new file mode 100644
index 0000000..2a71c1b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushDieUpRule.java
@@ -0,0 +1,48 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushDieUpRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op0.getInputs().size() == 0)
+ return false;
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) op0.getInputs().get(0).getOperator();
+
+ if (op1.getInputs().size() == 0)
+ return false;
+ LogicalOperatorTag tag = op1.getOperatorTag();
+ if (tag == LogicalOperatorTag.SINK || tag == LogicalOperatorTag.WRITE
+ || tag == LogicalOperatorTag.INSERT_DELETE || tag == LogicalOperatorTag.WRITE_RESULT)
+ return false;
+
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getOperator();
+ if (op2.getOperatorTag() == LogicalOperatorTag.DIE) {
+ op0.getInputs().get(0).setOperator(op2);
+ op1.getInputs().clear();
+ for (LogicalOperatorReference ref : op2.getInputs())
+ op1.getInputs().add(ref);
+ op2.getInputs().clear();
+ op2.getInputs().add(new LogicalOperatorReference(op1));
+
+ context.computeAndSetTypeEnvironmentForOperator(op0);
+ context.computeAndSetTypeEnvironmentForOperator(op1);
+ context.computeAndSetTypeEnvironmentForOperator(op2);
+ return true;
+ } else {
+ return false;
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java
new file mode 100644
index 0000000..467e8cb
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.LinkedList;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamLimitPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushLimitDownRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ /**
+ * When a global Limit over a merge-exchange is found, a local Limit is
+ * pushed down.
+ *
+ */
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.LIMIT) {
+ return false;
+ }
+ LimitOperator opLim = (LimitOperator) op;
+ if (!opLim.isTopmostLimitOp()) {
+ return false;
+ }
+
+ LogicalOperatorReference opRef2 = opLim.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+
+ if (context.checkAndAddToAlreadyCompared(op, op2)) {
+ return false;
+ }
+ if (op2.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+ return false;
+ }
+ PhysicalOperatorTag op2PTag = op2.getPhysicalOperator().getOperatorTag();
+ // we should test for any kind of merge
+ if (op2PTag != PhysicalOperatorTag.RANDOM_MERGE_EXCHANGE && op2PTag != PhysicalOperatorTag.SORT_MERGE_EXCHANGE) {
+ return false;
+ }
+
+ LinkedList<LogicalVariable> usedVars1 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(opLim, usedVars1);
+
+ do {
+ if (op2.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.LIMIT) {
+ return false;
+ }
+ if (op2.getInputs().size() > 1 || !op2.isMap()) {
+ break;
+ }
+ LinkedList<LogicalVariable> vars2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, vars2);
+ if (!OperatorPropertiesUtil.disjoint(vars2, usedVars1)) {
+ return false;
+ }
+ // we assume pipelineable ops. have only one input
+ opRef2 = op2.getInputs().get(0);
+ op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ } while (true);
+
+ LimitOperator clone2 = null;
+ if (opLim.getOffset().getExpression() == null) {
+ clone2 = new LimitOperator(opLim.getMaxObjects().getExpression(), false);
+ } else {
+ // push limit (max+offset)
+ IFunctionInfo finfoAdd = AlgebricksBuiltinFunctions
+ .getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.NUMERIC_ADD);
+ ScalarFunctionCallExpression maxPlusOffset = new ScalarFunctionCallExpression(finfoAdd, opLim
+ .getMaxObjects(), opLim.getOffset());
+ clone2 = new LimitOperator(maxPlusOffset, false);
+ }
+ clone2.setPhysicalOperator(new StreamLimitPOperator(false));
+ clone2.getInputs().add(new LogicalOperatorReference(op2));
+ clone2.setExecutionMode(op2.getExecutionMode());
+ clone2.recomputeSchema();
+ opRef2.setOperator(clone2);
+ context.computeAndSetTypeEnvironmentForOperator(clone2);
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
new file mode 100644
index 0000000..72bef8d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
@@ -0,0 +1,117 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+public class PushNestedOrderByUnderPreSortedGroupByRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ if (op.getPhysicalOperator() == null) {
+ return false;
+ }
+ AbstractPhysicalOperator pOp = (AbstractPhysicalOperator) op.getPhysicalOperator();
+ if (pOp.getOperatorTag() != PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY) {
+ return false;
+ }
+ GroupByOperator gby = (GroupByOperator) op;
+ ILogicalPlan plan = gby.getNestedPlans().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) plan.getRoots().get(0).getOperator();
+ if (op1.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+ LogicalOperatorReference opRef2 = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.ORDER) {
+ return false;
+ }
+ OrderOperator order1 = (OrderOperator) op2;
+ if (!isIndependentFromChildren(order1)) {
+ return false;
+ }
+ AbstractPhysicalOperator pOrder1 = (AbstractPhysicalOperator) op2.getPhysicalOperator();
+ if (pOrder1.getOperatorTag() != PhysicalOperatorTag.STABLE_SORT
+ && pOrder1.getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT) {
+ return false;
+ }
+ // StableSortPOperator sort1 = (StableSortPOperator) pOrder1;
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ if (op3.getOperatorTag() != LogicalOperatorTag.ORDER) {
+ return false;
+ }
+ AbstractPhysicalOperator pOp3 = (AbstractPhysicalOperator) op3.getPhysicalOperator();
+ if (pOp3.getOperatorTag() != PhysicalOperatorTag.STABLE_SORT) {
+ return false;
+ }
+ OrderOperator order2 = (OrderOperator) op3;
+ StableSortPOperator sort2 = (StableSortPOperator) pOp3;
+ // int n1 = sort1.getSortColumns().length;
+ // int n2 = sort2.getSortColumns().length;
+ // OrderColumn[] sortColumns = new OrderColumn[n2 + n1];
+ // System.arraycopy(sort2.getSortColumns(), 0, sortColumns, 0, n2);
+ // int k = 0;
+ for (Pair<IOrder, LogicalExpressionReference> oe : order1.getOrderExpressions()) {
+ order2.getOrderExpressions().add(oe);
+ // sortColumns[n2 + k] = sort1.getSortColumns()[k];
+ // ++k;
+ }
+ // sort2.setSortColumns(sortColumns);
+ sort2.computeDeliveredProperties(order2, null);
+ // remove order1
+ ILogicalOperator underOrder1 = order1.getInputs().get(0).getOperator();
+ opRef2.setOperator(underOrder1);
+ return true;
+ }
+
+ private boolean isIndependentFromChildren(OrderOperator order1) throws AlgebricksException {
+ Set<LogicalVariable> free = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(order1, free);
+ Set<LogicalVariable> usedInOrder = new HashSet<LogicalVariable>();
+ VariableUtilities.getUsedVariables(order1, usedInOrder);
+ return free.containsAll(usedInOrder);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
new file mode 100644
index 0000000..f393739
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
@@ -0,0 +1,218 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+
+/**
+ *
+ * Pushes projections through its input operator, provided that operator does
+ * not produce the projected variables.
+ *
+ * @author Nicola
+ *
+ */
+public class PushProjectDownRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ ProjectOperator pi = (ProjectOperator) op;
+ LogicalOperatorReference opRef2 = pi.getInputs().get(0);
+
+ HashSet<LogicalVariable> toPush = new HashSet<LogicalVariable>();
+ toPush.addAll(pi.getVariables());
+
+ Pair<Boolean, Boolean> p = pushThroughOp(toPush, opRef2, op, context);
+ boolean smthWasPushed = p.first;
+ if (p.second) { // the original projection is redundant
+ opRef.setOperator(op.getInputs().get(0).getOperator());
+ smthWasPushed = true;
+ }
+
+ return smthWasPushed;
+ }
+
+ private static Pair<Boolean, Boolean> pushThroughOp(HashSet<LogicalVariable> toPush,
+ LogicalOperatorReference opRef2, ILogicalOperator initialOp, IOptimizationContext context)
+ throws AlgebricksException {
+ List<LogicalVariable> initProjectList = new ArrayList<LogicalVariable>(toPush);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ do {
+ if (op2.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE
+ || op2.getOperatorTag() == LogicalOperatorTag.PROJECT
+ || op2.getOperatorTag() == LogicalOperatorTag.REPLICATE) {
+ return new Pair<Boolean, Boolean>(false, false);
+ }
+ if (!op2.isMap()) {
+ break;
+ }
+ LinkedList<LogicalVariable> usedVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op2, usedVars);
+ toPush.addAll(usedVars);
+ LinkedList<LogicalVariable> producedVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, producedVars);
+ toPush.removeAll(producedVars);
+ // we assume pipelineable ops. have only one input
+ opRef2 = op2.getInputs().get(0);
+ op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ } while (true);
+
+ LinkedList<LogicalVariable> produced2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, produced2);
+ LinkedList<LogicalVariable> used2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op2, used2);
+
+ boolean canCommuteProjection = initProjectList.containsAll(toPush) && initProjectList.containsAll(produced2)
+ && initProjectList.containsAll(used2);
+ // if true, we can get rid of the initial projection
+
+ // get rid of useless decor vars.
+ if (!canCommuteProjection && op2.getOperatorTag() == LogicalOperatorTag.GROUP) {
+ boolean gbyChanged = false;
+ GroupByOperator gby = (GroupByOperator) op2;
+ List<Pair<LogicalVariable, LogicalExpressionReference>> newDecorList = new ArrayList<Pair<LogicalVariable, LogicalExpressionReference>>();
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gby.getDecorList()) {
+ LogicalVariable decorVar = GroupByOperator.getDecorVariable(p);
+ if (!toPush.contains(decorVar)) {
+ used2.remove(decorVar);
+ gbyChanged = true;
+ } else {
+ newDecorList.add(p);
+ }
+ }
+ gby.getDecorList().clear();
+ gby.getDecorList().addAll(newDecorList);
+ if (gbyChanged) {
+ context.computeAndSetTypeEnvironmentForOperator(gby);
+ }
+ }
+ used2.clear();
+ VariableUtilities.getUsedVariables(op2, used2);
+
+ toPush.addAll(used2); // remember that toPush is a Set
+ toPush.removeAll(produced2);
+
+ if (toPush.isEmpty()) {
+ return new Pair<Boolean, Boolean>(false, false);
+ }
+
+ boolean smthWasPushed = false;
+ for (LogicalOperatorReference c : op2.getInputs()) {
+ if (pushNeededProjections(toPush, c, context, initialOp)) {
+ smthWasPushed = true;
+ }
+ }
+ if (op2.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans n = (AbstractOperatorWithNestedPlans) op2;
+ for (ILogicalPlan p : n.getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ if (pushNeededProjections(toPush, r, context, initialOp)) {
+ smthWasPushed = true;
+ }
+ }
+ }
+ }
+ return new Pair<Boolean, Boolean>(smthWasPushed, canCommuteProjection);
+ }
+
+ // It does not try to push above another Projection.
+ private static boolean pushNeededProjections(HashSet<LogicalVariable> toPush, LogicalOperatorReference opRef,
+ IOptimizationContext context, ILogicalOperator initialOp) throws AlgebricksException {
+ HashSet<LogicalVariable> allP = new HashSet<LogicalVariable>();
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ VariableUtilities.getLiveVariables(op, allP);
+
+ HashSet<LogicalVariable> toProject = new HashSet<LogicalVariable>();
+ for (LogicalVariable v : toPush) {
+ if (allP.contains(v)) {
+ toProject.add(v);
+ }
+ }
+ if (toProject.equals(allP)) {
+ // projection would be redundant, since we would project everything
+ // but we can try with the children
+ boolean push = false;
+ if (pushThroughOp(toProject, opRef, initialOp, context).first) {
+ push = true;
+ }
+ return push;
+ } else {
+ return pushAllProjectionsOnTopOf(toProject, opRef, context, initialOp);
+ }
+ }
+
+ // It does not try to push above another Projection.
+ private static boolean pushAllProjectionsOnTopOf(Collection<LogicalVariable> toPush,
+ LogicalOperatorReference opRef, IOptimizationContext context, ILogicalOperator initialOp)
+ throws AlgebricksException {
+ if (toPush.isEmpty()) {
+ return false;
+ }
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+
+ if (context.checkAndAddToAlreadyCompared(initialOp, op)) {
+ return false;
+ }
+
+ switch (op.getOperatorTag()) {
+ case EXCHANGE: {
+ opRef = opRef.getOperator().getInputs().get(0);
+ op = (AbstractLogicalOperator) opRef.getOperator();
+ break;
+ }
+ case PROJECT: {
+ return false;
+ }
+ }
+
+ ProjectOperator pi2 = new ProjectOperator(new ArrayList<LogicalVariable>(toPush));
+ pi2.getInputs().add(new LogicalOperatorReference(op));
+ opRef.setOperator(pi2);
+ pi2.setExecutionMode(op.getExecutionMode());
+ context.computeAndSetTypeEnvironmentForOperator(pi2);
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java
new file mode 100644
index 0000000..6358a7d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectIntoDataSourceScanRule.java
@@ -0,0 +1,43 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushProjectIntoDataSourceScanRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getInputs().size() <= 0)
+ return false;
+ AbstractLogicalOperator project = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ if (project.getOperatorTag() != LogicalOperatorTag.PROJECT)
+ return false;
+ AbstractLogicalOperator exchange = (AbstractLogicalOperator) project.getInputs().get(0).getOperator();
+ if (exchange.getOperatorTag() != LogicalOperatorTag.EXCHANGE)
+ return false;
+ AbstractLogicalOperator inputOp = (AbstractLogicalOperator) exchange.getInputs().get(0).getOperator();
+ if (inputOp.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN)
+ return false;
+ DataSourceScanOperator scanOp = (DataSourceScanOperator) inputOp;
+ ProjectOperator projectOp = (ProjectOperator) project;
+ scanOp.addProjectVariables(projectOp.getVariables());
+ if (op.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+ op.getInputs().set(0, project.getInputs().get(0));
+ } else {
+ op.getInputs().set(0, exchange.getInputs().get(0));
+ }
+ return true;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectDownRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectDownRule.java
new file mode 100644
index 0000000..972d257
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectDownRule.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushSelectDownRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+
+ LogicalOperatorReference opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+
+ if (context.checkAndAddToAlreadyCompared(op, op2)) {
+ return false;
+ }
+
+ LogicalOperatorTag tag2 = op2.getOperatorTag();
+
+ if (tag2 == LogicalOperatorTag.INNERJOIN || tag2 == LogicalOperatorTag.LEFTOUTERJOIN
+ || tag2 == LogicalOperatorTag.REPLICATE) {
+ return false;
+ } else { // not a join
+ boolean res = propagateSelectionRec(opRef, opRef2);
+ if (res) {
+ OperatorPropertiesUtil.typeOpRec(opRef, context);
+ }
+ return res;
+ }
+ }
+
+ private static boolean propagateSelectionRec(LogicalOperatorReference sigmaRef, LogicalOperatorReference opRef2)
+ throws AlgebricksException {
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ if (op2.getInputs().size() != 1 || op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ return false;
+ }
+
+ SelectOperator sigma = (SelectOperator) sigmaRef.getOperator();
+ LinkedList<LogicalVariable> usedInSigma = new LinkedList<LogicalVariable>();
+ sigma.getCondition().getExpression().getUsedVariables(usedInSigma);
+
+ LinkedList<LogicalVariable> produced2 = new LinkedList<LogicalVariable>();
+ VariableUtilities.getProducedVariables(op2, produced2);
+ if (OperatorPropertiesUtil.disjoint(produced2, usedInSigma)) {
+ // just swap
+ opRef2.setOperator(sigma);
+ sigmaRef.setOperator(op2);
+ List<LogicalOperatorReference> sigmaInpList = sigma.getInputs();
+ sigmaInpList.clear();
+ sigmaInpList.addAll(op2.getInputs());
+ List<LogicalOperatorReference> op2InpList = op2.getInputs();
+ op2InpList.clear();
+ op2InpList.add(opRef2);
+ propagateSelectionRec(opRef2, sigma.getInputs().get(0));
+ return true;
+
+ }
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
new file mode 100644
index 0000000..eea4978
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -0,0 +1,237 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushSelectIntoJoinRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ Collection<LogicalVariable> joinLiveVarsLeft = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> joinLiveVarsRight = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> liveInOpsToPushLeft = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> liveInOpsToPushRight = new HashSet<LogicalVariable>();
+
+ List<ILogicalOperator> pushedOnLeft = new ArrayList<ILogicalOperator>();
+ List<ILogicalOperator> pushedOnRight = new ArrayList<ILogicalOperator>();
+ LinkedList<ILogicalOperator> notPushedStack = new LinkedList<ILogicalOperator>();
+ Collection<LogicalVariable> usedVars = new HashSet<LogicalVariable>();
+ Collection<LogicalVariable> producedVars = new HashSet<LogicalVariable>();
+
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ SelectOperator select = (SelectOperator) op;
+ LogicalOperatorReference opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator son = (AbstractLogicalOperator) opRef2.getOperator();
+ AbstractLogicalOperator op2 = son;
+ boolean needToPushOps = false;
+ while (son.isMap()) {
+ needToPushOps = true;
+ LogicalOperatorReference opRefLink = son.getInputs().get(0);
+ son = (AbstractLogicalOperator) opRefLink.getOperator();
+ }
+
+ if (son.getOperatorTag() != LogicalOperatorTag.INNERJOIN
+ && son.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+ return false;
+ }
+ boolean isLoj = son.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN;
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) son;
+
+ LogicalOperatorReference joinBranchLeftRef = join.getInputs().get(0);
+ LogicalOperatorReference joinBranchRightRef = join.getInputs().get(1);
+
+ if (needToPushOps) {
+ ILogicalOperator joinBranchLeft = joinBranchLeftRef.getOperator();
+ ILogicalOperator joinBranchRight = joinBranchRightRef.getOperator();
+ VariableUtilities.getLiveVariables(joinBranchLeft, joinLiveVarsLeft);
+ VariableUtilities.getLiveVariables(joinBranchRight, joinLiveVarsRight);
+ LogicalOperatorReference opIterRef = opRef2;
+ ILogicalOperator opIter = op2;
+ while (opIter != join) {
+ LogicalOperatorTag tag = ((AbstractLogicalOperator) opIter).getOperatorTag();
+ if (tag == LogicalOperatorTag.PROJECT) {
+ notPushedStack.addFirst(opIter);
+ } else {
+ VariableUtilities.getUsedVariables(opIter, usedVars);
+ VariableUtilities.getProducedVariables(opIter, producedVars);
+ if (joinLiveVarsLeft.containsAll(usedVars)) {
+ pushedOnLeft.add(opIter);
+ liveInOpsToPushLeft.addAll(producedVars);
+ } else if (joinLiveVarsRight.containsAll(usedVars)) {
+ pushedOnRight.add(opIter);
+ liveInOpsToPushRight.addAll(producedVars);
+ } else {
+ return false;
+ }
+ }
+ opIterRef = opIter.getInputs().get(0);
+ opIter = opIterRef.getOperator();
+ }
+ if (isLoj && pushedOnLeft.isEmpty()) {
+ return false;
+ }
+ }
+
+ boolean intersectsAllBranches = true;
+ boolean[] intersectsBranch = new boolean[join.getInputs().size()];
+ LinkedList<LogicalVariable> selectVars = new LinkedList<LogicalVariable>();
+ select.getCondition().getExpression().getUsedVariables(selectVars);
+ int i = 0;
+ for (LogicalOperatorReference branch : join.getInputs()) {
+ LinkedList<LogicalVariable> branchVars = new LinkedList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(branch.getOperator(), branchVars);
+ if (i == 0) {
+ branchVars.addAll(liveInOpsToPushLeft);
+ } else {
+ branchVars.addAll(liveInOpsToPushRight);
+ }
+ if (OperatorPropertiesUtil.disjoint(selectVars, branchVars)) {
+ intersectsAllBranches = false;
+ } else {
+ intersectsBranch[i] = true;
+ }
+ i++;
+ }
+ if (!intersectsBranch[0] && !intersectsBranch[1]) {
+ return false;
+ }
+ if (intersectsAllBranches) {
+ if (needToPushOps) {
+ pushOps(pushedOnLeft, joinBranchLeftRef, context);
+ pushOps(pushedOnRight, joinBranchRightRef, context);
+ }
+ addCondToJoin(select, join);
+ } else { // push down
+ Iterator<LogicalOperatorReference> branchIter = join.getInputs().iterator();
+
+ for (int j = 0; j < intersectsBranch.length; j++) {
+ LogicalOperatorReference branch = branchIter.next();
+ boolean inter = intersectsBranch[j];
+ if (inter) {
+ if (needToPushOps) {
+ if (j == 0) {
+ pushOps(pushedOnLeft, joinBranchLeftRef, context);
+ } else {
+ pushOps(pushedOnRight, joinBranchRightRef, context);
+ }
+ }
+ copySelectToBranch(select, branch, context);
+ }
+
+ // if a left outer join, we can only push conditions into the
+ // outer branch.
+ if (j == 0 && isLoj) {
+ // stop at this branch
+ break;
+ }
+ }
+ }
+ ILogicalOperator top = join;
+ for (ILogicalOperator npOp : notPushedStack) {
+ List<LogicalOperatorReference> npInpList = npOp.getInputs();
+ npInpList.clear();
+ npInpList.add(new LogicalOperatorReference(top));
+ context.computeAndSetTypeEnvironmentForOperator(npOp);
+ top = npOp;
+ }
+ opRef.setOperator(top);
+ return true;
+
+ }
+
+ private void pushOps(List<ILogicalOperator> opList, LogicalOperatorReference joinBranch,
+ IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator topOp = joinBranch.getOperator();
+ ListIterator<ILogicalOperator> iter = opList.listIterator(opList.size());
+ while (iter.hasPrevious()) {
+ ILogicalOperator op = iter.previous();
+ List<LogicalOperatorReference> opInpList = op.getInputs();
+ opInpList.clear();
+ opInpList.add(new LogicalOperatorReference(topOp));
+ topOp = op;
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+ joinBranch.setOperator(topOp);
+ }
+
+ private static void addCondToJoin(SelectOperator select, AbstractBinaryJoinOperator join) {
+ ILogicalExpression cond = join.getCondition().getExpression();
+ if (OperatorPropertiesUtil.isAlwaysTrueCond(cond)) { // the join was a product
+ join.getCondition().setExpression(select.getCondition().getExpression());
+ } else {
+ boolean bAddedToConj = false;
+ if (cond.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ AbstractFunctionCallExpression fcond = (AbstractFunctionCallExpression) cond;
+ if (fcond.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
+ AbstractFunctionCallExpression newCond = new ScalarFunctionCallExpression(
+ AlgebricksBuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+ newCond.getArguments().add(select.getCondition());
+ newCond.getArguments().addAll(fcond.getArguments());
+ join.getCondition().setExpression(newCond);
+ bAddedToConj = true;
+ }
+ }
+ if (!bAddedToConj) {
+ AbstractFunctionCallExpression newCond = new ScalarFunctionCallExpression(AlgebricksBuiltinFunctions
+ .getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND), select.getCondition(),
+ new LogicalExpressionReference(join.getCondition().getExpression()));
+ join.getCondition().setExpression(newCond);
+ }
+ }
+ }
+
+ private static void copySelectToBranch(SelectOperator select, LogicalOperatorReference branch,
+ IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator newSelect = new SelectOperator(select.getCondition());
+ LogicalOperatorReference newRef = new LogicalOperatorReference(branch.getOperator());
+ newSelect.getInputs().add(newRef);
+ branch.setOperator(newSelect);
+ context.computeAndSetTypeEnvironmentForOperator(newSelect);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSubplanWithAggregateDownThroughProductRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSubplanWithAggregateDownThroughProductRule.java
new file mode 100644
index 0000000..32ff5c5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushSubplanWithAggregateDownThroughProductRule.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class PushSubplanWithAggregateDownThroughProductRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op1.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+ return false;
+ }
+ SubplanOperator subplan = (SubplanOperator) op1;
+ if (subplan.getNestedPlans().size() != 1) {
+ return false;
+ }
+ ILogicalPlan p = subplan.getNestedPlans().get(0);
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ LogicalOperatorReference r = p.getRoots().get(0);
+ if (((AbstractLogicalOperator) r.getOperator()).getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+ return false;
+ }
+
+ Set<LogicalVariable> free = new HashSet<LogicalVariable>();
+ OperatorPropertiesUtil.getFreeVariablesInSubplans(subplan, free);
+
+ LogicalOperatorReference op2Ref = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op2;
+ if (!OperatorPropertiesUtil.isAlwaysTrueCond(join.getCondition().getExpression())) {
+ return false;
+ }
+
+ LogicalOperatorReference b0Ref = op2.getInputs().get(0);
+ ILogicalOperator b0 = b0Ref.getOperator();
+ List<LogicalVariable> b0Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b0, b0Scm);
+ if (b0Scm.containsAll(free)) {
+ // push subplan on left branch
+ op2Ref.setOperator(b0);
+ b0Ref.setOperator(op1);
+ opRef.setOperator(op2);
+ return true;
+ } else {
+ LogicalOperatorReference b1Ref = op2.getInputs().get(1);
+ ILogicalOperator b1 = b1Ref.getOperator();
+ List<LogicalVariable> b1Scm = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(b1, b1Scm);
+ if (b1Scm.containsAll(free)) {
+ // push subplan on right branch
+ op2Ref.setOperator(b1);
+ b1Ref.setOperator(op1);
+ opRef.setOperator(op2);
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ReinferAllTypesRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ReinferAllTypesRule.java
new file mode 100644
index 0000000..55e83ae
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ReinferAllTypesRule.java
@@ -0,0 +1,47 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ReinferAllTypesRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getOperator())) {
+ return false;
+ }
+ typeOpRec(opRef, context);
+ return true;
+ }
+
+ private void typePlan(ILogicalPlan p, IOptimizationContext context) throws AlgebricksException {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ typeOpRec(r, context);
+ }
+ }
+
+ private void typeOpRec(LogicalOperatorReference r, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) r.getOperator();
+ for (LogicalOperatorReference i : op.getInputs()) {
+ typeOpRec(i, context);
+ }
+ if (op.hasNestedPlans()) {
+ for (ILogicalPlan p : ((AbstractOperatorWithNestedPlans) op).getNestedPlans()) {
+ typePlan(p, context);
+ }
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ context.addToDontApplySet(this, op);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantProjectionRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantProjectionRule.java
new file mode 100644
index 0000000..e11def8
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveRedundantProjectionRule.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/*
+ * project [var-list1]
+ * project [var-list2]
+ * P
+ *
+ * if var-list1.equals(var-list2) becomes
+ *
+ * project [var-list1]
+ * P
+ *
+ */
+
+public class RemoveRedundantProjectionRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op1.getOperatorTag() == LogicalOperatorTag.PROJECT) {
+ LogicalOperatorReference opRef2 = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ ProjectOperator pi2 = (ProjectOperator) op2;
+ opRef2.setOperator(pi2.getInputs().get(0).getOperator());
+ } else {
+ if (op1.getInputs().size() <= 0)
+ return false;
+ LogicalOperatorReference opRef2 = op1.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+ if (op2.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ if (op2.getInputs().size() <= 0)
+ return false;
+ LogicalOperatorReference opRef3 = op2.getInputs().get(0);
+ AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getOperator();
+
+ List<LogicalVariable> liveVars2 = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> liveVars3 = new ArrayList<LogicalVariable>();
+
+ VariableUtilities.getLiveVariables(op2, liveVars2);
+ VariableUtilities.getLiveVariables(op3, liveVars3);
+
+ if (!VariableUtilities.varListEqualUnordered(liveVars2, liveVars3))
+ return false;
+ opRef2.setOperator(op3);
+ }
+
+ return true;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
new file mode 100644
index 0000000..6d6bc70
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
@@ -0,0 +1,148 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class RemoveUnusedAssignAndAggregateRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getOperator())) {
+ return false;
+ }
+ Set<LogicalVariable> toRemove = new HashSet<LogicalVariable>();
+ collectUnusedAssignedVars((AbstractLogicalOperator) opRef.getOperator(), toRemove, true, context);
+ boolean smthToRemove = !toRemove.isEmpty();
+ if (smthToRemove) {
+ removeUnusedAssigns(opRef, toRemove, context);
+ }
+ return smthToRemove;
+ }
+
+ private void removeUnusedAssigns(LogicalOperatorReference opRef, Set<LogicalVariable> toRemove,
+ IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ while (removeFromAssigns(op, toRemove, context) == 0) {
+ if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ break;
+ }
+ op = (AbstractLogicalOperator) op.getInputs().get(0).getOperator();
+ opRef.setOperator(op);
+ }
+ Iterator<LogicalOperatorReference> childIter = op.getInputs().iterator();
+ while (childIter.hasNext()) {
+ LogicalOperatorReference cRef = childIter.next();
+ removeUnusedAssigns(cRef, toRemove, context);
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans opWithNest = (AbstractOperatorWithNestedPlans) op;
+ Iterator<ILogicalPlan> planIter = opWithNest.getNestedPlans().iterator();
+ while (planIter.hasNext()) {
+ ILogicalPlan p = planIter.next();
+ for (LogicalOperatorReference r : p.getRoots()) {
+ removeUnusedAssigns(r, toRemove, context);
+ }
+ }
+ }
+ }
+
+ private int removeFromAssigns(AbstractLogicalOperator op, Set<LogicalVariable> toRemove,
+ IOptimizationContext context) throws AlgebricksException {
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assign = (AssignOperator) op;
+ if (removeUnusedVarsAndExprs(toRemove, assign.getVariables(), assign.getExpressions())) {
+ context.computeAndSetTypeEnvironmentForOperator(assign);
+ }
+ return assign.getVariables().size();
+ } else if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ AggregateOperator agg = (AggregateOperator) op;
+ if (removeUnusedVarsAndExprs(toRemove, agg.getVariables(), agg.getExpressions())) {
+ context.computeAndSetTypeEnvironmentForOperator(agg);
+ }
+ return agg.getVariables().size();
+ }
+ return -1;
+ }
+
+ private boolean removeUnusedVarsAndExprs(Set<LogicalVariable> toRemove, List<LogicalVariable> varList,
+ List<LogicalExpressionReference> exprList) {
+ boolean changed = false;
+ Iterator<LogicalVariable> varIter = varList.iterator();
+ Iterator<LogicalExpressionReference> exprIter = exprList.iterator();
+ while (varIter.hasNext()) {
+ LogicalVariable v = varIter.next();
+ exprIter.next();
+ if (toRemove.contains(v)) {
+ varIter.remove();
+ exprIter.remove();
+ changed = true;
+ }
+ }
+ return changed;
+ }
+
+ private void collectUnusedAssignedVars(AbstractLogicalOperator op, Set<LogicalVariable> toRemove, boolean first,
+ IOptimizationContext context) throws AlgebricksException {
+ if (!first) {
+ context.addToDontApplySet(this, op);
+ }
+ for (LogicalOperatorReference c : op.getInputs()) {
+ collectUnusedAssignedVars((AbstractLogicalOperator) c.getOperator(), toRemove, false, context);
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans opWithNested = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan plan : opWithNested.getNestedPlans()) {
+ for (LogicalOperatorReference r : plan.getRoots()) {
+ collectUnusedAssignedVars((AbstractLogicalOperator) r.getOperator(), toRemove, false, context);
+ }
+ }
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assign = (AssignOperator) op;
+ toRemove.addAll(assign.getVariables());
+ } else if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+ AggregateOperator agg = (AggregateOperator) op;
+ toRemove.addAll(agg.getVariables());
+ }
+ List<LogicalVariable> used = new LinkedList<LogicalVariable>();
+ VariableUtilities.getUsedVariables(op, used);
+ toRemove.removeAll(used);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
new file mode 100644
index 0000000..23d607f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -0,0 +1,347 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.EmptyTupleSourcePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.IndexInsertDeletePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InsertDeletePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.MicroPreclusteredGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.NestedTupleSourcePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreSortedDistinctByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RunningAggregatePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SinkPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SinkWritePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamDiePOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamLimitPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StreamSelectPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.StringStreamingScriptPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SubplanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.UnionAllPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.UnnestPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.WriteResultPOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.rewriter.util.JoinUtils;
+
+public class SetAlgebricksPhysicalOperatorsRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ // if (context.checkIfInDontApplySet(this, op)) {
+ // return false;
+ // }
+ if (op.getPhysicalOperator() != null) {
+ return false;
+ }
+
+ computeDefaultPhysicalOp(op, true, context);
+ // context.addToDontApplySet(this, op);
+ return true;
+ }
+
+ private static void setPhysicalOperators(ILogicalPlan plan, boolean topLevelOp, IOptimizationContext context)
+ throws AlgebricksException {
+ for (LogicalOperatorReference root : plan.getRoots()) {
+ computeDefaultPhysicalOp((AbstractLogicalOperator) root.getOperator(), topLevelOp, context);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private static void computeDefaultPhysicalOp(AbstractLogicalOperator op, boolean topLevelOp,
+ IOptimizationContext context) throws AlgebricksException {
+ PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
+ if (op.getPhysicalOperator() == null) {
+ switch (op.getOperatorTag()) {
+ case AGGREGATE: {
+ op.setPhysicalOperator(new AggregatePOperator());
+ break;
+ }
+ case ASSIGN: {
+ op.setPhysicalOperator(new AssignPOperator());
+ break;
+ }
+ case DISTINCT: {
+ DistinctOperator distinct = (DistinctOperator) op;
+ distinct.setPhysicalOperator(new PreSortedDistinctByPOperator(distinct.getDistinctByVarList()));
+ break;
+ }
+ case EMPTYTUPLESOURCE: {
+ op.setPhysicalOperator(new EmptyTupleSourcePOperator());
+ break;
+ }
+ case EXCHANGE: {
+ if (op.getPhysicalOperator() == null) {
+ throw new AlgebricksException("Implementation for EXCHANGE operator was not set.");
+ }
+ // implem. choice for exchange should be set by a parent op.
+ break;
+ }
+ case GROUP: {
+ GroupByOperator gby = (GroupByOperator) op;
+
+ if (gby.getNestedPlans().size() == 1) {
+ ILogicalPlan p0 = gby.getNestedPlans().get(0);
+ if (p0.getRoots().size() == 1) {
+ if (gby.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY) == Boolean.TRUE
+ || gby.getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY) == Boolean.TRUE) {
+ if (!topLevelOp) {
+ throw new NotImplementedException(
+ "External hash group-by for nested grouping is not implemented.");
+ }
+ ExternalGroupByPOperator externalGby = new ExternalGroupByPOperator(
+ gby.getGroupByList(), physicalOptimizationConfig.getMaxFramesExternalGroupBy(),
+ physicalOptimizationConfig.getExternalGroupByTableSize());
+ op.setPhysicalOperator(externalGby);
+ generateMergeAggregationExpressions(gby, context);
+ break;
+ }
+ }
+ }
+
+ List<Pair<LogicalVariable, LogicalExpressionReference>> gbyList = gby.getGroupByList();
+ List<LogicalVariable> columnList = new ArrayList<LogicalVariable>(gbyList.size());
+ for (Pair<LogicalVariable, LogicalExpressionReference> p : gbyList) {
+ ILogicalExpression expr = p.second.getExpression();
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ columnList.add(varRef.getVariableReference());
+ }
+ }
+ if (topLevelOp) {
+ op.setPhysicalOperator(new PreclusteredGroupByPOperator(columnList));
+ } else {
+ op.setPhysicalOperator(new MicroPreclusteredGroupByPOperator(columnList));
+ }
+ break;
+ }
+ case INNERJOIN: {
+ JoinUtils.setJoinAlgorithmAndExchangeAlgo((InnerJoinOperator) op, context);
+ break;
+ }
+ case LEFTOUTERJOIN: {
+ JoinUtils.setJoinAlgorithmAndExchangeAlgo((LeftOuterJoinOperator) op, context);
+ break;
+ }
+ case LIMIT: {
+ LimitOperator opLim = (LimitOperator) op;
+ op.setPhysicalOperator(new StreamLimitPOperator(opLim.isTopmostLimitOp()
+ && opLim.getExecutionMode() == ExecutionMode.PARTITIONED));
+ break;
+ }
+ case NESTEDTUPLESOURCE: {
+ op.setPhysicalOperator(new NestedTupleSourcePOperator());
+ break;
+ }
+ case ORDER: {
+ OrderOperator oo = (OrderOperator) op;
+ for (Pair<IOrder, LogicalExpressionReference> p : oo.getOrderExpressions()) {
+ ILogicalExpression e = p.second.getExpression();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new AlgebricksException("Order expression " + e + " has not been normalized.");
+ }
+ }
+ if (topLevelOp) {
+ op.setPhysicalOperator(new StableSortPOperator(physicalOptimizationConfig
+ .getMaxFramesExternalSort()));
+ } else {
+ op.setPhysicalOperator(new InMemoryStableSortPOperator());
+ }
+ break;
+ }
+ case PROJECT: {
+ op.setPhysicalOperator(new StreamProjectPOperator());
+ break;
+ }
+ case RUNNINGAGGREGATE: {
+ op.setPhysicalOperator(new RunningAggregatePOperator());
+ break;
+ }
+ case REPLICATE: {
+ op.setPhysicalOperator(new ReplicatePOperator());
+ break;
+ }
+ case SCRIPT: {
+ op.setPhysicalOperator(new StringStreamingScriptPOperator());
+ break;
+ }
+ case SELECT: {
+ op.setPhysicalOperator(new StreamSelectPOperator());
+ break;
+ }
+ case SUBPLAN: {
+ op.setPhysicalOperator(new SubplanPOperator());
+ break;
+ }
+ case UNIONALL: {
+ op.setPhysicalOperator(new UnionAllPOperator());
+ break;
+ }
+
+ case UNNEST: {
+ op.setPhysicalOperator(new UnnestPOperator());
+ break;
+ }
+ case DATASOURCESCAN: {
+ DataSourceScanOperator scan = (DataSourceScanOperator) op;
+ IDataSource dataSource = scan.getDataSource();
+ DataSourceScanPOperator dss = new DataSourceScanPOperator(dataSource);
+ IMetadataProvider mp = context.getMetadataProvider();
+ if (mp.scannerOperatorIsLeaf(dataSource)) {
+ dss.disableJobGenBelowMe();
+ }
+ op.setPhysicalOperator(dss);
+ break;
+ }
+ case WRITE: {
+ op.setPhysicalOperator(new SinkWritePOperator());
+ break;
+ }
+ case WRITE_RESULT: {
+ WriteResultOperator opLoad = (WriteResultOperator) op;
+ LogicalVariable payload;
+ List<LogicalVariable> keys = new ArrayList<LogicalVariable>();
+ payload = getKeysAndLoad(opLoad.getPayloadExpression(), opLoad.getKeyExpressions(), keys);
+ op.setPhysicalOperator(new WriteResultPOperator(opLoad.getDataSource(), payload, keys));
+ break;
+ }
+ case INSERT_DELETE: {
+ InsertDeleteOperator opLoad = (InsertDeleteOperator) op;
+ LogicalVariable payload;
+ List<LogicalVariable> keys = new ArrayList<LogicalVariable>();
+ payload = getKeysAndLoad(opLoad.getPayloadExpression(), opLoad.getPrimaryKeyExpressions(), keys);
+ op.setPhysicalOperator(new InsertDeletePOperator(payload, keys, opLoad.getDataSource()));
+ break;
+ }
+ case INDEX_INSERT_DELETE: {
+ IndexInsertDeleteOperator opLoad = (IndexInsertDeleteOperator) op;
+ List<LogicalVariable> primaryKeys = new ArrayList<LogicalVariable>();
+ List<LogicalVariable> secondaryKeys = new ArrayList<LogicalVariable>();
+ getKeys(opLoad.getPrimaryKeyExpressions(), primaryKeys);
+ getKeys(opLoad.getSecondaryKeyExpressions(), secondaryKeys);
+ op.setPhysicalOperator(new IndexInsertDeletePOperator(primaryKeys, secondaryKeys, opLoad
+ .getDataSourceIndex()));
+ break;
+ }
+ case SINK: {
+ op.setPhysicalOperator(new SinkPOperator());
+ break;
+ }
+ case DIE: {
+ op.setPhysicalOperator(new StreamDiePOperator());
+ break;
+ }
+ }
+ }
+ if (op.hasNestedPlans()) {
+ AbstractOperatorWithNestedPlans nested = (AbstractOperatorWithNestedPlans) op;
+ for (ILogicalPlan p : nested.getNestedPlans()) {
+ setPhysicalOperators(p, false, context);
+ }
+ }
+ for (LogicalOperatorReference opRef : op.getInputs()) {
+ computeDefaultPhysicalOp((AbstractLogicalOperator) opRef.getOperator(), topLevelOp, context);
+ }
+ }
+
+ private static void getKeys(List<LogicalExpressionReference> keyExpressions, List<LogicalVariable> keys) {
+ for (LogicalExpressionReference kExpr : keyExpressions) {
+ ILogicalExpression e = kExpr.getExpression();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException();
+ }
+ keys.add(((VariableReferenceExpression) e).getVariableReference());
+ }
+ }
+
+ private static LogicalVariable getKeysAndLoad(LogicalExpressionReference payloadExpr,
+ List<LogicalExpressionReference> keyExpressions, List<LogicalVariable> keys) {
+ LogicalVariable payload;
+ if (payloadExpr.getExpression().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException();
+ }
+ payload = ((VariableReferenceExpression) payloadExpr.getExpression()).getVariableReference();
+
+ for (LogicalExpressionReference kExpr : keyExpressions) {
+ ILogicalExpression e = kExpr.getExpression();
+ if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException();
+ }
+ keys.add(((VariableReferenceExpression) e).getVariableReference());
+ }
+ return payload;
+ }
+
+ private static void generateMergeAggregationExpressions(GroupByOperator gby, IOptimizationContext context)
+ throws AlgebricksException {
+ if (gby.getNestedPlans().size() != 1) {
+ throw new AlgebricksException(
+ "External group-by currently works only for one nested plan with one root containing"
+ + "an aggregate and a nested-tuple-source.");
+ }
+ ILogicalPlan p0 = gby.getNestedPlans().get(0);
+ if (p0.getRoots().size() != 1) {
+ throw new AlgebricksException(
+ "External group-by currently works only for one nested plan with one root containing"
+ + "an aggregate and a nested-tuple-source.");
+ }
+ IMergeAggregationExpressionFactory mergeAggregationExpressionFactory = context
+ .getMergeAggregationExpressionFactory();
+ LogicalOperatorReference r0 = p0.getRoots().get(0);
+ AggregateOperator aggOp = (AggregateOperator) r0.getOperator();
+ List<LogicalExpressionReference> aggFuncRefs = aggOp.getExpressions();
+ int n = aggOp.getExpressions().size();
+ List<LogicalExpressionReference> mergeExpressionRefs = new ArrayList<LogicalExpressionReference>();
+ for (int i = 0; i < n; i++) {
+ ILogicalExpression mergeExpr = mergeAggregationExpressionFactory.createMergeAggregation(aggFuncRefs.get(i)
+ .getExpression(), context);
+ mergeExpressionRefs.add(new LogicalExpressionReference(mergeExpr));
+ }
+ aggOp.setMergeExpressions(mergeExpressionRefs);
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetExecutionModeRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetExecutionModeRule.java
new file mode 100644
index 0000000..5ecd14e
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetExecutionModeRule.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ *
+ * This rule sets the executionMode property of an operator, w/o introducing
+ * EXCHANGE operators in the plan. Previously, i.e. before having physical
+ * optimizations in place, we were using the IntroduceExchangeRule, which was
+ * doing both, to both set excutionMode and introduce data exchange ops.
+ *
+ *
+ * @author Nicola
+ *
+ */
+public class SetExecutionModeRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ boolean changed = OperatorManipulationUtil.setOperatorMode(op);
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.UNPARTITIONED
+ || op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.LOCAL) {
+ return changed;
+ }
+ switch (op.getOperatorTag()) {
+ // case DISTINCT:
+ // case AGGREGATE:
+ // case GROUP:
+ // case ORDER:
+ // case INNERJOIN:
+ // case LEFTOUTERJOIN: {
+ // op.setExecutionMode(ExecutionMode.GLOBAL);
+ // return true;
+ // }
+
+ case PARTITIONINGSPLIT: {
+ throw new NotImplementedException();
+ }
+ default: {
+ return changed;
+ }
+ }
+
+ }
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SimpleUnnestToProductRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SimpleUnnestToProductRule.java
new file mode 100644
index 0000000..25f21a9
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SimpleUnnestToProductRule.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class SimpleUnnestToProductRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getOperator();
+ if (op.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
+ return false;
+ }
+
+ LogicalOperatorReference opRef2 = op.getInputs().get(0);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getOperator();
+
+ if (!(op2 instanceof AbstractScanOperator) && !descOrSelfIsSourceScan(op2)) {
+ return false;
+ }
+ InnerJoinOperator product = new InnerJoinOperator(new LogicalExpressionReference(ConstantExpression.TRUE));
+
+ EmptyTupleSourceOperator ets = new EmptyTupleSourceOperator();
+ context.computeAndSetTypeEnvironmentForOperator(ets);
+ LogicalOperatorReference emptySrc = new LogicalOperatorReference(ets);
+ List<LogicalOperatorReference> opInpList = op.getInputs();
+ opInpList.clear();
+ opInpList.add(emptySrc);
+ product.getInputs().add(opRef2); // outer branch
+ product.getInputs().add(new LogicalOperatorReference(op));
+ opRef.setOperator(product); // plug the product in the plan
+ context.computeAndSetTypeEnvironmentForOperator(product);
+ return true;
+ }
+
+ private boolean descOrSelfIsSourceScan(AbstractLogicalOperator op2) {
+ if (op2.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ return true;
+ }
+ for (LogicalOperatorReference cRef : op2.getInputs()) {
+ AbstractLogicalOperator alo = (AbstractLogicalOperator) cRef.getOperator();
+ if (descOrSelfIsSourceScan(alo)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SubplanOutOfGroupRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SubplanOutOfGroupRule.java
new file mode 100644
index 0000000..bc86d53
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SubplanOutOfGroupRule.java
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ *
+ * Looks for a nested group-by plan ending in
+ *
+ * subplan {
+ *
+ * ...
+ *
+ * }
+ *
+ * select (function-call: algebricks:not, Args:[function-call:
+ * algebricks:is-null, Args:[...]])
+ *
+ * nested tuple source -- |UNPARTITIONED|
+ *
+ *
+ *
+ */
+
+public class SubplanOutOfGroupRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePost(LogicalOperatorReference opRef, IOptimizationContext context) throws AlgebricksException {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) opRef.getOperator();
+ if (op0.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ return false;
+ }
+ GroupByOperator gby = (GroupByOperator) op0;
+
+ Iterator<ILogicalPlan> plansIter = gby.getNestedPlans().iterator();
+ ILogicalPlan p = null;
+ while (plansIter.hasNext()) {
+ p = plansIter.next();
+ }
+ if (p == null) {
+ return false;
+ }
+ if (p.getRoots().size() != 1) {
+ return false;
+ }
+ LogicalOperatorReference op1Ref = p.getRoots().get(0);
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) op1Ref.getOperator();
+ boolean found = false;
+ while (op1.getInputs().size() == 1) {
+ if (op1.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+ SubplanOperator subplan = (SubplanOperator) op1;
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) subplan.getInputs().get(0).getOperator();
+ if (OperatorPropertiesUtil.isNullTest(op2)) {
+ if (subplan.getNestedPlans().size() == 1) {
+ ILogicalPlan p1 = subplan.getNestedPlans().get(0);
+ if (p1.getRoots().size() == 1) {
+ AbstractLogicalOperator r1 = (AbstractLogicalOperator) p1.getRoots().get(0).getOperator();
+ if (r1.getOperatorTag() == LogicalOperatorTag.INNERJOIN
+ || r1.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+ // now, check that it propagates all variables,
+ // so it can be pushed
+ List<LogicalVariable> op2Vars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(op2, op2Vars);
+ List<LogicalVariable> op1Vars = new ArrayList<LogicalVariable>();
+ VariableUtilities.getLiveVariables(subplan, op1Vars);
+ if (op1Vars.containsAll(op2Vars)) {
+ found = true;
+ break;
+ }
+ }
+ }
+ }
+ }
+ }
+ op1Ref = op1.getInputs().get(0);
+ op1 = (AbstractLogicalOperator) op1Ref.getOperator();
+ }
+ if (!found) {
+ return false;
+ }
+
+ ILogicalOperator subplan = op1;
+ ILogicalOperator op2 = op1.getInputs().get(0).getOperator();
+ op1Ref.setOperator(op2);
+ LogicalOperatorReference opUnderRef = gby.getInputs().get(0);
+ ILogicalOperator opUnder = opUnderRef.getOperator();
+ subplan.getInputs().clear();
+ subplan.getInputs().add(new LogicalOperatorReference(opUnder));
+ opUnderRef.setOperator(subplan);
+
+ return true;
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/JoinUtils.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/JoinUtils.java
new file mode 100644
index 0000000..711e859
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/JoinUtils.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.rewriter.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation.BroadcastSide;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalPropertiesVisitor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HybridHashJoinPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryHashJoinPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.NLJoinPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator.JoinPartitioningType;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+
+public class JoinUtils {
+
+ private final static int MB = 1048576;
+
+ private final static double DEFAULT_FUDGE_FACTOR = 1.3;
+ private final static int MAX_RECORDS_PER_FRAME = 512;
+ private final static int DEFAULT_FRAME_SIZE = 32768;
+ private final static int MAX_LEFT_INPUT_SIZE_HYBRID_HASH = (int) (140L * 1024 * MB / DEFAULT_FRAME_SIZE);
+ private final static int DEFAULT_MEMORY_SIZE_HYBRID_HASH = (int) (256L * MB / DEFAULT_FRAME_SIZE);
+
+ public static void setJoinAlgorithmAndExchangeAlgo(AbstractBinaryJoinOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ List<LogicalVariable> sideLeft = new LinkedList<LogicalVariable>();
+ List<LogicalVariable> sideRight = new LinkedList<LogicalVariable>();
+ List<LogicalVariable> varsLeft = op.getInputs().get(0).getOperator().getSchema();
+ List<LogicalVariable> varsRight = op.getInputs().get(1).getOperator().getSchema();
+ if (isHashJoinCondition(op.getCondition().getExpression(), varsLeft, varsRight, sideLeft, sideRight)) {
+ BroadcastSide side = getBroadcastJoinSide(op.getCondition().getExpression(), varsLeft, varsRight);
+ if (side == null) {
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ } else {
+ switch (side) {
+ case RIGHT:
+ setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideLeft, sideRight, context);
+ break;
+ case LEFT:
+ LogicalOperatorReference opRef0 = op.getInputs().get(0);
+ LogicalOperatorReference opRef1 = op.getInputs().get(1);
+ ILogicalOperator tmp = opRef0.getOperator();
+ opRef0.setOperator(opRef1.getOperator());
+ opRef1.setOperator(tmp);
+ setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideRight, sideLeft, context);
+ break;
+ default:
+ setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+ }
+ }
+ } else {
+ setNLJoinOp(op);
+ }
+ }
+
+ private static void setNLJoinOp(AbstractBinaryJoinOperator op) {
+ op.setPhysicalOperator(new NLJoinPOperator(op.getJoinKind(), JoinPartitioningType.BROADCAST,
+ DEFAULT_MEMORY_SIZE_HYBRID_HASH));
+ }
+
+ private static void setHashJoinOp(AbstractBinaryJoinOperator op, JoinPartitioningType partitioningType,
+ List<LogicalVariable> sideLeft, List<LogicalVariable> sideRight, IOptimizationContext context)
+ throws AlgebricksException {
+ op.setPhysicalOperator(new HybridHashJoinPOperator(op.getJoinKind(), partitioningType, sideLeft, sideRight,
+ DEFAULT_MEMORY_SIZE_HYBRID_HASH, MAX_LEFT_INPUT_SIZE_HYBRID_HASH, MAX_RECORDS_PER_FRAME,
+ DEFAULT_FUDGE_FACTOR));
+ if (partitioningType == JoinPartitioningType.BROADCAST) {
+ hybridToInMemHashJoin(op, context);
+ }
+ // op.setPhysicalOperator(new
+ // InMemoryHashJoinPOperator(op.getJoinKind(), partitioningType,
+ // sideLeft, sideRight,
+ // 1024 * 512));
+ }
+
+ private static void hybridToInMemHashJoin(AbstractBinaryJoinOperator op, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator opBuild = op.getInputs().get(1).getOperator();
+ LogicalPropertiesVisitor.computeLogicalPropertiesDFS(opBuild, context);
+ ILogicalPropertiesVector v = context.getLogicalPropertiesVector(opBuild);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("// HybridHashJoin inner branch -- Logical properties for " + opBuild
+ + ": " + v + "\n");
+ if (v != null) {
+ int size2 = v.getMaxOutputFrames();
+ HybridHashJoinPOperator hhj = (HybridHashJoinPOperator) op.getPhysicalOperator();
+ if (size2 > 0 && size2 * hhj.getFudgeFactor() <= hhj.getMemSizeInFrames()) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("// HybridHashJoin inner branch " + opBuild
+ + " fits in memory\n");
+ // maintains the local properties on the probe side
+ op.setPhysicalOperator(new InMemoryHashJoinPOperator(hhj.getKind(), hhj.getPartitioningType(), hhj
+ .getKeysLeftBranch(), hhj.getKeysRightBranch(), v.getNumberOfTuples() * 2));
+ }
+ }
+
+ }
+
+ private static boolean isHashJoinCondition(ILogicalExpression e, Collection<LogicalVariable> inLeftAll,
+ Collection<LogicalVariable> inRightAll, Collection<LogicalVariable> outLeftFields,
+ Collection<LogicalVariable> outRightFields) {
+ switch (e.getExpressionTag()) {
+ case FUNCTION_CALL: {
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e;
+ FunctionIdentifier fi = fexp.getFunctionIdentifier();
+ if (fi == AlgebricksBuiltinFunctions.AND) {
+ for (LogicalExpressionReference a : fexp.getArguments()) {
+ if (!isHashJoinCondition(a.getExpression(), inLeftAll, inRightAll, outLeftFields,
+ outRightFields)) {
+ return false;
+ }
+ }
+ return true;
+ } else {
+ ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(fi);
+ if (ck != ComparisonKind.EQ) {
+ return false;
+ }
+ ILogicalExpression opLeft = fexp.getArguments().get(0).getExpression();
+ ILogicalExpression opRight = fexp.getArguments().get(1).getExpression();
+ if (opLeft.getExpressionTag() != LogicalExpressionTag.VARIABLE
+ || opRight.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return false;
+ }
+ LogicalVariable var1 = ((VariableReferenceExpression) opLeft).getVariableReference();
+ if (inLeftAll.contains(var1) && !outLeftFields.contains(var1)) {
+ outLeftFields.add(var1);
+ } else if (inRightAll.contains(var1) && !outRightFields.contains(var1)) {
+ outRightFields.add(var1);
+ } else {
+ return false;
+ }
+ LogicalVariable var2 = ((VariableReferenceExpression) opRight).getVariableReference();
+ if (inLeftAll.contains(var2) && !outLeftFields.contains(var2)) {
+ outLeftFields.add(var2);
+ } else if (inRightAll.contains(var2) && !outRightFields.contains(var2)) {
+ outRightFields.add(var2);
+ } else {
+ return false;
+ }
+ return true;
+ }
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+
+ private static BroadcastSide getBroadcastJoinSide(ILogicalExpression e, List<LogicalVariable> varsLeft,
+ List<LogicalVariable> varsRight) {
+ if (e.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return null;
+ }
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e;
+ IExpressionAnnotation ann = fexp.getAnnotations().get(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY);
+ if (ann == null) {
+ return null;
+ }
+ BroadcastSide side = (BroadcastSide) ann.getObject();
+ if (side == null) {
+ return null;
+ }
+ int i;
+ switch (side) {
+ case LEFT:
+ i = 0;
+ break;
+ case RIGHT:
+ i = 1;
+ break;
+ default:
+ return null;
+ }
+ ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+ fexp.getArguments().get(i).getExpression().getUsedVariables(vars);
+ if (varsLeft.containsAll(vars)) {
+ return BroadcastSide.LEFT;
+ } else if (varsRight.containsAll(vars)) {
+ return BroadcastSide.RIGHT;
+ } else {
+ return null;
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/PhysicalOptimizationsUtil.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/PhysicalOptimizationsUtil.java
new file mode 100644
index 0000000..2af9fe4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/util/PhysicalOptimizationsUtil.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.hyracks.algebricks.rewriter.util;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorReference;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+
+public class PhysicalOptimizationsUtil {
+
+ public static void computeFDsAndEquivalenceClasses(AbstractLogicalOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ FDsAndEquivClassesVisitor visitor = new FDsAndEquivClassesVisitor();
+ Set<ILogicalOperator> visitSet = new HashSet<ILogicalOperator>();
+ computeFDsAndEqClassesWithVisitorRec(op, ctx, visitor, visitSet);
+ }
+
+ private static void computeFDsAndEqClassesWithVisitorRec(AbstractLogicalOperator op, IOptimizationContext ctx,
+ FDsAndEquivClassesVisitor visitor, Set<ILogicalOperator> visitSet) throws AlgebricksException {
+ visitSet.add(op);
+ for (LogicalOperatorReference i : op.getInputs()) {
+ computeFDsAndEqClassesWithVisitorRec((AbstractLogicalOperator) i.getOperator(), ctx, visitor, visitSet);
+ }
+ if (op.hasNestedPlans()) {
+ for (ILogicalPlan p : ((AbstractOperatorWithNestedPlans) op).getNestedPlans()) {
+ for (LogicalOperatorReference r : p.getRoots()) {
+ AbstractLogicalOperator rootOp = (AbstractLogicalOperator) r.getOperator();
+ computeFDsAndEqClassesWithVisitorRec(rootOp, ctx, visitor, visitSet);
+ }
+ }
+ }
+ if (op.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ NestedTupleSourceOperator nts = (NestedTupleSourceOperator) op;
+ ILogicalOperator source = nts.getDataSourceReference().getOperator().getInputs().get(0).getOperator();
+ if (!visitSet.contains(source)) {
+ computeFDsAndEqClassesWithVisitorRec((AbstractLogicalOperator) source, ctx, visitor, visitSet);
+ }
+ }
+ op.accept(visitor, ctx);
+ if (AlgebricksConfig.DEBUG) {
+ AlgebricksConfig.ALGEBRICKS_LOGGER.fine("--> op. type = " + op.getOperatorTag() + "\n"
+ + " equiv. classes = " + ctx.getEquivalenceClassMap(op) + "\n" + " FDs = "
+ + ctx.getFDList(op) + "\n");
+ }
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/simple/int-part1.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/simple/int-part1.tbl
new file mode 100644
index 0000000..91a5152
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/simple/int-part1.tbl
@@ -0,0 +1,7 @@
+0|
+2|
+4|
+6|
+8|
+10|
+12|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/simple/int-part2.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/simple/int-part2.tbl
new file mode 100644
index 0000000..a481d5d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/simple/int-part2.tbl
@@ -0,0 +1,6 @@
+1|
+3|
+5|
+7|
+9|
+11|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer-part1.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer-part1.tbl
new file mode 100644
index 0000000..8395f01
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer-part1.tbl
@@ -0,0 +1,75 @@
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+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. |
+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|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer-part2.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer-part2.tbl
new file mode 100644
index 0000000..3cb282c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer-part2.tbl
@@ -0,0 +1,75 @@
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer.tbl
new file mode 100644
index 0000000..5d39c80
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/customer.tbl
@@ -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 e|
+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. |
+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|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/lineitem.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/lineitem.tbl
new file mode 100644
index 0000000..58d47c6
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/lineitem.tbl
@@ -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 the|
+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|
+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. furious|
+194|168|7|7|21|22431.36|0.02|0.01|R|F|1992-05-06|1992-05-20|1992-05-07|COLLECT COD|REG AIR|accounts detect quickly dogged |
+195|85|6|1|6|5910.48|0.04|0.02|A|F|1994-01-09|1994-03-27|1994-01-28|COLLECT COD|REG AIR|y, even deposits haggle carefully. bli|
+195|94|8|2|41|40757.69|0.05|0.07|A|F|1994-02-24|1994-02-11|1994-03-20|NONE|TRUCK|rts detect in place of t|
+195|86|7|3|34|33526.72|0.08|0.08|R|F|1994-01-31|1994-02-11|1994-02-12|NONE|TRUCK| cajole furiously bold i|
+195|86|7|4|41|40429.28|0.06|0.04|R|F|1994-03-14|1994-03-13|1994-04-09|COLLECT COD|RAIL|ggle fluffily foxes. fluffily ironic ex|
+196|136|7|1|19|19686.47|0.03|0.02|R|F|1993-04-17|1993-05-27|1993-04-30|NONE|SHIP|sts maintain foxes. furiously regular p|
+196|10|3|2|15|13650.15|0.03|0.04|A|F|1993-07-05|1993-05-08|1993-07-06|TAKE BACK RETURN|SHIP|s accounts. furio|
+197|99|1|1|39|38964.51|0.02|0.04|N|O|1995-07-21|1995-07-01|1995-08-14|TAKE BACK RETURN|AIR|press accounts. daringly sp|
+197|178|8|2|8|8625.36|0.09|0.02|A|F|1995-04-17|1995-07-01|1995-04-27|DELIVER IN PERSON|SHIP|y blithely even deposits. blithely fina|
+197|156|4|3|17|17954.55|0.06|0.02|N|O|1995-08-02|1995-06-23|1995-08-03|COLLECT COD|REG AIR|ts. careful|
+197|18|5|4|25|22950.25|0.04|0.01|N|F|1995-06-13|1995-05-23|1995-06-24|TAKE BACK RETURN|FOB|s-- quickly final accounts|
+197|42|9|5|14|13188.56|0.09|0.01|R|F|1995-05-08|1995-05-24|1995-05-12|TAKE BACK RETURN|RAIL|use slyly slyly silent depo|
+197|106|1|6|1|1006.10|0.07|0.05|N|O|1995-07-15|1995-06-21|1995-08-11|COLLECT COD|RAIL| even, thin dependencies sno|
+198|57|8|1|33|31582.65|0.07|0.02|N|O|1998-01-05|1998-03-20|1998-01-10|TAKE BACK RETURN|TRUCK|carefully caref|
+198|16|10|2|20|18320.20|0.03|0.00|N|O|1998-01-15|1998-03-31|1998-01-25|DELIVER IN PERSON|FOB|carefully final escapades a|
+198|149|2|3|15|15737.10|0.04|0.02|N|O|1998-04-12|1998-02-26|1998-04-15|COLLECT COD|MAIL|es. quickly pending deposits s|
+198|11|5|4|35|31885.35|0.08|0.02|N|O|1998-02-27|1998-03-23|1998-03-14|TAKE BACK RETURN|RAIL|ests nod quickly furiously sly pinto be|
+198|102|3|5|33|33069.30|0.02|0.01|N|O|1998-03-22|1998-03-12|1998-04-14|DELIVER IN PERSON|SHIP|ending foxes acr|
+199|133|9|1|50|51656.50|0.02|0.00|N|O|1996-06-12|1996-06-03|1996-07-04|DELIVER IN PERSON|MAIL|essly regular ideas boost sly|
+199|134|5|2|30|31023.90|0.08|0.05|N|O|1996-03-27|1996-05-29|1996-04-14|NONE|TRUCK|ilent packages doze quickly. thinly |
+224|151|2|1|16|16818.40|0.04|0.00|A|F|1994-08-01|1994-07-30|1994-08-27|DELIVER IN PERSON|MAIL|y unusual foxes |
+224|109|2|2|34|34309.40|0.04|0.08|R|F|1994-07-13|1994-08-25|1994-07-31|COLLECT COD|TRUCK| carefully. final platelets |
+224|190|1|3|41|44697.79|0.07|0.04|A|F|1994-09-01|1994-09-15|1994-09-02|TAKE BACK RETURN|SHIP|after the furiou|
+224|167|4|4|12|12805.92|0.08|0.06|R|F|1994-10-12|1994-08-29|1994-10-20|DELIVER IN PERSON|MAIL|uriously regular packages. slyly fina|
+224|94|7|5|45|44734.05|0.07|0.07|R|F|1994-08-14|1994-09-02|1994-08-27|COLLECT COD|AIR|leep furiously regular requests. furiousl|
+224|51|3|6|4|3804.20|0.02|0.00|R|F|1994-09-08|1994-08-24|1994-10-04|DELIVER IN PERSON|FOB|tructions |
+225|172|3|1|4|4288.68|0.09|0.07|N|O|1995-08-05|1995-08-19|1995-09-03|TAKE BACK RETURN|SHIP|ng the ironic packages. asymptotes among |
+225|131|7|2|3|3093.39|0.00|0.08|N|O|1995-07-25|1995-07-08|1995-08-17|DELIVER IN PERSON|REG AIR| fluffily about the carefully bold a|
+225|199|2|3|45|49463.55|0.06|0.01|N|O|1995-08-17|1995-08-20|1995-08-30|TAKE BACK RETURN|FOB|the slyly even platelets use aro|
+225|147|4|4|24|25131.36|0.00|0.06|N|O|1995-09-23|1995-08-05|1995-10-16|COLLECT COD|MAIL|ironic accounts are final account|
+225|8|5|5|31|28148.00|0.04|0.06|N|O|1995-06-21|1995-07-24|1995-07-04|TAKE BACK RETURN|FOB|special platelets. quickly r|
+225|132|8|6|12|12385.56|0.00|0.00|A|F|1995-06-04|1995-07-15|1995-06-08|COLLECT COD|MAIL| unusual requests. bus|
+225|142|1|7|44|45854.16|0.10|0.06|N|O|1995-09-22|1995-08-16|1995-10-22|NONE|REG AIR|leep slyly |
+226|97|9|1|4|3988.36|0.00|0.00|R|F|1993-03-31|1993-04-30|1993-04-10|NONE|TRUCK|c foxes integrate carefully against th|
+226|138|4|2|46|47753.98|0.06|0.01|A|F|1993-07-06|1993-04-24|1993-07-13|COLLECT COD|FOB|s. carefully bold accounts cajol|
+226|38|4|3|35|32831.05|0.09|0.03|A|F|1993-03-31|1993-05-18|1993-04-01|NONE|RAIL|osits cajole. final, even foxes a|
+226|41|10|4|45|42346.80|0.10|0.02|R|F|1993-04-17|1993-05-27|1993-05-11|DELIVER IN PERSON|AIR| carefully pending pi|
+226|118|8|5|2|2036.22|0.07|0.02|R|F|1993-03-26|1993-04-13|1993-04-20|TAKE BACK RETURN|SHIP|al platelets. express somas |
+226|83|4|6|48|47187.84|0.02|0.00|A|F|1993-06-11|1993-05-15|1993-06-19|NONE|REG AIR|efully silent packages. final deposit|
+226|118|8|7|14|14253.54|0.09|0.00|R|F|1993-05-20|1993-06-05|1993-05-27|COLLECT COD|MAIL|ep carefully regular accounts. ironic|
+227|166|1|1|19|20257.04|0.05|0.06|N|O|1995-12-10|1996-01-30|1995-12-26|NONE|RAIL|s cajole furiously a|
+227|175|3|2|24|25804.08|0.07|0.07|N|O|1996-02-03|1995-12-24|1996-02-12|COLLECT COD|SHIP|uses across the blithe dependencies cajol|
+228|5|8|1|3|2715.00|0.10|0.08|A|F|1993-05-20|1993-04-08|1993-05-26|DELIVER IN PERSON|SHIP|ckages. sly|
+229|84|5|1|20|19681.60|0.02|0.03|R|F|1994-01-11|1994-01-31|1994-01-26|DELIVER IN PERSON|REG AIR|le. instructions use across the quickly fin|
+229|129|10|2|29|29844.48|0.07|0.00|A|F|1994-03-15|1994-03-02|1994-03-26|COLLECT COD|SHIP|s, final request|
+229|79|10|3|28|27413.96|0.02|0.02|R|F|1994-02-10|1994-02-02|1994-03-10|DELIVER IN PERSON|FOB| final, regular requests. platel|
+229|177|6|4|3|3231.51|0.02|0.08|R|F|1994-03-22|1994-03-24|1994-04-04|DELIVER IN PERSON|REG AIR|posits. furiously regular theodol|
+229|156|1|5|33|34852.95|0.03|0.06|R|F|1994-03-25|1994-02-11|1994-04-13|NONE|FOB| deposits; bold, ruthless theodolites|
+229|106|9|6|29|29176.90|0.04|0.00|R|F|1994-01-14|1994-02-16|1994-01-22|NONE|FOB|uriously pending |
+230|186|7|1|46|49964.28|0.09|0.00|R|F|1994-02-03|1994-01-15|1994-02-23|TAKE BACK RETURN|SHIP|old packages ha|
+230|195|7|2|6|6571.14|0.03|0.08|A|F|1994-01-26|1994-01-25|1994-02-13|NONE|REG AIR| sleep furiously about the p|
+230|8|5|3|1|908.00|0.07|0.06|R|F|1994-01-22|1994-01-03|1994-02-05|TAKE BACK RETURN|RAIL|blithely unusual dolphins. bold, ex|
+230|10|3|4|44|40040.44|0.08|0.06|R|F|1994-02-09|1994-01-18|1994-03-11|NONE|MAIL|deposits integrate slyly sile|
+230|19|9|5|8|7352.08|0.09|0.06|R|F|1993-11-03|1994-01-20|1993-11-11|TAKE BACK RETURN|TRUCK|g the instructions. fluffil|
+230|34|10|6|8|7472.24|0.00|0.05|R|F|1993-11-21|1994-01-05|1993-12-19|TAKE BACK RETURN|FOB|nal ideas. silent, reg|
+231|159|10|1|16|16946.40|0.04|0.08|R|F|1994-11-20|1994-10-29|1994-12-17|TAKE BACK RETURN|AIR|e furiously ironic pinto beans.|
+231|84|5|2|46|45267.68|0.04|0.05|R|F|1994-12-13|1994-12-02|1994-12-14|DELIVER IN PERSON|SHIP|affix blithely. bold requests among the f|
+231|199|1|3|50|54959.50|0.09|0.01|A|F|1994-12-11|1994-12-14|1994-12-13|NONE|RAIL|onic packages haggle fluffily a|
+231|57|8|4|31|29668.55|0.08|0.02|A|F|1994-11-05|1994-12-27|1994-11-30|TAKE BACK RETURN|SHIP|iously special decoys wake q|
+256|89|10|1|22|21759.76|0.09|0.02|R|F|1994-01-12|1993-12-28|1994-01-26|COLLECT COD|FOB|ke quickly ironic, ironic deposits. reg|
+256|119|6|2|40|40764.40|0.10|0.01|A|F|1993-11-30|1993-12-13|1993-12-02|NONE|FOB|nal theodolites. deposits cajole s|
+256|130|9|3|45|46355.85|0.02|0.08|R|F|1994-01-14|1994-01-17|1994-02-10|COLLECT COD|SHIP| grouches. ideas wake quickly ar|
+257|147|8|1|7|7329.98|0.05|0.02|N|O|1998-06-18|1998-05-15|1998-06-27|COLLECT COD|FOB|ackages sleep bold realms. f|
+258|107|4|1|8|8056.80|0.00|0.07|R|F|1994-01-20|1994-03-21|1994-02-09|NONE|REG AIR|ully about the fluffily silent dependencies|
+258|197|1|2|40|43887.60|0.10|0.01|A|F|1994-03-13|1994-02-23|1994-04-05|DELIVER IN PERSON|FOB|silent frets nod daringly busy, bold|
+258|162|3|3|45|47797.20|0.07|0.07|R|F|1994-03-04|1994-02-13|1994-03-30|DELIVER IN PERSON|TRUCK|regular excuses-- fluffily ruthl|
+258|133|9|4|31|32027.03|0.02|0.05|A|F|1994-04-20|1994-03-20|1994-04-28|COLLECT COD|REG AIR| slyly blithely special mul|
+258|36|2|5|25|23400.75|0.08|0.02|A|F|1994-04-13|1994-02-26|1994-04-29|TAKE BACK RETURN|TRUCK|leep pending packages.|
+258|147|4|6|36|37697.04|0.09|0.04|A|F|1994-01-11|1994-03-04|1994-01-18|DELIVER IN PERSON|AIR|nic asymptotes. slyly silent r|
+259|99|10|1|14|13987.26|0.00|0.08|A|F|1993-12-17|1993-12-09|1993-12-31|COLLECT COD|SHIP|ons against the express acco|
+259|162|1|2|14|14870.24|0.03|0.05|R|F|1993-11-10|1993-11-20|1993-11-17|DELIVER IN PERSON|FOB|ully even, regul|
+259|24|5|3|42|38808.84|0.09|0.00|R|F|1993-10-20|1993-11-18|1993-11-12|NONE|TRUCK|the slyly ironic pinto beans. fi|
+259|196|10|4|3|3288.57|0.08|0.06|R|F|1993-10-04|1993-11-07|1993-10-14|TAKE BACK RETURN|SHIP|ng slyly at the accounts.|
+259|193|6|5|6|6559.14|0.00|0.05|R|F|1993-12-05|1993-12-22|1993-12-21|COLLECT COD|TRUCK| requests sleep|
+260|156|7|1|50|52807.50|0.07|0.08|N|O|1997-03-24|1997-02-09|1997-04-20|TAKE BACK RETURN|REG AIR|c deposits |
+260|183|4|2|26|28162.68|0.02|0.07|N|O|1996-12-12|1997-02-06|1996-12-15|NONE|TRUCK|ld theodolites boost fl|
+260|42|1|3|27|25435.08|0.05|0.08|N|O|1997-03-23|1997-02-15|1997-04-22|TAKE BACK RETURN|RAIL|ions according to the|
+260|6|1|4|29|26274.00|0.10|0.06|N|O|1997-03-15|1997-01-14|1997-04-13|NONE|MAIL|fluffily even asymptotes. express wa|
+260|96|9|5|44|43827.96|0.01|0.05|N|O|1997-03-26|1997-02-03|1997-04-19|DELIVER IN PERSON|MAIL|above the blithely ironic instr|
+261|2|7|1|34|30668.00|0.05|0.08|R|F|1993-08-18|1993-09-24|1993-08-20|COLLECT COD|REG AIR|c packages. asymptotes da|
+261|66|7|2|20|19321.20|0.00|0.06|R|F|1993-10-21|1993-08-02|1993-11-04|DELIVER IN PERSON|RAIL|ites hinder |
+261|174|3|3|28|30076.76|0.08|0.03|R|F|1993-07-24|1993-08-20|1993-08-05|COLLECT COD|AIR|ironic packages nag slyly. carefully fin|
+261|119|3|4|49|49936.39|0.04|0.05|R|F|1993-09-12|1993-08-31|1993-10-07|COLLECT COD|SHIP|ions. bold accounts |
+261|61|6|5|49|47091.94|0.01|0.08|A|F|1993-09-29|1993-09-08|1993-10-01|COLLECT COD|SHIP| pinto beans haggle slyly furiously pending|
+261|97|9|6|20|19941.80|0.06|0.06|A|F|1993-10-15|1993-09-05|1993-11-07|NONE|AIR|ing to the special, ironic deposi|
+262|192|3|1|39|42595.41|0.01|0.05|N|O|1996-01-15|1996-02-18|1996-01-28|COLLECT COD|RAIL|usual, regular requests|
+262|61|6|2|33|31714.98|0.09|0.03|N|O|1996-03-10|1996-01-31|1996-03-27|TAKE BACK RETURN|AIR|atelets sleep furiously. requests cajole. b|
+262|59|1|3|35|33566.75|0.05|0.08|N|O|1996-03-12|1996-02-14|1996-04-11|COLLECT COD|MAIL|lites cajole along the pending packag|
+263|24|9|1|22|20328.44|0.06|0.08|R|F|1994-08-24|1994-06-20|1994-09-09|NONE|FOB|efully express fo|
+263|85|6|2|9|8865.72|0.08|0.00|A|F|1994-07-21|1994-07-16|1994-08-08|TAKE BACK RETURN|TRUCK|lms wake bl|
+263|143|2|3|50|52157.00|0.06|0.04|R|F|1994-08-18|1994-07-31|1994-08-22|NONE|TRUCK|re the packages. special|
+288|51|3|1|31|29482.55|0.00|0.03|N|O|1997-03-17|1997-04-28|1997-04-06|TAKE BACK RETURN|AIR|instructions wa|
+288|117|1|2|49|49838.39|0.08|0.05|N|O|1997-04-19|1997-05-19|1997-05-18|TAKE BACK RETURN|TRUCK|ic excuses sleep always spe|
+288|99|10|3|36|35967.24|0.02|0.02|N|O|1997-02-22|1997-05-07|1997-03-07|TAKE BACK RETURN|TRUCK|yly pending excu|
+288|79|10|4|19|18602.33|0.07|0.07|N|O|1997-03-14|1997-04-04|1997-03-26|NONE|MAIL|deposits. blithely quick courts ar|
+288|162|9|5|31|32926.96|0.10|0.04|N|O|1997-05-29|1997-04-24|1997-06-20|TAKE BACK RETURN|RAIL|ns. fluffily|
+289|174|2|1|25|26854.25|0.07|0.05|N|O|1997-03-18|1997-05-05|1997-04-15|DELIVER IN PERSON|FOB|out the quickly bold theodol|
+289|112|2|2|6|6072.66|0.06|0.05|N|O|1997-02-18|1997-05-08|1997-03-19|DELIVER IN PERSON|SHIP|d packages use fluffily furiously|
+289|17|4|3|44|40348.44|0.10|0.08|N|O|1997-06-05|1997-04-20|1997-07-02|COLLECT COD|MAIL|ly ironic foxes. asymptotes |
+289|40|6|4|48|45121.92|0.01|0.08|N|O|1997-03-14|1997-03-30|1997-03-24|DELIVER IN PERSON|RAIL|sits cajole. bold pinto beans x-ray fl|
+289|47|4|5|13|12311.52|0.10|0.03|N|O|1997-06-08|1997-04-06|1997-06-18|TAKE BACK RETURN|REG AIR|ts. quickly bold deposits alongside|
+290|6|1|1|35|31710.00|0.01|0.02|R|F|1994-04-01|1994-02-05|1994-04-27|NONE|MAIL|ove the final foxes detect slyly fluffily|
+290|129|4|2|2|2058.24|0.05|0.04|A|F|1994-01-30|1994-02-13|1994-02-21|TAKE BACK RETURN|TRUCK|. permanently furious reques|
+290|2|5|3|5|4510.00|0.03|0.05|A|F|1994-01-19|1994-02-24|1994-01-27|NONE|MAIL|ans integrate. requests sleep. fur|
+290|124|9|4|23|23554.76|0.05|0.08|R|F|1994-03-14|1994-02-21|1994-04-09|NONE|AIR|refully unusual packages. |
+291|123|6|1|21|21485.52|0.05|0.07|A|F|1994-05-26|1994-05-10|1994-06-23|COLLECT COD|TRUCK|y quickly regular theodolites. final t|
+291|138|9|2|19|19724.47|0.08|0.02|R|F|1994-06-14|1994-04-25|1994-06-19|NONE|REG AIR|e. ruthlessly final accounts after the|
+291|61|8|3|30|28831.80|0.10|0.02|R|F|1994-03-22|1994-04-30|1994-03-24|DELIVER IN PERSON|FOB| fluffily regular deposits. quickl|
+292|154|5|1|8|8433.20|0.10|0.03|R|F|1992-02-18|1992-03-30|1992-03-18|DELIVER IN PERSON|RAIL|sily bold deposits alongside of the ex|
+292|100|1|2|24|24002.40|0.08|0.04|R|F|1992-03-24|1992-03-06|1992-04-20|COLLECT COD|TRUCK| bold, pending theodolites u|
+293|9|6|1|14|12726.00|0.02|0.05|R|F|1992-10-19|1992-12-23|1992-11-10|DELIVER IN PERSON|SHIP|es. packages above the|
+293|187|8|2|11|11958.98|0.10|0.04|R|F|1992-12-24|1992-12-01|1993-01-12|COLLECT COD|MAIL| affix carefully quickly special idea|
+293|118|8|3|13|13235.43|0.04|0.02|A|F|1992-12-17|1992-12-26|1992-12-22|COLLECT COD|RAIL| wake after the quickly even deposits. bli|
+294|60|2|1|31|29761.86|0.00|0.01|R|F|1993-08-06|1993-08-19|1993-08-13|TAKE BACK RETURN|AIR|le fluffily along the quick|
+295|198|10|1|29|31847.51|0.02|0.07|A|F|1994-11-09|1994-12-08|1994-12-07|COLLECT COD|MAIL|inst the carefully ironic pinto beans. blit|
+295|92|6|2|26|25794.34|0.04|0.03|R|F|1994-12-13|1994-11-30|1995-01-06|DELIVER IN PERSON|AIR|ts above the slyly regular requests x-ray q|
+295|16|10|3|8|7328.08|0.10|0.07|R|F|1995-01-13|1994-11-17|1995-01-25|NONE|TRUCK| final instructions h|
+295|61|10|4|26|24987.56|0.10|0.04|A|F|1995-01-12|1994-11-22|1995-01-22|DELIVER IN PERSON|MAIL| carefully iron|
+320|5|2|1|30|27150.00|0.05|0.01|N|O|1997-12-04|1998-01-21|1997-12-13|NONE|RAIL| ironic, final accounts wake quick de|
+320|193|5|2|13|14211.47|0.03|0.00|N|O|1997-12-16|1997-12-26|1997-12-17|TAKE BACK RETURN|AIR|he furiously regular pinto beans. car|
+321|1|8|1|21|18921.00|0.01|0.08|A|F|1993-07-18|1993-04-24|1993-08-13|TAKE BACK RETURN|REG AIR|hockey players sleep slyly sl|
+321|141|4|2|41|42686.74|0.08|0.07|R|F|1993-06-21|1993-06-07|1993-07-09|NONE|REG AIR|special packages shall have to doze blit|
+322|153|8|1|12|12637.80|0.08|0.07|A|F|1992-06-29|1992-05-30|1992-07-11|NONE|AIR|ular theodolites promise qu|
+322|44|5|2|48|45313.92|0.02|0.07|A|F|1992-06-11|1992-06-16|1992-06-26|COLLECT COD|RAIL|dolites detect qu|
+322|13|3|3|20|18260.20|0.04|0.01|R|F|1992-04-26|1992-05-04|1992-05-22|DELIVER IN PERSON|MAIL|ckly toward |
+322|184|5|4|10|10841.80|0.06|0.03|R|F|1992-04-12|1992-05-13|1992-04-14|DELIVER IN PERSON|AIR| deposits grow slyly according to th|
+322|12|2|5|35|31920.35|0.07|0.06|A|F|1992-07-17|1992-05-03|1992-08-14|TAKE BACK RETURN|RAIL|egular accounts cajole carefully. even d|
+322|34|5|6|3|2802.09|0.08|0.05|A|F|1992-07-03|1992-05-10|1992-07-28|NONE|AIR|ending, ironic deposits along the blith|
+322|38|4|7|5|4690.15|0.01|0.02|A|F|1992-04-15|1992-05-12|1992-04-26|COLLECT COD|REG AIR| special grouches sleep quickly instructio|
+323|164|9|1|50|53208.00|0.05|0.04|A|F|1994-04-20|1994-04-25|1994-05-12|DELIVER IN PERSON|REG AIR|cial requests |
+323|96|8|2|18|17929.62|0.06|0.07|R|F|1994-04-13|1994-06-02|1994-05-10|DELIVER IN PERSON|TRUCK|posits cajole furiously pinto beans. |
+323|143|4|3|9|9388.26|0.07|0.04|A|F|1994-06-26|1994-06-10|1994-07-13|COLLECT COD|TRUCK|nic accounts. regular, regular pack|
+324|200|3|1|26|28605.20|0.07|0.01|R|F|1992-04-19|1992-05-28|1992-05-12|DELIVER IN PERSON|RAIL|ross the slyly regular s|
+325|159|1|1|34|36011.10|0.09|0.04|A|F|1993-10-28|1993-12-13|1993-11-17|TAKE BACK RETURN|MAIL|ly bold deposits. always iron|
+325|186|7|2|5|5430.90|0.07|0.08|A|F|1994-01-02|1994-01-05|1994-01-04|TAKE BACK RETURN|MAIL| theodolites. |
+325|19|3|3|35|32165.35|0.07|0.07|A|F|1993-12-06|1994-01-03|1993-12-26|DELIVER IN PERSON|REG AIR|packages wa|
+326|180|9|1|41|44287.38|0.06|0.03|N|O|1995-08-30|1995-07-09|1995-09-12|DELIVER IN PERSON|TRUCK|ily quickly bold ideas.|
+326|20|4|2|38|34960.76|0.02|0.08|N|O|1995-09-12|1995-08-23|1995-09-14|COLLECT COD|RAIL|es sleep slyly. carefully regular inst|
+326|184|5|3|25|27104.50|0.03|0.04|N|O|1995-08-03|1995-07-27|1995-08-16|NONE|AIR|ily furiously unusual accounts. |
+326|85|6|4|5|4925.40|0.03|0.08|N|O|1995-07-29|1995-07-13|1995-08-12|NONE|REG AIR|deas sleep according to the sometimes spe|
+326|35|6|5|31|28985.93|0.04|0.08|N|O|1995-09-27|1995-07-06|1995-10-22|NONE|TRUCK|cies sleep quick|
+326|157|9|6|41|43343.15|0.02|0.00|N|O|1995-07-05|1995-07-23|1995-07-20|TAKE BACK RETURN|REG AIR|to beans wake before the furiously re|
+326|43|10|7|47|44322.88|0.04|0.04|N|O|1995-09-16|1995-07-04|1995-10-04|NONE|REG AIR| special accounts sleep |
+327|144|3|1|16|16706.24|0.03|0.01|N|O|1995-07-05|1995-06-07|1995-07-09|TAKE BACK RETURN|TRUCK|cial ideas sleep af|
+327|42|9|2|9|8478.36|0.09|0.05|A|F|1995-05-24|1995-07-11|1995-06-05|NONE|AIR| asymptotes are fu|
+352|64|5|1|17|16389.02|0.07|0.05|R|F|1994-06-02|1994-05-31|1994-06-29|NONE|FOB|pending deposits sleep furiously |
+353|120|7|1|41|41824.92|0.00|0.06|A|F|1994-03-25|1994-03-31|1994-03-30|DELIVER IN PERSON|AIR|refully final theodoli|
+353|148|9|2|29|30396.06|0.09|0.00|A|F|1994-01-11|1994-03-19|1994-02-09|COLLECT COD|FOB|ctions impr|
+353|135|1|3|12|12421.56|0.06|0.01|R|F|1994-01-02|1994-03-26|1994-01-19|DELIVER IN PERSON|RAIL|g deposits cajole |
+353|78|7|4|46|44991.22|0.00|0.04|A|F|1994-04-14|1994-01-31|1994-05-05|DELIVER IN PERSON|FOB| ironic dolphins |
+353|117|4|5|9|9153.99|0.02|0.02|A|F|1994-03-15|1994-03-20|1994-03-18|TAKE BACK RETURN|RAIL|ual accounts! carefu|
+353|103|4|6|39|39120.90|0.02|0.05|A|F|1994-01-15|1994-03-30|1994-02-01|NONE|MAIL|losely quickly even accounts. c|
+354|50|7|1|14|13300.70|0.08|0.04|N|O|1996-04-12|1996-06-03|1996-05-08|NONE|SHIP|quickly regular grouches will eat. careful|
+354|194|8|2|24|26260.56|0.01|0.01|N|O|1996-05-08|1996-05-17|1996-06-07|DELIVER IN PERSON|AIR|y silent requests. regular, even accounts|
+354|59|10|3|50|47952.50|0.08|0.05|N|O|1996-03-21|1996-05-20|1996-04-04|COLLECT COD|TRUCK|to beans s|
+354|107|4|4|7|7049.70|0.06|0.01|N|O|1996-05-07|1996-04-18|1996-05-24|NONE|MAIL|ously idly ironic accounts-- quickl|
+354|31|2|5|18|16758.54|0.04|0.08|N|O|1996-03-31|1996-05-13|1996-04-27|DELIVER IN PERSON|RAIL| about the carefully unusual |
+354|62|1|6|36|34634.16|0.03|0.02|N|O|1996-03-19|1996-05-29|1996-03-30|NONE|AIR|onic requests thrash bold g|
+354|5|10|7|14|12670.00|0.01|0.07|N|O|1996-07-06|1996-06-08|1996-07-10|TAKE BACK RETURN|MAIL|t thinly above the ironic, |
+355|114|1|1|31|31437.41|0.09|0.07|A|F|1994-07-13|1994-08-18|1994-07-18|DELIVER IN PERSON|FOB|y unusual, ironic|
+355|97|1|2|41|40880.69|0.05|0.00|A|F|1994-08-15|1994-07-19|1994-09-06|DELIVER IN PERSON|TRUCK| deposits. carefully r|
+356|46|7|1|4|3784.16|0.10|0.01|A|F|1994-07-28|1994-08-01|1994-08-04|DELIVER IN PERSON|REG AIR| the dependencies nod unusual, final ac|
+356|108|3|2|48|48388.80|0.02|0.03|R|F|1994-08-12|1994-07-31|1994-08-26|NONE|FOB|unusual packages. furiously |
+356|119|3|3|35|35668.85|0.08|0.07|R|F|1994-10-14|1994-07-31|1994-10-23|COLLECT COD|TRUCK|s. unusual, final|
+356|56|1|4|41|39198.05|0.07|0.05|A|F|1994-09-28|1994-09-20|1994-10-07|COLLECT COD|SHIP| according to the express foxes will|
+356|125|8|5|37|37929.44|0.05|0.03|A|F|1994-07-15|1994-08-24|1994-08-09|DELIVER IN PERSON|FOB|ndencies are since the packag|
+357|114|5|1|26|26366.86|0.06|0.03|N|O|1996-12-28|1996-11-26|1997-01-13|NONE|FOB| carefully pending accounts use a|
+357|186|7|2|36|39102.48|0.07|0.06|N|O|1996-12-28|1996-11-13|1997-01-24|DELIVER IN PERSON|AIR|d the carefully even requests. |
+357|165|2|3|32|34085.12|0.05|0.07|N|O|1997-01-28|1996-12-29|1997-02-14|NONE|MAIL|y above the carefully final accounts|
+358|191|3|1|41|44738.79|0.06|0.01|A|F|1993-11-18|1993-11-14|1993-11-28|NONE|TRUCK|ely frets. furious deposits sleep |
+358|190|1|2|32|34886.08|0.05|0.08|A|F|1993-10-18|1993-12-12|1993-10-31|NONE|TRUCK|y final foxes sleep blithely sl|
+358|169|6|3|40|42766.40|0.09|0.01|A|F|1993-12-05|1993-11-04|1994-01-01|COLLECT COD|MAIL|ng the ironic theo|
+358|97|10|4|15|14956.35|0.08|0.08|A|F|1993-10-04|1993-12-17|1993-10-23|TAKE BACK RETURN|MAIL|out the blithely ironic deposits slee|
+358|29|2|5|18|16722.36|0.01|0.02|R|F|1993-10-07|1993-11-01|1993-10-26|COLLECT COD|SHIP|olphins haggle ironic accounts. f|
+358|162|3|6|32|33989.12|0.03|0.05|R|F|1993-12-21|1993-11-06|1994-01-17|DELIVER IN PERSON|RAIL|lyly express deposits |
+358|83|4|7|45|44238.60|0.05|0.02|A|F|1993-12-08|1993-10-29|1993-12-30|NONE|REG AIR|to beans. regular, unusual deposits sl|
+359|166|7|1|30|31984.80|0.00|0.08|A|F|1995-01-06|1995-02-20|1995-01-20|TAKE BACK RETURN|AIR|uses detect spec|
+359|12|9|2|18|16416.18|0.00|0.03|A|F|1995-01-27|1995-03-18|1995-01-31|DELIVER IN PERSON|RAIL|unusual warthogs. ironically sp|
+359|132|8|3|17|17546.21|0.07|0.06|A|F|1995-01-31|1995-03-18|1995-02-10|COLLECT COD|SHIP|sts according to the blithely|
+359|90|1|4|38|37623.42|0.10|0.08|R|F|1995-03-30|1995-01-20|1995-04-25|DELIVER IN PERSON|RAIL|g furiously. regular, sile|
+359|168|5|5|11|11749.76|0.01|0.03|A|F|1995-02-15|1995-01-27|1995-02-18|NONE|FOB|rets wake blithely. slyly final dep|
+359|183|4|6|23|24913.14|0.04|0.07|R|F|1995-01-31|1995-03-11|1995-02-16|DELIVER IN PERSON|REG AIR|ic courts snooze quickly furiously final fo|
+384|179|8|1|38|41008.46|0.07|0.01|R|F|1992-06-02|1992-04-18|1992-06-10|DELIVER IN PERSON|TRUCK|totes cajole blithely against the even|
+384|64|3|2|49|47238.94|0.09|0.07|A|F|1992-04-01|1992-04-25|1992-04-18|COLLECT COD|AIR|refully carefully ironic instructions. bl|
+384|182|3|3|11|11903.98|0.02|0.08|A|F|1992-04-02|1992-04-21|1992-04-15|COLLECT COD|MAIL|ash carefully|
+384|93|6|4|11|10923.99|0.00|0.06|R|F|1992-06-24|1992-05-29|1992-07-22|COLLECT COD|TRUCK|nic excuses are furiously above the blith|
+384|132|8|5|14|14449.82|0.08|0.06|R|F|1992-06-14|1992-05-29|1992-07-05|DELIVER IN PERSON|TRUCK|ckages are slyly after the slyly specia|
+385|167|6|1|7|7470.12|0.05|0.06|N|O|1996-05-23|1996-05-09|1996-06-06|DELIVER IN PERSON|REG AIR| special asymptote|
+385|54|9|2|46|43886.30|0.08|0.07|N|O|1996-03-29|1996-05-17|1996-04-18|NONE|REG AIR|lthily ironic f|
+386|153|5|1|39|41072.85|0.10|0.07|A|F|1995-05-10|1995-02-28|1995-05-25|NONE|SHIP|hely. carefully regular accounts hag|
+386|69|4|2|16|15504.96|0.06|0.01|A|F|1995-04-12|1995-04-18|1995-05-11|DELIVER IN PERSON|MAIL|lithely fluffi|
+386|131|2|3|37|38151.81|0.09|0.04|A|F|1995-05-23|1995-03-01|1995-05-25|TAKE BACK RETURN|MAIL|ending pearls breach fluffily. slyly pen|
+387|137|8|1|1|1037.13|0.08|0.03|N|O|1997-05-06|1997-04-23|1997-05-10|NONE|SHIP| pinto beans wake furiously carefu|
+387|153|4|2|42|44232.30|0.07|0.05|N|O|1997-05-25|1997-02-25|1997-05-29|DELIVER IN PERSON|RAIL|lithely final theodolites.|
+387|97|10|3|40|39883.60|0.09|0.02|N|O|1997-03-08|1997-04-18|1997-03-31|COLLECT COD|TRUCK| quickly ironic platelets are slyly. fluff|
+387|56|7|4|19|18164.95|0.08|0.00|N|O|1997-03-14|1997-04-21|1997-04-04|NONE|REG AIR|gular dependencies|
+387|149|6|5|32|33572.48|0.08|0.06|N|O|1997-05-02|1997-04-11|1997-05-11|DELIVER IN PERSON|TRUCK|gle. silent, fur|
+388|33|9|1|42|39187.26|0.05|0.06|R|F|1993-02-21|1993-02-26|1993-03-15|COLLECT COD|FOB|accounts sleep furiously|
+388|128|9|2|46|47293.52|0.07|0.01|A|F|1993-03-22|1993-01-26|1993-03-24|COLLECT COD|FOB|to beans nag about the careful reque|
+388|65|2|3|40|38602.40|0.06|0.01|A|F|1992-12-24|1993-01-28|1993-01-19|TAKE BACK RETURN|REG AIR|quests against the carefully unusual epi|
+389|190|1|1|2|2180.38|0.09|0.00|R|F|1994-04-13|1994-04-10|1994-04-25|TAKE BACK RETURN|RAIL|fts. courts eat blithely even dependenc|
+390|107|10|1|10|10071.00|0.02|0.05|N|O|1998-05-26|1998-07-06|1998-06-23|TAKE BACK RETURN|SHIP| requests. final accounts x-ray beside the|
+390|124|7|2|17|17410.04|0.09|0.06|N|O|1998-06-07|1998-06-14|1998-07-07|COLLECT COD|SHIP|ending, pending pinto beans wake slyl|
+390|184|5|3|46|49872.28|0.07|0.04|N|O|1998-06-06|1998-05-20|1998-06-14|DELIVER IN PERSON|SHIP|cial excuses. bold, pending packages|
+390|142|3|4|42|43769.88|0.01|0.05|N|O|1998-06-06|1998-06-22|1998-07-05|COLLECT COD|SHIP|counts nag across the sly, sil|
+390|128|3|5|13|13365.56|0.02|0.06|N|O|1998-07-08|1998-05-10|1998-07-18|DELIVER IN PERSON|SHIP|sleep carefully idle packages. blithely |
+390|125|4|6|11|11276.32|0.09|0.06|N|O|1998-05-05|1998-05-15|1998-06-01|DELIVER IN PERSON|SHIP|according to the foxes are furiously |
+390|85|6|7|24|23641.92|0.05|0.02|N|O|1998-04-18|1998-05-19|1998-04-28|TAKE BACK RETURN|AIR|y. enticingly final depos|
+391|122|1|1|14|14309.68|0.09|0.02|R|F|1995-02-11|1995-02-03|1995-02-13|TAKE BACK RETURN|TRUCK| escapades sleep furiously about |
+416|94|6|1|25|24852.25|0.00|0.05|A|F|1993-10-11|1993-11-26|1993-10-21|DELIVER IN PERSON|TRUCK|y final theodolites about|
+416|111|1|2|22|22244.42|0.10|0.00|R|F|1993-12-27|1993-12-17|1994-01-09|COLLECT COD|RAIL|rint blithely above the pending sentim|
+416|175|5|3|25|26879.25|0.07|0.01|R|F|1993-10-16|1993-12-03|1993-10-29|NONE|AIR|ses boost after the bold requests.|
+417|40|1|1|39|36661.56|0.01|0.02|A|F|1994-05-31|1994-05-02|1994-06-06|NONE|SHIP|y regular requests wake along |
+417|70|7|2|18|17461.26|0.00|0.01|R|F|1994-03-29|1994-04-10|1994-04-26|TAKE BACK RETURN|FOB|- final requests sle|
+417|45|2|3|41|38746.64|0.10|0.01|R|F|1994-04-11|1994-03-08|1994-05-06|COLLECT COD|RAIL|tes. regular requests across the |
+417|132|3|4|2|2064.26|0.01|0.03|R|F|1994-02-13|1994-04-19|1994-03-15|DELIVER IN PERSON|SHIP|uriously bol|
+418|19|3|1|31|28489.31|0.00|0.03|N|F|1995-06-05|1995-06-18|1995-06-26|COLLECT COD|FOB|final theodolites. fluffil|
+418|2|5|2|1|902.00|0.04|0.07|N|O|1995-06-23|1995-06-16|1995-07-23|DELIVER IN PERSON|AIR|regular, silent pinto|
+418|35|1|3|3|2805.09|0.04|0.06|N|O|1995-06-29|1995-07-12|1995-07-01|COLLECT COD|AIR|ly furiously regular w|
+419|153|8|1|33|34753.95|0.05|0.02|N|O|1996-11-06|1996-12-25|1996-11-20|TAKE BACK RETURN|TRUCK|y above the bli|
+419|65|2|2|32|30881.92|0.01|0.06|N|O|1996-12-04|1996-12-04|1996-12-24|COLLECT COD|SHIP|blithely regular requests. special pinto|
+419|71|1|3|15|14566.05|0.07|0.04|N|O|1996-12-17|1996-11-28|1996-12-19|TAKE BACK RETURN|REG AIR| sleep final, regular theodolites. fluffi|
+419|9|6|4|15|13635.00|0.01|0.02|N|O|1997-01-09|1996-12-22|1997-01-25|COLLECT COD|FOB|of the careful, thin theodolites. quickly s|
+419|149|2|5|17|17835.38|0.01|0.00|N|O|1997-01-13|1996-12-20|1997-02-01|COLLECT COD|REG AIR|lar dependencies: carefully regu|
+420|101|6|1|5|5005.50|0.04|0.03|N|O|1995-11-04|1996-01-02|1995-11-30|NONE|REG AIR|cajole blit|
+420|162|7|2|22|23367.52|0.05|0.04|N|O|1996-01-25|1995-12-16|1996-02-03|TAKE BACK RETURN|AIR|ly against the blithely re|
+420|48|1|3|45|42661.80|0.09|0.08|N|O|1996-01-14|1996-01-01|1996-01-26|COLLECT COD|FOB| final accounts. furiously express forges|
+420|75|6|4|12|11700.84|0.08|0.08|N|O|1996-02-05|1996-01-03|1996-02-12|TAKE BACK RETURN|REG AIR|c instructions are |
+420|73|2|5|37|36003.59|0.02|0.00|N|O|1995-11-16|1995-12-13|1995-11-19|DELIVER IN PERSON|SHIP|rbits. bold requests along the quickl|
+420|124|7|6|40|40964.80|0.01|0.05|N|O|1995-11-26|1995-12-26|1995-12-20|TAKE BACK RETURN|FOB| after the special|
+420|16|7|7|39|35724.39|0.00|0.08|N|O|1995-12-09|1995-12-16|1995-12-31|DELIVER IN PERSON|REG AIR|s. ironic waters about the car|
+421|134|5|1|1|1034.13|0.02|0.07|R|F|1992-05-29|1992-04-27|1992-06-09|NONE|TRUCK|oldly busy deposit|
+422|152|10|1|25|26303.75|0.10|0.07|N|O|1997-07-01|1997-08-17|1997-07-09|DELIVER IN PERSON|SHIP|carefully bold theodolit|
+422|171|1|2|10|10711.70|0.02|0.03|N|O|1997-06-15|1997-08-04|1997-07-08|TAKE BACK RETURN|AIR|he furiously ironic theodolite|
+422|176|4|3|46|49503.82|0.09|0.00|N|O|1997-06-21|1997-07-14|1997-06-27|DELIVER IN PERSON|RAIL| ideas. qu|
+422|162|7|4|25|26554.00|0.10|0.04|N|O|1997-08-24|1997-07-09|1997-09-22|NONE|FOB|ep along the furiousl|
+423|132|3|1|27|27867.51|0.06|0.03|N|O|1996-08-20|1996-08-01|1996-08-23|TAKE BACK RETURN|SHIP|ccounts. blithely regular pack|
+448|126|7|1|4|4104.48|0.00|0.04|N|O|1995-11-25|1995-10-20|1995-11-26|TAKE BACK RETURN|MAIL|nts thrash quickly among the b|
+448|173|1|2|46|49365.82|0.05|0.00|N|O|1995-08-31|1995-09-30|1995-09-09|COLLECT COD|SHIP| to the fluffily ironic packages.|
+448|27|6|3|35|32445.70|0.10|0.08|N|O|1995-09-27|1995-11-19|1995-10-20|COLLECT COD|REG AIR|ses nag quickly quickly ir|
+448|170|1|4|8|8561.36|0.10|0.00|N|O|1995-11-02|1995-10-16|1995-11-15|COLLECT COD|TRUCK|ounts wake blithely. furiously pending|
+448|138|9|5|23|23876.99|0.02|0.05|N|O|1995-09-26|1995-11-02|1995-10-17|NONE|SHIP|ious, final gifts|
+449|152|7|1|12|12625.80|0.02|0.08|N|O|1995-11-06|1995-08-25|1995-11-18|TAKE BACK RETURN|SHIP|ly. blithely ironic |
+449|109|6|2|4|4036.40|0.10|0.06|N|O|1995-10-27|1995-09-14|1995-11-21|DELIVER IN PERSON|FOB|are fluffily. requests are furiously|
+449|10|1|3|3|2730.03|0.07|0.08|N|O|1995-07-28|1995-09-11|1995-08-01|NONE|RAIL| bold deposits. express theodolites haggle|
+449|158|3|4|22|23279.30|0.07|0.00|N|O|1995-08-17|1995-09-04|1995-09-10|COLLECT COD|FOB|furiously final theodolites eat careful|
+450|162|7|1|42|44610.72|0.03|0.00|N|F|1995-06-07|1995-05-29|1995-06-23|TAKE BACK RETURN|SHIP|y asymptotes. regular depen|
+450|107|8|2|5|5035.50|0.03|0.02|A|F|1995-04-02|1995-05-06|1995-04-13|TAKE BACK RETURN|TRUCK|the pinto bea|
+450|143|6|3|32|33380.48|0.06|0.03|N|O|1995-07-02|1995-04-25|1995-07-30|TAKE BACK RETURN|SHIP| accounts nod fluffily even, pending|
+450|57|9|4|40|38282.00|0.05|0.03|R|F|1995-03-20|1995-05-25|1995-04-14|NONE|RAIL|ve. asymptote|
+450|79|10|5|2|1958.14|0.09|0.00|A|F|1995-03-11|1995-05-21|1995-03-16|COLLECT COD|AIR|y even pinto beans; qui|
+450|153|1|6|33|34753.95|0.08|0.05|R|F|1995-05-18|1995-05-22|1995-05-23|TAKE BACK RETURN|REG AIR|ily carefully final depo|
+451|130|9|1|36|37084.68|0.02|0.06|N|O|1998-06-18|1998-08-14|1998-06-20|TAKE BACK RETURN|AIR|rges can haggle carefully ironic, dogged |
+451|33|4|2|42|39187.26|0.05|0.01|N|O|1998-08-01|1998-08-05|1998-08-30|DELIVER IN PERSON|TRUCK|express excuses. blithely ironic pin|
+451|87|8|3|1|987.08|0.07|0.05|N|O|1998-07-13|1998-07-03|1998-08-04|DELIVER IN PERSON|AIR| carefully ironic packages solve furiously |
+451|77|5|4|28|27357.96|0.04|0.05|N|O|1998-06-16|1998-07-09|1998-06-17|DELIVER IN PERSON|SHIP| theodolites. even cou|
+452|115|6|1|2|2030.22|0.04|0.03|N|O|1997-12-26|1998-01-03|1998-01-12|COLLECT COD|FOB|y express instru|
+453|198|1|1|45|49418.55|0.01|0.00|N|O|1997-06-30|1997-08-20|1997-07-19|COLLECT COD|REG AIR|ifts wake carefully.|
+453|176|4|2|38|40894.46|0.08|0.04|N|O|1997-06-30|1997-07-08|1997-07-16|DELIVER IN PERSON|REG AIR| furiously f|
+453|14|1|3|38|34732.38|0.10|0.01|N|O|1997-08-10|1997-07-24|1997-09-07|NONE|SHIP|sts cajole. furiously un|
+453|96|7|4|45|44824.05|0.10|0.01|N|O|1997-09-18|1997-06-29|1997-10-14|TAKE BACK RETURN|AIR|ironic foxes. slyly pending depos|
+453|26|1|5|32|29632.64|0.04|0.01|N|O|1997-07-15|1997-06-27|1997-07-18|NONE|REG AIR|s. fluffily bold packages cajole. unu|
+453|95|7|6|28|27862.52|0.07|0.07|N|O|1997-08-16|1997-08-12|1997-08-27|NONE|MAIL|final dependencies. slyly special pl|
+454|118|8|1|24|24434.64|0.06|0.01|N|O|1996-04-26|1996-03-23|1996-05-20|NONE|TRUCK|le. deposits after the ideas nag unusual pa|
+455|157|9|1|42|44400.30|0.10|0.02|N|O|1997-01-26|1997-01-10|1997-02-22|DELIVER IN PERSON|REG AIR|around the quickly blit|
+455|28|9|2|44|40832.88|0.05|0.08|N|O|1997-01-17|1997-02-22|1997-02-12|TAKE BACK RETURN|TRUCK| accounts sleep slyly ironic asymptote|
+455|49|2|3|45|42706.80|0.04|0.06|N|O|1996-12-20|1997-01-31|1997-01-07|TAKE BACK RETURN|SHIP|thrash ironically regular packages. qui|
+455|171|9|4|11|11782.87|0.01|0.02|N|O|1997-03-15|1997-02-14|1997-03-26|DELIVER IN PERSON|MAIL|g deposits against the slyly idle foxes u|
+480|53|4|1|22|20967.10|0.04|0.02|A|F|1993-06-16|1993-07-28|1993-07-09|NONE|MAIL|into beans cajole furiously. accounts s|
+481|19|9|1|17|15623.17|0.07|0.05|A|F|1992-10-21|1992-12-09|1992-11-19|DELIVER IN PERSON|MAIL|. quickly final accounts among the |
+481|21|2|2|19|17499.38|0.08|0.01|R|F|1993-01-09|1992-11-27|1993-01-14|TAKE BACK RETURN|AIR|p blithely after t|
+481|186|7|3|42|45619.56|0.08|0.08|A|F|1992-11-27|1992-11-11|1992-12-08|COLLECT COD|RAIL|mptotes are furiously among the iron|
+481|82|3|4|11|10802.88|0.05|0.06|A|F|1993-01-12|1992-11-17|1993-02-05|NONE|FOB|eful attai|
+481|112|9|5|31|31375.41|0.05|0.01|A|F|1993-01-15|1992-12-31|1993-01-21|DELIVER IN PERSON|AIR|usly final packages believe. quick|
+482|138|9|1|32|33220.16|0.00|0.02|N|O|1996-05-22|1996-05-14|1996-05-29|NONE|SHIP|usual deposits affix against |
+482|122|5|2|1|1022.12|0.05|0.08|N|O|1996-05-29|1996-05-20|1996-05-31|COLLECT COD|AIR|es. quickly ironic escapades sleep furious|
+482|62|9|3|31|29823.86|0.04|0.03|N|O|1996-06-01|1996-05-06|1996-06-17|NONE|MAIL| blithe pin|
+482|196|7|4|8|8769.52|0.02|0.05|N|O|1996-04-19|1996-05-05|1996-04-21|NONE|TRUCK|tructions near the final, regular ideas de|
+482|39|10|5|46|43195.38|0.01|0.06|N|O|1996-07-19|1996-06-05|1996-08-10|NONE|MAIL|furiously thin realms. final, fina|
+482|79|10|6|19|18602.33|0.04|0.00|N|O|1996-03-27|1996-04-25|1996-04-15|NONE|FOB|ts hinder carefully silent requests|
+483|33|9|1|8|7464.24|0.00|0.08|N|O|1995-08-22|1995-08-23|1995-09-18|COLLECT COD|RAIL|osits. carefully fin|
+483|80|1|2|23|22541.84|0.04|0.06|N|O|1995-07-20|1995-08-11|1995-08-04|DELIVER IN PERSON|MAIL|requests was quickly against th|
+483|88|9|3|9|8892.72|0.04|0.03|N|O|1995-09-10|1995-09-02|1995-09-13|NONE|AIR| carefully express ins|
+484|31|2|1|49|45620.47|0.10|0.02|N|O|1997-03-06|1997-02-28|1997-03-23|COLLECT COD|TRUCK|ven accounts|
+484|32|8|2|45|41941.35|0.06|0.07|N|O|1997-04-09|1997-03-20|1997-04-19|DELIVER IN PERSON|TRUCK|usly final excuses boost slyly blithe|
+484|184|5|3|50|54209.00|0.06|0.05|N|O|1997-01-24|1997-03-27|1997-02-22|DELIVER IN PERSON|MAIL|uctions wake. final, silent requests haggle|
+484|165|6|4|22|23433.52|0.07|0.03|N|O|1997-04-29|1997-03-26|1997-05-17|TAKE BACK RETURN|SHIP|es are pending instructions. furiously unu|
+484|77|6|5|48|46899.36|0.00|0.05|N|O|1997-03-05|1997-02-08|1997-03-22|TAKE BACK RETURN|MAIL|l, bold packages? even mult|
+484|97|9|6|10|9970.90|0.01|0.08|N|O|1997-04-06|1997-02-14|1997-04-16|COLLECT COD|FOB|x fluffily carefully regular|
+485|150|1|1|50|52507.50|0.01|0.00|N|O|1997-03-28|1997-05-26|1997-04-18|TAKE BACK RETURN|MAIL|iously quick excuses. carefully final f|
+485|28|7|2|40|37120.80|0.08|0.01|N|O|1997-04-29|1997-05-08|1997-04-30|TAKE BACK RETURN|TRUCK|al escapades|
+485|137|3|3|22|22816.86|0.00|0.05|N|O|1997-04-06|1997-04-27|1997-05-01|DELIVER IN PERSON|TRUCK|refully final notornis haggle according |
+486|76|7|1|36|35138.52|0.00|0.01|N|O|1996-06-25|1996-05-06|1996-07-07|COLLECT COD|AIR|deposits around the quickly regular packa|
+486|68|9|2|40|38722.40|0.03|0.08|N|O|1996-05-21|1996-06-06|1996-06-07|COLLECT COD|SHIP|ts nag quickly among the slyl|
+486|136|2|3|26|26939.38|0.04|0.03|N|O|1996-03-16|1996-05-25|1996-03-31|NONE|RAIL|forges along the |
+486|72|1|4|38|36938.66|0.08|0.05|N|O|1996-05-07|1996-04-26|1996-05-26|TAKE BACK RETURN|TRUCK| blithely final pinto |
+486|29|2|5|3|2787.06|0.07|0.05|N|O|1996-07-07|1996-04-20|1996-07-23|DELIVER IN PERSON|RAIL|ccounts ha|
+486|47|4|6|46|43563.84|0.00|0.03|N|O|1996-04-18|1996-05-02|1996-04-20|COLLECT COD|AIR|theodolites eat carefully furious|
+487|92|3|1|47|46628.23|0.06|0.06|R|F|1992-09-30|1992-10-08|1992-10-24|NONE|TRUCK|tions. blithely reg|
+487|83|4|2|2|1966.16|0.02|0.06|R|F|1992-10-19|1992-11-04|1992-11-11|COLLECT COD|TRUCK|oss the unusual pinto beans. reg|
+512|189|10|1|19|20694.42|0.08|0.05|N|O|1995-07-12|1995-07-11|1995-08-04|COLLECT COD|MAIL| sleep. requests alongside of the fluff|
+512|23|2|2|37|34151.74|0.01|0.04|N|O|1995-06-20|1995-07-05|1995-07-16|NONE|RAIL|nic depths cajole? blithely b|
+512|180|1|3|40|43207.20|0.05|0.02|N|O|1995-07-06|1995-07-08|1995-07-08|COLLECT COD|TRUCK|quests are da|
+512|83|4|4|10|9830.80|0.09|0.02|N|O|1995-09-16|1995-07-29|1995-10-07|NONE|AIR|xes. pinto beans cajole carefully; |
+512|65|6|5|6|5790.36|0.03|0.05|R|F|1995-06-10|1995-06-21|1995-06-16|DELIVER IN PERSON|FOB|en ideas haggle |
+512|33|9|6|12|11196.36|0.04|0.00|R|F|1995-05-21|1995-08-03|1995-06-09|COLLECT COD|FOB|old furiously express deposits. specia|
+512|51|9|7|2|1902.10|0.09|0.08|N|O|1995-06-19|1995-08-13|1995-06-24|NONE|TRUCK|e slyly silent accounts serve with|
+513|62|7|1|20|19241.20|0.09|0.07|N|O|1995-07-12|1995-05-31|1995-07-31|NONE|AIR|efully ironic ideas doze slyl|
+513|122|5|2|44|44973.28|0.01|0.01|N|O|1995-07-14|1995-07-14|1995-08-12|NONE|MAIL|kages sleep boldly ironic theodolites. acco|
+514|79|9|1|21|20560.47|0.06|0.02|N|O|1996-06-09|1996-05-15|1996-07-07|DELIVER IN PERSON|RAIL|s sleep quickly blithely|
+514|118|2|2|34|34615.74|0.08|0.02|N|O|1996-04-14|1996-06-03|1996-04-23|COLLECT COD|REG AIR|ily even patterns. bold, silent instruc|
+514|13|7|3|6|5478.06|0.06|0.01|N|O|1996-05-30|1996-06-04|1996-06-28|COLLECT COD|SHIP|as haggle blithely; quickly s|
+514|116|7|4|43|43692.73|0.00|0.08|N|O|1996-06-07|1996-05-14|1996-07-01|TAKE BACK RETURN|FOB|thely regular |
+515|105|8|1|10|10051.00|0.03|0.02|A|F|1993-10-04|1993-11-03|1993-10-08|NONE|FOB|ar deposits th|
+515|148|1|2|38|39829.32|0.10|0.07|A|F|1993-09-19|1993-11-12|1993-10-03|DELIVER IN PERSON|SHIP|ays. furiously express requests haggle furi|
+515|183|4|3|11|11914.98|0.00|0.02|R|F|1993-09-04|1993-10-02|1993-09-05|DELIVER IN PERSON|FOB|ly pending accounts haggle blithel|
+515|109|10|4|34|34309.40|0.09|0.03|R|F|1993-10-03|1993-10-26|1993-10-15|DELIVER IN PERSON|REG AIR|ic dependencie|
+515|131|7|5|32|32996.16|0.01|0.07|R|F|1993-10-10|1993-10-08|1993-11-02|TAKE BACK RETURN|FOB|r sauternes boost. final theodolites wake a|
+515|109|4|6|25|25227.50|0.04|0.08|R|F|1993-11-14|1993-11-07|1993-12-03|DELIVER IN PERSON|MAIL|e packages engag|
+516|25|4|1|11|10175.22|0.01|0.06|N|O|1998-05-02|1998-05-23|1998-05-12|DELIVER IN PERSON|FOB|ongside of the blithely final reque|
+517|45|6|1|28|26461.12|0.03|0.02|N|O|1997-04-30|1997-05-18|1997-05-17|COLLECT COD|MAIL| requests. special, fi|
+517|156|4|2|15|15842.25|0.02|0.00|N|O|1997-04-09|1997-06-26|1997-05-01|NONE|TRUCK| slyly. express requests ar|
+517|41|8|3|9|8469.36|0.04|0.00|N|O|1997-05-03|1997-06-16|1997-05-24|COLLECT COD|SHIP| slyly stealthily express instructions. |
+517|133|4|4|11|11364.43|0.06|0.02|N|O|1997-06-20|1997-06-01|1997-06-27|NONE|REG AIR|ly throughout the fu|
+517|24|3|5|23|21252.46|0.00|0.01|N|O|1997-04-19|1997-05-07|1997-05-12|COLLECT COD|RAIL| kindle. furiously bold requests mus|
+518|165|6|1|30|31954.80|0.07|0.05|N|O|1998-02-18|1998-03-27|1998-03-16|COLLECT COD|TRUCK|slyly by the packages. carefull|
+518|84|5|2|23|22633.84|0.05|0.07|N|O|1998-02-20|1998-05-05|1998-03-11|COLLECT COD|TRUCK| special requests. fluffily ironic re|
+518|134|5|3|12|12409.56|0.01|0.06|N|O|1998-03-08|1998-03-31|1998-04-06|NONE|AIR| packages thrash slyly|
+518|122|3|4|46|47017.52|0.07|0.02|N|O|1998-04-07|1998-04-17|1998-04-29|NONE|MAIL|. blithely even ideas cajole furiously. b|
+518|71|2|5|16|15537.12|0.01|0.01|N|O|1998-03-15|1998-03-24|1998-04-08|NONE|MAIL|use quickly expre|
+518|197|10|6|39|42790.41|0.09|0.08|N|O|1998-02-26|1998-03-17|1998-03-21|DELIVER IN PERSON|FOB| the bold, special deposits are carefully |
+518|186|7|7|48|52136.64|0.03|0.07|N|O|1998-03-06|1998-04-22|1998-03-14|NONE|FOB| slyly final platelets; quickly even deposi|
+519|159|4|1|1|1059.15|0.07|0.07|N|O|1997-12-01|1998-01-26|1997-12-23|COLLECT COD|REG AIR|bold requests believe furiou|
+519|3|4|2|38|34314.00|0.05|0.08|N|O|1998-02-19|1997-12-15|1998-03-19|DELIVER IN PERSON|FOB|gular excuses detect quickly furiously |
+519|106|1|3|19|19115.90|0.00|0.02|N|O|1998-01-09|1998-01-03|1998-02-06|COLLECT COD|AIR|asymptotes. p|
+519|47|6|4|27|25570.08|0.08|0.06|N|O|1997-11-20|1997-12-06|1997-12-16|DELIVER IN PERSON|REG AIR|le. even, final dependencies|
+519|10|5|5|13|11830.13|0.06|0.08|N|O|1998-02-06|1997-12-02|1998-03-03|TAKE BACK RETURN|TRUCK|c accounts wake along the ironic so|
+519|151|6|6|3|3153.45|0.04|0.00|N|O|1998-02-01|1998-01-25|1998-02-27|TAKE BACK RETURN|FOB|erve blithely blithely ironic asymp|
+544|139|10|1|47|48839.11|0.08|0.06|R|F|1993-03-14|1993-03-27|1993-03-27|COLLECT COD|SHIP|ecial pains. deposits grow foxes. |
+545|170|1|1|4|4280.68|0.02|0.00|N|O|1996-02-23|1995-12-16|1996-03-21|DELIVER IN PERSON|FOB|, ironic grouches cajole over|
+545|171|10|2|18|19281.06|0.00|0.00|N|O|1996-02-21|1996-01-17|1996-02-26|NONE|RAIL|al, final packages affix. even a|
+546|85|6|1|16|15761.28|0.08|0.02|N|O|1997-02-04|1996-12-30|1997-02-25|DELIVER IN PERSON|TRUCK|de of the orbits. sometimes regula|
+547|71|10|1|44|42727.08|0.08|0.08|N|O|1996-10-18|1996-08-17|1996-10-27|TAKE BACK RETURN|FOB|thely express dependencies. qu|
+547|137|8|2|48|49782.24|0.01|0.04|N|O|1996-10-21|1996-08-04|1996-11-20|COLLECT COD|SHIP|thely specia|
+547|182|3|3|3|3246.54|0.05|0.02|N|O|1996-09-04|1996-08-01|1996-09-21|COLLECT COD|SHIP|pinto beans. ironi|
+548|197|8|1|2|2194.38|0.06|0.05|A|F|1994-11-26|1994-11-06|1994-12-06|COLLECT COD|MAIL|ests haggle quickly eve|
+548|5|6|2|6|5430.00|0.00|0.08|A|F|1995-01-18|1994-12-08|1995-02-10|NONE|TRUCK|sits wake furiously regular|
+548|1|8|3|21|18921.00|0.03|0.08|A|F|1995-01-13|1994-12-18|1995-01-25|NONE|AIR|ideas. special accounts above the furiou|
+548|57|9|4|21|20098.05|0.08|0.03|A|F|1994-10-27|1994-12-04|1994-11-21|DELIVER IN PERSON|AIR| engage quickly. regular theo|
+548|93|7|5|19|18868.71|0.00|0.02|A|F|1994-09-24|1994-11-24|1994-10-01|DELIVER IN PERSON|MAIL|courts boost care|
+548|153|8|6|32|33700.80|0.06|0.04|A|F|1994-12-16|1994-11-20|1994-12-29|NONE|REG AIR|c instruction|
+549|196|9|1|18|19731.42|0.07|0.04|R|F|1992-10-19|1992-08-12|1992-11-13|COLLECT COD|REG AIR|furiously according to the ironic, regular |
+549|189|10|2|38|41388.84|0.07|0.05|A|F|1992-08-17|1992-08-28|1992-09-05|COLLECT COD|RAIL|the regular, furious excuses. carefu|
+549|66|7|3|36|34778.16|0.08|0.04|R|F|1992-09-11|1992-10-11|1992-09-12|DELIVER IN PERSON|AIR|ts against the ironic, even theodolites eng|
+549|21|4|4|18|16578.36|0.09|0.01|A|F|1992-07-31|1992-09-11|1992-08-08|NONE|RAIL|ely regular accounts above the |
+549|24|7|5|38|35112.76|0.06|0.02|R|F|1992-08-23|1992-08-12|1992-08-25|COLLECT COD|REG AIR|eposits. carefully regular depos|
+550|191|3|1|31|33826.89|0.04|0.02|N|O|1995-10-24|1995-09-27|1995-11-04|COLLECT COD|AIR|thely silent packages. unusual|
+551|24|9|1|8|7392.16|0.08|0.02|N|O|1995-07-29|1995-07-18|1995-08-02|NONE|REG AIR| wake quickly slyly pending platel|
+551|159|4|2|20|21183.00|0.00|0.07|N|O|1995-09-18|1995-08-25|1995-10-11|COLLECT COD|TRUCK|r ideas. final, even ideas hinder alongside|
+551|162|9|3|16|16994.56|0.07|0.06|N|O|1995-07-29|1995-08-19|1995-08-10|COLLECT COD|MAIL|y along the carefully ex|
+576|87|8|1|2|1974.16|0.07|0.01|N|O|1997-05-15|1997-06-30|1997-05-28|NONE|RAIL|ccounts along the ac|
+576|34|5|2|6|5604.18|0.06|0.05|N|O|1997-05-15|1997-07-26|1997-06-03|DELIVER IN PERSON|TRUCK|al deposits. slyly even sauternes a|
+576|37|3|3|6|5622.18|0.08|0.07|N|O|1997-08-28|1997-06-16|1997-09-25|DELIVER IN PERSON|FOB|ts. ironic multipliers |
+576|138|9|4|5|5190.65|0.03|0.07|N|O|1997-06-11|1997-06-17|1997-07-05|NONE|REG AIR|l foxes boost slyly. accounts af|
+577|26|5|1|25|23150.50|0.06|0.01|A|F|1995-04-09|1995-02-20|1995-05-09|TAKE BACK RETURN|AIR|ve slyly of the frets. careful|
+577|64|1|2|14|13496.84|0.08|0.03|R|F|1995-03-19|1995-02-25|1995-04-09|DELIVER IN PERSON|RAIL|l accounts wake deposits. ironic packa|
+578|156|7|1|40|42246.00|0.02|0.08|N|O|1997-02-10|1997-03-18|1997-02-11|NONE|SHIP|usly even platel|
+578|188|9|2|23|25028.14|0.05|0.08|N|O|1997-03-06|1997-03-03|1997-03-20|TAKE BACK RETURN|FOB|nstructions. ironic deposits|
+579|151|6|1|9|9460.35|0.00|0.05|N|O|1998-06-20|1998-04-28|1998-07-19|DELIVER IN PERSON|RAIL|e ironic, express deposits are furiously|
+579|33|4|2|39|36388.17|0.02|0.01|N|O|1998-06-21|1998-06-03|1998-06-26|COLLECT COD|REG AIR|ncies. furiously final r|
+579|60|5|3|6|5760.36|0.03|0.00|N|O|1998-04-24|1998-05-03|1998-05-08|TAKE BACK RETURN|TRUCK|ickly final requests-- bold accou|
+579|7|10|4|41|37187.00|0.04|0.05|N|O|1998-05-28|1998-05-01|1998-06-04|COLLECT COD|REG AIR|bold, express requests sublate slyly. blith|
+579|13|7|5|28|25564.28|0.00|0.03|N|O|1998-07-10|1998-05-24|1998-07-19|NONE|RAIL|ic ideas until th|
+579|167|6|6|5|5335.80|0.05|0.08|N|O|1998-05-02|1998-04-25|1998-05-05|COLLECT COD|REG AIR|refully silent ideas cajole furious|
+580|85|6|1|33|32507.64|0.03|0.05|N|O|1997-10-11|1997-09-19|1997-10-16|TAKE BACK RETURN|FOB|y express theodolites cajole carefully |
+580|174|5|2|31|33299.27|0.04|0.08|N|O|1997-10-04|1997-09-08|1997-10-15|COLLECT COD|FOB|ose alongside of the sl|
+580|185|6|3|19|20618.42|0.04|0.04|N|O|1997-07-23|1997-09-21|1997-08-15|NONE|FOB|mong the special packag|
+581|64|1|1|41|39526.46|0.09|0.07|N|O|1997-05-26|1997-04-06|1997-06-10|TAKE BACK RETURN|MAIL|nts. quickly|
+581|93|5|2|14|13903.26|0.06|0.08|N|O|1997-05-17|1997-04-14|1997-06-08|NONE|MAIL|. deposits s|
+581|101|6|3|49|49053.90|0.10|0.02|N|O|1997-02-27|1997-04-24|1997-03-10|TAKE BACK RETURN|MAIL|. slyly regular pinto beans acr|
+581|75|4|4|30|29252.10|0.10|0.08|N|O|1997-06-19|1997-05-21|1997-06-22|TAKE BACK RETURN|TRUCK| regular ideas grow furio|
+582|57|9|1|7|6699.35|0.07|0.00|N|O|1997-11-16|1997-11-29|1997-12-10|TAKE BACK RETURN|FOB|ithely unusual t|
+582|51|2|2|49|46601.45|0.05|0.03|N|O|1997-12-17|1998-01-12|1997-12-31|COLLECT COD|REG AIR|nts according to the furiously regular pin|
+582|141|4|3|42|43727.88|0.07|0.00|N|O|1997-11-15|1997-12-21|1997-12-03|COLLECT COD|SHIP|iously beside the silent de|
+582|168|9|4|36|38453.76|0.06|0.01|N|O|1997-12-09|1997-11-27|1997-12-26|TAKE BACK RETURN|SHIP|lar requests. quickly |
+583|145|6|1|1|1045.14|0.07|0.07|N|O|1997-06-17|1997-04-29|1997-06-28|NONE|TRUCK| regular, regular ideas. even, bra|
+583|120|4|2|47|47945.64|0.10|0.06|N|O|1997-07-14|1997-05-12|1997-08-11|DELIVER IN PERSON|AIR|nts are fluffily. furiously even re|
+583|130|5|3|34|35024.42|0.01|0.02|N|O|1997-05-11|1997-04-24|1997-06-03|DELIVER IN PERSON|MAIL|express req|
+583|142|1|4|33|34390.62|0.10|0.01|N|O|1997-05-28|1997-04-25|1997-06-24|NONE|AIR|kages cajole slyly across the|
+583|189|10|5|13|14159.34|0.04|0.06|N|O|1997-06-23|1997-05-29|1997-07-08|COLLECT COD|TRUCK|y sly theodolites. ironi|
+608|154|6|1|19|20028.85|0.08|0.06|N|O|1996-04-19|1996-05-02|1996-05-03|DELIVER IN PERSON|RAIL|ideas. the|
+608|198|1|2|40|43927.60|0.03|0.01|N|O|1996-05-21|1996-04-11|1996-06-02|NONE|AIR| alongside of the regular tithes. sly|
+609|66|5|1|21|20287.26|0.01|0.05|R|F|1994-08-24|1994-08-23|1994-08-27|DELIVER IN PERSON|FOB|de of the special warthogs. excu|
+610|111|8|1|49|49544.39|0.10|0.07|N|O|1995-08-29|1995-10-26|1995-09-12|TAKE BACK RETURN|SHIP|ular instruc|
+610|68|3|2|11|10648.66|0.07|0.08|N|O|1995-10-31|1995-10-25|1995-11-18|NONE|MAIL|blithely final |
+610|118|9|3|26|26470.86|0.09|0.04|N|O|1995-11-22|1995-09-09|1995-12-04|TAKE BACK RETURN|AIR|cross the furiously even theodolites sl|
+610|186|7|4|17|18465.06|0.03|0.03|N|O|1995-11-01|1995-10-30|1995-11-04|COLLECT COD|FOB|p quickly instead of the slyly pending foxe|
+610|146|7|5|39|40799.46|0.08|0.05|N|O|1995-10-30|1995-10-21|1995-11-11|TAKE BACK RETURN|REG AIR|counts. ironic warhorses are |
+610|95|7|6|5|4975.45|0.00|0.07|N|O|1995-08-11|1995-10-22|1995-08-26|TAKE BACK RETURN|FOB|n pinto beans. iro|
+610|190|1|7|27|29435.13|0.06|0.03|N|O|1995-09-02|1995-09-19|1995-09-15|NONE|REG AIR| ironic pinto beans haggle. blithe|
+611|17|7|1|39|35763.39|0.05|0.06|R|F|1993-05-06|1993-04-09|1993-05-22|TAKE BACK RETURN|SHIP|nto beans |
+611|81|2|2|1|981.08|0.08|0.07|R|F|1993-05-17|1993-02-26|1993-06-15|DELIVER IN PERSON|MAIL|ts. pending platelets aff|
+611|120|4|3|39|39784.68|0.09|0.02|A|F|1993-03-10|1993-03-10|1993-03-17|TAKE BACK RETURN|TRUCK|the evenly bold requests. furious|
+612|185|6|1|5|5425.90|0.07|0.00|R|F|1992-11-08|1992-11-20|1992-12-03|TAKE BACK RETURN|RAIL|structions. q|
+612|195|7|2|28|30665.32|0.07|0.06|R|F|1993-01-02|1992-12-11|1993-01-30|DELIVER IN PERSON|TRUCK|regular instructions affix bl|
+612|67|4|3|49|47385.94|0.00|0.08|A|F|1993-01-08|1992-11-25|1993-01-17|TAKE BACK RETURN|REG AIR|theodolite|
+612|39|5|4|28|26292.84|0.05|0.00|A|F|1992-11-12|1992-12-05|1992-12-02|TAKE BACK RETURN|REG AIR|lyly regular asym|
+612|88|9|5|1|988.08|0.08|0.04|R|F|1992-12-18|1992-12-13|1992-12-20|TAKE BACK RETURN|FOB| requests.|
+612|189|10|6|33|35942.94|0.10|0.03|R|F|1992-11-30|1992-12-01|1992-12-12|COLLECT COD|MAIL|bove the blithely even ideas. careful|
+613|91|5|1|17|16848.53|0.06|0.06|N|O|1995-09-23|1995-08-04|1995-10-15|NONE|SHIP|ar dependencie|
+613|79|7|2|6|5874.42|0.05|0.05|N|O|1995-08-05|1995-08-09|1995-08-08|TAKE BACK RETURN|MAIL|y ironic deposits eat |
+613|186|7|3|3|3258.54|0.03|0.01|N|O|1995-09-27|1995-09-11|1995-10-05|NONE|TRUCK|ccounts cajole. |
+613|159|10|4|7|7414.05|0.02|0.04|N|O|1995-09-07|1995-08-02|1995-09-16|DELIVER IN PERSON|MAIL|ously blithely final pinto beans. regula|
+614|195|8|1|21|22998.99|0.00|0.03|R|F|1993-03-29|1993-01-06|1993-04-16|TAKE BACK RETURN|TRUCK|arefully. slyly express packag|
+614|187|8|2|48|52184.64|0.07|0.07|A|F|1993-03-09|1993-01-19|1993-03-19|DELIVER IN PERSON|SHIP|riously special excuses haggle along the|
+614|167|2|3|43|45887.88|0.05|0.00|A|F|1993-03-07|1993-02-22|1993-03-18|DELIVER IN PERSON|SHIP| express accounts wake. slyly ironic ins|
+614|147|6|4|14|14659.96|0.04|0.06|A|F|1992-12-03|1993-02-14|1992-12-27|DELIVER IN PERSON|SHIP|ular packages haggle about the pack|
+614|196|8|5|30|32885.70|0.08|0.07|R|F|1993-01-16|1993-02-08|1993-02-12|TAKE BACK RETURN|FOB|tructions are f|
+614|137|8|6|48|49782.24|0.04|0.08|A|F|1992-12-14|1993-01-22|1993-01-11|NONE|TRUCK| regular platelets cajole quickly eve|
+615|105|6|1|36|36183.60|0.10|0.01|A|F|1992-06-01|1992-07-14|1992-06-27|NONE|FOB| packages. carefully final pinto bea|
+640|93|7|1|49|48661.41|0.09|0.02|R|F|1993-03-27|1993-04-17|1993-04-15|NONE|RAIL|s haggle slyly|
+640|1|4|2|40|36040.00|0.09|0.05|A|F|1993-05-11|1993-04-11|1993-05-15|COLLECT COD|TRUCK|oach according to the bol|
+640|180|8|3|22|23763.96|0.05|0.07|A|F|1993-05-07|1993-04-14|1993-05-21|TAKE BACK RETURN|TRUCK|osits across the slyly regular theodo|
+640|32|3|4|45|41941.35|0.07|0.07|R|F|1993-04-15|1993-04-23|1993-04-21|DELIVER IN PERSON|REG AIR|ong the qui|
+641|126|9|1|18|18470.16|0.01|0.08|R|F|1993-10-17|1993-10-11|1993-10-29|DELIVER IN PERSON|AIR|p blithely bold packages. quick|
+641|100|2|2|1|1000.10|0.09|0.02|R|F|1993-12-03|1993-10-28|1993-12-26|TAKE BACK RETURN|RAIL| nag across the regular foxes.|
+641|95|7|3|40|39803.60|0.05|0.06|R|F|1993-11-22|1993-10-20|1993-12-11|DELIVER IN PERSON|REG AIR|lets. furiously regular requests cajo|
+641|71|10|4|25|24276.75|0.03|0.02|A|F|1993-12-04|1993-11-18|1993-12-18|TAKE BACK RETURN|FOB|d, regular d|
+641|4|9|5|41|37064.00|0.07|0.04|R|F|1993-11-29|1993-10-27|1993-12-04|TAKE BACK RETURN|FOB| asymptotes are quickly. bol|
+642|54|5|1|26|24805.30|0.10|0.03|A|F|1994-04-16|1994-02-01|1994-04-27|COLLECT COD|REG AIR|quests according to the unu|
+643|13|3|1|28|25564.28|0.00|0.08|A|F|1995-04-13|1995-05-12|1995-04-14|TAKE BACK RETURN|TRUCK|ly regular requests nag sly|
+643|51|2|2|48|45650.40|0.01|0.02|N|O|1995-07-10|1995-06-07|1995-08-01|NONE|FOB|ly ironic accounts|
+643|163|2|3|23|24452.68|0.05|0.03|N|O|1995-07-09|1995-05-18|1995-07-31|COLLECT COD|RAIL|sits are carefully according to the e|
+643|45|4|4|39|36856.56|0.08|0.04|A|F|1995-06-08|1995-06-16|1995-06-13|COLLECT COD|RAIL| the pains. carefully s|
+643|190|1|5|47|51238.93|0.10|0.03|R|F|1995-04-05|1995-06-14|1995-04-26|DELIVER IN PERSON|RAIL|y against |
+644|134|10|1|46|47569.98|0.02|0.01|A|F|1992-05-20|1992-06-14|1992-06-14|DELIVER IN PERSON|RAIL| special requests was sometimes expre|
+644|130|3|2|11|11331.43|0.05|0.02|A|F|1992-08-20|1992-07-21|1992-09-11|TAKE BACK RETURN|TRUCK|ealthy pinto beans use carefu|
+644|101|6|3|44|44048.40|0.04|0.04|R|F|1992-08-17|1992-07-26|1992-08-20|COLLECT COD|REG AIR|iously ironic pinto beans. bold packa|
+644|80|8|4|7|6860.56|0.01|0.02|A|F|1992-05-18|1992-07-01|1992-06-07|COLLECT COD|RAIL| regular requests are blithely. slyly|
+644|50|1|5|23|21851.15|0.02|0.04|R|F|1992-07-31|1992-07-28|1992-08-13|DELIVER IN PERSON|TRUCK|uctions nag quickly alongside of t|
+644|85|6|6|33|32507.64|0.00|0.07|R|F|1992-08-26|1992-07-27|1992-08-28|NONE|AIR|ages sleep. bold, bo|
+644|51|9|7|38|36139.90|0.08|0.06|R|F|1992-05-17|1992-07-10|1992-06-06|TAKE BACK RETURN|MAIL| packages. blithely slow accounts nag quic|
+645|160|8|1|33|34985.28|0.01|0.02|A|F|1994-12-09|1995-02-21|1995-01-03|NONE|TRUCK|heodolites b|
+645|170|1|2|47|50297.99|0.07|0.05|R|F|1995-02-16|1995-02-15|1995-02-25|COLLECT COD|TRUCK|hely regular instructions alon|
+645|70|7|3|46|44623.22|0.10|0.01|A|F|1995-01-04|1995-02-21|1995-01-21|COLLECT COD|REG AIR| regular dependencies across the speci|
+645|96|9|4|49|48808.41|0.05|0.03|R|F|1995-01-24|1995-01-06|1995-02-17|NONE|TRUCK|y. slyly iron|
+645|5|8|5|43|38915.00|0.06|0.02|A|F|1995-02-12|1995-02-27|1995-03-06|TAKE BACK RETURN|REG AIR| furiously accounts. slyly|
+645|34|5|6|18|16812.54|0.10|0.08|A|F|1995-03-02|1995-02-08|1995-03-03|COLLECT COD|RAIL|ep. slyly even |
+645|28|9|7|9|8352.18|0.03|0.03|A|F|1994-12-25|1995-01-04|1995-01-15|COLLECT COD|REG AIR|special deposits. regular, final th|
+646|109|6|1|31|31282.10|0.00|0.05|R|F|1994-12-17|1995-02-16|1995-01-04|COLLECT COD|MAIL|ag furiousl|
+646|127|8|2|1|1027.12|0.07|0.01|A|F|1994-12-05|1995-01-07|1994-12-31|TAKE BACK RETURN|MAIL|t blithely regular deposits. quic|
+646|30|9|3|24|22320.72|0.06|0.02|A|F|1995-02-20|1994-12-30|1995-03-16|TAKE BACK RETURN|TRUCK|regular accounts haggle dog|
+646|99|2|4|34|33969.06|0.01|0.00|R|F|1994-12-28|1994-12-27|1994-12-31|COLLECT COD|SHIP|slow accounts. fluffily idle instructions|
+646|90|1|5|17|16831.53|0.04|0.01|A|F|1994-12-31|1994-12-26|1995-01-01|DELIVER IN PERSON|REG AIR|inal packages haggle carefully |
+646|115|2|6|40|40604.40|0.10|0.01|R|F|1995-01-01|1995-01-13|1995-01-11|COLLECT COD|TRUCK|ronic packages sleep across th|
+647|17|1|1|41|37597.41|0.08|0.08|N|O|1997-11-19|1997-09-24|1997-12-15|COLLECT COD|REG AIR|r instructions. quickly unusu|
+647|113|10|2|5|5065.55|0.10|0.00|N|O|1997-09-25|1997-09-22|1997-10-25|TAKE BACK RETURN|AIR|ly express packages haggle caref|
+647|153|8|3|15|15797.25|0.08|0.00|N|O|1997-09-23|1997-10-09|1997-10-21|NONE|MAIL|ve the even, bold foxes sleep |
+672|173|4|1|41|43999.97|0.06|0.06|R|F|1994-06-20|1994-07-03|1994-06-22|COLLECT COD|REG AIR| dependencies in|
+672|190|1|2|9|9811.71|0.03|0.04|R|F|1994-06-25|1994-06-06|1994-07-19|TAKE BACK RETURN|TRUCK|haggle carefully carefully reg|
+672|143|2|3|35|36509.90|0.02|0.01|R|F|1994-07-13|1994-06-04|1994-07-14|COLLECT COD|RAIL| dependencies haggle quickly. theo|
+673|71|10|1|22|21363.54|0.03|0.02|R|F|1994-03-15|1994-04-27|1994-03-29|TAKE BACK RETURN|TRUCK| the regular, even requests. carefully fin|
+674|102|5|1|23|23048.30|0.06|0.07|A|F|1992-10-25|1992-10-15|1992-11-03|COLLECT COD|SHIP|ve the quickly even deposits. blithe|
+674|59|4|2|4|3836.20|0.02|0.07|R|F|1992-10-05|1992-11-22|1992-10-22|NONE|RAIL|ly express pinto beans sleep car|
+675|157|9|1|1|1057.15|0.04|0.08|N|O|1997-11-27|1997-09-30|1997-12-12|DELIVER IN PERSON|REG AIR|ide of the slyly regular packages. unus|
+675|137|3|2|35|36299.55|0.08|0.07|N|O|1997-08-19|1997-10-16|1997-09-17|DELIVER IN PERSON|REG AIR|s. furiously expre|
+675|176|6|3|34|36589.78|0.10|0.04|N|O|1997-11-17|1997-10-07|1997-11-27|NONE|FOB|y final accounts unwind around the |
+675|100|4|4|15|15001.50|0.09|0.05|N|O|1997-10-18|1997-09-28|1997-11-13|COLLECT COD|TRUCK|posits after the furio|
+675|5|8|5|46|41630.00|0.09|0.05|N|O|1997-09-18|1997-10-14|1997-10-01|DELIVER IN PERSON|AIR| deposits along the express foxes |
+676|51|3|1|9|8559.45|0.09|0.02|N|O|1997-04-03|1997-02-02|1997-04-08|COLLECT COD|REG AIR|aintain sl|
+676|78|6|2|20|19561.40|0.07|0.07|N|O|1997-02-02|1997-02-01|1997-02-11|NONE|REG AIR|riously around the blithely |
+676|163|4|3|35|37210.60|0.05|0.01|N|O|1996-12-30|1997-01-13|1997-01-19|DELIVER IN PERSON|RAIL|into beans. blithe|
+676|73|1|4|24|23353.68|0.01|0.06|N|O|1997-02-05|1997-01-16|1997-03-07|TAKE BACK RETURN|TRUCK|ress, regular dep|
+676|166|1|5|31|33050.96|0.01|0.06|N|O|1997-02-06|1997-02-28|1997-03-08|COLLECT COD|TRUCK|ial deposits cajo|
+676|76|7|6|33|32210.31|0.09|0.05|N|O|1997-03-02|1997-02-22|1997-03-19|TAKE BACK RETURN|TRUCK|as wake slyly furiously close pinto b|
+676|143|6|7|11|11474.54|0.07|0.02|N|O|1997-03-09|1997-03-06|1997-03-31|TAKE BACK RETURN|MAIL|he final acco|
+677|59|7|1|32|30689.60|0.04|0.08|R|F|1994-01-06|1994-01-31|1994-02-02|NONE|RAIL|slyly final|
+677|168|9|2|39|41658.24|0.00|0.07|R|F|1993-12-19|1994-02-11|1994-01-05|TAKE BACK RETURN|SHIP|ges. furiously regular packages use |
+677|24|5|3|46|42504.92|0.01|0.02|R|F|1993-12-02|1994-02-12|1993-12-06|COLLECT COD|RAIL|ng theodolites. furiously unusual theodo|
+677|148|7|4|1|1048.14|0.06|0.05|R|F|1993-12-01|1994-01-14|1993-12-26|DELIVER IN PERSON|MAIL|ly. regular |
+677|150|9|5|25|26253.75|0.00|0.05|A|F|1994-03-12|1994-02-02|1994-03-28|DELIVER IN PERSON|AIR| packages integrate blithely|
+678|146|7|1|20|20922.80|0.05|0.08|R|F|1993-06-21|1993-04-07|1993-07-10|TAKE BACK RETURN|MAIL|furiously express excuses. foxes eat fu|
+678|37|3|2|22|20614.66|0.01|0.02|A|F|1993-05-10|1993-04-29|1993-06-08|NONE|REG AIR|de of the carefully even requests. bl|
+678|143|10|3|16|16690.24|0.06|0.02|R|F|1993-03-20|1993-04-13|1993-04-16|DELIVER IN PERSON|REG AIR|equests cajole around the carefully regular|
+678|199|10|4|48|52761.12|0.08|0.08|R|F|1993-02-28|1993-04-04|1993-03-24|NONE|REG AIR|ithely. slyly express foxes|
+678|98|9|5|16|15969.44|0.06|0.04|R|F|1993-03-09|1993-04-18|1993-04-07|NONE|AIR| about the |
+678|43|4|6|11|10373.44|0.09|0.00|R|F|1993-04-28|1993-05-16|1993-05-11|COLLECT COD|TRUCK|ess deposits dazzle f|
+679|192|3|1|9|9829.71|0.09|0.00|N|O|1995-12-20|1996-01-27|1996-01-07|COLLECT COD|REG AIR|leep slyly. entici|
+704|190|1|1|40|43607.60|0.05|0.05|N|O|1997-01-30|1997-01-10|1997-02-20|COLLECT COD|AIR|ggle quickly. r|
+704|4|5|2|14|12656.00|0.07|0.08|N|O|1997-02-02|1996-12-26|1997-02-19|DELIVER IN PERSON|REG AIR|ve the quickly final forges. furiously p|
+705|189|10|1|46|50102.28|0.05|0.06|N|O|1997-04-18|1997-05-06|1997-05-05|DELIVER IN PERSON|SHIP|ss deposits. ironic packa|
+705|117|7|2|35|35598.85|0.10|0.04|N|O|1997-03-25|1997-03-20|1997-04-23|TAKE BACK RETURN|FOB|carefully ironic accounts|
+706|197|9|1|23|25235.37|0.05|0.00|N|O|1995-12-06|1995-12-02|1995-12-16|COLLECT COD|SHIP|ckey players. requests above the|
+707|155|6|1|34|35875.10|0.01|0.02|R|F|1994-12-08|1995-01-15|1995-01-02|NONE|RAIL| dependencies|
+707|43|10|2|22|20746.88|0.00|0.06|A|F|1995-01-12|1994-12-28|1995-01-16|DELIVER IN PERSON|REG AIR| kindle ironically|
+708|124|7|1|3|3072.36|0.05|0.02|N|O|1998-10-09|1998-09-22|1998-11-07|COLLECT COD|FOB|e slyly pending foxes. |
+708|180|1|2|19|20523.42|0.06|0.00|N|O|1998-10-28|1998-09-23|1998-11-25|COLLECT COD|SHIP| requests. even, thin ideas|
+708|122|5|3|33|33729.96|0.09|0.06|N|O|1998-09-10|1998-09-20|1998-09-22|COLLECT COD|RAIL|s boost carefully ruthless theodolites. f|
+708|56|7|4|5|4780.25|0.07|0.07|N|O|1998-07-22|1998-08-15|1998-07-28|TAKE BACK RETURN|REG AIR|c pinto beans nag after the account|
+708|143|2|5|36|37553.04|0.08|0.01|N|O|1998-07-16|1998-09-04|1998-08-11|NONE|SHIP|ests. even, regular hockey p|
+708|23|6|6|7|6461.14|0.10|0.03|N|O|1998-08-16|1998-08-15|1998-09-10|COLLECT COD|REG AIR|lly express ac|
+709|87|8|1|7|6909.56|0.00|0.00|N|O|1998-06-14|1998-06-08|1998-06-18|TAKE BACK RETURN|RAIL| special orbits cajole |
+709|198|10|2|15|16472.85|0.08|0.00|N|O|1998-07-10|1998-06-26|1998-08-09|NONE|RAIL|ily regular deposits. sauternes was accor|
+709|169|8|3|10|10691.60|0.01|0.02|N|O|1998-06-04|1998-06-30|1998-06-11|NONE|REG AIR|ts cajole boldly |
+709|108|9|4|40|40324.00|0.10|0.08|N|O|1998-08-12|1998-06-20|1998-08-20|DELIVER IN PERSON|RAIL|ggle fluffily carefully ironic|
+710|163|8|1|47|49968.52|0.06|0.08|A|F|1993-01-18|1993-03-24|1993-01-24|TAKE BACK RETURN|MAIL|usual ideas into th|
+710|193|4|2|38|41541.22|0.07|0.02|R|F|1993-04-18|1993-03-12|1993-05-15|COLLECT COD|FOB|sts boost fluffily aft|
+710|139|5|3|7|7273.91|0.04|0.06|R|F|1993-01-20|1993-03-28|1993-02-15|TAKE BACK RETURN|REG AIR|xpress, special ideas. bl|
+710|90|1|4|25|24752.25|0.00|0.05|R|F|1993-03-31|1993-02-05|1993-04-22|COLLECT COD|FOB|eas detect do|
+710|186|7|5|12|13034.16|0.01|0.02|A|F|1993-02-18|1993-02-27|1993-03-07|DELIVER IN PERSON|MAIL|ions. slyly express theodolites al|
+710|114|4|6|21|21296.31|0.04|0.06|R|F|1993-03-22|1993-03-05|1993-03-27|DELIVER IN PERSON|SHIP|es. furiously p|
+710|160|2|7|46|48767.36|0.03|0.07|R|F|1993-04-16|1993-03-27|1993-05-05|COLLECT COD|MAIL|ges use; blithely pending excuses inte|
+711|146|3|1|2|2092.28|0.10|0.04|R|F|1993-12-01|1993-12-09|1993-12-16|DELIVER IN PERSON|REG AIR|ely across t|
+711|103|8|2|27|27083.70|0.00|0.08|A|F|1993-10-02|1993-10-26|1993-10-08|DELIVER IN PERSON|MAIL|slyly. ironic asy|
+711|128|7|3|46|47293.52|0.10|0.00|R|F|1993-12-26|1993-11-19|1994-01-21|TAKE BACK RETURN|MAIL|deposits. permanen|
+711|128|9|4|20|20562.40|0.09|0.00|A|F|1994-01-17|1993-11-10|1994-01-31|DELIVER IN PERSON|TRUCK|kly regular acco|
+736|158|9|1|46|48674.90|0.05|0.01|N|O|1998-07-16|1998-09-01|1998-08-09|NONE|AIR|uctions cajole|
+736|80|1|2|23|22541.84|0.02|0.05|N|O|1998-10-08|1998-08-27|1998-10-19|TAKE BACK RETURN|AIR|k accounts are carefully|
+736|57|9|3|13|12441.65|0.00|0.03|N|O|1998-08-16|1998-07-26|1998-08-19|DELIVER IN PERSON|FOB|st furiously among the |
+736|98|2|4|14|13973.26|0.06|0.04|N|O|1998-10-04|1998-08-14|1998-10-16|COLLECT COD|REG AIR|nstructions.|
+736|169|6|5|32|34213.12|0.04|0.03|N|O|1998-07-30|1998-08-22|1998-08-12|DELIVER IN PERSON|RAIL|iously final accoun|
+737|182|3|1|12|12986.16|0.01|0.01|R|F|1992-04-28|1992-06-30|1992-05-08|COLLECT COD|RAIL|posits after the slyly bold du|
+738|198|1|1|34|37338.46|0.00|0.06|R|F|1993-06-09|1993-04-15|1993-07-09|TAKE BACK RETURN|TRUCK|s against the ironic exc|
+738|188|9|2|4|4352.72|0.00|0.03|A|F|1993-06-20|1993-04-08|1993-07-09|NONE|AIR|ar packages. fluffily bo|
+738|170|1|3|23|24613.91|0.04|0.08|A|F|1993-03-17|1993-04-02|1993-04-05|TAKE BACK RETURN|SHIP|nic, final excuses promise quickly regula|
+738|141|10|4|12|12493.68|0.04|0.08|A|F|1993-06-16|1993-05-05|1993-06-22|NONE|SHIP|ove the slyly regular p|
+738|175|4|5|30|32255.10|0.02|0.00|A|F|1993-06-12|1993-05-29|1993-06-25|NONE|AIR|ecial instructions haggle blithely regula|
+739|85|6|1|28|27582.24|0.00|0.03|N|O|1998-06-03|1998-08-04|1998-06-29|TAKE BACK RETURN|RAIL|elets about the pe|
+739|4|7|2|50|45200.00|0.07|0.06|N|O|1998-08-26|1998-07-16|1998-09-02|COLLECT COD|MAIL|ndencies. blith|
+739|49|2|3|12|11388.48|0.05|0.00|N|O|1998-08-20|1998-07-24|1998-08-22|NONE|MAIL|le slyly along the close i|
+739|44|3|4|47|44369.88|0.09|0.07|N|O|1998-08-12|1998-07-09|1998-08-28|NONE|REG AIR|deas according to the theodolites sn|
+739|188|9|5|30|32645.40|0.07|0.06|N|O|1998-06-19|1998-08-26|1998-07-02|DELIVER IN PERSON|REG AIR|above the even deposits. ironic requests|
+740|2|9|1|22|19844.00|0.10|0.02|N|O|1995-07-24|1995-09-11|1995-08-11|TAKE BACK RETURN|FOB|odolites cajole ironic, pending instruc|
+740|66|1|2|35|33812.10|0.00|0.00|N|O|1995-09-06|1995-08-22|1995-10-02|NONE|TRUCK|p quickly. fu|
+740|199|10|3|29|31876.51|0.06|0.05|N|O|1995-10-26|1995-09-17|1995-10-29|DELIVER IN PERSON|FOB|ntly bold pinto beans sleep quickl|
+741|187|8|1|25|27179.50|0.03|0.06|N|O|1998-07-15|1998-08-27|1998-08-12|DELIVER IN PERSON|MAIL|accounts. blithely bold pa|
+741|91|4|2|22|21803.98|0.09|0.01|N|O|1998-09-07|1998-09-28|1998-09-12|COLLECT COD|AIR|ven deposits about the regular, ironi|
+742|102|3|1|46|46096.60|0.04|0.08|A|F|1995-03-12|1995-03-20|1995-03-16|TAKE BACK RETURN|SHIP|e slyly bold deposits cajole according to|
+742|96|8|2|15|14941.35|0.08|0.05|A|F|1995-02-26|1995-03-20|1995-03-03|NONE|SHIP|blithely unusual pinto|
+742|102|5|3|24|24050.40|0.08|0.08|A|F|1995-02-12|1995-03-12|1995-02-14|DELIVER IN PERSON|SHIP|affix slyly. furiously i|
+742|192|4|4|16|17475.04|0.01|0.05|A|F|1995-01-15|1995-02-25|1995-01-24|COLLECT COD|AIR|eodolites haggle carefully regul|
+742|101|4|5|48|48052.80|0.09|0.08|R|F|1995-03-24|1995-01-23|1995-04-08|TAKE BACK RETURN|TRUCK| platelets |
+742|192|6|6|49|53517.31|0.02|0.07|A|F|1995-01-13|1995-02-13|1995-01-26|TAKE BACK RETURN|RAIL| carefully bold foxes sle|
+743|192|5|1|21|22935.99|0.01|0.04|N|O|1996-10-26|1996-11-05|1996-11-11|COLLECT COD|MAIL|d requests. packages afte|
+768|196|7|1|39|42751.41|0.06|0.08|N|O|1996-09-25|1996-10-27|1996-10-20|NONE|SHIP|out the ironic|
+768|18|9|2|2|1836.02|0.00|0.04|N|O|1996-11-13|1996-10-03|1996-11-25|DELIVER IN PERSON|SHIP|ular courts. slyly dogged accou|
+768|6|1|3|30|27180.00|0.06|0.05|N|O|1996-09-22|1996-11-03|1996-10-13|NONE|MAIL| furiously fluffy pinto beans haggle along|
+768|25|8|4|37|34225.74|0.10|0.00|N|O|1996-10-02|1996-09-23|1996-10-14|TAKE BACK RETURN|REG AIR|ending requests across the quickly|
+768|47|10|5|47|44510.88|0.06|0.05|N|O|1996-11-28|1996-10-30|1996-12-12|NONE|TRUCK|foxes. slyly ironic deposits a|
+768|112|9|6|43|43520.73|0.10|0.06|N|O|1996-09-22|1996-11-03|1996-10-22|TAKE BACK RETURN|AIR|sual ideas wake quickly|
+768|49|10|7|33|31318.32|0.01|0.04|N|O|1996-09-06|1996-09-29|1996-10-01|COLLECT COD|RAIL|sly ironic instructions. excuses can hagg|
+769|176|6|1|36|38742.12|0.02|0.02|A|F|1993-10-01|1993-08-07|1993-10-15|NONE|AIR|es. furiously iro|
+769|160|8|2|4|4240.64|0.01|0.04|R|F|1993-06-25|1993-08-12|1993-07-15|DELIVER IN PERSON|FOB| ideas. even|
+770|181|2|1|39|42166.02|0.09|0.06|N|O|1998-07-19|1998-08-09|1998-08-04|NONE|REG AIR|osits. foxes cajole |
+770|54|2|2|25|23851.25|0.03|0.02|N|O|1998-05-26|1998-07-23|1998-06-04|TAKE BACK RETURN|AIR| deposits dazzle fluffily alongside of |
+771|7|4|1|12|10884.00|0.10|0.08|N|O|1995-07-18|1995-08-02|1995-08-07|COLLECT COD|TRUCK|carefully. pending in|
+771|161|10|2|38|40324.08|0.03|0.08|N|O|1995-07-22|1995-09-10|1995-07-29|TAKE BACK RETURN|REG AIR| quickly final requests are final packages.|
+771|7|8|3|14|12698.00|0.02|0.05|N|O|1995-07-31|1995-08-13|1995-08-07|DELIVER IN PERSON|AIR|r, final packages are slyly iro|
+771|42|3|4|7|6594.28|0.06|0.02|N|O|1995-06-18|1995-08-31|1995-06-20|NONE|REG AIR|theodolites after the fluffily express |
+771|78|6|5|13|12714.91|0.09|0.01|N|O|1995-08-10|1995-08-21|1995-08-30|NONE|FOB|packages affix slyly about the quickly |
+771|82|3|6|23|22587.84|0.08|0.03|N|O|1995-06-19|1995-09-07|1995-07-09|COLLECT COD|FOB|cajole besides the quickly ironic pin|
+772|53|5|1|35|33356.75|0.10|0.06|R|F|1993-07-05|1993-06-05|1993-08-02|NONE|SHIP|kly thin packages wake slowly|
+772|84|5|2|10|9840.80|0.05|0.01|R|F|1993-05-20|1993-05-19|1993-06-15|DELIVER IN PERSON|MAIL| deposits cajole carefully instructions. t|
+772|86|7|3|35|34512.80|0.03|0.04|R|F|1993-04-18|1993-06-13|1993-05-01|COLLECT COD|MAIL|ng ideas. special packages haggle alon|
+772|180|8|4|10|10801.80|0.08|0.02|A|F|1993-05-17|1993-06-09|1993-05-29|COLLECT COD|AIR|o the furiously final deposits. furi|
+772|54|5|5|42|40070.10|0.02|0.07|A|F|1993-06-09|1993-07-16|1993-06-12|DELIVER IN PERSON|MAIL| express foxes abo|
+773|100|1|1|5|5000.50|0.06|0.04|A|F|1993-11-21|1993-12-19|1993-12-21|COLLECT COD|MAIL|ar requests. regular, thin packages u|
+773|11|5|2|31|28241.31|0.02|0.06|A|F|1993-12-30|1993-11-02|1994-01-01|TAKE BACK RETURN|MAIL|e slyly unusual deposit|
+773|151|3|3|39|40994.85|0.06|0.05|A|F|1994-01-04|1993-12-23|1994-01-26|DELIVER IN PERSON|FOB|quickly eve|
+773|29|8|4|28|26012.56|0.10|0.06|R|F|1994-01-19|1993-11-05|1994-01-23|NONE|TRUCK|he furiously slow deposits.|
+773|134|5|5|9|9307.17|0.09|0.02|R|F|1993-10-09|1993-12-25|1993-11-04|TAKE BACK RETURN|FOB|ent orbits haggle fluffily after the |
+773|40|1|6|43|40421.72|0.07|0.03|A|F|1993-11-06|1993-11-20|1993-11-08|TAKE BACK RETURN|SHIP|furiously bold dependencies. blithel|
+774|183|4|1|49|53075.82|0.08|0.03|N|O|1995-12-06|1996-01-07|1995-12-14|DELIVER IN PERSON|SHIP|ess accounts are carefully |
+774|17|4|2|3|2751.03|0.10|0.06|N|O|1996-02-13|1996-01-14|1996-03-04|COLLECT COD|FOB| slyly even courts nag blith|
+774|148|7|3|34|35636.76|0.02|0.07|N|O|1996-03-16|1996-01-03|1996-03-22|COLLECT COD|FOB|lar excuses are furiously final instr|
+774|15|6|4|8|7320.08|0.00|0.02|N|O|1996-01-24|1996-01-15|1996-02-13|COLLECT COD|RAIL|ully ironic requests c|
+774|177|5|5|44|47395.48|0.09|0.07|N|O|1996-02-29|1996-01-16|1996-03-06|NONE|REG AIR|s according to the deposits unwind ca|
+774|120|1|6|2|2040.24|0.07|0.03|N|O|1995-12-11|1996-02-10|1995-12-14|TAKE BACK RETURN|SHIP|accounts; slyly regular|
+775|32|3|1|16|14912.48|0.10|0.06|N|F|1995-05-23|1995-05-07|1995-06-19|NONE|TRUCK|un quickly slyly|
+775|174|2|2|21|22557.57|0.01|0.06|R|F|1995-05-01|1995-06-02|1995-05-13|DELIVER IN PERSON|FOB| quickly sile|
+775|108|5|3|20|20162.00|0.01|0.08|N|F|1995-06-17|1995-05-22|1995-07-13|COLLECT COD|AIR|en dependencies nag slowly |
+800|72|1|1|38|36938.66|0.00|0.05|N|O|1998-07-21|1998-09-25|1998-08-07|TAKE BACK RETURN|TRUCK|according to the bold, final dependencies |
+800|85|6|2|21|20686.68|0.04|0.05|N|O|1998-07-23|1998-10-01|1998-08-20|TAKE BACK RETURN|RAIL|ckly even requests after the carefully r|
+800|176|5|3|26|27980.42|0.01|0.02|N|O|1998-07-23|1998-10-08|1998-07-25|DELIVER IN PERSON|FOB|bove the pending requests.|
+801|6|3|1|13|11778.00|0.10|0.02|R|F|1992-04-25|1992-04-24|1992-05-16|COLLECT COD|RAIL|s are fluffily stealthily expres|
+801|95|8|2|21|20896.89|0.05|0.02|A|F|1992-03-14|1992-04-01|1992-04-05|COLLECT COD|AIR|wake silently furiously idle deposits. |
+801|3|4|3|21|18963.00|0.05|0.03|A|F|1992-04-25|1992-03-20|1992-05-04|COLLECT COD|REG AIR|cial, special packages.|
+801|164|9|4|12|12769.92|0.08|0.04|A|F|1992-06-06|1992-04-14|1992-06-12|TAKE BACK RETURN|RAIL|s. ironic pinto b|
+801|74|2|5|45|43833.15|0.01|0.06|R|F|1992-03-22|1992-03-22|1992-03-25|COLLECT COD|REG AIR| even asymptotes|
+801|122|7|6|10|10221.20|0.08|0.01|A|F|1992-06-05|1992-05-15|1992-06-21|DELIVER IN PERSON|MAIL|al accounts. carefully regular foxes wake|
+801|26|5|7|11|10186.22|0.01|0.03|A|F|1992-05-09|1992-04-19|1992-05-15|DELIVER IN PERSON|REG AIR|y special pinto beans cajole |
+802|143|6|1|40|41725.60|0.08|0.08|A|F|1995-01-07|1995-04-03|1995-01-23|DELIVER IN PERSON|RAIL|y bold accou|
+802|133|4|2|34|35126.42|0.08|0.06|A|F|1995-03-01|1995-03-15|1995-03-12|COLLECT COD|AIR|instructions cajole carefully. quietl|
+802|131|2|3|44|45369.72|0.07|0.04|R|F|1995-01-09|1995-02-04|1995-01-18|TAKE BACK RETURN|SHIP|rmanently idly special requ|
+802|157|2|4|18|19028.70|0.09|0.02|R|F|1995-03-06|1995-02-07|1995-03-19|TAKE BACK RETURN|RAIL|y regular requests engage furiously final d|
+802|132|3|5|19|19610.47|0.08|0.06|A|F|1995-04-01|1995-02-20|1995-04-23|DELIVER IN PERSON|REG AIR|old, furious|
+803|54|9|1|8|7632.40|0.07|0.01|N|O|1997-08-04|1997-06-19|1997-08-12|NONE|SHIP|ronic theodo|
+803|99|10|2|21|20980.89|0.08|0.06|N|O|1997-08-25|1997-06-30|1997-09-10|TAKE BACK RETURN|AIR|ironic packages cajole slyly. un|
+804|126|7|1|30|30783.60|0.08|0.04|A|F|1993-03-29|1993-05-07|1993-04-14|COLLECT COD|REG AIR|ehind the quietly regular pac|
+804|199|3|2|2|2198.38|0.02|0.00|A|F|1993-06-23|1993-04-30|1993-06-25|NONE|TRUCK|slyly silent |
+804|76|5|3|44|42947.08|0.04|0.05|R|F|1993-07-06|1993-04-13|1993-07-28|DELIVER IN PERSON|TRUCK|ly final deposits? special |
+804|38|9|4|21|19698.63|0.01|0.00|A|F|1993-04-12|1993-06-06|1993-04-20|DELIVER IN PERSON|RAIL|ular, ironic foxes. quickly even accounts|
+805|198|10|1|25|27454.75|0.07|0.06|N|O|1995-08-05|1995-09-30|1995-08-06|NONE|AIR|ide of the pending, sly requests. quickly f|
+805|57|5|2|29|27754.45|0.07|0.01|N|O|1995-08-24|1995-08-15|1995-09-16|TAKE BACK RETURN|AIR|dolites according to the slyly f|
+805|47|8|3|12|11364.48|0.01|0.06|N|O|1995-07-13|1995-09-27|1995-08-02|TAKE BACK RETURN|REG AIR| regular foxes. furio|
+805|76|6|4|26|25377.82|0.08|0.07|N|O|1995-08-28|1995-09-24|1995-09-11|TAKE BACK RETURN|RAIL|. ironic deposits sleep across |
+806|105|2|1|1|1005.10|0.04|0.07|N|O|1996-07-14|1996-09-12|1996-07-25|COLLECT COD|RAIL|ar accounts? pending, pending foxes a|
+806|160|5|2|22|23323.52|0.08|0.02|N|O|1996-10-03|1996-08-11|1996-10-20|DELIVER IN PERSON|REG AIR|fily pending |
+806|91|3|3|4|3964.36|0.04|0.03|N|O|1996-08-09|1996-09-18|1996-08-13|COLLECT COD|TRUCK|eans. quickly ironic ideas |
+807|117|7|1|49|49838.39|0.00|0.00|R|F|1993-12-05|1994-01-13|1993-12-25|COLLECT COD|REG AIR| furiously according to the un|
+807|155|10|2|49|51702.35|0.01|0.06|A|F|1994-01-17|1994-01-24|1994-01-22|COLLECT COD|TRUCK|y regular requests haggle.|
+807|181|2|3|48|51896.64|0.07|0.07|A|F|1994-01-08|1994-02-02|1994-01-15|DELIVER IN PERSON|SHIP|kly across the f|
+807|80|1|4|10|9800.80|0.09|0.00|R|F|1994-01-19|1994-02-12|1994-01-28|NONE|TRUCK|furiously final depths sleep a|
+807|143|6|5|30|31294.20|0.02|0.01|R|F|1994-01-19|1994-01-09|1994-01-27|NONE|RAIL|cial accoun|
+807|12|2|6|11|10032.11|0.02|0.04|R|F|1994-03-25|1994-01-26|1994-04-14|NONE|FOB|unts above the slyly final ex|
+807|1|6|7|19|17119.00|0.08|0.05|A|F|1994-02-10|1994-02-20|1994-03-06|NONE|SHIP|ns haggle quickly across the furi|
+832|103|6|1|45|45139.50|0.01|0.02|A|F|1992-05-08|1992-06-06|1992-06-04|COLLECT COD|MAIL|foxes engage slyly alon|
+832|48|1|2|24|22752.96|0.05|0.06|A|F|1992-06-15|1992-07-14|1992-06-17|NONE|TRUCK|ully. carefully speci|
+833|54|5|1|1|954.05|0.04|0.04|R|F|1994-04-26|1994-04-05|1994-04-29|COLLECT COD|MAIL|ffily ironic theodolites|
+833|112|6|2|38|38460.18|0.05|0.05|A|F|1994-04-05|1994-04-21|1994-05-01|COLLECT COD|TRUCK| platelets promise furiously. |
+833|162|7|3|9|9559.44|0.05|0.07|A|F|1994-02-28|1994-04-26|1994-03-20|TAKE BACK RETURN|FOB|ecial, even requests. even, bold instructi|
+834|145|2|1|36|37625.04|0.06|0.04|R|F|1994-06-28|1994-07-25|1994-07-07|TAKE BACK RETURN|SHIP|ccounts haggle after the furiously |
+834|7|2|2|11|9977.00|0.03|0.00|A|F|1994-09-18|1994-08-03|1994-10-02|DELIVER IN PERSON|TRUCK|inst the regular packa|
+835|107|2|1|33|33234.30|0.09|0.06|N|O|1995-11-01|1995-12-02|1995-11-24|DELIVER IN PERSON|RAIL|onic instructions among the carefully iro|
+835|185|6|2|28|30385.04|0.02|0.02|N|O|1995-12-27|1995-12-11|1996-01-21|NONE|SHIP| fluffily furious pinto beans|
+836|188|9|1|6|6529.08|0.09|0.03|N|O|1996-12-09|1997-01-31|1996-12-29|COLLECT COD|TRUCK|fully bold theodolites are daringly across|
+836|84|5|2|18|17713.44|0.03|0.05|N|O|1997-02-27|1997-02-11|1997-03-22|NONE|REG AIR|y pending packages use alon|
+836|141|8|3|46|47892.44|0.05|0.07|N|O|1997-03-21|1997-02-06|1997-04-05|NONE|REG AIR|boldly final pinto beans haggle furiously|
+837|57|5|1|39|37324.95|0.03|0.08|A|F|1994-07-22|1994-08-10|1994-08-11|NONE|RAIL|ecial pinto bea|
+837|88|9|2|24|23713.92|0.08|0.00|R|F|1994-06-27|1994-09-02|1994-07-27|DELIVER IN PERSON|FOB|p carefully. theodolites use. bold courts a|
+838|134|10|1|20|20682.60|0.10|0.07|N|O|1998-04-11|1998-03-25|1998-04-19|COLLECT COD|TRUCK| furiously final ideas. slow, bold |
+838|29|10|2|27|25083.54|0.05|0.07|N|O|1998-02-15|1998-04-03|1998-02-20|DELIVER IN PERSON|SHIP| pending pinto beans haggle about t|
+838|95|7|3|23|22887.07|0.10|0.07|N|O|1998-03-26|1998-04-17|1998-04-02|COLLECT COD|AIR|ets haggle furiously furiously regular r|
+838|44|5|4|18|16992.72|0.09|0.00|N|O|1998-03-28|1998-04-06|1998-03-31|TAKE BACK RETURN|AIR|hely unusual foxes. furio|
+839|158|10|1|23|24337.45|0.07|0.02|N|O|1995-10-17|1995-11-03|1995-11-04|COLLECT COD|AIR|ng ideas haggle accord|
+839|189|10|2|47|51191.46|0.08|0.00|N|O|1995-10-17|1995-11-06|1995-11-10|NONE|AIR|refully final excuses about |
+864|130|5|1|34|35024.42|0.03|0.04|N|O|1997-12-16|1997-10-23|1998-01-12|TAKE BACK RETURN|SHIP|gside of the furiously special|
+864|98|1|2|7|6986.63|0.01|0.02|N|O|1997-11-13|1997-10-07|1997-12-13|TAKE BACK RETURN|MAIL|ven requests should sleep along |
+864|80|10|3|34|33322.72|0.03|0.00|N|O|1997-09-14|1997-11-04|1997-09-21|TAKE BACK RETURN|REG AIR|to the furiously ironic platelets! |
+865|198|10|1|16|17571.04|0.07|0.03|R|F|1993-08-24|1993-06-26|1993-08-28|TAKE BACK RETURN|TRUCK|y even accounts. quickly bold decoys|
+865|20|7|2|3|2760.06|0.02|0.05|A|F|1993-07-17|1993-07-14|1993-08-01|NONE|MAIL|fully regular the|
+865|87|8|3|15|14806.20|0.00|0.06|R|F|1993-07-05|1993-06-25|1993-07-26|NONE|SHIP| deposits sleep quickl|
+865|169|4|4|34|36351.44|0.09|0.06|A|F|1993-05-09|1993-07-28|1993-05-18|DELIVER IN PERSON|REG AIR|furiously fluffily unusual account|
+866|136|7|1|5|5180.65|0.08|0.00|R|F|1993-01-22|1993-01-14|1993-02-07|TAKE BACK RETURN|AIR|tegrate fluffily. carefully f|
+867|139|10|1|7|7273.91|0.04|0.07|A|F|1994-02-19|1993-12-25|1994-02-25|DELIVER IN PERSON|TRUCK|pendencies-- slyly unusual packages hagg|
+868|168|9|1|8|8545.28|0.06|0.03|R|F|1992-10-07|1992-08-01|1992-10-16|NONE|MAIL|l deposits. blithely regular pint|
+868|29|8|2|13|12077.26|0.05|0.07|R|F|1992-07-25|1992-08-26|1992-08-04|NONE|AIR|gged instructi|
+868|68|5|3|19|18393.14|0.09|0.06|R|F|1992-06-22|1992-08-27|1992-07-04|COLLECT COD|SHIP|lyly ironic platelets wake. rut|
+868|122|1|4|43|43951.16|0.02|0.04|A|F|1992-07-02|1992-07-22|1992-07-21|COLLECT COD|SHIP|kly silent deposits wake dar|
+868|25|8|5|27|24975.54|0.04|0.01|R|F|1992-08-01|1992-08-25|1992-08-12|TAKE BACK RETURN|RAIL|oss the fluffily unusual pinto |
+868|125|6|6|19|19477.28|0.02|0.05|R|F|1992-09-20|1992-07-18|1992-10-04|NONE|FOB|ely even deposits lose blithe|
+869|63|2|1|27|26002.62|0.07|0.07|N|O|1997-01-30|1997-02-17|1997-02-26|TAKE BACK RETURN|TRUCK|uffily even excuses? slyly even deposits |
+869|47|4|2|36|34093.44|0.04|0.01|N|O|1997-05-03|1997-03-17|1997-05-24|NONE|RAIL|ong the furiously bold instructi|
+870|50|9|1|36|34201.80|0.04|0.07|A|F|1993-10-18|1993-09-16|1993-11-15|COLLECT COD|MAIL|fily. furiously final accounts are |
+870|186|7|2|5|5430.90|0.06|0.05|A|F|1993-08-13|1993-09-11|1993-08-24|COLLECT COD|FOB|e slyly excuses. ironi|
+871|97|8|1|48|47860.32|0.10|0.03|N|O|1996-02-25|1996-02-09|1996-03-18|NONE|AIR|coys dazzle slyly slow notornis. f|
+871|55|10|2|47|44887.35|0.07|0.03|N|O|1995-12-25|1996-02-01|1996-01-24|TAKE BACK RETURN|RAIL|ss, final dep|
+871|108|5|3|13|13105.30|0.09|0.01|N|O|1996-01-25|1996-01-24|1996-02-03|NONE|REG AIR| haggle furiou|
+871|190|1|4|29|31615.51|0.06|0.07|N|O|1995-11-16|1996-01-27|1995-12-16|DELIVER IN PERSON|RAIL|ests are carefu|
+871|128|7|5|8|8224.96|0.00|0.01|N|O|1995-11-25|1996-01-12|1995-12-12|DELIVER IN PERSON|AIR|lar ideas-- slyly even accou|
+871|143|2|6|26|27121.64|0.00|0.06|N|O|1996-02-07|1996-01-05|1996-02-25|COLLECT COD|AIR|symptotes use quickly near the |
+871|174|3|7|4|4296.68|0.00|0.07|N|O|1996-03-09|1996-01-20|1996-03-26|COLLECT COD|FOB|l, regular dependencies w|
+896|39|10|1|47|44134.41|0.07|0.08|R|F|1993-05-28|1993-05-15|1993-06-15|DELIVER IN PERSON|TRUCK|ly even pinto beans integrate. b|
+896|198|2|2|10|10981.90|0.03|0.07|A|F|1993-07-07|1993-06-03|1993-07-24|COLLECT COD|SHIP| quickly even theodolites. carefully regu|
+896|2|9|3|7|6314.00|0.09|0.02|A|F|1993-05-02|1993-05-24|1993-05-31|DELIVER IN PERSON|MAIL| requests |
+896|152|3|4|11|11573.65|0.08|0.04|A|F|1993-05-19|1993-05-22|1993-06-08|COLLECT COD|MAIL|the multipliers sleep|
+896|188|9|5|34|36998.12|0.00|0.05|R|F|1993-05-21|1993-06-01|1993-05-23|NONE|TRUCK|ular, close requests cajo|
+896|177|6|6|44|47395.48|0.09|0.08|R|F|1993-05-19|1993-04-14|1993-06-02|DELIVER IN PERSON|FOB|lar, pending packages. deposits are q|
+896|109|2|7|11|11100.10|0.01|0.07|A|F|1993-05-01|1993-04-09|1993-05-06|TAKE BACK RETURN|FOB|rding to the pinto beans wa|
+897|91|4|1|15|14866.35|0.07|0.04|R|F|1995-05-25|1995-05-09|1995-06-07|COLLECT COD|REG AIR|r ideas. slyly spec|
+897|184|5|2|26|28188.68|0.05|0.08|N|O|1995-07-01|1995-06-10|1995-07-14|COLLECT COD|MAIL|tions sleep according to the special|
+897|126|1|3|13|13339.56|0.07|0.00|A|F|1995-03-30|1995-05-17|1995-04-21|TAKE BACK RETURN|MAIL|bold accounts mold carefully! braids|
+897|102|7|4|2|2004.20|0.08|0.08|R|F|1995-05-22|1995-05-07|1995-06-16|COLLECT COD|RAIL|into beans. slyly special fox|
+898|161|2|1|9|9550.44|0.07|0.08|A|F|1993-07-04|1993-07-09|1993-07-25|NONE|AIR|e slyly across the blithe|
+898|179|7|2|37|39929.29|0.03|0.05|A|F|1993-08-17|1993-08-04|1993-09-01|DELIVER IN PERSON|REG AIR|packages sleep furiously|
+898|49|8|3|11|10439.44|0.01|0.00|A|F|1993-09-13|1993-08-31|1993-09-25|TAKE BACK RETURN|MAIL|etly bold accounts |
+898|193|6|4|36|39354.84|0.04|0.07|R|F|1993-08-04|1993-07-25|1993-08-23|DELIVER IN PERSON|REG AIR| after the carefully |
+899|61|10|1|18|17299.08|0.04|0.05|N|O|1998-08-06|1998-05-09|1998-09-05|DELIVER IN PERSON|AIR|re daring, pending deposits. blit|
+899|47|4|2|25|23676.00|0.00|0.07|N|O|1998-07-21|1998-05-12|1998-08-16|NONE|REG AIR|rly final sentiments. bold pinto beans |
+899|85|6|3|4|3940.32|0.09|0.05|N|O|1998-06-02|1998-06-28|1998-06-14|TAKE BACK RETURN|REG AIR|ter the carefully regular deposits are agai|
+899|180|9|4|14|15122.52|0.05|0.03|N|O|1998-05-21|1998-05-28|1998-06-03|TAKE BACK RETURN|FOB|ades impress carefully|
+899|71|10|5|4|3884.28|0.06|0.02|N|O|1998-04-11|1998-05-14|1998-04-27|NONE|TRUCK|ges. blithe, ironic waters cajole care|
+899|120|4|6|47|47945.64|0.00|0.04|N|O|1998-04-14|1998-05-30|1998-05-13|DELIVER IN PERSON|TRUCK|furiously final foxes after the s|
+899|14|1|7|11|10054.11|0.02|0.08|N|O|1998-06-03|1998-06-15|1998-06-20|COLLECT COD|REG AIR|t the ironic|
+900|199|1|1|44|48364.36|0.01|0.06|R|F|1994-12-15|1994-12-03|1994-12-27|COLLECT COD|MAIL| detect quick|
+900|115|6|2|48|48725.28|0.08|0.04|A|F|1994-12-22|1994-11-08|1995-01-19|COLLECT COD|TRUCK|cial pinto beans nag |
+900|75|6|3|24|23401.68|0.03|0.00|R|F|1994-10-21|1994-12-25|1994-10-22|TAKE BACK RETURN|SHIP|-ray furiously un|
+901|22|7|1|36|33192.72|0.01|0.01|N|O|1998-08-11|1998-10-09|1998-08-27|DELIVER IN PERSON|REG AIR|. accounts are care|
+901|46|7|2|2|1892.08|0.09|0.02|N|O|1998-10-25|1998-09-27|1998-11-01|DELIVER IN PERSON|AIR|d foxes use slyly|
+901|43|10|3|37|34892.48|0.04|0.08|N|O|1998-11-01|1998-09-13|1998-11-05|NONE|AIR|ickly final deposits |
+901|18|9|4|11|10098.11|0.00|0.06|N|O|1998-11-13|1998-10-19|1998-11-14|TAKE BACK RETURN|TRUCK|ourts among the quickly expre|
+902|111|2|1|3|3033.33|0.06|0.00|R|F|1994-10-01|1994-10-25|1994-10-28|COLLECT COD|MAIL|into beans thrash blithely about the flu|
+902|118|2|2|8|8144.88|0.06|0.07|R|F|1994-10-25|1994-09-20|1994-11-07|COLLECT COD|RAIL| orbits al|
+902|165|2|3|24|25563.84|0.02|0.05|R|F|1994-11-08|1994-10-12|1994-11-26|NONE|FOB|. blithely even accounts poach furiously i|
+903|65|10|1|27|26056.62|0.04|0.03|N|O|1995-09-18|1995-09-20|1995-10-02|TAKE BACK RETURN|SHIP|lly pending foxes. furiously|
+903|9|2|2|35|31815.00|0.06|0.05|N|O|1995-09-18|1995-08-21|1995-10-12|TAKE BACK RETURN|TRUCK|rets wake fin|
+903|9|2|3|33|29997.00|0.02|0.03|N|O|1995-09-24|1995-09-01|1995-10-12|COLLECT COD|MAIL|ely ironic packages wake blithely|
+903|56|1|4|9|8604.45|0.09|0.00|N|O|1995-10-06|1995-09-14|1995-10-24|NONE|TRUCK|he slyly ev|
+903|42|3|5|1|942.04|0.04|0.00|N|O|1995-10-22|1995-09-13|1995-11-03|NONE|AIR|y final platelets sublate among the |
+903|168|9|6|13|13886.08|0.07|0.02|N|O|1995-09-11|1995-10-04|1995-10-03|COLLECT COD|SHIP|sleep along the final|
+928|169|10|1|29|31005.64|0.07|0.02|R|F|1995-05-17|1995-05-12|1995-05-21|NONE|REG AIR|ly alongside of the s|
+928|48|7|2|24|22752.96|0.05|0.08|A|F|1995-04-06|1995-05-08|1995-04-24|DELIVER IN PERSON|AIR|s the furiously regular warthogs im|
+928|152|10|3|46|48398.90|0.08|0.00|A|F|1995-05-09|1995-04-09|1995-06-01|DELIVER IN PERSON|REG AIR| beans sleep against the carefully ir|
+928|52|4|4|43|40938.15|0.10|0.05|A|F|1995-04-14|1995-04-21|1995-05-09|NONE|REG AIR|blithely. express, silent requests doze at|
+928|12|3|5|38|34656.38|0.02|0.08|N|F|1995-06-08|1995-04-15|1995-06-30|TAKE BACK RETURN|SHIP|xpress grouc|
+928|55|6|6|50|47752.50|0.05|0.00|N|F|1995-06-07|1995-04-15|1995-07-01|DELIVER IN PERSON|TRUCK| slyly slyly special request|
+928|11|5|7|11|10021.11|0.00|0.01|A|F|1995-04-29|1995-04-16|1995-04-30|NONE|AIR|longside of|
+929|129|8|1|45|46310.40|0.09|0.01|R|F|1993-01-24|1992-12-06|1993-02-16|DELIVER IN PERSON|REG AIR|ges haggle careful|
+929|175|5|2|44|47307.48|0.02|0.00|A|F|1992-10-09|1992-11-20|1992-10-22|DELIVER IN PERSON|SHIP|s. excuses cajole. carefully regu|
+929|74|5|3|14|13636.98|0.06|0.07|A|F|1992-10-21|1992-11-17|1992-11-15|NONE|FOB|gainst the|
+929|102|5|4|7|7014.70|0.06|0.01|A|F|1992-12-24|1992-12-19|1993-01-08|TAKE BACK RETURN|TRUCK|ithely. slyly c|
+930|45|4|1|36|34021.44|0.10|0.04|R|F|1994-12-21|1995-02-20|1994-12-24|COLLECT COD|RAIL|quickly regular pinto beans sle|
+930|18|8|2|47|43146.47|0.08|0.00|A|F|1995-03-20|1995-02-04|1995-04-04|DELIVER IN PERSON|AIR|ackages. fluffily e|
+930|65|10|3|10|9650.60|0.07|0.08|A|F|1994-12-18|1995-01-27|1995-01-16|COLLECT COD|AIR|ckly regular requests: regular instructions|
+930|100|2|4|21|21002.10|0.06|0.02|A|F|1995-02-16|1995-03-03|1995-03-13|DELIVER IN PERSON|SHIP|foxes. regular deposits integrate carefu|
+930|164|9|5|50|53208.00|0.03|0.06|A|F|1995-04-03|1995-01-29|1995-04-22|COLLECT COD|MAIL| excuses among the furiously express ideas |
+930|145|4|6|10|10451.40|0.00|0.04|A|F|1995-02-09|1995-02-17|1995-02-16|NONE|SHIP|blithely bold i|
+930|167|4|7|30|32014.80|0.07|0.08|R|F|1995-01-20|1995-02-28|1995-02-04|TAKE BACK RETURN|RAIL|g accounts sleep along the platelets.|
+931|40|1|1|18|16920.72|0.00|0.05|A|F|1993-04-04|1993-01-11|1993-04-13|NONE|RAIL|slyly ironic re|
+931|17|7|2|10|9170.10|0.05|0.07|A|F|1993-03-01|1993-01-09|1993-03-07|TAKE BACK RETURN|SHIP|ajole quickly. slyly sil|
+931|147|6|3|48|50262.72|0.01|0.08|A|F|1993-02-03|1993-03-02|1993-02-09|TAKE BACK RETURN|FOB|ep alongside of the fluffy |
+931|82|3|4|38|37319.04|0.08|0.08|A|F|1993-03-06|1993-02-24|1993-03-27|DELIVER IN PERSON|RAIL|usly final packages integrate carefully|
+932|44|1|1|41|38705.64|0.01|0.05|N|O|1997-06-05|1997-07-22|1997-06-26|COLLECT COD|RAIL|foxes. ironic pl|
+933|49|8|1|23|21827.92|0.02|0.04|R|F|1992-08-13|1992-09-18|1992-08-25|DELIVER IN PERSON|MAIL| the furiously bold dinos. sly|
+933|13|4|2|27|24651.27|0.02|0.01|R|F|1992-10-03|1992-10-02|1992-10-26|DELIVER IN PERSON|RAIL|ests. express|
+933|100|2|3|26|26002.60|0.05|0.00|A|F|1992-11-09|1992-11-03|1992-11-16|DELIVER IN PERSON|AIR| the deposits affix slyly after t|
+934|118|5|1|18|18325.98|0.07|0.01|N|O|1996-09-10|1996-09-20|1996-09-25|COLLECT COD|RAIL|y unusual requests dazzle above t|
+935|28|3|1|23|21344.46|0.05|0.00|N|O|1997-11-11|1997-11-22|1997-11-29|COLLECT COD|REG AIR|ular accounts about|
+935|65|10|2|23|22196.38|0.02|0.01|N|O|1998-01-11|1997-11-25|1998-02-05|COLLECT COD|TRUCK|hes haggle furiously dolphins. qu|
+935|135|1|3|36|37264.68|0.06|0.00|N|O|1997-11-05|1997-12-05|1997-11-25|TAKE BACK RETURN|AIR|leep about the exp|
+935|58|3|4|13|12454.65|0.08|0.04|N|O|1998-01-13|1997-11-30|1998-02-08|DELIVER IN PERSON|TRUCK|ld platelet|
+935|13|7|5|8|7304.08|0.02|0.05|N|O|1998-01-12|1997-11-02|1998-02-05|NONE|TRUCK|cept the quickly regular p|
+935|59|1|6|1|959.05|0.01|0.08|N|O|1997-12-14|1997-11-22|1998-01-08|DELIVER IN PERSON|TRUCK| instructions. ironic acc|
+960|107|10|1|1|1007.10|0.07|0.00|A|F|1994-12-24|1994-10-26|1995-01-20|DELIVER IN PERSON|AIR|y ironic packages. quickly even |
+960|117|7|2|25|25427.75|0.06|0.08|R|F|1994-12-01|1994-10-29|1994-12-27|DELIVER IN PERSON|RAIL|ts. fluffily regular requests |
+960|175|3|3|32|34405.44|0.01|0.08|R|F|1995-01-19|1994-12-17|1995-02-04|DELIVER IN PERSON|FOB|around the blithe, even pl|
+961|118|5|1|7|7126.77|0.10|0.00|N|O|1995-07-23|1995-07-20|1995-08-11|TAKE BACK RETURN|RAIL|usual dolphins. ironic pearls sleep blit|
+961|91|2|2|18|17839.62|0.09|0.05|N|O|1995-07-01|1995-08-14|1995-07-04|DELIVER IN PERSON|AIR|rmanent foxes haggle speci|
+961|97|8|3|42|41877.78|0.06|0.01|N|O|1995-08-24|1995-08-21|1995-09-10|TAKE BACK RETURN|SHIP|ests do cajole blithely. furiously bo|
+961|34|10|4|29|27086.87|0.00|0.07|N|F|1995-06-10|1995-08-20|1995-06-26|TAKE BACK RETURN|TRUCK|l accounts use blithely against the|
+961|26|7|5|38|35188.76|0.03|0.05|N|O|1995-08-21|1995-07-19|1995-08-27|NONE|RAIL|he blithely special requests. furiousl|
+961|197|8|6|30|32915.70|0.09|0.03|N|O|1995-07-06|1995-07-20|1995-07-26|DELIVER IN PERSON|MAIL|warhorses slee|
+962|57|8|1|36|34453.80|0.01|0.03|R|F|1994-08-09|1994-07-10|1994-09-02|COLLECT COD|TRUCK|al foxes. iron|
+962|36|2|2|27|25272.81|0.09|0.02|A|F|1994-05-11|1994-07-10|1994-06-03|TAKE BACK RETURN|SHIP|y slyly express deposits. final i|
+962|80|1|3|3|2940.24|0.07|0.08|A|F|1994-05-08|1994-07-06|1994-06-02|DELIVER IN PERSON|FOB|ag furiously. even pa|
+962|57|5|4|20|19141.00|0.04|0.02|R|F|1994-08-26|1994-06-27|1994-09-11|DELIVER IN PERSON|SHIP| deposits use fluffily according to |
+962|152|7|5|12|12625.80|0.02|0.00|A|F|1994-06-09|1994-06-07|1994-06-11|COLLECT COD|TRUCK|across the furiously regular escapades daz|
+962|188|9|6|5|5440.90|0.02|0.05|A|F|1994-08-29|1994-07-15|1994-09-19|COLLECT COD|TRUCK|efully bold packages run slyly caref|
+963|194|8|1|7|7659.33|0.01|0.00|R|F|1994-09-12|1994-07-18|1994-09-17|DELIVER IN PERSON|REG AIR|s. slyly regular depe|
+963|98|10|2|48|47908.32|0.10|0.06|R|F|1994-08-25|1994-08-12|1994-09-21|DELIVER IN PERSON|RAIL|ages. quickly express deposits cajole pe|
+964|199|10|1|39|42868.41|0.04|0.01|N|O|1995-06-21|1995-07-24|1995-06-24|NONE|AIR|se furiously regular instructions. blith|
+964|113|4|2|1|1013.11|0.02|0.05|N|O|1995-08-20|1995-07-29|1995-09-10|DELIVER IN PERSON|REG AIR|unts. quickly even platelets s|
+964|57|5|3|49|46895.45|0.01|0.03|N|O|1995-09-06|1995-08-10|1995-10-05|NONE|MAIL|ounts. blithely regular packag|
+964|55|3|4|44|42022.20|0.05|0.02|N|O|1995-09-18|1995-08-02|1995-10-17|TAKE BACK RETURN|TRUCK|ronic deposit|
+965|108|1|1|20|20162.00|0.04|0.05|N|F|1995-06-16|1995-07-20|1995-07-06|COLLECT COD|MAIL|kly. carefully pending requ|
+965|18|5|2|23|21114.23|0.06|0.08|N|O|1995-07-12|1995-07-08|1995-08-11|COLLECT COD|MAIL|ld kindle carefully across th|
+966|180|8|1|19|20523.42|0.07|0.01|N|O|1998-05-26|1998-07-15|1998-05-29|COLLECT COD|FOB|efully final pinto beans. quickly |
+966|117|4|2|42|42718.62|0.02|0.06|N|O|1998-06-28|1998-06-20|1998-07-05|NONE|TRUCK|tions boost furiously car|
+966|22|1|3|42|38724.84|0.06|0.08|N|O|1998-06-15|1998-06-08|1998-07-05|NONE|RAIL|sly ironic asymptotes hagg|
+966|5|2|4|20|18100.00|0.04|0.01|N|O|1998-07-19|1998-07-15|1998-07-27|NONE|TRUCK|pecial ins|
+967|59|4|1|41|39321.05|0.05|0.05|R|F|1992-09-21|1992-08-15|1992-10-21|NONE|MAIL|ld foxes wake closely special|
+967|85|6|2|4|3940.32|0.01|0.02|A|F|1992-07-15|1992-07-27|1992-07-18|DELIVER IN PERSON|REG AIR|platelets hang carefully along |
+967|132|8|3|10|10321.30|0.00|0.02|A|F|1992-09-18|1992-08-06|1992-09-19|DELIVER IN PERSON|MAIL|old pinto beans alongside of the exp|
+967|148|7|4|49|51358.86|0.01|0.04|A|F|1992-09-28|1992-09-15|1992-10-14|NONE|SHIP|the slyly even ideas. carefully even|
+967|17|1|5|41|37597.41|0.08|0.04|A|F|1992-07-23|1992-08-07|1992-08-13|TAKE BACK RETURN|FOB|efully special ide|
+967|106|9|6|17|17103.70|0.05|0.06|A|F|1992-10-02|1992-08-19|1992-10-25|NONE|MAIL|y ironic foxes caj|
+967|161|8|7|18|19100.88|0.00|0.02|A|F|1992-10-06|1992-08-05|1992-10-15|DELIVER IN PERSON|RAIL|ngage blith|
+992|60|2|1|14|13440.84|0.10|0.03|N|O|1998-01-29|1997-12-29|1998-02-18|TAKE BACK RETURN|MAIL|the unusual, even dependencies affix fluff|
+992|38|9|2|34|31893.02|0.02|0.00|N|O|1997-11-29|1998-01-21|1997-11-30|NONE|RAIL|s use silently. blithely regular ideas b|
+992|105|6|3|30|30153.00|0.10|0.00|N|O|1997-12-15|1998-02-02|1998-01-12|NONE|SHIP|nic instructions n|
+992|48|5|4|21|19908.84|0.06|0.06|N|O|1997-11-13|1997-12-28|1997-12-10|NONE|TRUCK|fily. quickly special deposit|
+992|92|4|5|7|6944.63|0.09|0.05|N|O|1997-11-30|1997-12-24|1997-12-16|DELIVER IN PERSON|TRUCK|ideas haggle. special theodolit|
+992|75|3|6|41|39977.87|0.10|0.01|N|O|1997-11-14|1998-02-04|1997-11-23|TAKE BACK RETURN|AIR|eodolites cajole across the accounts.|
+993|175|5|1|33|35480.61|0.01|0.05|N|O|1996-01-03|1995-11-28|1996-01-23|DELIVER IN PERSON|AIR| the deposits affix agains|
+993|3|6|2|28|25284.00|0.06|0.08|N|O|1995-10-24|1995-11-20|1995-11-06|DELIVER IN PERSON|RAIL|lites. even theodolite|
+993|40|1|3|10|9400.40|0.03|0.08|N|O|1995-12-17|1995-11-13|1995-12-20|NONE|RAIL|encies wake fur|
+993|191|4|4|40|43647.60|0.01|0.01|N|O|1995-11-16|1995-11-01|1995-12-05|TAKE BACK RETURN|RAIL|gle above the furiously |
+993|146|7|5|33|34522.62|0.09|0.08|N|O|1995-09-28|1995-10-24|1995-10-03|COLLECT COD|RAIL|fluffily. quiet excuses sleep furiously sly|
+993|137|3|6|35|36299.55|0.04|0.02|N|O|1995-10-26|1995-10-20|1995-11-05|DELIVER IN PERSON|FOB|es. ironic, ironic requests|
+993|5|2|7|15|13575.00|0.09|0.03|N|O|1995-09-27|1995-10-21|1995-10-17|DELIVER IN PERSON|MAIL|sits. pending pinto beans haggle? ca|
+994|65|6|1|4|3860.24|0.07|0.03|R|F|1994-07-05|1994-05-21|1994-07-20|COLLECT COD|SHIP|aggle carefully acc|
+994|10|3|2|11|10010.11|0.01|0.00|R|F|1994-05-03|1994-06-10|1994-05-22|NONE|AIR|ular accounts sleep |
+994|31|7|3|5|4655.15|0.08|0.08|A|F|1994-06-24|1994-06-14|1994-06-26|NONE|MAIL|ainst the pending requests. packages sl|
+994|131|7|4|25|25778.25|0.10|0.00|A|F|1994-06-03|1994-06-02|1994-06-06|COLLECT COD|RAIL|usual pinto beans.|
+995|173|1|1|15|16097.55|0.08|0.05|N|O|1995-06-30|1995-08-04|1995-07-27|NONE|REG AIR|uses. fluffily fina|
+995|129|4|2|28|28815.36|0.08|0.03|N|F|1995-06-12|1995-07-20|1995-06-19|DELIVER IN PERSON|SHIP|pades. quick, final frays use flu|
+995|166|3|3|45|47977.20|0.00|0.05|N|O|1995-08-02|1995-07-21|1995-08-03|DELIVER IN PERSON|SHIP|lar packages detect blithely above t|
+995|66|3|4|25|24151.50|0.01|0.08|N|O|1995-09-08|1995-08-05|1995-09-28|NONE|TRUCK|lyly even |
+995|24|5|5|18|16632.36|0.06|0.03|N|O|1995-07-03|1995-07-29|1995-07-22|TAKE BACK RETURN|AIR| even accounts unwind c|
+996|173|2|1|43|46146.31|0.03|0.06|N|O|1998-03-27|1998-03-25|1998-04-06|COLLECT COD|SHIP| the blithely ironic foxes. slyly silent d|
+997|163|4|1|11|11694.76|0.00|0.02|N|O|1997-06-16|1997-07-21|1997-07-14|DELIVER IN PERSON|TRUCK|p furiously according to t|
+997|48|9|2|17|16116.68|0.03|0.00|N|O|1997-07-28|1997-07-26|1997-08-20|DELIVER IN PERSON|SHIP|aggle quickly furiously|
+998|10|7|1|22|20020.22|0.04|0.05|A|F|1994-12-03|1995-02-17|1994-12-19|TAKE BACK RETURN|RAIL|lites. qui|
+998|181|2|2|7|7568.26|0.10|0.05|R|F|1995-03-24|1995-01-18|1995-04-03|NONE|MAIL|nic deposits. even asym|
+998|142|9|3|30|31264.20|0.05|0.07|A|F|1994-12-02|1995-01-23|1994-12-23|NONE|SHIP|lyly idle Tir|
+998|11|8|4|6|5466.06|0.09|0.05|R|F|1995-03-20|1994-12-27|1995-04-13|DELIVER IN PERSON|MAIL|refully accounts. carefully express ac|
+998|73|2|5|1|973.07|0.04|0.00|R|F|1995-01-05|1995-01-06|1995-01-13|NONE|SHIP|es sleep. regular dependencies use bl|
+999|61|6|1|34|32676.04|0.00|0.08|R|F|1993-10-30|1993-10-17|1993-10-31|TAKE BACK RETURN|SHIP|its. daringly final instruc|
+999|199|1|2|41|45066.79|0.08|0.01|A|F|1993-10-16|1993-12-04|1993-11-03|DELIVER IN PERSON|REG AIR|us depths. carefully ironic instruc|
+999|118|5|3|15|15271.65|0.07|0.06|A|F|1993-12-12|1993-10-18|1994-01-08|COLLECT COD|REG AIR|y ironic requests. carefully regu|
+999|3|4|4|10|9030.00|0.05|0.02|A|F|1993-11-23|1993-12-02|1993-11-29|NONE|MAIL|efully pending|
+999|19|10|5|3|2757.03|0.03|0.00|R|F|1993-09-17|1993-10-22|1993-10-13|NONE|FOB|nic, pending ideas. bl|
+999|181|2|6|37|40003.66|0.00|0.04|R|F|1994-01-03|1993-10-28|1994-01-12|DELIVER IN PERSON|TRUCK|ckly slyly unusual packages: packages hagg|
+1024|199|2|1|49|53860.31|0.03|0.05|N|O|1998-03-06|1998-01-26|1998-03-29|TAKE BACK RETURN|FOB|ts. asymptotes nag fur|
+1024|126|5|2|34|34888.08|0.00|0.01|N|O|1998-01-06|1998-02-05|1998-01-26|COLLECT COD|SHIP|des the slyly even|
+1024|44|3|3|28|26433.12|0.04|0.01|N|O|1998-03-04|1998-03-12|1998-03-15|TAKE BACK RETURN|TRUCK|e blithely regular pi|
+1024|184|5|4|13|14094.34|0.02|0.04|N|O|1998-04-11|1998-02-26|1998-04-18|NONE|FOB|e slyly around the slyly special instructi|
+1024|21|4|5|49|45129.98|0.02|0.04|N|O|1998-02-27|1998-03-10|1998-03-27|COLLECT COD|FOB| carefully bold |
+1025|150|1|1|36|37805.40|0.03|0.04|A|F|1995-05-15|1995-07-05|1995-06-10|COLLECT COD|FOB|e unusual, regular instr|
+1025|69|10|2|23|22288.38|0.08|0.03|N|F|1995-06-02|1995-07-29|1995-06-23|COLLECT COD|RAIL| regular platelets nag carefu|
+1025|23|2|3|25|23075.50|0.06|0.05|R|F|1995-05-29|1995-06-21|1995-06-13|DELIVER IN PERSON|REG AIR|xpress foxes. furiousl|
+1026|38|4|1|36|33769.08|0.10|0.02|N|O|1997-06-14|1997-07-20|1997-06-23|NONE|SHIP|st the ide|
+1026|37|8|2|6|5622.18|0.10|0.08|N|O|1997-07-07|1997-08-16|1997-07-14|TAKE BACK RETURN|TRUCK|to beans. special, regular packages hagg|
+1027|156|1|1|43|45414.45|0.07|0.08|R|F|1992-06-17|1992-08-28|1992-07-10|DELIVER IN PERSON|MAIL|oxes. carefully regular deposits|
+1027|113|10|2|20|20262.20|0.01|0.02|A|F|1992-06-08|1992-08-29|1992-06-14|NONE|TRUCK|ar excuses eat f|
+1027|126|9|3|2|2052.24|0.01|0.02|R|F|1992-08-28|1992-07-09|1992-09-10|NONE|FOB|s. quickly unusual waters inside |
+1027|100|4|4|13|13001.30|0.08|0.01|R|F|1992-08-22|1992-07-10|1992-09-12|DELIVER IN PERSON|RAIL|ily ironic ideas use|
+1027|136|2|5|22|22794.86|0.02|0.00|A|F|1992-09-03|1992-08-14|1992-10-01|DELIVER IN PERSON|FOB|the furiously express ex|
+1027|105|8|6|10|10051.00|0.06|0.08|R|F|1992-08-28|1992-08-06|1992-09-03|COLLECT COD|REG AIR|ilent, express foxes near the blithely sp|
+1028|128|3|1|2|2056.24|0.09|0.03|A|F|1994-01-10|1994-03-22|1994-01-26|COLLECT COD|FOB|s alongside of the regular asymptotes sleep|
+1028|112|9|2|39|39472.29|0.06|0.05|R|F|1994-02-18|1994-03-22|1994-03-06|TAKE BACK RETURN|MAIL| final dependencies affix a|
+1028|100|3|3|8|8000.80|0.03|0.07|A|F|1994-02-14|1994-03-28|1994-02-22|NONE|AIR|e carefully final packages. furiously fi|
+1028|32|8|4|26|24232.78|0.07|0.02|A|F|1994-03-18|1994-02-08|1994-03-19|TAKE BACK RETURN|RAIL|ronic platelets. carefully f|
+1028|29|2|5|27|25083.54|0.00|0.04|A|F|1994-04-03|1994-02-07|1994-04-26|NONE|REG AIR|ial accounts nag. slyly|
+1028|26|1|6|39|36114.78|0.03|0.02|A|F|1994-02-27|1994-02-16|1994-03-02|DELIVER IN PERSON|AIR|c theodoli|
+1028|31|2|7|22|20482.66|0.03|0.00|R|F|1994-04-24|1994-02-27|1994-05-08|NONE|REG AIR| Tiresias alongside of the carefully spec|
+1029|137|3|1|45|46670.85|0.05|0.07|R|F|1994-07-21|1994-08-30|1994-07-29|TAKE BACK RETURN|FOB|sits boost blithely|
+1030|65|10|1|17|16406.02|0.06|0.06|R|F|1994-10-13|1994-08-01|1994-11-10|DELIVER IN PERSON|RAIL|ly. carefully even packages dazz|
+1031|46|7|1|15|14190.60|0.10|0.08|A|F|1994-11-07|1994-10-29|1994-11-09|TAKE BACK RETURN|FOB|about the carefully bold a|
+1031|165|4|2|28|29824.48|0.05|0.01|A|F|1994-12-10|1994-10-29|1994-12-18|COLLECT COD|FOB|ly ironic accounts across the q|
+1031|187|8|3|27|29353.86|0.07|0.02|R|F|1994-09-20|1994-10-18|1994-10-10|DELIVER IN PERSON|SHIP|gular deposits cajole. blithely unus|
+1031|88|9|4|7|6916.56|0.03|0.03|R|F|1994-12-07|1994-11-11|1994-12-30|COLLECT COD|FOB|r instructions. car|
+1031|191|5|5|44|48012.36|0.01|0.07|R|F|1994-11-20|1994-11-24|1994-12-11|NONE|AIR|re slyly above the furio|
+1056|121|6|1|37|37781.44|0.04|0.06|R|F|1995-02-18|1995-04-01|1995-03-20|NONE|TRUCK| special packages. qui|
+1057|193|5|1|29|31702.51|0.10|0.01|A|F|1992-05-05|1992-05-05|1992-06-03|TAKE BACK RETURN|SHIP|es wake according to the q|
+1057|169|8|2|11|11760.76|0.00|0.02|R|F|1992-03-31|1992-04-18|1992-04-18|COLLECT COD|AIR|yly final theodolites. furi|
+1057|85|6|3|21|20686.68|0.03|0.04|A|F|1992-02-28|1992-05-01|1992-03-10|NONE|REG AIR|ar orbits boost bli|
+1057|182|3|4|20|21643.60|0.06|0.03|R|F|1992-03-02|1992-05-19|1992-03-13|DELIVER IN PERSON|TRUCK|s wake bol|
+1057|97|1|5|7|6979.63|0.06|0.05|R|F|1992-06-05|1992-04-30|1992-06-20|NONE|TRUCK|y slyly express theodolites. slyly bo|
+1057|52|7|6|19|18088.95|0.04|0.07|A|F|1992-05-31|1992-05-09|1992-06-02|DELIVER IN PERSON|FOB|r-- packages haggle alon|
+1058|140|6|1|24|24963.36|0.08|0.04|A|F|1993-07-09|1993-05-28|1993-07-22|DELIVER IN PERSON|TRUCK|fully ironic accounts. express accou|
+1058|89|10|2|5|4945.40|0.04|0.07|R|F|1993-05-11|1993-05-29|1993-05-27|COLLECT COD|TRUCK|refully even requests boost along|
+1058|90|1|3|44|43563.96|0.10|0.01|R|F|1993-06-26|1993-06-21|1993-07-20|COLLECT COD|TRUCK|uriously f|
+1058|5|2|4|25|22625.00|0.09|0.01|A|F|1993-05-27|1993-06-10|1993-06-20|TAKE BACK RETURN|MAIL| the final requests believe carefully |
+1059|178|9|1|16|17250.72|0.07|0.02|A|F|1994-04-24|1994-03-31|1994-04-28|DELIVER IN PERSON|SHIP|y ironic pinto |
+1059|29|2|2|7|6503.14|0.07|0.06|R|F|1994-03-30|1994-04-01|1994-04-24|DELIVER IN PERSON|MAIL|the furiously silent excuses are e|
+1059|88|9|3|45|44463.60|0.00|0.02|R|F|1994-06-10|1994-05-08|1994-06-21|COLLECT COD|RAIL|riously even theodolites. slyly regula|
+1059|110|7|4|26|26262.86|0.09|0.01|A|F|1994-03-17|1994-04-18|1994-03-26|DELIVER IN PERSON|TRUCK|ar pinto beans at the furiously |
+1059|139|5|5|37|38447.81|0.09|0.04|R|F|1994-03-31|1994-05-08|1994-04-06|COLLECT COD|RAIL| packages lose in place of the slyly unusu|
+1059|190|1|6|50|54509.50|0.00|0.03|A|F|1994-06-15|1994-05-11|1994-06-29|NONE|MAIL|s impress furiously about|
+1059|123|4|7|13|13300.56|0.01|0.03|R|F|1994-06-12|1994-05-11|1994-07-02|COLLECT COD|TRUCK|usly regular theodo|
+1060|196|10|1|8|8769.52|0.07|0.04|R|F|1993-05-21|1993-05-06|1993-06-10|DELIVER IN PERSON|FOB|iously. furiously regular in|
+1060|8|5|2|26|23608.00|0.06|0.08|R|F|1993-04-12|1993-04-01|1993-04-20|DELIVER IN PERSON|TRUCK|counts; even deposits are carefull|
+1060|164|3|3|11|11705.76|0.01|0.07|A|F|1993-05-13|1993-05-08|1993-05-17|TAKE BACK RETURN|MAIL|e regular deposits: re|
+1060|110|7|4|16|16161.76|0.03|0.06|A|F|1993-06-15|1993-04-18|1993-07-05|COLLECT COD|SHIP|ccounts. foxes maintain care|
+1060|53|8|5|1|953.05|0.04|0.06|A|F|1993-06-19|1993-05-10|1993-06-21|COLLECT COD|RAIL|posits detect carefully abo|
+1060|72|2|6|26|25273.82|0.01|0.03|A|F|1993-02-28|1993-04-01|1993-03-09|TAKE BACK RETURN|FOB|quickly abo|
+1060|121|10|7|36|36760.32|0.09|0.01|R|F|1993-03-14|1993-03-24|1993-04-02|TAKE BACK RETURN|FOB|r the quickly|
+1061|151|6|1|7|7358.05|0.04|0.04|N|O|1998-08-09|1998-08-12|1998-08-16|COLLECT COD|FOB|es are slyly expr|
+1061|119|3|2|2|2038.22|0.06|0.02|N|O|1998-08-15|1998-08-05|1998-08-24|COLLECT COD|MAIL|. regular accounts impre|
+1061|111|8|3|26|26288.86|0.08|0.02|N|O|1998-06-18|1998-07-25|1998-06-24|TAKE BACK RETURN|AIR|ave to slee|
+1061|136|7|4|41|42481.33|0.00|0.05|N|O|1998-06-29|1998-07-02|1998-07-27|NONE|MAIL|s are. ironic theodolites cajole. dep|
+1061|131|2|5|50|51556.50|0.04|0.08|N|O|1998-05-25|1998-07-22|1998-06-22|COLLECT COD|AIR|nding excuses are around the e|
+1061|144|1|6|35|36544.90|0.09|0.05|N|O|1998-07-05|1998-07-07|1998-07-30|TAKE BACK RETURN|MAIL|ending requests nag careful|
+1062|137|8|1|38|39410.94|0.00|0.01|N|O|1997-01-27|1997-03-07|1997-02-16|DELIVER IN PERSON|TRUCK|deas. pending acc|
+1063|96|9|1|42|41835.78|0.03|0.02|A|F|1994-07-10|1994-05-25|1994-07-26|NONE|RAIL|tructions about the blithely ex|
+1088|107|8|1|30|30213.00|0.07|0.03|A|F|1992-05-22|1992-06-25|1992-06-11|TAKE BACK RETURN|SHIP|long the packages snooze careful|
+1088|37|3|2|11|10307.33|0.06|0.00|A|F|1992-08-30|1992-07-25|1992-09-10|TAKE BACK RETURN|AIR|inal requests. fluffily express theod|
+1088|181|2|3|5|5405.90|0.03|0.07|A|F|1992-07-01|1992-07-25|1992-07-02|NONE|AIR|refully ironic packages. r|
+1088|124|5|4|3|3072.36|0.09|0.03|A|F|1992-06-15|1992-08-02|1992-06-18|DELIVER IN PERSON|MAIL|pecial theodolites |
+1089|151|3|1|47|49404.05|0.05|0.06|N|O|1996-06-26|1996-06-25|1996-07-11|NONE|TRUCK|aggle furiously among the bravely eve|
+1089|50|7|2|35|33251.75|0.03|0.00|N|O|1996-08-14|1996-07-10|1996-08-26|NONE|TRUCK|ly express deposits haggle|
+1089|26|7|3|23|21298.46|0.10|0.05|N|O|1996-06-24|1996-07-25|1996-07-20|DELIVER IN PERSON|AIR|g dolphins. deposits integrate. s|
+1089|141|10|4|1|1041.14|0.01|0.03|N|O|1996-07-08|1996-07-07|1996-07-17|COLLECT COD|RAIL|n courts among the caref|
+1090|22|3|1|5|4610.10|0.02|0.05|N|O|1998-02-19|1997-12-25|1998-02-24|DELIVER IN PERSON|AIR|s above the |
+1090|113|10|2|28|28367.08|0.08|0.08|N|O|1998-02-20|1998-01-03|1998-03-19|NONE|FOB|s cajole above the regular|
+1091|38|9|1|40|37521.20|0.10|0.06|N|O|1996-12-17|1996-10-14|1996-12-24|TAKE BACK RETURN|REG AIR|platelets. regular packag|
+1092|184|5|1|48|52040.64|0.04|0.04|N|O|1995-06-25|1995-04-06|1995-07-18|DELIVER IN PERSON|AIR|unusual accounts. fluffi|
+1092|153|5|2|1|1053.15|0.01|0.06|A|F|1995-03-10|1995-04-21|1995-04-06|COLLECT COD|RAIL|lent, pending requests-- requests nag accor|
+1092|161|8|3|28|29712.48|0.05|0.08|R|F|1995-04-08|1995-05-01|1995-05-02|DELIVER IN PERSON|FOB|affix carefully. u|
+1092|86|7|4|2|1972.16|0.05|0.07|R|F|1995-04-09|1995-05-12|1995-05-03|TAKE BACK RETURN|TRUCK|ans. slyly eve|
+1093|87|8|1|7|6909.56|0.04|0.02|N|O|1997-11-24|1997-09-23|1997-11-25|TAKE BACK RETURN|SHIP|bold deposits. blithely ironic depos|
+1093|177|5|2|37|39855.29|0.08|0.04|N|O|1997-11-06|1997-10-08|1997-11-22|COLLECT COD|FOB|le furiously across the carefully sp|
+1093|61|2|3|34|32676.04|0.01|0.06|N|O|1997-11-07|1997-09-06|1997-11-28|TAKE BACK RETURN|REG AIR|sits. express accounts play carefully. bol|
+1094|115|6|1|9|9135.99|0.07|0.06|N|O|1997-12-28|1998-03-16|1998-01-18|DELIVER IN PERSON|AIR|as. slyly pe|
+1095|137|3|1|33|34225.29|0.01|0.02|N|O|1995-10-03|1995-09-22|1995-10-13|NONE|MAIL|slyly around the iron|
+1095|136|2|2|24|24867.12|0.04|0.06|N|O|1995-08-24|1995-10-20|1995-09-09|COLLECT COD|TRUCK|packages nod furiously above the carefully |
+1095|156|4|3|13|13729.95|0.06|0.01|N|O|1995-08-24|1995-10-19|1995-09-02|TAKE BACK RETURN|REG AIR|ously even accounts. slyly bold a|
+1095|135|1|4|28|28983.64|0.08|0.03|N|O|1995-09-20|1995-11-18|1995-10-02|DELIVER IN PERSON|SHIP| regular pac|
+1095|112|2|5|40|40484.40|0.09|0.03|N|O|1995-10-18|1995-11-14|1995-11-09|NONE|MAIL| bold accounts haggle slyly furiously even|
+1095|181|2|6|37|40003.66|0.07|0.08|N|O|1995-10-04|1995-11-13|1995-10-12|NONE|SHIP|. quickly even dolphins sle|
+1120|178|8|1|10|10781.70|0.08|0.05|N|O|1997-12-17|1998-01-21|1997-12-23|DELIVER IN PERSON|MAIL|dependencies. blithel|
+1120|20|1|2|49|45080.98|0.01|0.07|N|O|1998-01-03|1998-02-02|1998-01-09|TAKE BACK RETURN|RAIL|heodolites. quick re|
+1120|76|6|3|21|20497.47|0.06|0.01|N|O|1998-01-11|1998-02-04|1998-01-19|COLLECT COD|REG AIR|s: fluffily even packages c|
+1120|46|9|4|22|20812.88|0.09|0.08|N|O|1997-11-15|1998-01-25|1997-12-07|TAKE BACK RETURN|REG AIR|ons. slyly silent requests sleep silent|
+1120|83|4|5|10|9830.80|0.07|0.08|N|O|1997-11-10|1998-02-01|1997-11-28|TAKE BACK RETURN|AIR|ages haggle furiously |
+1121|168|3|1|42|44862.72|0.04|0.05|N|O|1997-03-05|1997-03-18|1997-03-14|DELIVER IN PERSON|SHIP|nts are slyly special packages. f|
+1121|161|10|2|27|28651.32|0.08|0.00|N|O|1997-05-08|1997-03-28|1997-05-14|NONE|MAIL|ly ironic accounts cajole slyly abou|
+1121|157|5|3|10|10571.50|0.00|0.04|N|O|1997-04-17|1997-03-18|1997-05-02|TAKE BACK RETURN|RAIL|dencies. quickly regular theodolites n|
+1121|166|1|4|29|30918.64|0.02|0.01|N|O|1997-03-07|1997-04-02|1997-04-01|DELIVER IN PERSON|REG AIR| use furiously. quickly silent package|
+1121|30|9|5|47|43711.41|0.09|0.03|N|O|1997-04-27|1997-03-28|1997-05-14|COLLECT COD|FOB|ly idle, i|
+1121|200|1|6|50|55010.00|0.06|0.03|N|O|1997-04-21|1997-02-16|1997-04-25|NONE|TRUCK|odolites. slyly even accounts|
+1121|80|8|7|37|36262.96|0.06|0.01|N|O|1997-02-27|1997-03-04|1997-03-02|COLLECT COD|RAIL|special packages. fluffily final requests s|
+1122|92|6|1|8|7936.72|0.10|0.06|N|O|1997-02-02|1997-04-03|1997-02-22|TAKE BACK RETURN|RAIL|c foxes are along the slyly r|
+1122|182|3|2|29|31383.22|0.05|0.04|N|O|1997-05-07|1997-04-07|1997-05-15|COLLECT COD|SHIP|ptotes. quickl|
+1122|147|6|3|25|26178.50|0.09|0.01|N|O|1997-03-21|1997-03-03|1997-04-07|TAKE BACK RETURN|RAIL|d furiously. pinto |
+1122|106|9|4|40|40244.00|0.08|0.08|N|O|1997-02-07|1997-03-25|1997-02-25|NONE|REG AIR|packages sleep after the asym|
+1122|151|2|5|15|15767.25|0.05|0.03|N|O|1997-04-15|1997-03-15|1997-05-07|COLLECT COD|SHIP|olve blithely regular, |
+1122|162|7|6|24|25491.84|0.04|0.01|N|O|1997-03-08|1997-02-20|1997-04-05|NONE|RAIL|blithely requests. slyly pending r|
+1122|1|6|7|38|34238.00|0.00|0.08|N|O|1997-01-23|1997-04-02|1997-02-16|NONE|TRUCK|t theodolites sleep. even, ironic|
+1123|12|2|1|10|9120.10|0.05|0.08|N|O|1996-11-12|1996-10-04|1996-11-30|NONE|MAIL|ckages are above the depths. slyly ir|
+1123|178|8|2|39|42048.63|0.03|0.08|N|O|1996-08-25|1996-10-21|1996-09-04|DELIVER IN PERSON|REG AIR|rding to the furiously ironic requests: r|
+1123|101|4|3|38|38041.80|0.03|0.08|N|O|1996-09-23|1996-10-04|1996-09-27|DELIVER IN PERSON|FOB| blithely carefully unusual reques|
+1124|198|2|1|1|1098.19|0.09|0.08|N|O|1998-10-06|1998-10-02|1998-10-30|NONE|REG AIR| instructions cajole qu|
+1124|6|1|2|13|11778.00|0.05|0.04|N|O|1998-09-05|1998-10-03|1998-09-30|DELIVER IN PERSON|SHIP|t the slyly |
+1124|93|5|3|35|34758.15|0.10|0.05|N|O|1998-11-25|1998-10-08|1998-12-25|TAKE BACK RETURN|AIR|ut the slyly bold pinto beans; fi|
+1124|50|1|4|25|23751.25|0.08|0.05|N|O|1998-08-05|1998-10-14|1998-08-11|NONE|MAIL|ggle slyly according|
+1124|75|5|5|33|32177.31|0.05|0.04|N|O|1998-10-19|1998-09-17|1998-10-26|TAKE BACK RETURN|SHIP|eposits sleep slyly. stealthily f|
+1124|27|6|6|43|39861.86|0.01|0.03|N|O|1998-09-19|1998-10-28|1998-10-10|COLLECT COD|MAIL|across the |
+1124|95|6|7|1|995.09|0.09|0.01|N|O|1998-10-07|1998-08-31|1998-10-12|NONE|TRUCK|ly bold accou|
+1125|133|4|1|4|4132.52|0.08|0.02|A|F|1994-12-10|1994-12-28|1994-12-30|NONE|MAIL| quickly express packages a|
+1125|138|9|2|24|24915.12|0.10|0.03|R|F|1995-01-31|1994-12-02|1995-02-20|COLLECT COD|AIR|es about the slyly s|
+1125|122|7|3|26|26575.12|0.05|0.04|A|F|1995-02-24|1995-01-18|1995-03-05|COLLECT COD|TRUCK|l instruction|
+1125|98|1|4|29|28944.61|0.06|0.00|A|F|1994-11-29|1994-12-20|1994-12-10|DELIVER IN PERSON|RAIL| platelets wake against the carefully i|
+1126|36|2|1|44|41185.32|0.08|0.03|N|O|1998-05-07|1998-04-02|1998-05-29|NONE|TRUCK|es. carefully special|
+1126|58|3|2|7|6706.35|0.06|0.01|N|O|1998-05-02|1998-03-22|1998-05-21|COLLECT COD|MAIL|ons. final, unusual|
+1126|147|10|3|14|14659.96|0.07|0.07|N|O|1998-04-17|1998-04-15|1998-05-12|DELIVER IN PERSON|TRUCK|nstructions. blithe|
+1127|43|10|1|35|33006.40|0.02|0.03|N|O|1995-11-25|1995-11-03|1995-12-17|NONE|TRUCK|l instructions boost blithely according |
+1127|110|5|2|38|38384.18|0.09|0.05|N|O|1995-11-07|1995-11-11|1995-11-26|DELIVER IN PERSON|RAIL|. never final packages boost acro|
+1127|20|1|3|29|26680.58|0.09|0.07|N|O|1995-09-20|1995-11-21|1995-10-11|DELIVER IN PERSON|REG AIR|y. blithely r|
+1127|175|6|4|7|7526.19|0.07|0.05|N|O|1995-11-05|1995-11-02|1995-11-11|DELIVER IN PERSON|FOB| idly pending pains |
+1152|9|10|1|23|20907.00|0.06|0.04|A|F|1994-10-14|1994-10-22|1994-10-21|DELIVER IN PERSON|MAIL|equests alongside of the unusual |
+1152|100|2|2|25|25002.50|0.04|0.08|R|F|1994-10-20|1994-09-18|1994-10-28|DELIVER IN PERSON|REG AIR|efully ironic accounts. sly instructions wa|
+1152|42|9|3|6|5652.24|0.07|0.03|A|F|1994-12-07|1994-11-05|1994-12-25|DELIVER IN PERSON|FOB|p furiously; packages above th|
+1153|86|7|1|15|14791.20|0.00|0.08|N|O|1996-04-24|1996-07-17|1996-04-29|TAKE BACK RETURN|SHIP|uctions boost fluffily according to|
+1153|169|8|2|50|53458.00|0.00|0.07|N|O|1996-06-27|1996-07-13|1996-07-05|COLLECT COD|REG AIR|ronic asymptotes nag slyly. |
+1153|44|5|3|25|23601.00|0.00|0.05|N|O|1996-06-18|1996-06-28|1996-07-09|NONE|TRUCK| theodolites|
+1153|92|3|4|43|42659.87|0.01|0.00|N|O|1996-06-09|1996-06-01|1996-07-04|DELIVER IN PERSON|MAIL|special instructions are. unusual, final du|
+1153|142|5|5|45|46896.30|0.00|0.02|N|O|1996-06-18|1996-06-20|1996-07-03|TAKE BACK RETURN|AIR|oss the ex|
+1153|136|7|6|26|26939.38|0.02|0.03|N|O|1996-08-16|1996-07-12|1996-09-08|NONE|MAIL|kages haggle carefully. f|
+1153|192|4|7|5|5460.95|0.02|0.03|N|O|1996-05-03|1996-06-12|1996-05-28|TAKE BACK RETURN|FOB|special excuses promi|
+1154|143|10|1|31|32337.34|0.06|0.06|A|F|1992-04-17|1992-04-26|1992-05-17|COLLECT COD|AIR|ithely. final, blithe |
+1154|148|7|2|50|52407.00|0.07|0.06|A|F|1992-04-22|1992-04-21|1992-05-01|NONE|TRUCK|ove the furiously bold Tires|
+1154|97|1|3|5|4985.45|0.09|0.04|A|F|1992-06-07|1992-05-07|1992-07-05|DELIVER IN PERSON|MAIL|the furiously |
+1154|1|2|4|35|31535.00|0.00|0.07|A|F|1992-03-30|1992-04-02|1992-04-21|DELIVER IN PERSON|TRUCK|the carefully regular pinto beans boost|
+1154|36|2|5|18|16848.54|0.02|0.03|A|F|1992-02-26|1992-03-24|1992-03-20|TAKE BACK RETURN|REG AIR|y regular excuses cajole blithely. fi|
+1154|196|8|6|50|54809.50|0.06|0.03|A|F|1992-03-04|1992-04-01|1992-04-01|TAKE BACK RETURN|TRUCK| even, special |
+1155|70|1|1|4|3880.28|0.09|0.05|N|O|1997-10-19|1997-12-09|1997-11-02|DELIVER IN PERSON|SHIP|ic foxes according to the carefully final |
+1155|196|9|2|39|42751.41|0.08|0.05|N|O|1998-01-29|1998-01-03|1998-02-01|COLLECT COD|TRUCK|ckly final pinto beans was.|
+1155|147|4|3|23|24084.22|0.08|0.03|N|O|1997-11-24|1997-11-28|1997-12-06|DELIVER IN PERSON|FOB|ly unusual packages. iro|
+1155|140|1|4|12|12481.68|0.01|0.06|N|O|1997-11-01|1998-01-03|1997-11-19|DELIVER IN PERSON|RAIL|packages do|
+1155|5|2|5|49|44345.00|0.04|0.08|N|O|1997-12-07|1997-12-30|1997-12-08|NONE|AIR|ccounts are alongside of t|
+1156|87|8|1|15|14806.20|0.07|0.06|N|O|1996-12-21|1997-01-03|1997-01-10|TAKE BACK RETURN|AIR|the furiously pen|
+1156|33|4|2|21|19593.63|0.02|0.08|N|O|1996-11-07|1997-01-14|1996-12-03|NONE|AIR|dolphins. fluffily ironic packages sleep re|
+1156|12|2|3|29|26448.29|0.09|0.06|N|O|1997-01-24|1996-12-26|1997-02-04|DELIVER IN PERSON|TRUCK|ts sleep sly|
+1156|172|3|4|42|45031.14|0.02|0.00|N|O|1997-01-18|1997-01-12|1997-02-13|NONE|REG AIR|s. quickly bold pains are|
+1156|74|4|5|49|47729.43|0.04|0.01|N|O|1996-11-16|1996-12-02|1996-12-05|COLLECT COD|AIR|ithely unusual in|
+1156|195|9|6|42|45997.98|0.02|0.06|N|O|1997-01-27|1997-01-09|1997-01-28|DELIVER IN PERSON|MAIL|even requests boost ironic deposits. pe|
+1156|47|6|7|20|18940.80|0.08|0.07|N|O|1997-01-01|1997-01-06|1997-01-16|COLLECT COD|MAIL|deposits sleep bravel|
+1157|49|2|1|16|15184.64|0.06|0.00|N|O|1998-04-12|1998-03-09|1998-04-23|DELIVER IN PERSON|MAIL|tions hang|
+1157|83|4|2|4|3932.32|0.10|0.05|N|O|1998-02-24|1998-03-30|1998-03-24|DELIVER IN PERSON|SHIP|ounts. ironic deposits|
+1157|48|7|3|8|7584.32|0.02|0.00|N|O|1998-03-25|1998-03-16|1998-03-29|NONE|REG AIR|blithely even pa|
+1157|77|8|4|46|44945.22|0.07|0.08|N|O|1998-04-19|1998-03-13|1998-04-23|NONE|FOB|slyly regular excuses. accounts|
+1157|160|5|5|14|14842.24|0.03|0.03|N|O|1998-04-17|1998-03-03|1998-05-01|NONE|FOB|theodolites. fluffily re|
+1158|45|2|1|5|4725.20|0.02|0.04|N|O|1996-10-20|1996-07-30|1996-11-14|COLLECT COD|AIR|symptotes along the care|
+1158|157|9|2|23|24314.45|0.00|0.08|N|O|1996-10-21|1996-08-19|1996-10-31|COLLECT COD|MAIL|ularly ironic requests use care|
+1159|109|10|1|39|39354.90|0.01|0.00|A|F|1992-11-20|1992-10-28|1992-12-18|TAKE BACK RETURN|FOB| blithely express reques|
+1159|96|9|2|7|6972.63|0.08|0.00|A|F|1992-11-25|1992-10-27|1992-12-20|NONE|AIR|olve somet|
+1159|98|10|3|11|10978.99|0.10|0.03|R|F|1992-12-09|1992-12-07|1992-12-18|DELIVER IN PERSON|MAIL|h furiousl|
+1184|47|4|1|27|25570.08|0.01|0.00|N|O|1998-01-10|1997-12-02|1998-02-06|TAKE BACK RETURN|REG AIR|s wake fluffily. fl|
+1184|147|10|2|4|4188.56|0.04|0.03|N|O|1997-12-25|1998-01-24|1998-01-18|DELIVER IN PERSON|RAIL| express packages. slyly expres|
+1184|164|5|3|7|7449.12|0.05|0.00|N|O|1998-02-14|1998-01-06|1998-03-11|COLLECT COD|TRUCK|ckly warthogs. blithely bold foxes hag|
+1184|126|9|4|3|3078.36|0.02|0.05|N|O|1998-01-15|1997-12-19|1998-02-02|NONE|REG AIR|ar packages. final packages cajol|
+1185|72|1|1|8|7776.56|0.01|0.06|A|F|1992-12-05|1992-10-05|1992-12-28|DELIVER IN PERSON|MAIL|ely according to the furiously regular r|
+1185|31|2|2|28|26068.84|0.07|0.06|A|F|1992-09-24|1992-10-07|1992-10-10|DELIVER IN PERSON|REG AIR|ke. slyly regular t|
+1185|190|1|3|12|13082.28|0.05|0.06|R|F|1992-10-12|1992-09-26|1992-11-11|NONE|REG AIR|instructions. daringly pend|
+1186|3|4|1|28|25284.00|0.08|0.07|N|O|1996-12-08|1996-10-17|1996-12-15|TAKE BACK RETURN|TRUCK|ffily spec|
+1186|92|5|2|11|10912.99|0.07|0.05|N|O|1996-10-03|1996-10-21|1996-10-17|DELIVER IN PERSON|AIR|s haggle furiously; slyl|
+1186|101|2|3|20|20022.00|0.07|0.07|N|O|1996-08-20|1996-10-23|1996-09-05|COLLECT COD|FOB|ely alongside of the blithel|
+1186|106|7|4|27|27164.70|0.06|0.04|N|O|1996-10-08|1996-11-06|1996-10-09|TAKE BACK RETURN|SHIP|accounts. express, e|
+1187|178|6|1|29|31266.93|0.01|0.04|R|F|1992-12-10|1993-02-09|1992-12-29|TAKE BACK RETURN|RAIL|riously express ac|
+1187|131|7|2|15|15466.95|0.03|0.04|A|F|1992-12-22|1993-01-13|1993-01-01|NONE|TRUCK|ests. foxes wake. carefu|
+1187|78|8|3|40|39122.80|0.08|0.06|R|F|1993-03-05|1992-12-31|1993-03-12|NONE|TRUCK|ar, brave deposits nag blithe|
+1188|115|9|1|2|2030.22|0.00|0.04|N|O|1996-05-22|1996-05-23|1996-06-06|COLLECT COD|RAIL|its breach blit|
+1188|113|4|2|9|9117.99|0.01|0.08|N|O|1996-08-04|1996-06-04|1996-08-19|NONE|REG AIR|ow carefully ironic d|
+1188|179|10|3|41|44245.97|0.07|0.04|N|O|1996-06-29|1996-05-21|1996-07-21|TAKE BACK RETURN|TRUCK|althy packages. fluffily unusual ideas h|
+1189|51|2|1|23|21874.15|0.06|0.00|R|F|1994-07-25|1994-06-07|1994-08-02|COLLECT COD|FOB|s. fluffy Tiresias run quickly. bra|
+1189|105|2|2|32|32163.20|0.09|0.02|R|F|1994-05-06|1994-07-03|1994-05-15|TAKE BACK RETURN|FOB|e regular deposits. quickly quiet deposi|
+1189|57|5|3|22|21055.10|0.05|0.03|R|F|1994-06-09|1994-06-29|1994-06-23|DELIVER IN PERSON|TRUCK|quickly unusual platelets lose forges. ca|
+1190|84|5|1|32|31490.56|0.07|0.06|N|O|1997-05-08|1997-04-17|1997-06-01|COLLECT COD|FOB|y final packages? slyly even|
+1191|49|6|1|29|27522.16|0.00|0.04|N|O|1996-01-24|1996-01-28|1996-02-17|COLLECT COD|AIR| regular pin|
+1216|97|1|1|8|7976.72|0.03|0.04|R|F|1993-02-01|1993-03-06|1993-02-08|TAKE BACK RETURN|TRUCK| of the carefully express|
+1216|75|3|2|48|46803.36|0.10|0.01|R|F|1993-01-17|1993-02-01|1993-02-13|COLLECT COD|SHIP|symptotes use against th|
+1216|42|3|3|18|16956.72|0.00|0.03|A|F|1993-01-20|1993-01-28|1993-02-02|COLLECT COD|MAIL|y final packages nod |
+1217|60|5|1|45|43202.70|0.07|0.02|A|F|1992-07-01|1992-06-23|1992-07-06|COLLECT COD|AIR|riously close ideas|
+1218|140|6|1|16|16642.24|0.04|0.07|A|F|1994-06-26|1994-08-07|1994-06-30|TAKE BACK RETURN|FOB|ven realms be|
+1218|94|6|2|41|40757.69|0.06|0.06|R|F|1994-08-04|1994-08-05|1994-08-11|TAKE BACK RETURN|SHIP|dolphins. theodolites beyond th|
+1218|48|7|3|44|41713.76|0.07|0.06|A|F|1994-10-05|1994-09-03|1994-10-30|COLLECT COD|TRUCK|thely ironic accounts wake slyly|
+1218|42|9|4|1|942.04|0.01|0.08|R|F|1994-09-15|1994-09-07|1994-10-03|COLLECT COD|TRUCK|press furio|
+1219|132|3|1|6|6192.78|0.08|0.04|N|O|1995-11-13|1995-12-24|1995-11-18|NONE|MAIL|pecial, ironic requ|
+1219|129|4|2|4|4116.48|0.01|0.04|N|O|1995-11-24|1995-11-22|1995-12-07|TAKE BACK RETURN|SHIP|lly quick requests. blithely even h|
+1220|169|4|1|25|26729.00|0.10|0.03|N|O|1996-10-15|1996-11-07|1996-11-06|COLLECT COD|REG AIR| regular orbi|
+1220|160|5|2|36|38165.76|0.01|0.02|N|O|1996-12-10|1996-11-14|1997-01-07|COLLECT COD|SHIP|ar packages. blithely final acc|
+1220|37|8|3|3|2811.09|0.08|0.06|N|O|1996-09-06|1996-11-03|1996-09-10|COLLECT COD|REG AIR| final theodolites. blithely silent |
+1220|6|1|4|36|32616.00|0.07|0.03|N|O|1996-12-12|1996-10-03|1996-12-15|TAKE BACK RETURN|TRUCK|unusual, silent pinto beans aga|
+1220|49|2|5|25|23726.00|0.03|0.08|N|O|1996-09-11|1996-10-09|1996-09-25|DELIVER IN PERSON|RAIL|packages affi|
+1221|81|2|1|43|42186.44|0.05|0.05|R|F|1992-06-22|1992-07-15|1992-07-20|DELIVER IN PERSON|FOB|y slyly above the slyly unusual ideas|
+1221|170|1|2|12|12842.04|0.00|0.08|R|F|1992-08-07|1992-06-24|1992-08-13|COLLECT COD|AIR|yly ironic |
+1221|69|6|3|3|2907.18|0.10|0.08|R|F|1992-07-01|1992-06-04|1992-07-27|COLLECT COD|TRUCK|ing to the fluffily|
+1221|120|10|4|41|41824.92|0.06|0.02|A|F|1992-04-28|1992-07-02|1992-05-19|NONE|RAIL|ns. bold deposit|
+1221|108|1|5|13|13105.30|0.10|0.00|R|F|1992-08-01|1992-06-29|1992-08-27|TAKE BACK RETURN|AIR|ajole furiously. blithely expres|
+1221|85|6|6|7|6895.56|0.08|0.06|A|F|1992-06-27|1992-06-16|1992-07-23|TAKE BACK RETURN|RAIL|xpress accounts |
+1222|72|10|1|12|11664.84|0.09|0.02|A|F|1993-02-12|1993-03-14|1993-03-12|TAKE BACK RETURN|RAIL|s print permanently unusual packages. |
+1222|159|7|2|12|12709.80|0.08|0.01|A|F|1993-05-05|1993-03-27|1993-05-18|TAKE BACK RETURN|REG AIR| furiously bold instructions|
+1222|8|1|3|26|23608.00|0.02|0.08|R|F|1993-02-13|1993-03-20|1993-02-22|TAKE BACK RETURN|MAIL|, even accounts are ironic|
+1223|100|1|1|28|28002.80|0.10|0.06|N|O|1996-08-07|1996-07-24|1996-08-13|TAKE BACK RETURN|MAIL| quickly ironic requests. furious|
+1248|164|5|1|45|47887.20|0.00|0.08|A|F|1992-04-17|1992-03-31|1992-05-13|NONE|RAIL|ter the pending pl|
+1248|151|9|2|37|38892.55|0.06|0.06|R|F|1992-01-26|1992-02-05|1992-02-13|COLLECT COD|TRUCK|. final requests integrate quickly. blit|
+1248|56|8|3|26|24857.30|0.09|0.06|A|F|1992-01-16|1992-03-01|1992-02-06|TAKE BACK RETURN|AIR| ironic dependen|
+1248|156|7|4|49|51751.35|0.02|0.01|A|F|1992-04-24|1992-02-18|1992-05-03|TAKE BACK RETURN|AIR|beans run quickly according to the carefu|
+1248|122|7|5|20|20442.40|0.08|0.00|A|F|1992-03-12|1992-03-23|1992-04-07|TAKE BACK RETURN|AIR|nal foxes cajole carefully slyl|
+1248|62|9|6|30|28861.80|0.10|0.01|R|F|1992-02-01|1992-03-24|1992-02-08|TAKE BACK RETURN|MAIL|fily special foxes kindle am|
+1249|59|4|1|49|46993.45|0.07|0.05|A|F|1994-03-03|1994-02-28|1994-03-08|NONE|RAIL|ffily express theodo|
+1250|2|3|1|15|13530.00|0.10|0.06|A|F|1992-11-05|1992-12-17|1992-12-03|TAKE BACK RETURN|SHIP| regular, i|
+1251|4|5|1|37|33448.00|0.08|0.08|N|O|1997-12-21|1998-01-12|1997-12-26|COLLECT COD|AIR|. furiously|
+1251|78|9|2|36|35210.52|0.07|0.04|N|O|1997-11-29|1998-01-07|1997-12-03|TAKE BACK RETURN|RAIL|y ironic Tiresias are slyly furio|
+1251|99|3|3|37|36966.33|0.09|0.02|N|O|1998-01-11|1997-12-01|1998-01-23|DELIVER IN PERSON|RAIL|finally bold requests|
+1251|150|9|4|7|7351.05|0.07|0.00|N|O|1998-01-08|1997-12-27|1998-01-18|COLLECT COD|MAIL|riously pe|
+1251|188|9|5|1|1088.18|0.02|0.03|N|O|1997-12-08|1998-01-06|1998-01-01|DELIVER IN PERSON|REG AIR| use quickly final packages. iron|
+1252|87|8|1|13|12832.04|0.10|0.01|N|O|1997-09-07|1997-09-12|1997-10-01|COLLECT COD|REG AIR|sts dazzle|
+1252|111|8|2|27|27299.97|0.00|0.08|N|O|1997-10-22|1997-10-10|1997-11-10|TAKE BACK RETURN|REG AIR|packages hag|
+1252|40|1|3|19|17860.76|0.07|0.02|N|O|1997-10-13|1997-10-23|1997-10-18|NONE|AIR|ts wake carefully-- packages sleep. quick |
+1252|92|4|4|11|10912.99|0.10|0.01|N|O|1997-10-16|1997-09-22|1997-10-28|COLLECT COD|AIR|s are. slyly final requests among the|
+1252|79|10|5|26|25455.82|0.05|0.05|N|O|1997-08-05|1997-10-24|1997-08-07|DELIVER IN PERSON|SHIP|onic pinto beans haggle furiously |
+1253|180|8|1|14|15122.52|0.00|0.06|R|F|1993-04-03|1993-04-16|1993-04-27|TAKE BACK RETURN|MAIL|lar foxes sleep furiously final, final pack|
+1253|54|9|2|13|12402.65|0.01|0.06|A|F|1993-03-05|1993-04-26|1993-03-08|DELIVER IN PERSON|FOB|al packages|
+1253|70|1|3|22|21341.54|0.05|0.06|A|F|1993-02-23|1993-04-06|1993-03-07|TAKE BACK RETURN|SHIP|telets cajole alongside of the final reques|
+1253|176|5|4|23|24751.91|0.09|0.02|R|F|1993-04-18|1993-04-18|1993-05-07|COLLECT COD|FOB| the slyly silent re|
+1253|114|8|5|19|19268.09|0.05|0.05|A|F|1993-04-01|1993-04-22|1993-04-14|TAKE BACK RETURN|AIR|al pinto bea|
+1254|193|5|1|6|6559.14|0.08|0.01|N|O|1996-02-02|1996-03-21|1996-02-29|NONE|REG AIR|lithely even deposits eat!|
+1254|200|3|2|47|51709.40|0.05|0.06|N|O|1996-03-07|1996-02-20|1996-04-05|COLLECT COD|MAIL| platelets cajol|
+1254|135|6|3|35|36229.55|0.05|0.06|N|O|1996-04-08|1996-02-29|1996-04-18|DELIVER IN PERSON|FOB|ckages boost. furious warhorses cajole|
+1255|192|4|1|12|13106.28|0.00|0.02|A|F|1994-08-17|1994-06-29|1994-09-04|TAKE BACK RETURN|REG AIR| regular, express accounts are |
+1255|194|8|2|46|50332.74|0.07|0.05|R|F|1994-07-06|1994-07-14|1994-08-05|NONE|MAIL|ons nag qui|
+1280|129|8|1|17|17495.04|0.01|0.01|A|F|1993-02-04|1993-04-10|1993-02-07|NONE|FOB|ructions integrate across the th|
+1280|189|10|2|6|6535.08|0.05|0.06|R|F|1993-03-30|1993-02-16|1993-04-18|DELIVER IN PERSON|AIR|gular deposits |
+1280|33|4|3|13|12129.39|0.03|0.02|R|F|1993-03-06|1993-03-11|1993-03-18|DELIVER IN PERSON|TRUCK|blithely final accounts use evenly |
+1280|175|3|4|5|5375.85|0.06|0.03|R|F|1993-02-03|1993-02-11|1993-02-23|DELIVER IN PERSON|AIR|beans haggle. quickly bold instructions h|
+1280|52|10|5|24|22849.20|0.07|0.02|R|F|1993-03-20|1993-03-01|1993-04-09|COLLECT COD|RAIL|y pending orbits boost after the slyly|
+1280|66|3|6|9|8694.54|0.00|0.05|R|F|1993-04-18|1993-03-28|1993-05-04|DELIVER IN PERSON|FOB|usual accou|
+1280|92|6|7|19|18849.71|0.02|0.06|A|F|1993-02-07|1993-02-28|1993-02-12|NONE|TRUCK|lyly along the furiously regular |
+1281|138|4|1|33|34258.29|0.07|0.08|R|F|1995-02-01|1995-01-18|1995-03-03|NONE|REG AIR|dencies. thinly final pinto beans wake|
+1281|7|2|2|37|33559.00|0.08|0.03|A|F|1995-03-19|1995-02-02|1995-03-27|NONE|AIR|ounts detect|
+1281|94|7|3|2|1988.18|0.05|0.06|A|F|1994-12-27|1995-01-26|1995-01-21|TAKE BACK RETURN|FOB|ly unusual requests. final reques|
+1281|154|2|4|38|40057.70|0.04|0.06|R|F|1995-03-28|1995-01-11|1995-04-14|TAKE BACK RETURN|MAIL| ideas-- blithely regular|
+1281|152|10|5|13|13677.95|0.03|0.07|A|F|1995-02-06|1995-02-13|1995-02-18|DELIVER IN PERSON|TRUCK|fully final platelets wa|
+1281|50|9|6|4|3800.20|0.07|0.04|R|F|1995-03-15|1995-02-21|1995-03-20|NONE|SHIP|ggle against the even requests. requests |
+1281|78|6|7|43|42057.01|0.10|0.02|R|F|1995-01-28|1995-02-08|1995-02-10|DELIVER IN PERSON|AIR|final accounts. final packages slee|
+1282|23|4|1|14|12922.28|0.04|0.02|R|F|1992-06-29|1992-04-05|1992-07-21|TAKE BACK RETURN|REG AIR|ecial deposit|
+1282|30|9|2|10|9300.30|0.09|0.06|R|F|1992-04-10|1992-04-16|1992-05-01|DELIVER IN PERSON|SHIP|r theodolite|
+1282|160|1|3|19|20143.04|0.01|0.03|R|F|1992-05-07|1992-04-07|1992-05-13|NONE|RAIL|ts x-ray across the furi|
+1282|59|10|4|19|18221.95|0.00|0.05|A|F|1992-06-20|1992-04-17|1992-07-05|DELIVER IN PERSON|REG AIR|nto beans. carefully close theodo|
+1283|93|7|1|47|46675.23|0.05|0.03|N|O|1996-10-21|1996-10-29|1996-11-12|DELIVER IN PERSON|TRUCK|even instructions boost slyly blithely |
+1283|106|1|2|1|1006.10|0.00|0.08|N|O|1996-10-07|1996-10-12|1996-10-08|NONE|RAIL|d the sauternes. slyly ev|
+1283|138|4|3|18|18686.34|0.02|0.01|N|O|1996-10-14|1996-11-07|1996-10-22|DELIVER IN PERSON|AIR|equests use along the fluff|
+1283|192|4|4|40|43687.60|0.07|0.03|N|O|1996-11-09|1996-11-23|1996-11-28|NONE|MAIL|riously. even, ironic instructions after|
+1283|124|9|5|43|44037.16|0.01|0.04|N|O|1996-09-29|1996-11-19|1996-10-26|TAKE BACK RETURN|RAIL|requests sleep slyly about the |
+1283|8|5|6|30|27240.00|0.06|0.07|N|O|1996-11-22|1996-11-22|1996-12-15|COLLECT COD|TRUCK|t the fluffily|
+1283|197|8|7|21|23040.99|0.04|0.03|N|O|1996-09-12|1996-10-02|1996-10-12|NONE|REG AIR|fully regular |
+1284|178|7|1|49|52830.33|0.00|0.06|N|O|1996-04-11|1996-03-04|1996-04-16|NONE|MAIL|lar packages. special packages ac|
+1284|6|7|2|4|3624.00|0.07|0.06|N|O|1996-02-29|1996-02-11|1996-03-01|TAKE BACK RETURN|TRUCK| regular asymptotes. |
+1284|133|4|3|39|40292.07|0.08|0.00|N|O|1996-01-11|1996-02-07|1996-02-05|COLLECT COD|MAIL|even accoun|
+1284|59|10|4|1|959.05|0.01|0.07|N|O|1996-04-28|1996-04-02|1996-05-08|DELIVER IN PERSON|SHIP|al packages use carefully express de|
+1284|34|5|5|9|8406.27|0.05|0.06|N|O|1996-03-03|1996-03-19|1996-04-01|DELIVER IN PERSON|REG AIR|after the pending|
+1285|22|3|1|12|11064.24|0.00|0.06|A|F|1992-06-21|1992-08-16|1992-07-12|COLLECT COD|MAIL|ss foxes. blithe theodolites cajole slyly|
+1285|143|10|2|45|46941.30|0.01|0.02|R|F|1992-09-05|1992-08-08|1992-10-02|COLLECT COD|REG AIR| special requests haggle blithely.|
+1285|189|10|3|4|4356.72|0.09|0.06|A|F|1992-07-20|1992-08-17|1992-07-26|DELIVER IN PERSON|FOB|l packages sleep slyly quiet i|
+1285|188|9|4|39|42439.02|0.05|0.01|A|F|1992-09-15|1992-08-05|1992-10-05|DELIVER IN PERSON|TRUCK|uctions. car|
+1285|84|5|5|33|32474.64|0.00|0.08|R|F|1992-09-08|1992-08-25|1992-09-16|NONE|SHIP|ites affix|
+1286|178|9|1|49|52830.33|0.08|0.01|R|F|1993-06-24|1993-08-12|1993-06-26|DELIVER IN PERSON|SHIP|gged accoun|
+1286|49|6|2|48|45553.92|0.01|0.04|A|F|1993-07-11|1993-07-11|1993-08-01|COLLECT COD|TRUCK|unts alongs|
+1286|189|10|3|11|11980.98|0.03|0.04|R|F|1993-08-08|1993-07-30|1993-09-05|DELIVER IN PERSON|FOB| slyly even packages. requ|
+1286|184|5|4|37|40114.66|0.00|0.02|R|F|1993-05-27|1993-07-11|1993-06-01|COLLECT COD|SHIP|lyly ironic pinto beans cajole furiously s|
+1286|165|10|5|14|14912.24|0.00|0.01|R|F|1993-05-23|1993-08-09|1993-06-01|NONE|REG AIR|blithely bo|
+1286|146|5|6|41|42891.74|0.04|0.05|R|F|1993-08-02|1993-08-06|1993-08-07|TAKE BACK RETURN|FOB| the furiously expre|
+1287|174|3|1|35|37595.95|0.09|0.06|A|F|1994-09-07|1994-09-12|1994-09-30|TAKE BACK RETURN|FOB|s wake unusual grou|
+1287|95|8|2|10|9950.90|0.08|0.03|R|F|1994-07-08|1994-08-28|1994-07-10|TAKE BACK RETURN|RAIL|thely alongside of the unusual, ironic pa|
+1287|1|2|3|30|27030.00|0.00|0.07|R|F|1994-07-12|1994-09-23|1994-08-07|NONE|RAIL|ar packages. even, even|
+1287|62|7|4|10|9620.60|0.01|0.05|A|F|1994-09-03|1994-08-12|1994-09-16|TAKE BACK RETURN|REG AIR|ding, regular accounts|
+1287|179|8|5|21|22662.57|0.06|0.02|A|F|1994-10-06|1994-09-25|1994-10-16|TAKE BACK RETURN|TRUCK|y quickly bold theodoli|
+1287|21|10|6|26|23946.52|0.03|0.08|R|F|1994-10-03|1994-09-27|1994-10-30|DELIVER IN PERSON|RAIL|egular foxes. theodolites nag along t|
+1312|81|2|1|9|8829.72|0.04|0.08|R|F|1994-07-19|1994-06-29|1994-07-24|TAKE BACK RETURN|MAIL|. furiously |
+1312|136|7|2|28|29011.64|0.06|0.06|A|F|1994-09-09|1994-08-01|1994-10-02|TAKE BACK RETURN|FOB|uriously final frays should use quick|
+1312|173|1|3|18|19317.06|0.03|0.07|A|F|1994-09-13|1994-07-08|1994-09-22|TAKE BACK RETURN|MAIL|. slyly ironic|
+1313|52|4|1|48|45698.40|0.01|0.03|A|F|1994-12-20|1994-10-29|1995-01-07|COLLECT COD|MAIL|s are quick|
+1314|198|10|1|5|5490.95|0.03|0.01|A|F|1994-05-26|1994-08-06|1994-05-31|TAKE BACK RETURN|AIR|equests nag across the furious|
+1314|110|5|2|39|39394.29|0.01|0.03|R|F|1994-08-09|1994-06-14|1994-08-31|TAKE BACK RETURN|TRUCK| unusual accounts slee|
+1314|41|2|3|11|10351.44|0.01|0.04|A|F|1994-05-16|1994-07-30|1994-05-31|COLLECT COD|REG AIR|tegrate furious|
+1315|96|8|1|27|26894.43|0.01|0.03|N|O|1998-07-04|1998-06-13|1998-07-28|NONE|SHIP|latelets. fluffily ironic account|
+1315|16|6|2|15|13740.15|0.05|0.01|N|O|1998-07-12|1998-06-10|1998-08-07|COLLECT COD|AIR|. foxes integrate carefully special|
+1315|168|3|3|25|26704.00|0.01|0.08|N|O|1998-06-26|1998-06-10|1998-07-06|TAKE BACK RETURN|FOB|lites. unusual foxes affi|
+1315|161|6|4|19|20162.04|0.02|0.05|N|O|1998-07-05|1998-05-23|1998-08-04|TAKE BACK RETURN|SHIP|nal, regular warhorses about the fu|
+1315|159|7|5|32|33892.80|0.10|0.05|N|O|1998-03-30|1998-06-12|1998-04-25|NONE|SHIP|neath the final p|
+1316|127|6|1|46|47247.52|0.05|0.04|A|F|1994-01-13|1994-01-24|1994-02-03|COLLECT COD|TRUCK|ges haggle of the|
+1316|79|9|2|15|14686.05|0.02|0.01|R|F|1994-03-12|1994-03-02|1994-03-14|COLLECT COD|FOB|se. furiously final depo|
+1316|198|9|3|33|36240.27|0.10|0.06|R|F|1994-03-31|1994-01-23|1994-04-20|TAKE BACK RETURN|AIR|manently; blithely special deposits|
+1316|66|3|4|15|14490.90|0.00|0.06|R|F|1993-12-17|1994-02-04|1993-12-20|NONE|RAIL|fully express dugouts. furiously silent ide|
+1316|41|2|5|40|37641.60|0.01|0.03|R|F|1994-02-04|1994-02-09|1994-02-27|NONE|REG AIR|l dugouts. co|
+1316|4|7|6|7|6328.00|0.05|0.04|A|F|1993-12-09|1994-01-12|1993-12-30|TAKE BACK RETURN|MAIL|. furiously even accounts a|
+1316|163|8|7|8|8505.28|0.10|0.04|A|F|1994-03-26|1994-02-08|1994-04-19|NONE|SHIP|packages against the express requests wa|
+1317|134|5|1|34|35160.42|0.08|0.04|N|O|1995-08-13|1995-08-08|1995-09-10|COLLECT COD|RAIL|deposits boost thinly blithely final id|
+1317|160|2|2|7|7421.12|0.05|0.01|A|F|1995-06-08|1995-08-03|1995-06-16|TAKE BACK RETURN|SHIP| pinto beans according to the final, pend|
+1317|158|9|3|26|27511.90|0.01|0.02|N|O|1995-07-13|1995-06-26|1995-08-06|COLLECT COD|RAIL|leep along th|
+1317|106|3|4|35|35213.50|0.05|0.02|N|O|1995-07-16|1995-07-07|1995-07-22|TAKE BACK RETURN|FOB|r packages impress blithely car|
+1317|150|9|5|36|37805.40|0.02|0.00|N|O|1995-09-03|1995-07-06|1995-09-04|DELIVER IN PERSON|AIR| deposits. quic|
+1318|114|4|1|24|24338.64|0.08|0.06|N|O|1998-09-27|1998-09-15|1998-10-12|TAKE BACK RETURN|AIR|ual, unusual packages. fluffy, iro|
+1318|46|3|2|26|24597.04|0.01|0.03|N|O|1998-09-26|1998-08-09|1998-10-07|DELIVER IN PERSON|FOB|ly. regular, u|
+1318|129|4|3|31|31902.72|0.01|0.04|N|O|1998-08-25|1998-07-31|1998-08-31|COLLECT COD|AIR|ve the carefully expr|
+1319|61|8|1|21|20182.26|0.03|0.04|N|O|1996-10-05|1996-12-02|1996-10-28|COLLECT COD|FOB|s: carefully express |
+1319|37|8|2|12|11244.36|0.09|0.05|N|O|1996-11-05|1996-12-12|1996-11-29|DELIVER IN PERSON|TRUCK|packages integrate furiously. expres|
+1344|141|4|1|15|15617.10|0.10|0.07|A|F|1992-06-22|1992-06-24|1992-06-23|TAKE BACK RETURN|MAIL|rding to the blithely ironic theodolite|
+1344|190|1|2|29|31615.51|0.09|0.00|A|F|1992-07-17|1992-06-07|1992-07-21|NONE|REG AIR|ffily quiet foxes wake blithely. slyly |
+1345|198|9|1|49|53811.31|0.08|0.00|A|F|1992-12-27|1993-01-23|1993-01-06|NONE|FOB|sly. furiously final accounts are blithely |
+1345|12|9|2|37|33744.37|0.10|0.07|A|F|1992-11-27|1992-12-11|1992-12-07|COLLECT COD|FOB|e slyly express requests. ironic accounts c|
+1345|57|8|3|31|29668.55|0.08|0.07|R|F|1992-12-02|1992-12-29|1992-12-14|COLLECT COD|REG AIR|. slyly silent accounts sublat|
+1346|160|8|1|29|30744.64|0.07|0.05|A|F|1992-08-18|1992-09-15|1992-09-17|TAKE BACK RETURN|REG AIR|the pinto |
+1346|125|6|2|48|49205.76|0.06|0.03|A|F|1992-09-28|1992-07-22|1992-10-13|TAKE BACK RETURN|REG AIR| along the carefully spec|
+1346|54|5|3|13|12402.65|0.10|0.04|A|F|1992-07-22|1992-08-10|1992-08-06|NONE|SHIP|arefully brave deposits into the slyly iro|
+1346|124|5|4|6|6144.72|0.02|0.02|R|F|1992-09-13|1992-07-21|1992-09-27|TAKE BACK RETURN|AIR|inst the furiously final theodolites. caref|
+1346|187|8|5|30|32615.40|0.01|0.07|R|F|1992-10-01|1992-07-22|1992-10-24|NONE|SHIP| nag blithely. unusual, ru|
+1346|16|6|6|45|41220.45|0.02|0.04|A|F|1992-09-11|1992-08-06|1992-09-12|COLLECT COD|FOB|press deposits.|
+1347|81|2|1|45|44148.60|0.02|0.05|N|O|1997-08-24|1997-09-03|1997-09-08|COLLECT COD|AIR|ages wake around t|
+1347|143|6|2|34|35466.76|0.07|0.04|N|O|1997-06-25|1997-09-08|1997-07-24|COLLECT COD|FOB|r packages. f|
+1347|185|6|3|23|24959.14|0.03|0.04|N|O|1997-07-31|1997-08-25|1997-08-21|COLLECT COD|SHIP|ronic pinto beans. express reques|
+1347|113|7|4|28|28367.08|0.01|0.00|N|O|1997-07-30|1997-07-22|1997-08-18|TAKE BACK RETURN|FOB|foxes after the blithely special i|
+1347|65|6|5|9|8685.54|0.01|0.03|N|O|1997-08-28|1997-09-16|1997-09-26|DELIVER IN PERSON|AIR| detect blithely above the fina|
+1347|153|8|6|21|22116.15|0.06|0.04|N|O|1997-10-10|1997-08-16|1997-11-02|NONE|FOB|g pinto beans affix car|
+1347|51|3|7|10|9510.50|0.02|0.07|N|O|1997-07-04|1997-07-23|1997-07-05|DELIVER IN PERSON|SHIP|y ironic pin|
+1348|95|7|1|13|12936.17|0.01|0.01|N|O|1998-04-28|1998-06-05|1998-05-12|TAKE BACK RETURN|SHIP| blithely r|
+1348|22|5|2|41|37802.82|0.07|0.03|N|O|1998-05-02|1998-05-26|1998-05-09|COLLECT COD|RAIL|kages. platelets about the ca|
+1348|199|10|3|40|43967.60|0.07|0.05|N|O|1998-08-14|1998-07-10|1998-08-27|COLLECT COD|AIR|fter the regu|
+1348|98|1|4|2|1996.18|0.01|0.04|N|O|1998-05-30|1998-06-20|1998-06-05|COLLECT COD|MAIL|lly final packages use fluffily express ac|
+1349|181|2|1|1|1081.18|0.06|0.03|N|O|1998-01-07|1998-01-14|1998-02-03|COLLECT COD|REG AIR| express inst|
+1349|118|2|2|45|45814.95|0.03|0.02|N|O|1997-12-24|1998-01-17|1997-12-28|NONE|AIR| ironic, unusual deposits wake carefu|
+1350|54|9|1|21|20035.05|0.04|0.04|A|F|1993-12-17|1993-10-17|1993-12-25|COLLECT COD|REG AIR|lyly above the evenly |
+1350|44|5|2|32|30209.28|0.03|0.00|R|F|1993-11-18|1993-09-30|1993-12-16|COLLECT COD|MAIL|ic, final |
+1351|108|9|1|25|25202.50|0.06|0.04|N|O|1998-06-02|1998-05-25|1998-06-22|COLLECT COD|SHIP|iously regul|
+1376|169|8|1|22|23521.52|0.01|0.03|N|O|1997-08-05|1997-07-08|1997-09-03|NONE|REG AIR|inst the final, pending |
+1377|154|6|1|5|5270.75|0.06|0.05|N|O|1998-05-06|1998-07-08|1998-06-01|TAKE BACK RETURN|FOB| final, final grouches. accoun|
+1377|33|9|2|3|2799.09|0.10|0.04|N|O|1998-04-30|1998-07-02|1998-05-14|DELIVER IN PERSON|REG AIR|yly enticing requ|
+1377|84|5|3|26|25586.08|0.07|0.07|N|O|1998-05-28|1998-06-11|1998-06-25|COLLECT COD|SHIP|egular deposits. quickly regular acco|
+1377|121|4|4|39|39823.68|0.00|0.03|N|O|1998-07-27|1998-07-18|1998-08-13|DELIVER IN PERSON|SHIP|e ironic, regular requests. carefully |
+1377|33|9|5|19|17727.57|0.10|0.00|N|O|1998-06-20|1998-06-27|1998-07-20|NONE|AIR|ught to are bold foxes|
+1377|154|6|6|17|17920.55|0.03|0.04|N|O|1998-06-19|1998-07-20|1998-07-14|NONE|REG AIR|s must have to mold b|
+1378|197|10|1|34|37304.46|0.09|0.07|N|O|1996-07-08|1996-04-23|1996-07-09|COLLECT COD|RAIL|le furiously slyly final accounts. careful|
+1378|124|9|2|18|18434.16|0.05|0.02|N|O|1996-06-19|1996-05-16|1996-06-21|DELIVER IN PERSON|RAIL| theodolites. i|
+1378|73|4|3|11|10703.77|0.10|0.03|N|O|1996-06-07|1996-05-09|1996-07-05|COLLECT COD|TRUCK| blithely express hoc|
+1378|171|2|4|12|12854.04|0.02|0.06|N|O|1996-06-16|1996-05-23|1996-07-09|COLLECT COD|SHIP|notornis. b|
+1378|156|7|5|9|9505.35|0.06|0.05|N|O|1996-04-20|1996-04-13|1996-05-09|COLLECT COD|REG AIR|e carefully. carefully iron|
+1378|194|6|6|29|31731.51|0.05|0.05|N|O|1996-04-15|1996-04-23|1996-05-14|NONE|REG AIR|ual packages are furiously blith|
+1379|73|3|1|13|12649.91|0.04|0.01|N|O|1998-06-08|1998-07-13|1998-06-16|NONE|AIR|ully across the furiously iron|
+1379|118|2|2|50|50905.50|0.07|0.08|N|O|1998-08-31|1998-07-13|1998-09-02|TAKE BACK RETURN|FOB|olphins. ca|
+1379|13|7|3|24|21912.24|0.05|0.02|N|O|1998-07-06|1998-07-09|1998-07-29|DELIVER IN PERSON|MAIL|ages cajole carefully idly express re|
+1380|149|2|1|6|6294.84|0.00|0.04|N|O|1996-08-06|1996-10-01|1996-08-14|NONE|RAIL|e foxes. slyly specia|
+1380|141|4|2|40|41645.60|0.02|0.02|N|O|1996-10-01|1996-08-14|1996-10-20|COLLECT COD|RAIL|ly final frets. ironic,|
+1380|78|9|3|15|14671.05|0.05|0.02|N|O|1996-07-14|1996-08-12|1996-08-03|NONE|FOB|riously ironic foxes aff|
+1380|61|10|4|33|31714.98|0.04|0.07|N|O|1996-08-23|1996-10-01|1996-09-18|TAKE BACK RETURN|SHIP|e ironic, even excuses haggle |
+1381|144|1|1|47|49074.58|0.08|0.04|N|O|1998-09-22|1998-08-12|1998-10-12|DELIVER IN PERSON|AIR|ly ironic deposits|
+1381|34|10|2|12|11208.36|0.07|0.08|N|O|1998-08-13|1998-08-12|1998-08-28|TAKE BACK RETURN|AIR| furiously regular package|
+1382|162|3|1|18|19118.88|0.08|0.03|R|F|1993-08-30|1993-10-19|1993-09-03|DELIVER IN PERSON|AIR|hely regular deposits. fluffy s|
+1382|181|2|2|29|31354.22|0.08|0.04|A|F|1993-10-08|1993-11-11|1993-10-10|COLLECT COD|FOB| haggle: closely even asymptot|
+1382|178|7|3|43|46361.31|0.10|0.04|A|F|1993-09-02|1993-10-06|1993-09-15|DELIVER IN PERSON|AIR|ress deposits. slyly ironic foxes are blit|
+1382|181|2|4|11|11892.98|0.04|0.04|R|F|1993-09-17|1993-09-29|1993-09-21|NONE|SHIP|furiously unusual packages play quickly |
+1382|157|8|5|31|32771.65|0.07|0.03|R|F|1993-10-26|1993-10-15|1993-11-09|TAKE BACK RETURN|FOB|hely regular dependencies. f|
+1382|10|5|6|38|34580.38|0.07|0.07|R|F|1993-11-17|1993-09-28|1993-11-20|COLLECT COD|SHIP|ake pending pinto beans. s|
+1382|23|4|7|5|4615.10|0.07|0.01|R|F|1993-10-02|1993-09-29|1993-10-12|DELIVER IN PERSON|REG AIR|ter the carefully final excuses. blit|
+1383|193|7|1|14|15304.66|0.07|0.06|A|F|1993-08-25|1993-07-09|1993-09-12|DELIVER IN PERSON|RAIL|ole carefully silent requests. car|
+1383|161|10|2|19|20162.04|0.06|0.04|R|F|1993-05-24|1993-07-07|1993-06-14|NONE|AIR|lyly unusual accounts sle|
+1408|148|7|1|29|30396.06|0.03|0.04|N|O|1998-03-12|1998-02-14|1998-03-17|COLLECT COD|MAIL|en accounts grow furiousl|
+1408|173|2|2|7|7512.19|0.05|0.06|N|O|1998-01-14|1998-03-21|1998-01-29|COLLECT COD|AIR|fully final instructions. theodolites ca|
+1408|76|6|3|11|10736.77|0.00|0.03|N|O|1998-04-04|1998-01-29|1998-04-18|NONE|REG AIR|y even accounts thrash care|
+1408|148|5|4|20|20962.80|0.06|0.00|N|O|1998-04-21|1998-01-25|1998-05-12|DELIVER IN PERSON|TRUCK| blithely fluffi|
+1408|170|1|5|41|43876.97|0.02|0.06|N|O|1998-02-25|1998-02-03|1998-03-13|COLLECT COD|REG AIR|ep along the fina|
+1408|134|10|6|42|43433.46|0.05|0.08|N|O|1998-01-30|1998-02-07|1998-02-18|TAKE BACK RETURN|REG AIR|even packages. even accounts cajole|
+1408|55|6|7|26|24831.30|0.00|0.00|N|O|1998-03-19|1998-03-14|1998-04-01|COLLECT COD|RAIL|ic foxes ca|
+1409|99|1|1|23|22979.07|0.01|0.03|A|F|1993-04-18|1993-02-25|1993-05-06|DELIVER IN PERSON|FOB|ions. slyly ironic packages wake quick|
+1409|65|2|2|36|34742.16|0.09|0.02|A|F|1993-01-27|1993-01-31|1993-02-07|COLLECT COD|FOB|ncies sleep carefully r|
+1409|160|1|3|17|18022.72|0.07|0.00|R|F|1993-04-15|1993-03-01|1993-04-29|NONE|REG AIR|pending accounts poach. care|
+1410|121|10|1|15|15316.80|0.06|0.05|N|O|1997-05-25|1997-07-08|1997-06-15|NONE|SHIP| bold packages are fluf|
+1410|179|9|2|18|19425.06|0.03|0.00|N|O|1997-06-03|1997-05-17|1997-06-07|TAKE BACK RETURN|RAIL|gle furiously fluffily regular requests|
+1410|109|4|3|37|37336.70|0.02|0.01|N|O|1997-04-17|1997-06-18|1997-04-19|COLLECT COD|TRUCK|to beans b|
+1410|188|9|4|22|23939.96|0.10|0.00|N|O|1997-07-31|1997-05-17|1997-08-19|TAKE BACK RETURN|RAIL|gular account|
+1410|66|1|5|25|24151.50|0.09|0.02|N|O|1997-05-07|1997-07-10|1997-05-16|NONE|REG AIR|unts haggle against the furiously fina|
+1411|17|7|1|9|8253.09|0.06|0.04|A|F|1995-03-08|1995-03-04|1995-03-11|DELIVER IN PERSON|AIR|accounts. furiou|
+1411|107|8|2|26|26184.60|0.02|0.02|A|F|1995-04-12|1995-01-24|1995-05-03|TAKE BACK RETURN|TRUCK|c packages. |
+1411|27|6|3|37|34299.74|0.00|0.06|A|F|1995-02-27|1995-03-02|1995-03-24|NONE|MAIL|d excuses. furiously final pear|
+1411|200|3|4|20|22004.00|0.01|0.03|R|F|1995-04-06|1995-03-16|1995-04-17|COLLECT COD|FOB|s against the|
+1411|83|4|5|46|45221.68|0.08|0.05|A|F|1995-04-03|1995-01-20|1995-04-05|DELIVER IN PERSON|REG AIR|ly daring instructions|
+1411|77|6|6|30|29312.10|0.09|0.04|A|F|1995-01-12|1995-02-01|1995-01-23|DELIVER IN PERSON|MAIL|ious foxes wake courts. caref|
+1412|58|3|1|37|35447.85|0.06|0.01|A|F|1993-04-10|1993-04-19|1993-04-12|DELIVER IN PERSON|RAIL|hely express excuses are |
+1412|156|1|2|20|21123.00|0.10|0.05|A|F|1993-07-04|1993-05-18|1993-07-22|DELIVER IN PERSON|REG AIR|odolites sleep ironically|
+1412|23|2|3|2|1846.04|0.10|0.07|R|F|1993-04-01|1993-05-03|1993-04-12|DELIVER IN PERSON|REG AIR|s among the requests are a|
+1412|167|8|4|11|11738.76|0.05|0.07|R|F|1993-05-27|1993-05-30|1993-06-07|DELIVER IN PERSON|MAIL|en packages. regular packages dete|
+1412|158|6|5|11|11639.65|0.08|0.06|A|F|1993-03-30|1993-05-25|1993-04-21|NONE|FOB|se slyly. special, unusual accounts nag bl|
+1413|178|9|1|18|19407.06|0.08|0.05|N|O|1997-10-11|1997-08-17|1997-10-25|NONE|FOB|yly bold packages haggle quickly acr|
+1413|165|10|2|49|52192.84|0.07|0.06|N|O|1997-08-28|1997-08-23|1997-09-12|DELIVER IN PERSON|MAIL|nstructions br|
+1413|42|9|3|6|5652.24|0.04|0.02|N|O|1997-09-07|1997-07-30|1997-09-21|TAKE BACK RETURN|MAIL|lithely excuses. f|
+1414|38|4|1|39|36583.17|0.10|0.03|N|O|1995-09-22|1995-09-30|1995-10-07|NONE|MAIL|quickly aro|
+1414|107|8|2|4|4028.40|0.02|0.05|N|O|1995-09-16|1995-11-01|1995-10-02|COLLECT COD|AIR| haggle quickly|
+1415|149|10|1|25|26228.50|0.06|0.00|A|F|1994-09-03|1994-07-12|1994-09-13|DELIVER IN PERSON|RAIL|ect never fluff|
+1440|193|6|1|3|3279.57|0.06|0.01|N|O|1995-10-30|1995-10-17|1995-11-08|COLLECT COD|SHIP|instructions boost. fluffily regul|
+1440|114|4|2|46|46649.06|0.02|0.03|N|O|1995-09-21|1995-10-19|1995-10-19|NONE|RAIL|blithely even instructions. |
+1441|144|7|1|5|5220.70|0.04|0.01|N|O|1997-05-17|1997-05-11|1997-05-30|COLLECT COD|MAIL|egular courts. fluffily even grouches |
+1441|177|7|2|5|5385.85|0.02|0.05|N|O|1997-04-25|1997-04-16|1997-05-23|COLLECT COD|FOB|he quickly enticing pac|
+1441|118|5|3|14|14253.54|0.01|0.03|N|O|1997-06-30|1997-04-29|1997-07-24|DELIVER IN PERSON|REG AIR|special requests ha|
+1441|160|8|4|37|39225.92|0.01|0.00|N|O|1997-04-26|1997-04-27|1997-04-29|NONE|REG AIR|accounts. slyly special dolphins b|
+1441|72|10|5|34|33050.38|0.09|0.00|N|O|1997-06-12|1997-05-11|1997-06-29|TAKE BACK RETURN|RAIL|e carefully. blithely ironic dep|
+1441|25|4|6|15|13875.30|0.09|0.08|N|O|1997-05-21|1997-05-06|1997-06-04|NONE|REG AIR| dependencies-- cour|
+1441|96|10|7|50|49804.50|0.03|0.01|N|O|1997-06-07|1997-05-12|1997-06-08|NONE|SHIP| requests. blithely e|
+1442|26|5|1|8|7408.16|0.05|0.01|A|F|1994-10-31|1994-09-04|1994-11-25|COLLECT COD|AIR|c deposits haggle after the even|
+1443|34|10|1|47|43899.41|0.04|0.06|N|O|1997-02-05|1997-02-02|1997-03-03|NONE|RAIL|carefully ironic requests sl|
+1444|170|5|1|42|44947.14|0.01|0.02|R|F|1994-12-22|1995-03-03|1994-12-31|NONE|SHIP|ly bold packages boost regular ideas. spe|
+1444|57|2|2|34|32539.70|0.04|0.08|A|F|1995-02-22|1995-02-15|1995-03-19|TAKE BACK RETURN|AIR|y. doggedly pend|
+1444|155|3|3|34|35875.10|0.02|0.07|R|F|1994-12-17|1995-01-12|1995-01-03|COLLECT COD|AIR|ular accounts |
+1444|119|6|4|6|6114.66|0.06|0.03|A|F|1995-01-07|1995-03-05|1995-01-17|COLLECT COD|RAIL|al accounts. br|
+1444|20|1|5|35|32200.70|0.02|0.05|A|F|1995-02-25|1995-03-05|1995-03-24|DELIVER IN PERSON|SHIP|aggle furiou|
+1444|33|4|6|42|39187.26|0.00|0.02|A|F|1994-12-16|1995-02-18|1994-12-22|DELIVER IN PERSON|RAIL|ss requests. ironic ideas wake above|
+1444|82|3|7|12|11784.96|0.00|0.03|R|F|1994-12-23|1995-01-15|1995-01-13|COLLECT COD|TRUCK|ly among the bol|
+1445|100|1|1|24|24002.40|0.01|0.00|A|F|1995-02-21|1995-02-22|1995-03-18|DELIVER IN PERSON|SHIP|al accounts use furiously a|
+1445|67|8|2|48|46418.88|0.10|0.02|A|F|1995-02-28|1995-03-16|1995-03-12|COLLECT COD|MAIL|. final ideas are carefully dar|
+1445|192|4|3|7|7645.33|0.10|0.04|A|F|1995-04-25|1995-02-25|1995-05-10|NONE|SHIP|structions: slyly regular re|
+1445|28|1|4|17|15776.34|0.04|0.07|A|F|1995-04-02|1995-04-04|1995-05-01|COLLECT COD|FOB|ges. furiously regular pint|
+1445|135|1|5|24|24843.12|0.10|0.06|R|F|1995-04-23|1995-02-16|1995-05-18|NONE|REG AIR|rate after the carefully reg|
+1445|168|9|6|39|41658.24|0.03|0.02|A|F|1995-02-05|1995-02-20|1995-02-06|NONE|MAIL|ully unusual reques|
+1446|72|3|1|31|30134.17|0.10|0.02|N|O|1998-05-01|1998-05-17|1998-05-30|NONE|REG AIR|. slyly reg|
+1447|167|4|1|19|20276.04|0.06|0.04|A|F|1993-01-31|1992-12-07|1993-02-04|COLLECT COD|MAIL|. quickly ironic |
+1447|32|3|2|6|5592.18|0.01|0.05|A|F|1992-10-24|1992-12-10|1992-11-05|DELIVER IN PERSON|AIR|as! regular packages poach above the|
+1447|39|5|3|9|8451.27|0.04|0.00|R|F|1992-11-15|1993-01-07|1992-11-29|DELIVER IN PERSON|MAIL|counts wake s|
+1447|22|5|4|8|7376.16|0.09|0.08|R|F|1992-11-20|1993-01-12|1992-12-14|COLLECT COD|FOB|ost carefully |
+1447|130|1|5|23|23692.99|0.02|0.07|A|F|1992-12-07|1992-12-25|1993-01-06|TAKE BACK RETURN|AIR| dazzle quickly deposits. f|
+1447|200|3|6|41|45108.20|0.08|0.02|R|F|1993-01-06|1993-01-05|1993-01-13|TAKE BACK RETURN|MAIL|rts boost s|
+1472|8|5|1|36|32688.00|0.04|0.05|N|O|1996-11-06|1996-11-13|1996-11-12|COLLECT COD|SHIP|riously silent deposits to the pending d|
+1472|133|4|2|26|26861.38|0.03|0.05|N|O|1996-11-08|1996-11-13|1996-12-02|DELIVER IN PERSON|FOB|ic packages w|
+1472|1|8|3|6|5406.00|0.08|0.01|N|O|1996-10-24|1996-11-19|1996-11-23|COLLECT COD|FOB|onic theodolites hinder slyly slyly r|
+1473|54|9|1|50|47702.50|0.04|0.03|N|O|1997-05-05|1997-05-20|1997-05-09|NONE|TRUCK|requests wake express deposits. special, ir|
+1473|68|3|2|32|30977.92|0.00|0.08|N|O|1997-04-18|1997-05-12|1997-05-10|DELIVER IN PERSON|REG AIR|out the packages lose furiously ab|
+1474|15|5|1|5|4575.05|0.05|0.04|A|F|1995-04-22|1995-02-20|1995-05-06|COLLECT COD|SHIP|ully final a|
+1474|123|8|2|30|30693.60|0.04|0.02|A|F|1995-03-23|1995-02-11|1995-04-17|DELIVER IN PERSON|TRUCK|usly. evenly express |
+1474|92|5|3|18|17857.62|0.06|0.02|A|F|1995-01-23|1995-03-28|1995-02-03|NONE|RAIL|after the special|
+1475|168|3|1|15|16022.40|0.08|0.06|N|O|1998-02-12|1997-12-17|1998-03-02|TAKE BACK RETURN|SHIP|xpress requests haggle after the final, fi|
+1475|118|9|2|18|18325.98|0.07|0.00|N|O|1998-03-08|1998-01-18|1998-03-10|TAKE BACK RETURN|AIR|al deposits use. ironic packages along the |
+1475|144|1|3|30|31324.20|0.03|0.02|N|O|1998-03-11|1997-12-30|1998-03-15|COLLECT COD|REG AIR| regular theodolites mold across th|
+1475|187|8|4|50|54359.00|0.03|0.05|N|O|1997-12-14|1997-12-13|1997-12-21|COLLECT COD|AIR|. slyly bold re|
+1475|32|3|5|33|30756.99|0.01|0.06|N|O|1998-01-02|1998-01-27|1998-01-11|NONE|FOB|quickly fluffy|
+1475|50|7|6|12|11400.60|0.04|0.04|N|O|1998-01-09|1997-12-30|1998-01-23|NONE|TRUCK|arefully-- excuses sublate|
+1475|112|3|7|23|23278.53|0.02|0.00|N|O|1998-02-13|1998-02-05|1998-03-08|NONE|TRUCK|hely regular hocke|
+1476|31|7|1|20|18620.60|0.02|0.03|N|O|1996-08-11|1996-09-18|1996-08-26|TAKE BACK RETURN|AIR|. bold deposits are carefully amo|
+1477|72|1|1|31|30134.17|0.00|0.06|N|O|1997-12-16|1997-09-30|1997-12-17|COLLECT COD|RAIL| requests. fluffily final |
+1477|110|7|2|8|8080.88|0.09|0.05|N|O|1997-10-25|1997-10-18|1997-11-16|COLLECT COD|MAIL|ironic realms wake unusual, even ac|
+1477|125|6|3|42|43055.04|0.06|0.00|N|O|1997-11-02|1997-11-02|1997-11-20|DELIVER IN PERSON|SHIP|lithely after the ir|
+1477|107|8|4|32|32227.20|0.05|0.08|N|O|1997-09-12|1997-10-26|1997-10-12|TAKE BACK RETURN|AIR|; quickly regula|
+1477|115|6|5|41|41619.51|0.04|0.06|N|O|1997-12-16|1997-10-31|1998-01-12|DELIVER IN PERSON|REG AIR|y. final pearls kindle. accounts |
+1477|69|6|6|49|47483.94|0.06|0.00|N|O|1997-11-18|1997-11-06|1997-11-27|COLLECT COD|FOB|ise according to the sly, bold p|
+1477|120|4|7|33|33663.96|0.06|0.00|N|O|1997-11-12|1997-11-06|1997-11-24|DELIVER IN PERSON|TRUCK|yly regular p|
+1478|34|5|1|21|19614.63|0.00|0.06|N|O|1997-09-20|1997-10-25|1997-10-06|TAKE BACK RETURN|MAIL| fluffily pending acc|
+1479|149|6|1|33|34621.62|0.10|0.01|N|O|1996-03-12|1996-02-28|1996-03-31|DELIVER IN PERSON|FOB| carefully special courts affix. fluff|
+1504|82|3|1|42|41247.36|0.02|0.03|R|F|1992-10-18|1992-10-14|1992-11-10|TAKE BACK RETURN|FOB|ep. carefully ironic excuses haggle quickl|
+1504|103|10|2|22|22068.20|0.04|0.03|A|F|1992-09-09|1992-10-29|1992-09-10|NONE|REG AIR| accounts sleep. furiou|
+1504|178|8|3|9|9703.53|0.07|0.02|R|F|1992-11-02|1992-10-12|1992-11-15|TAKE BACK RETURN|RAIL|y slyly regular courts.|
+1504|115|2|4|10|10151.10|0.04|0.07|A|F|1992-09-22|1992-10-22|1992-10-13|TAKE BACK RETURN|TRUCK|final theodolites. furiously e|
+1504|20|10|5|7|6440.14|0.02|0.00|R|F|1992-11-20|1992-11-23|1992-12-13|COLLECT COD|MAIL|y final packa|
+1505|120|7|1|4|4080.48|0.09|0.00|A|F|1992-12-14|1992-11-11|1993-01-02|COLLECT COD|SHIP|side of the s|
+1505|123|8|2|50|51156.00|0.00|0.02|R|F|1992-11-22|1992-09-24|1992-11-26|TAKE BACK RETURN|FOB|lyly special platelets. requests ar|
+1506|133|4|1|46|47523.98|0.04|0.05|R|F|1993-01-18|1992-11-11|1993-02-09|COLLECT COD|REG AIR|sits whithout the blithely ironic packages|
+1506|114|4|2|30|30423.30|0.07|0.02|A|F|1992-11-22|1992-10-25|1992-12-04|DELIVER IN PERSON|FOB|deposits cajole |
+1506|191|3|3|28|30553.32|0.10|0.06|A|F|1992-09-22|1992-11-19|1992-10-09|TAKE BACK RETURN|AIR| unwind carefully: theodolit|
+1506|28|7|4|37|34336.74|0.00|0.03|R|F|1992-11-04|1992-12-01|1992-11-23|TAKE BACK RETURN|TRUCK|carefully bold dolphins. accounts su|
+1506|195|8|5|15|16427.85|0.05|0.00|R|F|1992-09-24|1992-11-11|1992-10-05|NONE|REG AIR| carefully fluffy packages-- caref|
+1506|50|3|6|38|36101.90|0.05|0.02|R|F|1992-12-02|1992-12-19|1992-12-29|NONE|REG AIR|xpress, regular excuse|
+1506|169|6|7|4|4276.64|0.07|0.00|R|F|1993-01-03|1992-12-06|1993-01-05|COLLECT COD|REG AIR|posits. furiou|
+1507|68|5|1|25|24201.50|0.01|0.08|R|F|1994-01-07|1994-01-06|1994-01-11|NONE|RAIL|xes. slyly busy de|
+1507|40|6|2|33|31021.32|0.04|0.02|A|F|1993-10-29|1993-12-23|1993-11-14|DELIVER IN PERSON|REG AIR| asymptotes nag furiously above t|
+1507|86|7|3|39|38457.12|0.03|0.07|R|F|1993-11-04|1993-12-16|1993-12-03|TAKE BACK RETURN|REG AIR|ly even instructions.|
+1508|51|3|1|16|15216.80|0.02|0.06|N|O|1998-06-21|1998-05-30|1998-07-11|COLLECT COD|MAIL|riously across the ironic, unusua|
+1508|25|4|2|20|18500.40|0.06|0.01|N|O|1998-04-17|1998-06-11|1998-05-17|DELIVER IN PERSON|MAIL|nic platelets. carefully final fra|
+1508|93|7|3|43|42702.87|0.01|0.02|N|O|1998-06-01|1998-06-24|1998-06-03|TAKE BACK RETURN|TRUCK|ndencies h|
+1508|148|7|4|1|1048.14|0.02|0.02|N|O|1998-07-13|1998-06-03|1998-07-17|TAKE BACK RETURN|AIR|s the blithely bold instruction|
+1508|135|6|5|29|30018.77|0.02|0.00|N|O|1998-08-03|1998-07-08|1998-08-22|COLLECT COD|RAIL|r instructions. carefully|
+1508|3|10|6|5|4515.00|0.06|0.08|N|O|1998-05-22|1998-07-06|1998-06-04|COLLECT COD|REG AIR|cording to the furiously ironic depe|
+1508|117|8|7|38|38650.18|0.03|0.06|N|O|1998-04-30|1998-06-23|1998-05-18|DELIVER IN PERSON|RAIL|tes wake furiously regular w|
+1509|28|7|1|14|12992.28|0.04|0.01|A|F|1993-10-04|1993-09-25|1993-10-21|NONE|TRUCK|nal realms|
+1509|11|2|2|46|41906.46|0.08|0.02|A|F|1993-10-15|1993-10-04|1993-11-01|TAKE BACK RETURN|FOB|uriously regula|
+1509|107|8|3|17|17120.70|0.06|0.05|A|F|1993-07-25|1993-08-28|1993-08-19|DELIVER IN PERSON|AIR| furiously. blithely regular ideas haggle c|
+1509|20|4|4|11|10120.22|0.03|0.08|R|F|1993-11-04|1993-10-03|1993-11-14|TAKE BACK RETURN|FOB|ily ironic packages nod carefully.|
+1509|90|1|5|37|36633.33|0.01|0.08|A|F|1993-08-31|1993-09-10|1993-09-24|NONE|FOB|he slyly even deposits wake a|
+1509|187|8|6|31|33702.58|0.04|0.03|A|F|1993-07-14|1993-08-21|1993-08-06|COLLECT COD|SHIP|ic deposits cajole carefully. quickly bold |
+1509|157|2|7|27|28543.05|0.01|0.01|A|F|1993-09-29|1993-09-08|1993-10-04|TAKE BACK RETURN|FOB|lithely after the |
+1510|98|2|1|11|10978.99|0.09|0.04|N|O|1996-09-23|1996-12-03|1996-10-01|DELIVER IN PERSON|RAIL|e of the unusual accounts. stealthy deposit|
+1510|84|5|2|24|23617.92|0.05|0.04|N|O|1996-10-07|1996-10-22|1996-11-03|DELIVER IN PERSON|REG AIR|yly brave theod|
+1510|190|1|3|36|39246.84|0.07|0.02|N|O|1996-10-02|1996-11-23|1996-10-05|NONE|SHIP|old deposits along the carefully|
+1510|182|3|4|8|8657.44|0.01|0.08|N|O|1996-10-26|1996-11-07|1996-10-30|TAKE BACK RETURN|RAIL|blithely express|
+1510|59|10|5|27|25894.35|0.08|0.06|N|O|1996-10-20|1996-12-05|1996-11-02|NONE|MAIL|he blithely regular req|
+1510|14|5|6|3|2742.03|0.05|0.02|N|O|1996-10-31|1996-12-03|1996-11-13|COLLECT COD|RAIL|along the slyly regular pin|
+1510|22|1|7|50|46101.00|0.04|0.05|N|O|1996-11-01|1996-10-17|1996-11-28|NONE|MAIL|even packages. carefully regular fo|
+1511|98|2|1|29|28944.61|0.01|0.04|N|O|1997-03-17|1997-02-11|1997-03-27|DELIVER IN PERSON|AIR|s cajole furiously against |
+1511|62|9|2|32|30785.92|0.04|0.01|N|O|1997-01-06|1997-03-21|1997-01-26|TAKE BACK RETURN|REG AIR| deposits. carefully ironi|
+1536|194|5|1|5|5470.95|0.08|0.03|N|O|1997-02-08|1997-03-11|1997-03-02|COLLECT COD|MAIL|requests sleep pe|
+1537|18|2|1|17|15606.17|0.01|0.03|A|F|1992-04-12|1992-04-19|1992-04-13|NONE|TRUCK|he regular pack|
+1537|179|8|2|50|53958.50|0.08|0.00|R|F|1992-05-30|1992-05-14|1992-06-23|TAKE BACK RETURN|MAIL|special packages haggle slyly at the silent|
+1537|13|4|3|44|40172.44|0.05|0.04|R|F|1992-04-01|1992-03-31|1992-04-21|NONE|TRUCK|lar courts.|
+1537|140|6|4|3|3120.42|0.08|0.07|R|F|1992-03-20|1992-04-14|1992-03-21|TAKE BACK RETURN|SHIP|s, final ideas detect sl|
+1538|102|5|1|32|32067.20|0.05|0.05|N|O|1995-07-08|1995-07-29|1995-08-01|TAKE BACK RETURN|RAIL|uses maintain blithely. fluffily|
+1538|192|3|2|27|29489.13|0.05|0.01|N|O|1995-09-19|1995-08-03|1995-09-24|DELIVER IN PERSON|TRUCK|ngly even packag|
+1538|130|3|3|36|37084.68|0.08|0.04|N|O|1995-07-11|1995-09-10|1995-07-26|DELIVER IN PERSON|MAIL|al deposits mo|
+1538|104|1|4|28|28114.80|0.10|0.04|N|O|1995-09-19|1995-08-27|1995-10-10|COLLECT COD|RAIL|bout the fluffily unusual|
+1538|178|7|5|13|14016.21|0.01|0.05|N|O|1995-06-26|1995-07-30|1995-07-25|NONE|SHIP|ly. packages sleep f|
+1538|128|3|6|42|43181.04|0.08|0.08|N|O|1995-10-10|1995-09-12|1995-11-08|DELIVER IN PERSON|TRUCK|equests cajole blithely |
+1539|196|9|1|21|23019.99|0.08|0.02|R|F|1995-04-19|1995-05-10|1995-04-27|COLLECT COD|TRUCK|ounts haggle. busy|
+1539|86|7|2|11|10846.88|0.01|0.08|A|F|1995-05-27|1995-04-13|1995-06-10|TAKE BACK RETURN|TRUCK|ly express requests. furiously |
+1539|68|5|3|7|6776.42|0.09|0.04|R|F|1995-05-14|1995-04-16|1995-05-30|DELIVER IN PERSON|AIR|. fluffily reg|
+1540|173|1|1|38|40780.46|0.03|0.01|R|F|1992-09-30|1992-10-27|1992-10-12|TAKE BACK RETURN|SHIP| final grouches bo|
+1540|60|2|2|35|33602.10|0.02|0.07|R|F|1992-10-31|1992-09-04|1992-11-05|TAKE BACK RETURN|SHIP|e blithely a|
+1540|8|3|3|25|22700.00|0.08|0.04|R|F|1992-11-15|1992-10-24|1992-12-14|DELIVER IN PERSON|SHIP|ironic deposits amo|
+1540|25|8|4|6|5550.12|0.09|0.03|R|F|1992-08-28|1992-09-17|1992-09-14|COLLECT COD|MAIL|ing to the slyly express asymptote|
+1540|87|8|5|27|26651.16|0.10|0.08|R|F|1992-12-02|1992-10-18|1992-12-31|NONE|SHIP|carefully final packages; b|
+1541|64|3|1|44|42418.64|0.10|0.05|N|O|1995-08-24|1995-07-13|1995-08-26|TAKE BACK RETURN|MAIL|o beans boost fluffily abou|
+1541|26|7|2|8|7408.16|0.10|0.08|N|F|1995-06-05|1995-08-07|1995-06-21|TAKE BACK RETURN|TRUCK|y pending packages. blithely fi|
+1542|58|9|1|37|35447.85|0.07|0.06|A|F|1993-12-15|1993-10-17|1994-01-07|TAKE BACK RETURN|REG AIR|e blithely unusual accounts. quic|
+1542|3|6|2|12|10836.00|0.09|0.06|R|F|1993-10-29|1993-11-02|1993-11-09|TAKE BACK RETURN|RAIL|carefully |
+1542|6|7|3|18|16308.00|0.05|0.05|R|F|1993-10-17|1993-11-15|1993-10-26|TAKE BACK RETURN|FOB|pending instr|
+1542|143|10|4|21|21905.94|0.01|0.05|R|F|1993-10-13|1993-12-13|1993-11-12|NONE|RAIL|y pending foxes nag blithely |
+1542|155|7|5|46|48536.90|0.00|0.00|R|F|1993-09-28|1993-11-03|1993-10-15|COLLECT COD|FOB|ial instructions. ironically|
+1543|71|10|1|34|33016.38|0.02|0.08|N|O|1997-05-25|1997-03-30|1997-06-04|NONE|AIR|ic requests are ac|
+1543|115|9|2|6|6090.66|0.09|0.01|N|O|1997-04-16|1997-05-20|1997-05-16|DELIVER IN PERSON|MAIL| among the carefully bold or|
+1543|67|8|3|42|40616.52|0.06|0.01|N|O|1997-05-26|1997-03-30|1997-06-12|DELIVER IN PERSON|FOB|its sleep until the fur|
+1543|189|10|4|42|45745.56|0.05|0.06|N|O|1997-04-11|1997-04-11|1997-04-23|TAKE BACK RETURN|MAIL|xpress instructions. regular acc|
+1543|40|1|5|9|8460.36|0.08|0.06|N|O|1997-03-14|1997-05-19|1997-03-26|DELIVER IN PERSON|FOB|ravely special requests |
+1543|49|8|6|3|2847.12|0.10|0.04|N|O|1997-03-29|1997-05-10|1997-04-22|COLLECT COD|MAIL|sleep along the furiou|
+1543|68|7|7|3|2904.18|0.00|0.02|N|O|1997-03-22|1997-04-06|1997-03-30|NONE|AIR|quickly. final accounts haggle slyl|
+1568|90|1|1|36|35643.24|0.02|0.03|N|O|1997-05-31|1997-04-22|1997-06-21|TAKE BACK RETURN|RAIL|platelets-- furiously sly excu|
+1568|9|2|2|46|41814.00|0.04|0.00|N|O|1997-04-06|1997-04-08|1997-04-23|TAKE BACK RETURN|MAIL|g the blithely even acco|
+1569|75|3|1|5|4875.35|0.07|0.00|N|O|1998-04-16|1998-06-21|1998-04-18|COLLECT COD|REG AIR| packages. ironic, even excuses a|
+1569|39|10|2|16|15024.48|0.01|0.08|N|O|1998-04-26|1998-06-16|1998-05-26|COLLECT COD|MAIL|deposits. blithely final asymptotes ac|
+1569|49|10|3|43|40808.72|0.10|0.03|N|O|1998-06-05|1998-05-31|1998-06-28|DELIVER IN PERSON|FOB| instructions.|
+1569|70|1|4|30|29102.10|0.02|0.03|N|O|1998-07-19|1998-06-04|1998-08-10|NONE|SHIP|packages. excuses lose evenly carefully reg|
+1570|183|4|1|25|27079.50|0.00|0.06|N|O|1998-05-03|1998-06-02|1998-06-02|DELIVER IN PERSON|REG AIR|its. slyly regular sentiments|
+1570|86|7|2|7|6902.56|0.05|0.05|N|O|1998-07-10|1998-06-01|1998-07-23|TAKE BACK RETURN|MAIL|requests boost quickly re|
+1571|52|3|1|47|44746.35|0.00|0.05|R|F|1992-12-07|1993-02-24|1993-01-01|TAKE BACK RETURN|REG AIR|ng to the fluffily unusual |
+1571|183|4|2|6|6499.08|0.03|0.00|A|F|1993-01-08|1993-02-13|1993-02-07|COLLECT COD|SHIP| special, ironic depo|
+1571|59|7|3|18|17262.90|0.05|0.08|A|F|1993-01-09|1993-01-12|1993-01-31|COLLECT COD|AIR| pending grouches |
+1571|101|4|4|48|48052.80|0.05|0.05|A|F|1992-12-28|1993-01-04|1993-01-04|DELIVER IN PERSON|RAIL|slyly pending p|
+1571|42|5|5|10|9420.40|0.03|0.06|R|F|1992-12-12|1993-02-13|1992-12-29|DELIVER IN PERSON|AIR|lets. carefully regular ideas wake|
+1571|34|10|6|24|22416.72|0.05|0.07|A|F|1993-03-22|1993-01-31|1993-04-09|NONE|TRUCK|warthogs wake carefully acro|
+1572|24|5|1|41|37884.82|0.02|0.00|N|O|1996-05-16|1996-04-09|1996-05-28|TAKE BACK RETURN|REG AIR|. pinto beans alongside|
+1572|93|7|2|10|9930.90|0.04|0.06|N|O|1996-05-17|1996-03-26|1996-05-19|NONE|AIR| accounts affix slyly. |
+1573|186|7|1|5|5430.90|0.05|0.01|A|F|1993-04-24|1993-03-13|1993-05-17|TAKE BACK RETURN|MAIL|ymptotes could u|
+1573|31|2|2|17|15827.51|0.00|0.06|R|F|1993-02-24|1993-02-16|1993-03-08|TAKE BACK RETURN|TRUCK|carefully regular deposits. |
+1573|83|4|3|16|15729.28|0.04|0.03|A|F|1993-03-15|1993-03-16|1993-03-31|COLLECT COD|AIR|ely. furiously final requests wake slyl|
+1573|194|7|4|11|12036.09|0.09|0.01|R|F|1993-03-23|1993-03-24|1993-04-12|TAKE BACK RETURN|RAIL|nently pending|
+1573|137|8|5|7|7259.91|0.00|0.01|R|F|1993-01-30|1993-03-14|1993-02-27|DELIVER IN PERSON|SHIP|eodolites sleep slyly. slyly f|
+1573|154|6|6|30|31624.50|0.03|0.01|A|F|1992-12-29|1993-03-06|1993-01-02|DELIVER IN PERSON|TRUCK|. blithely even theodolites boos|
+1574|48|7|1|41|38869.64|0.06|0.02|N|O|1997-03-08|1997-02-09|1997-04-01|COLLECT COD|AIR|s. slyly regular depen|
+1574|191|5|2|50|54559.50|0.00|0.05|N|O|1996-12-14|1997-02-14|1996-12-16|TAKE BACK RETURN|FOB|le regular, regular foxes. blithely e|
+1574|55|3|3|25|23876.25|0.06|0.02|N|O|1997-01-16|1997-02-14|1997-02-12|DELIVER IN PERSON|TRUCK|ly silent accounts.|
+1574|191|4|4|6|6547.14|0.03|0.05|N|O|1997-02-24|1997-02-03|1997-03-01|NONE|AIR|e silent, final packages. speci|
+1574|109|4|5|6|6054.60|0.05|0.05|N|O|1997-02-09|1997-03-02|1997-02-14|COLLECT COD|MAIL|nic, final ideas snooze. |
+1574|5|2|6|42|38010.00|0.07|0.01|N|O|1996-12-19|1997-01-13|1996-12-28|NONE|FOB|o beans according t|
+1574|136|7|7|14|14505.82|0.04|0.01|N|O|1996-12-30|1997-01-19|1997-01-20|NONE|AIR|ily bold a|
+1575|29|10|1|42|39018.84|0.05|0.08|N|O|1995-10-21|1995-11-25|1995-10-24|DELIVER IN PERSON|RAIL|ly pending pinto beans.|
+1575|36|7|2|39|36505.17|0.00|0.06|N|O|1995-10-30|1995-10-15|1995-11-10|COLLECT COD|TRUCK| ironic requests snooze ironic, regular acc|
+1575|2|5|3|12|10824.00|0.01|0.05|N|O|1995-12-27|1995-11-11|1996-01-23|TAKE BACK RETURN|AIR| bold accounts. furi|
+1575|111|1|4|39|39433.29|0.07|0.00|N|O|1995-09-23|1995-11-05|1995-09-25|TAKE BACK RETURN|TRUCK| after the unusual asym|
+1575|83|4|5|10|9830.80|0.09|0.00|N|O|1996-01-10|1995-11-20|1996-01-13|DELIVER IN PERSON|RAIL|k excuses. pinto beans wake a|
+1575|178|6|6|14|15094.38|0.08|0.06|N|O|1995-10-31|1995-12-06|1995-11-30|NONE|AIR|beans breach among the furiously specia|
+1575|117|1|7|48|48821.28|0.08|0.04|N|O|1995-11-19|1995-10-25|1995-12-07|DELIVER IN PERSON|TRUCK|cies. regu|
+1600|172|10|1|20|21443.40|0.02|0.01|R|F|1993-06-16|1993-04-23|1993-07-02|COLLECT COD|FOB|pths sleep blithely about the|
+1600|44|3|2|48|45313.92|0.07|0.02|R|F|1993-04-17|1993-04-14|1993-05-03|DELIVER IN PERSON|FOB|furiously silent foxes could wake. car|
+1600|39|10|3|8|7512.24|0.04|0.07|R|F|1993-03-07|1993-04-22|1993-03-26|TAKE BACK RETURN|FOB|cajole furiously fluf|
+1600|69|8|4|25|24226.50|0.00|0.06|A|F|1993-05-25|1993-04-07|1993-06-05|TAKE BACK RETURN|REG AIR|press packages. ironic excuses bo|
+1600|147|8|5|30|31414.20|0.03|0.08|R|F|1993-06-03|1993-05-03|1993-06-07|DELIVER IN PERSON|RAIL|al escapades alongside of the depo|
+1601|167|8|1|6|6402.96|0.00|0.00|A|F|1994-10-19|1994-09-28|1994-10-23|COLLECT COD|SHIP| bold sheaves. furiously per|
+1601|175|3|2|50|53758.50|0.03|0.02|R|F|1994-12-24|1994-10-23|1995-01-11|COLLECT COD|FOB|ideas doubt|
+1601|90|1|3|14|13861.26|0.04|0.08|R|F|1994-09-17|1994-11-22|1994-10-03|DELIVER IN PERSON|RAIL|he special, fin|
+1602|183|4|1|4|4332.72|0.08|0.06|R|F|1993-10-31|1993-09-05|1993-11-21|NONE|RAIL|y. even excuses|
+1603|39|5|1|1|939.03|0.08|0.00|R|F|1993-08-17|1993-09-04|1993-08-22|TAKE BACK RETURN|REG AIR|d accounts. special warthogs use fur|
+1603|66|5|2|29|28015.74|0.06|0.08|A|F|1993-09-28|1993-09-20|1993-10-28|NONE|SHIP|ses wake furiously. theodolite|
+1604|42|3|1|15|14130.60|0.09|0.08|R|F|1993-09-22|1993-09-03|1993-09-29|TAKE BACK RETURN|MAIL| instructions haggle|
+1604|141|4|2|37|38522.18|0.06|0.06|A|F|1993-08-22|1993-09-21|1993-09-10|COLLECT COD|SHIP|requests. blithely ironic somas s|
+1604|114|8|3|19|19268.09|0.09|0.07|A|F|1993-10-15|1993-10-04|1993-11-09|COLLECT COD|RAIL| ideas. bol|
+1604|175|4|4|15|16127.55|0.03|0.00|R|F|1993-09-10|1993-08-31|1993-09-30|TAKE BACK RETURN|RAIL|ending realms along the special, p|
+1604|21|4|5|23|21183.46|0.08|0.05|A|F|1993-10-11|1993-08-30|1993-10-18|DELIVER IN PERSON|RAIL|en requests. blithely fin|
+1605|142|1|1|47|48980.58|0.00|0.01|N|O|1998-04-29|1998-06-12|1998-05-20|DELIVER IN PERSON|AIR|. carefully r|
+1605|180|8|2|18|19443.24|0.10|0.00|N|O|1998-05-13|1998-06-17|1998-06-03|COLLECT COD|REG AIR|ly regular foxes wake carefully. bol|
+1605|59|10|3|39|37402.95|0.02|0.03|N|O|1998-07-12|1998-06-05|1998-08-09|DELIVER IN PERSON|MAIL|nal dependencies-- quickly final frets acc|
+1605|183|4|4|25|27079.50|0.06|0.02|N|O|1998-05-26|1998-06-14|1998-06-05|COLLECT COD|AIR|ole carefully car|
+1606|115|6|1|21|21317.31|0.04|0.00|N|O|1997-06-02|1997-07-02|1997-06-27|DELIVER IN PERSON|RAIL| pending theodolites prom|
+1606|174|3|2|35|37595.95|0.00|0.02|N|O|1997-06-20|1997-06-19|1997-06-22|COLLECT COD|TRUCK|carefully sil|
+1606|100|4|3|23|23002.30|0.00|0.06|N|O|1997-04-19|1997-06-26|1997-04-30|NONE|MAIL|ously final requests. slowly ironic ex|
+1606|97|9|4|20|19941.80|0.02|0.04|N|O|1997-05-01|1997-05-26|1997-05-28|TAKE BACK RETURN|TRUCK|fily carefu|
+1606|71|10|5|14|13594.98|0.10|0.01|N|O|1997-05-19|1997-07-05|1997-06-10|COLLECT COD|FOB|structions haggle f|
+1607|190|1|1|2|2180.38|0.02|0.00|N|O|1996-01-11|1996-02-15|1996-01-19|DELIVER IN PERSON|MAIL|packages haggle. regular requests boost s|
+1607|119|3|2|37|37707.07|0.05|0.02|N|O|1996-02-27|1996-02-18|1996-03-16|NONE|AIR|alongside |
+1607|123|4|3|39|39901.68|0.00|0.00|N|O|1996-02-01|1996-02-12|1996-02-16|NONE|FOB|uches cajole. accounts ar|
+1607|76|6|4|34|33186.38|0.05|0.06|N|O|1996-01-06|1996-02-24|1996-01-10|DELIVER IN PERSON|SHIP| quickly above the |
+1607|178|8|5|48|51752.16|0.00|0.05|N|O|1996-02-22|1996-02-13|1996-03-09|TAKE BACK RETURN|MAIL|ular forges. deposits a|
+1632|191|5|1|47|51285.93|0.08|0.00|N|O|1997-01-25|1997-02-09|1997-02-19|TAKE BACK RETURN|RAIL|g to the closely special no|
+1632|148|7|2|14|14673.96|0.08|0.05|N|O|1997-01-15|1997-02-25|1997-01-28|NONE|RAIL|oxes. deposits nag slyly along the slyly |
+1632|177|6|3|47|50626.99|0.03|0.04|N|O|1997-01-29|1997-03-03|1997-02-21|NONE|MAIL|sts. blithely regular |
+1632|57|9|4|33|31582.65|0.09|0.02|N|O|1997-04-01|1997-02-24|1997-04-29|TAKE BACK RETURN|REG AIR|ructions! slyly|
+1632|142|1|5|43|44812.02|0.10|0.03|N|O|1997-02-24|1997-02-19|1997-03-25|DELIVER IN PERSON|FOB|ts. blithe, bold ideas cajo|
+1633|178|7|1|35|37735.95|0.01|0.02|N|O|1996-01-09|1995-12-02|1996-01-21|COLLECT COD|REG AIR|ly against the dolph|
+1633|5|6|2|15|13575.00|0.00|0.05|N|O|1995-12-13|1995-11-13|1996-01-04|TAKE BACK RETURN|FOB|ges wake fluffil|
+1634|48|9|1|21|19908.84|0.00|0.00|N|O|1996-10-04|1996-10-22|1996-11-01|NONE|MAIL|counts alo|
+1634|172|3|2|44|47175.48|0.05|0.01|N|O|1996-09-17|1996-11-09|1996-10-03|COLLECT COD|SHIP|requests affix slyly. quickly even pack|
+1634|19|10|3|21|19299.21|0.06|0.07|N|O|1996-11-16|1996-10-21|1996-11-27|NONE|TRUCK|y along the excuses.|
+1634|68|3|4|17|16457.02|0.08|0.07|N|O|1996-10-29|1996-10-15|1996-11-02|TAKE BACK RETURN|SHIP|cial, bold platelets alongside of the f|
+1634|76|7|5|2|1952.14|0.07|0.04|N|O|1996-11-22|1996-10-28|1996-12-17|NONE|SHIP|ly. carefully regular asymptotes wake|
+1634|170|9|6|11|11771.87|0.01|0.08|N|O|1996-10-04|1996-12-06|1996-10-14|DELIVER IN PERSON|SHIP|final requests |
+1634|13|7|7|35|31955.35|0.06|0.02|N|O|1996-11-25|1996-11-25|1996-12-12|TAKE BACK RETURN|RAIL|cies. regular, special de|
+1635|71|1|1|3|2913.21|0.06|0.08|N|O|1997-03-13|1997-03-25|1997-03-27|COLLECT COD|FOB| quickly ironic r|
+1635|90|1|2|8|7920.72|0.04|0.05|N|O|1997-04-30|1997-04-21|1997-05-09|DELIVER IN PERSON|AIR|ravely carefully express |
+1635|114|5|3|20|20282.20|0.07|0.01|N|O|1997-05-19|1997-04-01|1997-06-17|TAKE BACK RETURN|FOB|oost according to the carefully even accou|
+1635|77|5|4|40|39082.80|0.01|0.04|N|O|1997-02-25|1997-03-20|1997-03-12|TAKE BACK RETURN|RAIL|uriously up the ironic deposits. slyly i|
+1636|85|6|1|2|1970.16|0.09|0.03|N|O|1997-09-26|1997-08-22|1997-10-05|NONE|TRUCK|nal foxes cajole above the blithely reg|
+1636|169|10|2|45|48112.20|0.03|0.01|N|O|1997-07-14|1997-08-08|1997-07-27|COLLECT COD|RAIL|ely express reque|
+1636|108|1|3|24|24194.40|0.07|0.08|N|O|1997-10-07|1997-08-12|1997-11-04|TAKE BACK RETURN|MAIL|e carefully unusual ideas are f|
+1636|153|1|4|43|45285.45|0.06|0.00|N|O|1997-08-23|1997-08-10|1997-09-17|NONE|REG AIR|blithely special r|
+1636|19|6|5|22|20218.22|0.05|0.02|N|O|1997-07-22|1997-08-18|1997-08-03|COLLECT COD|AIR|ular, regu|
+1636|63|2|6|34|32744.04|0.10|0.01|N|O|1997-08-11|1997-09-09|1997-08-23|NONE|TRUCK|ular depos|
+1636|114|1|7|7|7098.77|0.04|0.00|N|O|1997-07-28|1997-09-10|1997-07-31|NONE|MAIL|ronic instructions. final|
+1637|86|7|1|49|48317.92|0.02|0.03|N|F|1995-06-08|1995-04-19|1995-07-01|COLLECT COD|REG AIR|. blithely i|
+1637|73|2|2|1|973.07|0.10|0.02|A|F|1995-02-14|1995-03-26|1995-03-09|TAKE BACK RETURN|AIR|ly final pinto beans. furiously|
+1637|22|1|3|10|9220.20|0.02|0.05|R|F|1995-02-21|1995-03-17|1995-03-11|NONE|AIR|uriously? blithely even sauternes wake. |
+1637|93|5|4|42|41709.78|0.06|0.01|A|F|1995-03-18|1995-04-24|1995-03-31|COLLECT COD|SHIP|blithely a|
+1637|5|8|5|25|22625.00|0.05|0.00|R|F|1995-06-07|1995-03-26|1995-06-08|COLLECT COD|RAIL| haggle carefully silent accou|
+1637|109|4|6|38|38345.80|0.02|0.08|R|F|1995-03-20|1995-05-05|1995-04-14|DELIVER IN PERSON|SHIP|even, pending foxes nod regular|
+1637|52|10|7|21|19993.05|0.07|0.08|A|F|1995-04-30|1995-04-30|1995-05-05|COLLECT COD|SHIP|ly ironic theodolites use b|
+1638|6|7|1|46|41676.00|0.03|0.02|N|O|1997-10-16|1997-10-28|1997-11-09|COLLECT COD|MAIL|otes haggle before the slyly bold instructi|
+1638|149|10|2|30|31474.20|0.00|0.04|N|O|1997-12-05|1997-09-17|1997-12-06|NONE|REG AIR|s cajole boldly bold requests. closely |
+1638|31|7|3|5|4655.15|0.08|0.07|N|O|1997-10-15|1997-11-01|1997-11-08|DELIVER IN PERSON|FOB|xcuses sleep furiou|
+1638|56|8|4|19|18164.95|0.00|0.08|N|O|1997-10-15|1997-10-27|1997-11-03|DELIVER IN PERSON|MAIL| quickly expres|
+1638|143|6|5|25|26078.50|0.05|0.03|N|O|1997-10-06|1997-09-30|1997-11-02|DELIVER IN PERSON|REG AIR|gle final, ironic pinto beans. |
+1638|155|10|6|46|48536.90|0.07|0.08|N|O|1997-08-20|1997-10-10|1997-09-09|COLLECT COD|AIR|ckages are carefully even instru|
+1639|187|8|1|24|26092.32|0.07|0.00|N|O|1995-08-24|1995-10-06|1995-08-31|COLLECT COD|REG AIR| the regular packages. courts dou|
+1639|43|6|2|38|35835.52|0.01|0.04|N|O|1995-08-23|1995-11-09|1995-08-29|TAKE BACK RETURN|FOB|y regular packages. b|
+1639|171|10|3|41|43917.97|0.04|0.02|N|O|1995-12-19|1995-11-11|1996-01-12|DELIVER IN PERSON|FOB|structions w|
+1664|118|5|1|48|48869.28|0.04|0.02|N|O|1996-06-21|1996-05-01|1996-07-19|TAKE BACK RETURN|RAIL| use. ironic deposits integrate. slyly unu|
+1664|173|2|2|30|32195.10|0.06|0.05|N|O|1996-04-04|1996-05-04|1996-05-03|COLLECT COD|FOB|ess multip|
+1664|151|2|3|10|10511.50|0.00|0.06|N|O|1996-04-10|1996-05-13|1996-05-07|TAKE BACK RETURN|RAIL|instructions up the acc|
+1664|155|3|4|35|36930.25|0.00|0.04|N|O|1996-03-06|1996-05-16|1996-03-09|DELIVER IN PERSON|REG AIR|y regular ide|
+1664|57|8|5|9|8613.45|0.07|0.04|N|O|1996-04-15|1996-05-14|1996-05-11|DELIVER IN PERSON|TRUCK|ges. fluffil|
+1664|141|8|6|40|41645.60|0.09|0.07|N|O|1996-04-02|1996-04-22|1996-04-17|COLLECT COD|REG AIR|se blithely unusual pains. carefully|
+1665|47|6|1|4|3788.16|0.02|0.03|A|F|1994-09-01|1994-06-07|1994-09-12|DELIVER IN PERSON|TRUCK|ely final requests. requests|
+1665|78|6|2|1|978.07|0.03|0.05|R|F|1994-05-22|1994-07-06|1994-05-24|TAKE BACK RETURN|TRUCK|sly final p|
+1666|185|6|1|30|32555.40|0.04|0.03|N|O|1995-10-28|1995-11-30|1995-11-18|TAKE BACK RETURN|AIR| breach evenly final accounts. r|
+1666|64|1|2|20|19281.20|0.01|0.00|N|O|1996-01-27|1995-12-12|1996-01-31|NONE|REG AIR|uietly regular foxes wake quick|
+1666|134|10|3|31|32058.03|0.05|0.07|N|O|1996-02-11|1996-01-11|1996-02-28|COLLECT COD|RAIL|ding to the express, bold accounts. fu|
+1666|169|8|4|41|43835.56|0.06|0.08|N|O|1995-11-29|1996-01-04|1995-12-24|NONE|TRUCK|ly regular excuses; regular ac|
+1667|21|4|1|6|5526.12|0.04|0.02|N|O|1997-12-07|1997-11-16|1998-01-02|COLLECT COD|FOB|riously busy requests. blithely final a|
+1667|22|1|2|29|26738.58|0.06|0.07|N|O|1997-10-15|1997-11-09|1997-11-11|TAKE BACK RETURN|MAIL|l accounts. furiously final courts h|
+1667|95|8|3|48|47764.32|0.05|0.01|N|O|1998-01-27|1998-01-06|1998-02-09|TAKE BACK RETURN|SHIP|tes sleep furiously. carefully eve|
+1667|59|1|4|24|23017.20|0.04|0.01|N|O|1997-10-14|1997-12-01|1997-11-09|TAKE BACK RETURN|MAIL|hrash final requests. care|
+1667|195|9|5|2|2190.38|0.07|0.00|N|O|1997-12-17|1997-11-22|1998-01-16|NONE|SHIP|pecial requests hag|
+1667|48|7|6|6|5688.24|0.01|0.03|N|O|1998-01-21|1997-12-19|1998-01-28|NONE|TRUCK| nag quickly above th|
+1667|40|6|7|19|17860.76|0.09|0.03|N|O|1998-01-23|1997-11-24|1998-01-26|DELIVER IN PERSON|SHIP|around the pinto beans. express, special|
+1668|132|8|1|8|8257.04|0.06|0.01|N|O|1997-07-23|1997-10-09|1997-08-06|DELIVER IN PERSON|FOB|arefully regular tithes! slyl|
+1668|1|8|2|25|22525.00|0.01|0.06|N|O|1997-08-08|1997-09-28|1997-09-01|NONE|TRUCK|y ironic requests. bold, final ideas a|
+1668|75|5|3|42|40952.94|0.08|0.01|N|O|1997-08-09|1997-09-08|1997-08-31|NONE|FOB|ole carefully excuses. final|
+1668|191|5|4|9|9820.71|0.05|0.03|N|O|1997-10-17|1997-09-05|1997-11-01|COLLECT COD|RAIL|wake furiously even instructions. sil|
+1668|128|9|5|25|25703.00|0.01|0.02|N|O|1997-10-08|1997-09-20|1997-10-11|DELIVER IN PERSON|REG AIR|even platelets across the silent |
+1668|10|3|6|38|34580.38|0.07|0.01|N|O|1997-08-26|1997-09-17|1997-09-05|DELIVER IN PERSON|TRUCK|ep slyly across the furi|
+1669|79|10|1|24|23497.68|0.04|0.08|N|O|1997-09-04|1997-07-30|1997-09-20|DELIVER IN PERSON|RAIL| regular, final deposits use quick|
+1670|32|3|1|41|38213.23|0.07|0.01|N|O|1997-07-19|1997-08-20|1997-07-23|DELIVER IN PERSON|TRUCK|thely according to the sly|
+1670|122|3|2|10|10221.20|0.07|0.03|N|O|1997-09-14|1997-08-16|1997-09-23|NONE|SHIP|fily special ideas |
+1670|186|7|3|41|44533.38|0.07|0.07|N|O|1997-07-19|1997-08-05|1997-07-26|COLLECT COD|SHIP|al gifts. speci|
+1671|149|2|1|21|22031.94|0.02|0.07|N|O|1996-07-28|1996-09-28|1996-08-08|TAKE BACK RETURN|AIR|s accounts slee|
+1671|96|10|2|4|3984.36|0.05|0.00|N|O|1996-08-30|1996-09-19|1996-09-23|DELIVER IN PERSON|TRUCK|lyly regular ac|
+1671|124|3|3|11|11265.32|0.06|0.08|N|O|1996-09-16|1996-10-21|1996-09-18|NONE|SHIP|tes sleep blithely|
+1671|178|7|4|5|5390.85|0.00|0.00|N|O|1996-11-14|1996-10-20|1996-11-25|TAKE BACK RETURN|FOB|luffily regular deposits|
+1671|127|8|5|12|12325.44|0.07|0.04|N|O|1996-11-17|1996-09-02|1996-12-17|COLLECT COD|RAIL|special, ironic|
+1671|197|9|6|46|50470.74|0.08|0.05|N|O|1996-09-13|1996-10-14|1996-09-28|TAKE BACK RETURN|REG AIR|. slyly bold instructions boost. furiousl|
+1696|16|3|1|8|7328.08|0.04|0.02|N|O|1998-04-28|1998-02-07|1998-05-10|NONE|TRUCK|the blithely|
+1696|139|5|2|13|13508.69|0.08|0.06|N|O|1998-03-01|1998-03-25|1998-03-24|TAKE BACK RETURN|TRUCK|tructions play slyly q|
+1696|2|5|3|19|17138.00|0.08|0.05|N|O|1998-05-03|1998-03-13|1998-05-28|TAKE BACK RETURN|REG AIR|its maintain alongside of the f|
+1696|193|4|4|21|22956.99|0.05|0.00|N|O|1998-05-04|1998-02-18|1998-05-07|NONE|MAIL|y players sleep along the final, pending |
+1696|94|7|5|43|42745.87|0.03|0.06|N|O|1998-02-14|1998-03-29|1998-02-20|COLLECT COD|FOB|arefully regular dep|
+1697|75|5|1|6|5850.42|0.05|0.00|N|O|1997-01-28|1996-11-27|1997-01-31|NONE|FOB|accounts breach slyly even de|
+1697|104|7|2|24|24098.40|0.00|0.08|N|O|1996-12-29|1996-12-19|1997-01-10|NONE|SHIP|ts cajole carefully above the carefully|
+1697|124|9|3|27|27651.24|0.06|0.00|N|O|1997-01-20|1996-12-02|1997-02-05|COLLECT COD|MAIL|ly regular packages across the silent, b|
+1697|94|5|4|49|48710.41|0.08|0.04|N|O|1996-12-07|1997-01-02|1996-12-31|COLLECT COD|TRUCK|lar foxes. fluffily furious ideas doubt qu|
+1697|35|1|5|19|17765.57|0.03|0.07|N|O|1997-01-08|1996-11-12|1997-01-11|DELIVER IN PERSON|FOB|ons? special, special accounts after|
+1698|97|8|1|44|43871.96|0.05|0.05|N|O|1997-05-16|1997-07-05|1997-05-27|NONE|RAIL|ts wake slyly after t|
+1698|93|5|2|6|5958.54|0.08|0.00|N|O|1997-08-21|1997-06-08|1997-09-03|DELIVER IN PERSON|RAIL| pending packages affix ne|
+1698|21|6|3|22|20262.44|0.03|0.04|N|O|1997-08-07|1997-05-28|1997-08-24|DELIVER IN PERSON|TRUCK|oward the furiously iro|
+1698|112|6|4|19|19230.09|0.00|0.07|N|O|1997-07-04|1997-06-21|1997-08-01|NONE|RAIL| fluffily e|
+1698|53|4|5|37|35262.85|0.00|0.03|N|O|1997-05-16|1997-05-29|1997-05-27|NONE|AIR|ly regular ideas. deposit|
+1698|166|7|6|15|15992.40|0.10|0.01|N|O|1997-07-20|1997-06-07|1997-07-21|TAKE BACK RETURN|RAIL|final ideas. even, ironic |
+1699|38|9|1|50|46901.50|0.00|0.06|A|F|1994-03-26|1994-03-23|1994-04-20|NONE|FOB|to the final requests are carefully silent |
+1699|135|6|2|17|17597.21|0.07|0.02|R|F|1994-01-12|1994-03-12|1994-02-08|NONE|AIR|haggle blithely slyly|
+1700|140|1|1|38|39525.32|0.04|0.04|N|O|1996-10-03|1996-07-27|1996-10-22|NONE|RAIL|ular dependencies engage slyly |
+1700|156|7|2|49|51751.35|0.04|0.00|N|O|1996-09-26|1996-07-28|1996-10-16|NONE|TRUCK|kly even dependencies haggle fluffi|
+1701|150|9|1|47|49357.05|0.08|0.05|R|F|1992-05-25|1992-06-29|1992-06-15|NONE|RAIL|slyly final requests cajole requests. f|
+1701|54|5|2|2|1908.10|0.01|0.04|R|F|1992-06-24|1992-07-12|1992-06-29|COLLECT COD|SHIP|ween the pending, final accounts. |
+1701|35|1|3|26|24310.78|0.10|0.06|R|F|1992-06-04|1992-07-11|1992-07-04|DELIVER IN PERSON|FOB| accounts. blithely pending pinto be|
+1702|67|2|1|19|18374.14|0.02|0.01|N|F|1995-06-02|1995-06-30|1995-06-29|NONE|REG AIR|ies haggle blith|
+1702|30|5|2|38|35341.14|0.00|0.00|N|O|1995-09-01|1995-06-10|1995-09-10|DELIVER IN PERSON|REG AIR|as believe blithely. bo|
+1702|195|6|3|46|50378.74|0.00|0.08|N|O|1995-07-14|1995-06-30|1995-07-20|NONE|FOB|y even foxes. carefully final dependencies |
+1702|93|4|4|28|27806.52|0.07|0.05|R|F|1995-06-10|1995-07-26|1995-06-16|TAKE BACK RETURN|AIR|nts haggle along the packa|
+1702|89|10|5|34|33628.72|0.01|0.06|N|O|1995-07-04|1995-06-08|1995-07-28|DELIVER IN PERSON|AIR|y careful packages; dogged acco|
+1702|42|9|6|28|26377.12|0.10|0.00|N|O|1995-08-14|1995-07-31|1995-09-08|COLLECT COD|RAIL|ackages sleep. furiously even excuses snooz|
+1703|166|5|1|36|38381.76|0.09|0.01|R|F|1993-04-22|1993-03-05|1993-04-24|DELIVER IN PERSON|SHIP|riously express |
+1703|137|8|2|35|36299.55|0.01|0.08|R|F|1993-04-14|1993-03-31|1993-04-27|NONE|RAIL|he carefully|
+1703|124|5|3|48|49157.76|0.06|0.02|R|F|1993-02-07|1993-04-20|1993-02-24|TAKE BACK RETURN|AIR|ggle slyly furiously regular theodol|
+1728|126|5|1|1|1026.12|0.07|0.04|N|O|1996-09-16|1996-08-19|1996-09-18|COLLECT COD|FOB|lly. carefully ex|
+1728|105|8|2|23|23117.30|0.05|0.02|N|O|1996-09-08|1996-07-24|1996-09-20|NONE|FOB|ns. pending, final ac|
+1728|165|10|3|44|46867.04|0.08|0.07|N|O|1996-07-31|1996-06-22|1996-08-06|COLLECT COD|FOB|ide of the slyly blithe|
+1728|27|8|4|34|31518.68|0.08|0.05|N|O|1996-08-28|1996-07-20|1996-09-12|DELIVER IN PERSON|MAIL|special req|
+1728|199|2|5|31|34074.89|0.09|0.02|N|O|1996-07-26|1996-06-28|1996-08-14|NONE|REG AIR|kly sly theodolites.|
+1729|157|8|1|12|12685.80|0.08|0.04|A|F|1992-08-11|1992-07-24|1992-08-16|COLLECT COD|RAIL|y pending packages detect. carefully re|
+1730|166|5|1|41|43712.56|0.01|0.03|N|O|1998-08-11|1998-08-29|1998-09-02|TAKE BACK RETURN|TRUCK| instructions. unusual, even Tiresi|
+1730|162|3|2|15|15932.40|0.07|0.04|N|O|1998-09-07|1998-09-12|1998-09-30|TAKE BACK RETURN|AIR|pinto beans cajole. bravely bold|
+1730|162|1|3|9|9559.44|0.10|0.00|N|O|1998-09-18|1998-09-15|1998-09-21|DELIVER IN PERSON|FOB|gular dependencies wake. blithely final e|
+1730|10|7|4|40|36400.40|0.02|0.03|N|O|1998-10-02|1998-10-06|1998-10-03|NONE|SHIP|ven dinos slee|
+1730|141|4|5|43|44769.02|0.04|0.06|N|O|1998-10-26|1998-10-22|1998-11-02|DELIVER IN PERSON|TRUCK|ng deposits cajo|
+1731|184|5|1|36|39030.48|0.10|0.00|N|O|1996-04-18|1996-04-03|1996-04-29|TAKE BACK RETURN|MAIL|ngside of the even instruct|
+1731|139|10|2|7|7273.91|0.04|0.07|N|O|1996-04-11|1996-02-13|1996-04-30|DELIVER IN PERSON|REG AIR|fily quick asymptotes|
+1731|51|9|3|50|47552.50|0.05|0.04|N|O|1996-01-14|1996-03-13|1996-01-29|COLLECT COD|RAIL|ly slyly speci|
+1731|196|10|4|23|25212.37|0.10|0.04|N|O|1996-04-22|1996-02-25|1996-05-16|TAKE BACK RETURN|RAIL|rays? bold, express pac|
+1731|53|4|5|37|35262.85|0.10|0.05|N|O|1996-04-30|1996-03-17|1996-05-27|TAKE BACK RETURN|RAIL| beans use furiously slyly b|
+1731|124|7|6|41|41988.92|0.03|0.08|N|O|1996-04-05|1996-02-28|1996-05-01|TAKE BACK RETURN|RAIL|haggle across the blithely ironi|
+1732|5|6|1|50|45250.00|0.02|0.01|R|F|1993-12-05|1994-01-23|1993-12-20|TAKE BACK RETURN|FOB|fily final asymptotes according |
+1732|99|10|2|36|35967.24|0.01|0.03|A|F|1994-03-15|1994-02-09|1994-04-02|DELIVER IN PERSON|TRUCK|ve the accounts. slowly ironic multip|
+1732|161|8|3|41|43507.56|0.00|0.04|R|F|1994-02-20|1994-01-07|1994-02-27|TAKE BACK RETURN|AIR|quests sublate against the silent |
+1732|152|3|4|9|9469.35|0.04|0.04|A|F|1994-02-25|1994-01-29|1994-03-16|TAKE BACK RETURN|FOB|ular platelets. deposits wak|
+1732|169|8|5|25|26729.00|0.02|0.05|A|F|1994-02-15|1994-01-07|1994-02-21|COLLECT COD|REG AIR|nag slyly. even, special de|
+1732|73|1|6|16|15569.12|0.01|0.05|R|F|1994-01-07|1994-01-02|1994-01-25|COLLECT COD|SHIP|ix carefully at the furiously regular pac|
+1733|111|5|1|41|41455.51|0.08|0.01|N|O|1996-06-13|1996-07-08|1996-07-07|TAKE BACK RETURN|AIR|ess notornis. fur|
+1733|24|7|2|16|14784.32|0.00|0.04|N|O|1996-08-28|1996-07-25|1996-09-27|COLLECT COD|MAIL|slyly express deposits sleep abo|
+1733|120|10|3|29|29583.48|0.10|0.06|N|O|1996-07-16|1996-08-08|1996-07-28|NONE|TRUCK|ns detect among the special accounts. qu|
+1733|136|7|4|38|39372.94|0.01|0.03|N|O|1996-08-26|1996-07-23|1996-08-28|NONE|FOB| deposits |
+1733|34|5|5|22|20548.66|0.06|0.07|N|O|1996-07-16|1996-07-24|1996-07-30|COLLECT COD|AIR|gainst the final deposits. carefully final |
+1733|66|7|6|9|8694.54|0.06|0.08|N|O|1996-05-25|1996-07-23|1996-06-10|COLLECT COD|TRUCK|ven foxes was according to t|
+1733|146|9|7|13|13599.82|0.02|0.03|N|O|1996-08-03|1996-08-02|1996-08-18|NONE|MAIL|olites sleep furious|
+1734|155|3|1|38|40095.70|0.03|0.03|R|F|1994-08-09|1994-09-07|1994-08-12|COLLECT COD|FOB|ts doubt b|
+1734|118|2|2|4|4072.44|0.06|0.03|A|F|1994-08-20|1994-07-17|1994-08-25|DELIVER IN PERSON|AIR|final warhorses.|
+1735|156|7|1|43|45414.45|0.02|0.06|A|F|1993-01-14|1993-03-25|1993-02-02|DELIVER IN PERSON|FOB|iously after the |
+1735|139|5|2|49|50917.37|0.03|0.04|A|F|1992-12-31|1993-02-03|1993-01-25|TAKE BACK RETURN|TRUCK|y express accounts above the exp|
+1760|96|9|1|38|37851.42|0.09|0.03|N|O|1996-06-15|1996-06-29|1996-07-11|NONE|MAIL|tions. blithely regular orbits against the |
+1760|8|9|2|3|2724.00|0.00|0.06|N|O|1996-07-18|1996-07-01|1996-08-01|NONE|RAIL|lyly bold dolphins haggle carefully. sl|
+1760|137|8|3|44|45633.72|0.05|0.01|N|O|1996-06-11|1996-06-16|1996-07-02|COLLECT COD|REG AIR|instructions poach slyly ironic theodolites|
+1761|52|4|1|33|31417.65|0.09|0.03|R|F|1994-01-03|1994-01-23|1994-01-31|NONE|FOB|s. excuses a|
+1761|52|3|2|37|35225.85|0.02|0.07|R|F|1994-02-17|1994-03-08|1994-03-16|NONE|RAIL| integrate. quickly unusual|
+1761|49|6|3|37|35114.48|0.06|0.04|R|F|1994-01-02|1994-03-12|1994-01-25|DELIVER IN PERSON|TRUCK|regular packages wake after|
+1761|73|1|4|49|47680.43|0.06|0.07|R|F|1994-01-08|1994-03-03|1994-02-05|TAKE BACK RETURN|FOB|y even packages promise|
+1761|157|5|5|37|39114.55|0.03|0.04|R|F|1994-04-24|1994-03-14|1994-04-29|TAKE BACK RETURN|MAIL|express requests print blithely around the|
+1761|24|7|6|12|11088.24|0.01|0.05|A|F|1994-04-16|1994-03-08|1994-04-21|DELIVER IN PERSON|AIR| sleep furiously. deposits are acco|
+1761|1|6|7|13|11713.00|0.03|0.08|R|F|1994-03-06|1994-03-18|1994-03-22|DELIVER IN PERSON|TRUCK|ons boost fu|
+1762|26|5|1|15|13890.30|0.04|0.08|A|F|1994-12-18|1994-10-29|1995-01-17|TAKE BACK RETURN|REG AIR|old packages thrash. care|
+1762|50|3|2|39|37051.95|0.10|0.02|A|F|1994-09-12|1994-11-09|1994-10-08|DELIVER IN PERSON|MAIL| ironic platelets sleep along t|
+1762|32|8|3|7|6524.21|0.05|0.01|R|F|1994-09-03|1994-10-02|1994-09-10|NONE|REG AIR|uickly express packages wake slyly-- regul|
+1762|145|2|4|24|25083.36|0.03|0.03|A|F|1994-11-30|1994-11-02|1994-12-20|NONE|REG AIR|accounts solve alongside of the fluffily |
+1762|8|9|5|49|44492.00|0.08|0.05|A|F|1994-10-20|1994-11-02|1994-11-10|TAKE BACK RETURN|SHIP| packages sleep fluffily pen|
+1762|94|7|6|35|34793.15|0.05|0.05|A|F|1994-11-25|1994-10-21|1994-11-28|COLLECT COD|AIR|ind quickly. accounts ca|
+1762|73|3|7|47|45734.29|0.03|0.01|A|F|1994-11-02|1994-10-07|1994-11-08|NONE|SHIP| blithely brave|
+1763|12|9|1|22|20064.22|0.09|0.06|N|O|1997-01-17|1997-01-15|1997-02-03|TAKE BACK RETURN|SHIP|ld. fluffily final ideas boos|
+1763|157|5|2|43|45457.45|0.04|0.04|N|O|1996-11-04|1996-12-09|1996-11-28|DELIVER IN PERSON|FOB|r deposits integrate blithely pending, quic|
+1763|25|10|3|16|14800.32|0.06|0.02|N|O|1996-12-12|1996-12-04|1996-12-25|DELIVER IN PERSON|RAIL|ously pending asymptotes a|
+1763|61|6|4|44|42286.64|0.04|0.05|N|O|1996-12-04|1997-01-06|1996-12-25|DELIVER IN PERSON|REG AIR| instructions need to integrate deposits. |
+1763|147|4|5|13|13612.82|0.03|0.05|N|O|1996-11-23|1997-01-24|1996-12-05|TAKE BACK RETURN|SHIP|s sleep carefully. fluffily unusua|
+1763|143|4|6|3|3129.42|0.05|0.03|N|O|1996-12-10|1996-12-06|1997-01-04|TAKE BACK RETURN|FOB|ut the slyly pending deposi|
+1763|184|5|7|2|2168.36|0.05|0.07|N|O|1997-02-27|1996-12-04|1997-03-27|COLLECT COD|FOB|even pinto beans snooze fluffi|
+1764|121|2|1|20|20422.40|0.09|0.02|A|F|1992-06-09|1992-05-22|1992-07-06|COLLECT COD|MAIL|y quickly regular packages. car|
+1764|67|4|2|3|2901.18|0.07|0.07|R|F|1992-05-13|1992-06-07|1992-05-26|COLLECT COD|RAIL|es wake slowly. |
+1764|78|6|3|27|26407.89|0.07|0.04|A|F|1992-05-06|1992-05-11|1992-05-23|COLLECT COD|TRUCK|ly final foxes wake blithely even requests|
+1765|161|2|1|36|38201.76|0.08|0.04|N|O|1996-03-02|1996-02-17|1996-03-14|DELIVER IN PERSON|SHIP|he blithely pending accou|
+1766|87|8|1|32|31586.56|0.08|0.01|N|O|1997-01-08|1996-11-11|1997-01-31|TAKE BACK RETURN|AIR|ess accounts. stealthily ironic accou|
+1766|34|10|2|12|11208.36|0.05|0.01|N|O|1996-10-28|1996-12-18|1996-11-15|DELIVER IN PERSON|AIR|heodolites above the final, regular acc|
+1766|111|1|3|1|1011.11|0.10|0.02|N|O|1997-01-21|1997-01-07|1997-02-19|NONE|TRUCK|ly blithely pending accounts. reg|
+1767|25|4|1|32|29600.64|0.08|0.04|A|F|1995-05-22|1995-05-14|1995-05-23|COLLECT COD|SHIP|to the bravely ironic requests i|
+1767|42|1|2|1|942.04|0.09|0.05|N|O|1995-06-23|1995-05-25|1995-07-03|TAKE BACK RETURN|RAIL|ing to the slyly fin|
+1767|174|5|3|24|25780.08|0.06|0.03|R|F|1995-03-16|1995-04-29|1995-04-11|DELIVER IN PERSON|RAIL|luffy theodolites need to detect furi|
+1767|23|8|4|50|46151.00|0.01|0.02|R|F|1995-05-29|1995-04-14|1995-06-15|NONE|REG AIR|y unusual foxe|
+1767|52|10|5|40|38082.00|0.06|0.00|R|F|1995-04-16|1995-05-06|1995-04-21|TAKE BACK RETURN|AIR|ep. accounts nag blithely fu|
+1792|88|9|1|9|8892.72|0.09|0.04|R|F|1994-02-28|1993-12-11|1994-03-12|TAKE BACK RETURN|AIR|final packages s|
+1792|9|6|2|5|4545.00|0.04|0.02|R|F|1994-02-13|1994-01-03|1994-02-28|DELIVER IN PERSON|TRUCK|ely regular accounts are slyly. pending, bo|
+1792|9|2|3|8|7272.00|0.01|0.04|A|F|1994-02-21|1994-01-26|1994-02-27|DELIVER IN PERSON|RAIL|nts. fluffily special instructions integr|
+1792|191|3|4|45|49103.55|0.00|0.01|A|F|1994-02-27|1993-12-24|1994-03-07|DELIVER IN PERSON|MAIL|ests are. ironic, regular asy|
+1792|199|2|5|35|38471.65|0.06|0.05|R|F|1994-01-31|1994-01-20|1994-02-17|NONE|FOB|e against the quic|
+1793|48|5|1|29|27493.16|0.01|0.06|R|F|1992-10-24|1992-09-20|1992-11-23|NONE|MAIL|ar excuses. |
+1793|126|9|2|4|4104.48|0.07|0.05|A|F|1992-07-28|1992-08-26|1992-08-21|COLLECT COD|RAIL|nic foxes along the even|
+1793|131|7|3|6|6186.78|0.01|0.05|R|F|1992-09-21|1992-09-05|1992-10-01|DELIVER IN PERSON|REG AIR|uctions; depo|
+1793|118|8|4|4|4072.44|0.00|0.08|R|F|1992-09-27|1992-09-21|1992-10-07|DELIVER IN PERSON|AIR|equests nod ac|
+1793|25|6|5|42|38850.84|0.03|0.03|A|F|1992-10-13|1992-10-02|1992-11-06|NONE|RAIL|uctions sleep carefully special, fl|
+1794|168|9|1|36|38453.76|0.09|0.08|N|O|1997-11-07|1997-11-01|1997-11-18|TAKE BACK RETURN|FOB|ely fluffily ironi|
+1794|95|8|2|3|2985.27|0.02|0.03|N|O|1997-11-15|1997-12-16|1997-11-20|DELIVER IN PERSON|FOB| sentiments according to the q|
+1794|117|8|3|23|23393.53|0.08|0.04|N|O|1997-10-13|1997-11-30|1997-10-28|TAKE BACK RETURN|AIR|usly unusual theodolites doze about |
+1794|85|6|4|34|33492.72|0.06|0.08|N|O|1997-09-29|1997-11-13|1997-10-07|TAKE BACK RETURN|SHIP|rs above the accoun|
+1794|117|4|5|47|47804.17|0.10|0.06|N|O|1998-01-15|1997-11-30|1998-02-14|DELIVER IN PERSON|TRUCK| haggle slyly. furiously express orbit|
+1794|91|3|6|37|36670.33|0.01|0.01|N|O|1998-01-12|1997-12-21|1998-01-17|DELIVER IN PERSON|MAIL|ackages. pinto|
+1795|137|8|1|44|45633.72|0.08|0.08|A|F|1994-04-28|1994-05-24|1994-05-27|NONE|AIR|ites sleep carefully slyly p|
+1795|114|5|2|34|34479.74|0.08|0.00|A|F|1994-04-24|1994-06-01|1994-05-08|DELIVER IN PERSON|SHIP|closely regular instructions wake. |
+1795|168|3|3|25|26704.00|0.07|0.01|A|F|1994-05-18|1994-05-22|1994-05-20|TAKE BACK RETURN|RAIL|he always express accounts ca|
+1795|125|8|4|32|32803.84|0.03|0.06|R|F|1994-05-10|1994-04-21|1994-05-17|DELIVER IN PERSON|SHIP| asymptotes across the bold,|
+1795|163|8|5|11|11694.76|0.08|0.02|R|F|1994-06-19|1994-04-24|1994-07-02|TAKE BACK RETURN|TRUCK|slyly. special pa|
+1796|10|1|1|28|25480.28|0.08|0.04|A|F|1992-12-01|1993-01-01|1992-12-24|DELIVER IN PERSON|FOB|y quickly ironic accounts.|
+1796|185|6|2|8|8681.44|0.00|0.08|R|F|1993-01-07|1993-01-04|1993-01-10|NONE|SHIP|slyly bold accounts are furiously agains|
+1797|31|7|1|17|15827.51|0.01|0.02|N|O|1996-08-06|1996-07-11|1996-08-29|NONE|TRUCK| cajole carefully. unusual Tiresias e|
+1797|145|2|2|16|16722.24|0.01|0.00|N|O|1996-06-03|1996-07-21|1996-06-07|NONE|FOB|o beans wake regular accounts. blit|
+1797|12|9|3|21|19152.21|0.02|0.01|N|O|1996-08-05|1996-08-05|1996-08-06|DELIVER IN PERSON|AIR|ns. regular, regular deposit|
+1798|109|10|1|43|43391.30|0.01|0.08|N|O|1997-08-27|1997-10-23|1997-09-09|DELIVER IN PERSON|MAIL|ld packages sleep furiously. depend|
+1799|52|10|1|8|7616.40|0.04|0.08|R|F|1994-06-14|1994-05-27|1994-06-27|TAKE BACK RETURN|MAIL|ealms upon the special, ironic waters|
+1799|27|10|2|42|38934.84|0.02|0.02|R|F|1994-04-05|1994-04-28|1994-04-09|DELIVER IN PERSON|FOB|es pending |
+1824|120|10|1|45|45905.40|0.03|0.02|R|F|1994-08-21|1994-06-21|1994-09-19|NONE|RAIL|ent Tiresias. quickly express |
+1824|69|4|2|40|38762.40|0.10|0.03|A|F|1994-05-08|1994-07-24|1994-06-06|NONE|FOB|es mold furiously final instructions. s|
+1825|156|1|1|43|45414.45|0.05|0.05|A|F|1994-02-18|1994-02-19|1994-03-02|TAKE BACK RETURN|RAIL| accounts breach fluffily spe|
+1825|148|5|2|39|40877.46|0.00|0.00|R|F|1994-04-01|1994-01-12|1994-04-21|DELIVER IN PERSON|REG AIR|ual, bold ideas haggle above the quickly ir|
+1825|17|4|3|7|6419.07|0.04|0.03|A|F|1994-01-02|1994-01-30|1994-01-30|TAKE BACK RETURN|REG AIR|fully ironic requests. requests cajole ex|
+1825|121|10|4|23|23485.76|0.05|0.01|R|F|1994-01-08|1994-02-08|1994-01-19|NONE|MAIL| wake express, even r|
+1825|178|9|5|33|35579.61|0.04|0.04|A|F|1993-12-07|1994-03-01|1993-12-16|TAKE BACK RETURN|RAIL|about the ne|
+1826|27|10|1|4|3708.08|0.06|0.00|R|F|1992-07-05|1992-06-12|1992-08-04|DELIVER IN PERSON|MAIL|alongside of the quickly unusual re|
+1826|68|3|2|9|8712.54|0.07|0.07|R|F|1992-07-12|1992-07-11|1992-07-15|DELIVER IN PERSON|TRUCK| blithely special|
+1826|176|4|3|14|15066.38|0.05|0.01|A|F|1992-04-28|1992-05-31|1992-05-25|COLLECT COD|TRUCK|uriously bold pinto beans are carefully ag|
+1826|180|9|4|6|6481.08|0.05|0.04|R|F|1992-06-30|1992-05-17|1992-07-30|DELIVER IN PERSON|RAIL|kages. blithely silent|
+1826|135|1|5|46|47615.98|0.05|0.06|R|F|1992-05-02|1992-06-25|1992-05-26|TAKE BACK RETURN|FOB|ously? quickly pe|
+1826|108|3|6|43|43348.30|0.02|0.03|A|F|1992-07-28|1992-06-14|1992-08-03|NONE|MAIL|ss tithes use even ideas. fluffily final t|
+1827|90|1|1|47|46534.23|0.00|0.01|N|O|1996-08-01|1996-08-07|1996-08-23|TAKE BACK RETURN|RAIL|. pending courts about the even e|
+1827|154|9|2|48|50599.20|0.03|0.05|N|O|1996-08-28|1996-09-15|1996-09-01|COLLECT COD|RAIL|oxes. special, final asymptote|
+1827|200|1|3|37|40707.40|0.01|0.07|N|O|1996-07-20|1996-08-18|1996-08-08|DELIVER IN PERSON|REG AIR|ously ironic theodolites serve quickly af|
+1827|127|10|4|4|4108.48|0.04|0.04|N|O|1996-07-22|1996-09-10|1996-08-11|DELIVER IN PERSON|RAIL|special requests. blithely|
+1827|80|10|5|24|23521.92|0.00|0.08|N|O|1996-08-07|1996-09-01|1996-09-04|DELIVER IN PERSON|SHIP|al gifts! re|
+1827|21|2|6|7|6447.14|0.10|0.02|N|O|1996-08-28|1996-08-07|1996-08-31|DELIVER IN PERSON|AIR|egular foxes|
+1827|6|7|7|38|34428.00|0.05|0.01|N|O|1996-10-17|1996-08-29|1996-11-07|TAKE BACK RETURN|SHIP| blithely. express, bo|
+1828|100|4|1|33|33003.30|0.05|0.04|R|F|1994-06-27|1994-06-10|1994-07-24|COLLECT COD|FOB|s boost carefully. pending d|
+1828|13|3|2|40|36520.40|0.08|0.07|R|F|1994-05-05|1994-07-02|1994-05-19|COLLECT COD|REG AIR|s use above the quietly fin|
+1828|196|7|3|11|12058.09|0.07|0.08|R|F|1994-07-21|1994-05-28|1994-08-13|DELIVER IN PERSON|FOB| wake blithely |
+1828|8|3|4|45|40860.00|0.02|0.05|R|F|1994-05-15|1994-05-29|1994-05-28|COLLECT COD|RAIL| accounts run slyly |
+1828|79|7|5|14|13706.98|0.01|0.08|A|F|1994-05-20|1994-06-02|1994-05-25|TAKE BACK RETURN|SHIP|. final packages along the carefully bold|
+1829|150|7|1|12|12601.80|0.05|0.06|A|F|1994-08-23|1994-07-13|1994-09-04|DELIVER IN PERSON|FOB|ges wake furiously express pinto|
+1829|5|6|2|11|9955.00|0.04|0.05|A|F|1994-05-18|1994-06-13|1994-06-07|COLLECT COD|MAIL|ding orbits|
+1829|104|9|3|49|49200.90|0.09|0.08|A|F|1994-08-26|1994-08-01|1994-09-16|NONE|TRUCK|ound the quickly |
+1829|153|4|4|14|14744.10|0.03|0.06|A|F|1994-08-15|1994-06-08|1994-08-30|TAKE BACK RETURN|AIR|regular deposits alongside of the flu|
+1829|166|5|5|6|6396.96|0.02|0.07|A|F|1994-08-09|1994-08-05|1994-09-05|DELIVER IN PERSON|MAIL|s haggle! slyl|
+1829|115|5|6|36|36543.96|0.09|0.04|R|F|1994-06-10|1994-06-23|1994-06-22|NONE|FOB|ackages-- express requests sleep; pen|
+1830|120|4|1|38|38764.56|0.00|0.07|R|F|1995-04-20|1995-05-22|1995-04-24|TAKE BACK RETURN|TRUCK|ely even a|
+1830|25|10|2|9|8325.18|0.05|0.07|R|F|1995-03-09|1995-05-24|1995-03-14|NONE|SHIP|st furiously among |
+1830|82|3|3|36|35354.88|0.07|0.07|R|F|1995-04-21|1995-04-14|1995-05-10|DELIVER IN PERSON|SHIP| slowly unusual orbits. carefull|
+1831|136|2|1|9|9325.17|0.02|0.03|A|F|1993-12-17|1994-01-27|1993-12-26|NONE|TRUCK|mptotes. furiously regular dolphins al|
+1831|48|9|2|9|8532.36|0.07|0.06|R|F|1994-03-22|1994-01-07|1994-04-06|COLLECT COD|MAIL|ent deposits. regular saute|
+1831|115|5|3|17|17256.87|0.02|0.08|R|F|1994-01-18|1994-02-12|1994-01-30|TAKE BACK RETURN|MAIL|s boost ironic foxe|
+1831|95|8|4|23|22887.07|0.06|0.02|R|F|1993-12-21|1994-02-08|1994-01-04|NONE|SHIP|ests. express pinto beans abou|
+1856|55|10|1|10|9550.50|0.05|0.07|R|F|1992-05-11|1992-05-20|1992-06-02|TAKE BACK RETURN|FOB|he furiously even theodolites. account|
+1856|97|10|2|47|46863.23|0.07|0.07|R|F|1992-03-22|1992-06-09|1992-04-17|DELIVER IN PERSON|FOB|ingly blithe theodolites. slyly pending |
+1856|117|7|3|20|20342.20|0.04|0.06|R|F|1992-05-04|1992-05-06|1992-05-11|DELIVER IN PERSON|MAIL|ost carefully. slyly bold accounts|
+1856|150|1|4|22|23103.30|0.08|0.02|A|F|1992-05-02|1992-05-26|1992-05-20|TAKE BACK RETURN|REG AIR|platelets detect slyly regular packages. ca|
+1856|190|1|5|14|15262.66|0.01|0.01|A|F|1992-04-14|1992-05-02|1992-05-11|COLLECT COD|SHIP|ans are even requests. deposits caj|
+1856|23|6|6|36|33228.72|0.03|0.05|A|F|1992-06-19|1992-05-12|1992-06-28|TAKE BACK RETURN|TRUCK|ly even foxes kindle blithely even realm|
+1856|130|3|7|42|43265.46|0.04|0.00|R|F|1992-05-23|1992-06-06|1992-06-19|COLLECT COD|RAIL|usly final deposits|
+1857|174|5|1|15|16112.55|0.10|0.03|R|F|1993-04-05|1993-02-28|1993-04-13|COLLECT COD|RAIL|egular, regular inst|
+1857|167|6|2|40|42686.40|0.10|0.00|R|F|1993-02-15|1993-03-08|1993-02-21|NONE|AIR|slyly close d|
+1857|119|3|3|8|8152.88|0.01|0.07|R|F|1993-01-27|1993-04-04|1993-02-20|TAKE BACK RETURN|AIR|slyly about the fluffily silent req|
+1857|100|3|4|41|41004.10|0.07|0.07|A|F|1993-04-16|1993-02-16|1993-04-18|NONE|REG AIR| the slyly|
+1858|14|8|1|33|30162.33|0.01|0.02|N|O|1997-12-28|1998-02-03|1998-01-13|NONE|RAIL|tect along the slyly final|
+1859|75|6|1|18|17551.26|0.10|0.00|N|O|1997-08-08|1997-06-30|1997-08-26|TAKE BACK RETURN|SHIP|e carefully a|
+1859|188|9|2|36|39174.48|0.02|0.01|N|O|1997-05-05|1997-07-08|1997-05-25|TAKE BACK RETURN|REG AIR|regular requests. carefully unusual theo|
+1859|158|10|3|5|5290.75|0.06|0.03|N|O|1997-06-20|1997-05-20|1997-07-19|TAKE BACK RETURN|AIR|across the p|
+1859|191|2|4|21|22914.99|0.00|0.03|N|O|1997-08-06|1997-05-29|1997-08-26|TAKE BACK RETURN|REG AIR|lar packages wake quickly exp|
+1859|46|3|5|11|10406.44|0.06|0.06|N|O|1997-07-15|1997-06-05|1997-07-29|TAKE BACK RETURN|SHIP|ffily ironic pac|
+1859|105|8|6|12|12061.20|0.08|0.03|N|O|1997-05-22|1997-06-08|1997-06-07|COLLECT COD|TRUCK|es. unusual, silent request|
+1860|113|4|1|9|9117.99|0.04|0.04|N|O|1996-08-03|1996-05-31|1996-08-04|DELIVER IN PERSON|TRUCK|c realms print carefully car|
+1861|68|5|1|7|6776.42|0.08|0.05|A|F|1994-01-14|1994-04-03|1994-01-16|COLLECT COD|RAIL|s foxes. slyly|
+1861|27|8|2|31|28737.62|0.10|0.05|R|F|1994-01-29|1994-03-07|1994-02-15|TAKE BACK RETURN|RAIL|arefully unusual|
+1861|24|9|3|23|21252.46|0.00|0.08|A|F|1994-04-09|1994-03-04|1994-04-11|DELIVER IN PERSON|MAIL|in packages sleep silent dolphins; sly|
+1861|116|6|4|38|38612.18|0.10|0.05|R|F|1994-02-26|1994-02-05|1994-03-01|NONE|RAIL|pending deposits cajole quic|
+1861|16|3|5|2|1832.02|0.03|0.08|R|F|1994-04-26|1994-03-15|1994-05-15|TAKE BACK RETURN|MAIL|e final, regular requests. carefully |
+1862|30|5|1|41|38131.23|0.10|0.00|N|O|1998-06-05|1998-05-17|1998-07-04|COLLECT COD|FOB| carefully along|
+1862|166|7|2|37|39447.92|0.06|0.02|N|O|1998-04-15|1998-05-15|1998-05-14|TAKE BACK RETURN|MAIL|l deposits. carefully even dep|
+1862|104|1|3|26|26106.60|0.02|0.01|N|O|1998-03-25|1998-05-17|1998-04-17|TAKE BACK RETURN|TRUCK|g carefully: thinly ironic deposits af|
+1863|63|2|1|48|46226.88|0.09|0.04|A|F|1993-10-10|1993-12-09|1993-10-19|NONE|FOB|ans hinder furiou|
+1863|157|2|2|48|50743.20|0.04|0.08|A|F|1993-11-08|1993-11-05|1993-12-08|COLLECT COD|AIR|onic theodolites alongside of the pending a|
+1888|98|10|1|27|26948.43|0.03|0.06|R|F|1994-02-13|1994-01-16|1994-02-25|NONE|REG AIR|. carefully special dolphins sle|
+1888|74|5|2|38|37014.66|0.03|0.03|R|F|1993-11-29|1994-01-16|1993-12-08|TAKE BACK RETURN|TRUCK|dazzle carefull|
+1888|80|1|3|49|48023.92|0.07|0.05|A|F|1994-02-27|1994-01-14|1994-03-28|DELIVER IN PERSON|FOB|lar accounts haggle carefu|
+1888|19|10|4|9|8271.09|0.01|0.04|A|F|1994-02-09|1994-01-22|1994-02-19|NONE|AIR| packages are blithely. carefu|
+1888|160|1|5|4|4240.64|0.03|0.06|R|F|1993-12-28|1993-12-19|1994-01-11|COLLECT COD|FOB|lphins. ironically special theodolit|
+1888|53|8|6|48|45746.40|0.08|0.08|R|F|1994-02-28|1993-12-16|1994-03-15|COLLECT COD|TRUCK|ar ideas cajole. regular p|
+1888|167|6|7|50|53358.00|0.04|0.07|R|F|1993-12-22|1994-01-10|1994-01-06|DELIVER IN PERSON|FOB|ependencies affix blithely regular warhors|
+1889|152|4|1|41|43138.15|0.10|0.02|N|O|1997-06-15|1997-05-10|1997-07-08|NONE|AIR|s! furiously pending r|
+1889|172|3|2|13|13938.21|0.05|0.00|N|O|1997-06-12|1997-04-28|1997-06-23|NONE|REG AIR|to the regular accounts. carefully express|
+1889|138|9|3|36|37372.68|0.05|0.07|N|O|1997-05-19|1997-06-14|1997-05-23|NONE|SHIP|l pinto beans kindle |
+1889|168|5|4|5|5340.80|0.02|0.07|N|O|1997-06-26|1997-06-09|1997-07-21|COLLECT COD|AIR|ording to the blithely silent r|
+1890|141|8|1|26|27069.64|0.03|0.07|N|O|1997-04-02|1997-03-13|1997-04-22|DELIVER IN PERSON|FOB|ngage. slyly ironic |
+1890|100|1|2|43|43004.30|0.07|0.03|N|O|1996-12-30|1997-01-31|1997-01-19|DELIVER IN PERSON|FOB|p ironic, express accounts. fu|
+1890|59|1|3|24|23017.20|0.06|0.04|N|O|1997-02-09|1997-02-10|1997-02-12|COLLECT COD|MAIL|is wake carefully above the even id|
+1890|68|9|4|43|41626.58|0.09|0.04|N|O|1997-04-08|1997-02-19|1997-04-30|TAKE BACK RETURN|FOB|lyly. instructions across the furiously|
+1890|122|3|5|45|45995.40|0.08|0.05|N|O|1997-04-15|1997-03-16|1997-04-19|COLLECT COD|FOB|he carefully regular sauternes. ironic fret|
+1890|181|2|6|16|17298.88|0.08|0.02|N|O|1997-02-13|1997-02-18|1997-03-12|TAKE BACK RETURN|TRUCK|ged pinto beans. regular, regular id|
+1890|121|4|7|10|10211.20|0.01|0.04|N|O|1996-12-24|1997-02-19|1997-01-01|DELIVER IN PERSON|AIR|. even, unusual inst|
+1891|77|8|1|45|43968.15|0.07|0.04|A|F|1994-12-20|1995-01-16|1995-01-05|NONE|RAIL|ests along|
+1891|184|5|2|18|19515.24|0.06|0.00|A|F|1995-01-24|1995-01-29|1995-02-14|NONE|RAIL| foxes above the carefu|
+1891|198|9|3|15|16472.85|0.03|0.00|R|F|1995-03-11|1995-03-05|1995-03-18|TAKE BACK RETURN|MAIL| accounts are furiou|
+1892|113|7|1|48|48629.28|0.02|0.01|A|F|1994-06-16|1994-06-16|1994-06-28|NONE|RAIL|tornis detect regul|
+1892|43|2|2|35|33006.40|0.04|0.08|R|F|1994-04-05|1994-05-09|1994-05-03|NONE|MAIL|hes nod furiously around the instruc|
+1892|134|5|3|37|38262.81|0.10|0.03|R|F|1994-04-11|1994-06-04|1994-04-24|TAKE BACK RETURN|SHIP|nts. slyly regular asymptot|
+1892|197|9|4|14|15360.66|0.06|0.07|R|F|1994-04-08|1994-06-12|1994-04-27|DELIVER IN PERSON|FOB|furiously about the furiously|
+1893|99|1|1|43|42960.87|0.10|0.00|N|O|1998-01-25|1998-01-06|1998-02-14|COLLECT COD|SHIP|he carefully regular |
+1893|148|9|2|49|51358.86|0.03|0.05|N|O|1998-01-19|1998-01-28|1998-02-02|TAKE BACK RETURN|FOB|y final foxes bo|
+1893|45|6|3|3|2835.12|0.03|0.02|N|O|1998-02-10|1998-01-18|1998-02-25|DELIVER IN PERSON|MAIL|gular, even ideas. fluffily bol|
+1893|101|6|4|18|18019.80|0.07|0.06|N|O|1998-01-24|1998-01-12|1998-02-13|TAKE BACK RETURN|RAIL|g packages. fluffily final reques|
+1893|53|4|5|6|5718.30|0.10|0.02|N|O|1998-01-23|1997-12-22|1998-02-09|DELIVER IN PERSON|TRUCK|ar accounts use. daringly ironic packag|
+1894|169|10|1|40|42766.40|0.03|0.07|R|F|1992-06-07|1992-05-11|1992-07-01|DELIVER IN PERSON|FOB|ily furiously bold packages. flu|
+1895|161|6|1|43|45629.88|0.09|0.07|R|F|1994-07-26|1994-07-19|1994-08-11|NONE|AIR| carefully eve|
+1920|96|7|1|24|23906.16|0.04|0.05|N|O|1998-09-27|1998-08-23|1998-10-15|DELIVER IN PERSON|AIR|thely. bold, pend|
+1920|51|6|2|31|29482.55|0.05|0.06|N|O|1998-08-01|1998-08-30|1998-08-17|COLLECT COD|SHIP|lly. ideas wa|
+1920|18|2|3|6|5508.06|0.01|0.05|N|O|1998-10-01|1998-08-20|1998-10-24|COLLECT COD|SHIP|l ideas boost slyly pl|
+1920|84|5|4|50|49204.00|0.09|0.06|N|O|1998-10-03|1998-08-04|1998-10-29|DELIVER IN PERSON|MAIL|e blithely unusual foxes. brave packages|
+1920|34|10|5|14|13076.42|0.08|0.05|N|O|1998-10-22|1998-08-10|1998-10-27|DELIVER IN PERSON|AIR|ickly ironic d|
+1921|21|10|1|9|8289.18|0.08|0.00|R|F|1994-02-01|1994-03-20|1994-03-01|DELIVER IN PERSON|FOB|to beans. even excuses integrate specia|
+1921|140|6|2|21|21842.94|0.02|0.06|R|F|1994-02-08|1994-03-28|1994-02-15|COLLECT COD|FOB|ckly regula|
+1921|71|2|3|27|26218.89|0.00|0.04|A|F|1994-04-26|1994-04-07|1994-04-30|TAKE BACK RETURN|FOB|ing pinto beans above the pend|
+1922|10|5|1|13|11830.13|0.05|0.03|N|O|1996-10-24|1996-09-21|1996-11-15|NONE|SHIP|quests. furiously|
+1923|37|8|1|9|8433.27|0.01|0.08|N|O|1997-08-29|1997-09-13|1997-09-07|NONE|FOB|lites. ironic instructions integrate bravel|
+1923|178|8|2|23|24797.91|0.07|0.05|N|O|1997-09-08|1997-08-11|1997-09-14|TAKE BACK RETURN|MAIL|aggle carefully. furiously permanent|
+1923|180|1|3|11|11881.98|0.03|0.03|N|O|1997-07-12|1997-09-04|1997-08-01|TAKE BACK RETURN|REG AIR|ages wake slyly about the furiously regular|
+1923|193|5|4|49|53566.31|0.06|0.05|N|O|1997-07-21|1997-08-08|1997-07-26|NONE|AIR|de of the carefully expre|
+1923|184|5|5|25|27104.50|0.10|0.08|N|O|1997-08-18|1997-08-20|1997-09-12|DELIVER IN PERSON|TRUCK|the ideas: slyly pendin|
+1923|37|3|6|50|46851.50|0.03|0.03|N|O|1997-11-04|1997-08-08|1997-11-25|NONE|TRUCK|uickly along the bold courts. bold the|
+1924|73|1|1|7|6811.49|0.06|0.07|N|O|1997-01-01|1996-12-02|1997-01-08|COLLECT COD|SHIP|osits. even accounts nag furious|
+1924|18|8|2|47|43146.47|0.02|0.06|N|O|1996-11-24|1996-10-18|1996-12-13|COLLECT COD|REG AIR|silent requests cajole blithely final pack|
+1924|57|8|3|40|38282.00|0.04|0.08|N|O|1996-10-31|1996-11-30|1996-11-21|NONE|REG AIR|ains sleep carefully|
+1924|34|5|4|31|28954.93|0.03|0.03|N|O|1996-09-20|1996-10-19|1996-10-19|DELIVER IN PERSON|SHIP| the slyly regular foxes. ruthle|
+1924|36|7|5|17|15912.51|0.04|0.05|N|O|1996-12-31|1996-11-12|1997-01-25|COLLECT COD|TRUCK|e carefully theodolites. ironically ironic |
+1924|76|4|6|15|14641.05|0.02|0.04|N|O|1997-01-04|1996-11-13|1997-01-27|NONE|SHIP|he package|
+1924|40|1|7|21|19740.84|0.09|0.03|N|O|1996-09-21|1996-11-12|1996-10-02|TAKE BACK RETURN|AIR| blithely reg|
+1925|184|5|1|50|54209.00|0.01|0.02|R|F|1992-04-12|1992-04-23|1992-05-08|TAKE BACK RETURN|TRUCK|usual pinto|
+1925|135|1|2|35|36229.55|0.06|0.06|R|F|1992-05-11|1992-04-10|1992-05-14|TAKE BACK RETURN|AIR|counts. carefully ironic packages boost ab|
+1925|116|10|3|40|40644.40|0.08|0.08|A|F|1992-05-17|1992-05-20|1992-06-08|TAKE BACK RETURN|AIR|e carefully regul|
+1925|30|5|4|17|15810.51|0.06|0.02|R|F|1992-05-18|1992-04-06|1992-06-16|TAKE BACK RETURN|MAIL|instructions sleep. pinto bea|
+1926|51|9|1|24|22825.20|0.06|0.05|N|O|1996-05-04|1996-03-14|1996-06-01|DELIVER IN PERSON|RAIL|e theodolites.|
+1926|106|9|2|29|29176.90|0.09|0.08|N|O|1996-02-26|1996-03-14|1996-03-14|TAKE BACK RETURN|TRUCK|es. dependencies according to the fl|
+1926|178|6|3|10|10781.70|0.02|0.03|N|O|1996-05-23|1996-03-02|1996-06-04|NONE|AIR|usly bold accounts. express accounts|
+1926|68|9|4|13|12584.78|0.04|0.02|N|O|1996-04-26|1996-04-13|1996-05-08|DELIVER IN PERSON|MAIL|eans wake bli|
+1926|40|1|5|29|27261.16|0.06|0.00|N|O|1996-02-29|1996-03-13|1996-03-24|DELIVER IN PERSON|MAIL|hily unusual packages are fluffily am|
+1927|68|5|1|3|2904.18|0.00|0.05|N|O|1995-10-06|1995-12-08|1995-11-05|COLLECT COD|FOB|ccounts affi|
+1927|73|2|2|15|14596.05|0.08|0.08|N|O|1995-12-25|1995-12-26|1995-12-31|COLLECT COD|RAIL| carefully regular requests sleep car|
+1927|65|10|3|6|5790.36|0.05|0.05|N|O|1995-11-29|1995-11-20|1995-12-08|TAKE BACK RETURN|TRUCK|furiously even wat|
+1952|53|8|1|7|6671.35|0.04|0.05|A|F|1994-05-06|1994-06-11|1994-05-12|NONE|RAIL|about the express, even requ|
+1952|142|5|2|6|6252.84|0.06|0.05|A|F|1994-05-09|1994-05-21|1994-05-26|DELIVER IN PERSON|AIR|packages haggle. |
+1953|128|1|1|25|25703.00|0.07|0.06|A|F|1994-01-07|1994-01-28|1994-01-29|TAKE BACK RETURN|RAIL|ular, regular i|
+1953|14|5|2|35|31990.35|0.06|0.06|R|F|1994-02-03|1994-02-25|1994-02-14|DELIVER IN PERSON|FOB|among the fur|
+1954|152|7|1|31|32616.65|0.06|0.06|N|O|1997-08-18|1997-07-07|1997-09-03|DELIVER IN PERSON|RAIL|against the packages. bold, ironic e|
+1954|182|3|2|1|1082.18|0.03|0.01|N|O|1997-09-16|1997-07-08|1997-10-07|COLLECT COD|MAIL|te. furiously final deposits hag|
+1954|199|2|3|11|12091.09|0.07|0.07|N|O|1997-08-07|1997-07-23|1997-08-25|DELIVER IN PERSON|TRUCK|y carefully ironi|
+1954|159|4|4|12|12709.80|0.02|0.08|N|O|1997-07-19|1997-07-04|1997-08-06|COLLECT COD|AIR|ongside of the slyly unusual requests. reg|
+1954|170|7|5|29|31034.93|0.08|0.08|N|O|1997-08-25|1997-07-15|1997-09-02|DELIVER IN PERSON|RAIL|use thinly furiously regular asy|
+1954|177|8|6|13|14003.21|0.00|0.07|N|O|1997-06-15|1997-08-22|1997-06-20|TAKE BACK RETURN|MAIL|y ironic instructions cajole|
+1954|194|5|7|49|53615.31|0.05|0.06|N|O|1997-06-04|1997-08-29|1997-06-14|COLLECT COD|TRUCK|eans. final pinto beans sleep furiousl|
+1955|137|3|1|32|33188.16|0.02|0.02|A|F|1992-07-05|1992-06-29|1992-08-03|TAKE BACK RETURN|TRUCK|g to the carefully sile|
+1955|18|8|2|2|1836.02|0.03|0.01|R|F|1992-07-06|1992-07-06|1992-08-01|COLLECT COD|TRUCK|ickly aroun|
+1955|158|6|3|41|43384.15|0.08|0.06|A|F|1992-08-01|1992-06-04|1992-08-07|COLLECT COD|AIR| carefully against the furiously reg|
+1955|9|4|4|16|14544.00|0.03|0.07|A|F|1992-04-30|1992-06-23|1992-05-23|TAKE BACK RETURN|FOB|odolites eat s|
+1955|159|10|5|11|11650.65|0.09|0.01|A|F|1992-06-03|1992-07-04|1992-06-07|NONE|REG AIR|ously quickly pendi|
+1956|177|8|1|8|8617.36|0.02|0.04|A|F|1992-12-25|1992-11-24|1993-01-12|TAKE BACK RETURN|AIR|efully about the ironic, ironic de|
+1956|103|6|2|16|16049.60|0.00|0.05|R|F|1992-11-11|1992-11-11|1992-11-30|NONE|FOB|es cajole blithely. pen|
+1956|139|5|3|39|40526.07|0.08|0.02|A|F|1992-09-24|1992-11-26|1992-10-15|DELIVER IN PERSON|REG AIR|r theodolites sleep above the b|
+1956|29|10|4|11|10219.22|0.10|0.00|A|F|1992-12-19|1992-10-29|1993-01-07|TAKE BACK RETURN|AIR| the braids slee|
+1956|155|10|5|16|16882.40|0.08|0.02|R|F|1992-09-28|1992-10-21|1992-09-30|TAKE BACK RETURN|FOB| wake after the |
+1957|79|9|1|50|48953.50|0.09|0.05|N|O|1998-08-08|1998-09-28|1998-08-27|COLLECT COD|FOB|gainst the re|
+1957|119|3|2|31|31592.41|0.10|0.08|N|O|1998-08-13|1998-08-31|1998-08-16|NONE|REG AIR|express packages maintain fluffi|
+1958|73|2|1|9|8757.63|0.01|0.05|N|O|1995-12-08|1995-12-17|1995-12-18|DELIVER IN PERSON|REG AIR|ickly. slyly bold |
+1958|176|7|2|29|31208.93|0.05|0.06|N|O|1996-01-19|1995-12-05|1996-02-14|COLLECT COD|SHIP|d pinto beans|
+1958|102|3|3|4|4008.40|0.04|0.02|N|O|1995-10-24|1995-12-09|1995-10-28|DELIVER IN PERSON|AIR|he slyly even dependencies |
+1958|83|4|4|38|37357.04|0.09|0.07|N|O|1995-10-09|1995-11-26|1995-11-05|COLLECT COD|TRUCK|yly. slyly regular courts use silentl|
+1958|101|8|5|31|31034.10|0.08|0.01|N|O|1995-10-31|1995-11-12|1995-11-07|TAKE BACK RETURN|TRUCK|r deposits c|
+1958|17|4|6|44|40348.44|0.08|0.04|N|O|1995-12-17|1995-11-30|1996-01-15|TAKE BACK RETURN|RAIL|c theodolites after the unusual deposit|
+1958|39|5|7|29|27231.87|0.02|0.05|N|O|1995-10-14|1995-11-06|1995-11-01|NONE|REG AIR|final requests nag according to the |
+1959|169|10|1|46|49181.36|0.04|0.00|N|O|1997-05-05|1997-03-03|1997-05-24|TAKE BACK RETURN|AIR| furiously ex|
+1959|120|7|2|15|15301.80|0.08|0.07|N|O|1997-01-20|1997-02-18|1997-02-08|DELIVER IN PERSON|MAIL| quickly sp|
+1984|53|5|1|45|42887.25|0.03|0.04|N|O|1998-04-09|1998-06-11|1998-05-01|COLLECT COD|AIR|p. quickly final ideas sle|
+1984|70|7|2|35|33952.45|0.01|0.07|N|O|1998-05-18|1998-05-04|1998-06-01|COLLECT COD|RAIL|tes. quickly pending packages haggle boldl|
+1985|28|1|1|33|30624.66|0.10|0.03|R|F|1994-12-04|1994-11-01|1994-12-05|DELIVER IN PERSON|FOB|s are express packages. pendin|
+1985|21|6|2|50|46051.00|0.04|0.02|R|F|1994-09-30|1994-10-18|1994-10-12|COLLECT COD|AIR|ate carefully. carefully|
+1985|134|10|3|20|20682.60|0.07|0.03|R|F|1994-10-29|1994-11-12|1994-11-27|NONE|TRUCK|regular requests. furiously express|
+1985|199|10|4|30|32975.70|0.05|0.07|R|F|1994-09-06|1994-10-10|1994-09-26|NONE|RAIL|uickly. instr|
+1985|124|9|5|42|43013.04|0.05|0.05|R|F|1994-10-25|1994-11-03|1994-11-19|DELIVER IN PERSON|SHIP| patterns? final requests after the sp|
+1985|20|7|6|2|1840.04|0.02|0.00|A|F|1994-11-25|1994-10-09|1994-12-25|TAKE BACK RETURN|FOB| silent inst|
+1986|92|3|1|12|11905.08|0.06|0.05|A|F|1994-08-17|1994-06-28|1994-09-02|COLLECT COD|RAIL|sleep furiously fluffily final|
+1986|105|8|2|10|10051.00|0.10|0.03|R|F|1994-05-14|1994-06-21|1994-06-02|COLLECT COD|REG AIR|yly into the carefully even |
+1986|63|2|3|14|13482.84|0.04|0.02|R|F|1994-07-14|1994-06-19|1994-08-08|NONE|SHIP|the packages. pending, unusual|
+1987|16|6|1|7|6412.07|0.03|0.03|A|F|1994-07-30|1994-07-06|1994-08-29|NONE|REG AIR| regular a|
+1988|72|1|1|36|34994.52|0.09|0.04|N|O|1996-01-21|1995-11-24|1996-01-27|NONE|RAIL|gular theodolites. |
+1988|199|3|2|19|20884.61|0.08|0.08|N|O|1996-02-03|1995-12-10|1996-02-14|COLLECT COD|FOB|lly about the slyly thin instructions. f|
+1988|54|6|3|8|7632.40|0.06|0.01|N|O|1995-10-20|1995-11-11|1995-11-18|DELIVER IN PERSON|AIR|le quickly ac|
+1988|36|2|4|27|25272.81|0.08|0.00|N|O|1996-01-27|1995-12-24|1996-02-24|TAKE BACK RETURN|TRUCK|uests. regular requests are according to t|
+1988|79|8|5|26|25455.82|0.08|0.04|N|O|1996-01-25|1995-12-15|1996-01-26|COLLECT COD|SHIP| ironic dolphins haggl|
+1988|86|7|6|9|8874.72|0.08|0.03|N|O|1995-12-26|1996-01-02|1996-01-25|DELIVER IN PERSON|MAIL|lar platelets. slyly ironic packa|
+1989|10|7|1|47|42770.47|0.10|0.02|R|F|1994-06-21|1994-05-27|1994-06-22|TAKE BACK RETURN|REG AIR|final deposits s|
+1990|101|2|1|46|46050.60|0.01|0.07|R|F|1994-12-29|1995-03-14|1995-01-13|NONE|TRUCK|ar sentiments.|
+1991|110|3|1|39|39394.29|0.06|0.02|A|F|1993-01-01|1992-11-29|1993-01-10|TAKE BACK RETURN|TRUCK|ckages? carefully bold depos|
+1991|53|1|2|49|46699.45|0.08|0.06|R|F|1992-10-19|1992-11-29|1992-10-25|NONE|SHIP|nd the ideas affi|
+1991|174|5|3|6|6445.02|0.02|0.01|A|F|1992-11-02|1992-10-08|1992-11-14|TAKE BACK RETURN|REG AIR|hes nag slyly|
+1991|138|9|4|6|6228.78|0.10|0.06|A|F|1992-11-21|1992-11-03|1992-11-27|NONE|RAIL|uickly blithely final de|
+1991|60|8|5|49|47042.94|0.06|0.00|R|F|1992-09-10|1992-11-30|1992-10-07|NONE|AIR|quests cajole blithely|
+2016|147|4|1|2|2094.28|0.02|0.07|N|O|1996-10-12|1996-11-09|1996-10-31|DELIVER IN PERSON|TRUCK|carefully according to the |
+2016|63|8|2|15|14445.90|0.04|0.05|N|O|1996-09-24|1996-10-05|1996-10-21|TAKE BACK RETURN|MAIL|uests haggle carefully furiously regul|
+2016|122|7|3|8|8176.96|0.09|0.02|N|O|1996-09-19|1996-10-21|1996-10-13|TAKE BACK RETURN|SHIP|mptotes haggle ideas. packages wake flu|
+2017|103|4|1|49|49151.90|0.10|0.06|N|O|1998-05-26|1998-07-01|1998-06-06|COLLECT COD|TRUCK| after the unusual instructions. sly|
+2017|71|2|2|14|13594.98|0.07|0.04|N|O|1998-06-28|1998-06-15|1998-07-11|NONE|TRUCK|ily final w|
+2017|84|5|3|11|10824.88|0.05|0.02|N|O|1998-05-22|1998-07-13|1998-05-26|TAKE BACK RETURN|TRUCK|gside of the slyly dogged dolp|
+2018|195|6|1|2|2190.38|0.02|0.07|N|O|1995-06-25|1995-06-20|1995-07-04|NONE|TRUCK|ly ironic accounts against the slyly sly|
+2018|129|10|2|23|23669.76|0.05|0.01|R|F|1995-05-05|1995-05-12|1995-05-22|TAKE BACK RETURN|RAIL|ingly even theodolites s|
+2019|4|9|1|31|28024.00|0.07|0.03|R|F|1992-11-18|1992-12-26|1992-11-24|DELIVER IN PERSON|FOB|l ideas across the slowl|
+2019|52|7|2|18|17136.90|0.04|0.03|R|F|1993-01-24|1992-12-22|1993-02-02|NONE|MAIL|are carefully furiously regular requ|
+2020|34|10|1|50|46701.50|0.06|0.01|R|F|1993-07-12|1993-08-28|1993-08-02|COLLECT COD|TRUCK|ts against the pending ideas serve along|
+2020|176|4|2|40|43046.80|0.09|0.00|A|F|1993-10-17|1993-09-14|1993-10-29|TAKE BACK RETURN|RAIL|ently across the|
+2020|14|4|3|30|27420.30|0.07|0.04|A|F|1993-09-08|1993-08-11|1993-09-29|TAKE BACK RETURN|AIR|ly about the blithely ironic foxes. bold|
+2020|61|8|4|27|25948.62|0.05|0.06|A|F|1993-07-14|1993-09-02|1993-08-03|NONE|FOB|e of the bold foxes haggle |
+2021|85|6|1|7|6895.56|0.08|0.04|N|O|1995-10-17|1995-09-29|1995-10-20|NONE|MAIL| accounts boost blithely. blithely reg|
+2021|166|3|2|19|20257.04|0.04|0.05|N|O|1995-08-14|1995-09-05|1995-08-23|NONE|RAIL| above the slyly fl|
+2022|169|8|1|38|40628.08|0.00|0.08|R|F|1992-07-05|1992-04-20|1992-07-13|TAKE BACK RETURN|REG AIR| against the express accounts wake ca|
+2022|55|3|2|38|36291.90|0.05|0.04|R|F|1992-06-17|1992-05-15|1992-06-28|COLLECT COD|SHIP|instructions dazzle carefull|
+2022|49|10|3|48|45553.92|0.10|0.02|A|F|1992-06-14|1992-06-04|1992-07-12|DELIVER IN PERSON|SHIP|counts. slyly enticing accounts are during |
+2022|182|3|4|16|17314.88|0.05|0.03|R|F|1992-06-23|1992-05-22|1992-07-07|NONE|TRUCK|ages wake slyly care|
+2022|100|1|5|36|36003.60|0.05|0.02|R|F|1992-03-24|1992-05-07|1992-04-13|NONE|MAIL|ly after the foxes. regular, final inst|
+2022|129|2|6|20|20582.40|0.08|0.08|A|F|1992-03-31|1992-04-17|1992-04-02|NONE|SHIP|r deposits kindle |
+2022|78|9|7|13|12714.91|0.06|0.08|R|F|1992-04-04|1992-05-30|1992-04-21|NONE|FOB| orbits haggle fluffily fl|
+2023|127|10|1|9|9244.08|0.05|0.04|R|F|1992-06-04|1992-06-30|1992-06-10|NONE|AIR|ly regular pinto beans poa|
+2023|38|4|2|2|1876.06|0.01|0.00|R|F|1992-08-27|1992-07-16|1992-08-29|DELIVER IN PERSON|RAIL|ing packages. fluffily silen|
+2023|19|6|3|25|22975.25|0.10|0.03|A|F|1992-07-19|1992-07-07|1992-08-15|NONE|REG AIR| wake furiously among the slyly final|
+2023|185|6|4|9|9766.62|0.02|0.00|A|F|1992-07-23|1992-07-04|1992-08-20|TAKE BACK RETURN|AIR|nts maintain blithely alongside of the|
+2023|20|10|5|22|20240.44|0.04|0.06|A|F|1992-06-15|1992-07-13|1992-06-21|TAKE BACK RETURN|SHIP|ronic attainments. |
+2023|43|2|6|29|27348.16|0.02|0.06|A|F|1992-08-29|1992-07-28|1992-09-18|COLLECT COD|RAIL|usual instructions. bli|
+2023|134|10|7|50|51706.50|0.00|0.03|R|F|1992-06-20|1992-07-04|1992-06-23|DELIVER IN PERSON|FOB|its! carefully ex|
+2048|35|1|1|7|6545.21|0.06|0.01|R|F|1993-12-07|1994-01-31|1994-01-05|TAKE BACK RETURN|REG AIR|lent platelets boost deposits. carefully sp|
+2048|8|5|2|5|4540.00|0.04|0.04|A|F|1994-01-18|1994-02-01|1994-01-29|TAKE BACK RETURN|TRUCK|affix carefully against |
+2048|101|2|3|12|12013.20|0.01|0.05|R|F|1994-01-28|1994-01-19|1994-02-08|NONE|AIR| even theodoli|
+2048|97|1|4|11|10967.99|0.10|0.03|R|F|1993-12-20|1994-01-19|1994-01-04|TAKE BACK RETURN|MAIL|totes. idly ironic packages nag|
+2049|189|10|1|25|27229.50|0.08|0.00|N|O|1996-03-31|1996-02-29|1996-04-15|DELIVER IN PERSON|MAIL| excuses above the |
+2049|35|1|2|31|28985.93|0.10|0.05|N|O|1995-12-25|1996-02-25|1995-12-29|TAKE BACK RETURN|MAIL| packages are slyly alongside|
+2049|67|6|3|18|17407.08|0.05|0.05|N|O|1996-01-09|1996-01-22|1996-01-25|TAKE BACK RETURN|AIR| sleep fluffily. dependencies use never|
+2049|6|7|4|39|35334.00|0.02|0.05|N|O|1996-01-17|1996-01-21|1996-02-03|TAKE BACK RETURN|MAIL|the even pinto beans |
+2049|126|1|5|30|30783.60|0.04|0.06|N|O|1995-12-16|1996-02-04|1995-12-22|NONE|TRUCK|ial accounts are among the furiously perma|
+2049|84|5|6|17|16729.36|0.07|0.00|N|O|1996-02-04|1996-03-01|1996-02-24|NONE|FOB|al, regular foxes. pending, |
+2050|73|2|1|47|45734.29|0.05|0.03|A|F|1994-08-25|1994-07-18|1994-09-15|DELIVER IN PERSON|TRUCK|tside the blithely pending packages eat f|
+2050|152|3|2|48|50503.20|0.05|0.01|A|F|1994-09-30|1994-08-23|1994-10-29|COLLECT COD|AIR| final packages. pinto|
+2050|113|4|3|41|41537.51|0.10|0.04|A|F|1994-06-08|1994-08-27|1994-06-23|NONE|AIR| final theodolites. depende|
+2050|32|8|4|11|10252.33|0.02|0.01|A|F|1994-07-27|1994-08-18|1994-08-02|DELIVER IN PERSON|REG AIR|ns. bold, final ideas cajole among the fi|
+2050|168|9|5|16|17090.56|0.07|0.01|R|F|1994-08-17|1994-07-28|1994-09-05|DELIVER IN PERSON|REG AIR|al accounts. closely even |
+2050|49|2|6|29|27522.16|0.00|0.05|A|F|1994-09-23|1994-08-01|1994-10-23|TAKE BACK RETURN|MAIL|oxes alongsid|
+2050|48|5|7|25|23701.00|0.10|0.00|R|F|1994-08-18|1994-07-04|1994-09-04|TAKE BACK RETURN|RAIL|y according to |
+2051|25|6|1|43|39775.86|0.08|0.04|N|O|1996-04-22|1996-06-16|1996-04-28|COLLECT COD|RAIL|ounts sleep fluffily even requ|
+2051|130|1|2|48|49446.24|0.01|0.02|N|O|1996-05-04|1996-06-14|1996-05-19|NONE|TRUCK|unts. pending platelets believe about|
+2052|68|7|1|50|48403.00|0.09|0.08|R|F|1992-06-22|1992-06-03|1992-07-19|DELIVER IN PERSON|AIR|wake after the decoy|
+2052|135|1|2|35|36229.55|0.09|0.05|A|F|1992-05-29|1992-05-24|1992-06-11|NONE|TRUCK|ts according t|
+2052|43|2|3|16|15088.64|0.01|0.08|A|F|1992-06-30|1992-07-09|1992-07-12|NONE|SHIP|y final deposits cajole according |
+2052|96|7|4|47|46816.23|0.08|0.01|A|F|1992-06-18|1992-05-16|1992-07-02|TAKE BACK RETURN|REG AIR|final requests. stealt|
+2053|101|4|1|20|20022.00|0.09|0.00|A|F|1995-04-25|1995-04-12|1995-05-13|NONE|TRUCK|ly ironic foxes haggle slyly speci|
+2053|33|4|2|34|31723.02|0.07|0.00|A|F|1995-03-15|1995-03-20|1995-04-09|TAKE BACK RETURN|TRUCK|ions. unusual dependencies|
+2053|65|2|3|46|44392.76|0.01|0.03|R|F|1995-04-01|1995-04-02|1995-04-18|NONE|RAIL|tions. furiously even requests hagg|
+2053|121|6|4|31|31654.72|0.06|0.08|R|F|1995-03-23|1995-03-13|1995-04-16|DELIVER IN PERSON|SHIP|ts. fluffily final mul|
+2054|113|4|1|11|11144.21|0.03|0.05|R|F|1992-08-13|1992-08-26|1992-08-22|NONE|AIR|ular accou|
+2054|120|7|2|31|31623.72|0.05|0.08|A|F|1992-08-18|1992-09-04|1992-08-24|NONE|FOB|se bold, regular accounts. unusual depos|
+2054|121|2|3|32|32675.84|0.06|0.00|A|F|1992-06-23|1992-07-08|1992-07-22|NONE|FOB| packages thrash. carefully final|
+2054|174|3|4|14|15038.38|0.10|0.05|R|F|1992-06-25|1992-09-05|1992-07-14|DELIVER IN PERSON|SHIP|uickly final|
+2054|6|1|5|40|36240.00|0.08|0.06|R|F|1992-06-23|1992-08-09|1992-07-04|TAKE BACK RETURN|RAIL|n pinto beans. ironic courts are iro|
+2054|134|10|6|17|17580.21|0.08|0.01|A|F|1992-06-09|1992-08-28|1992-06-16|NONE|AIR|ges nag acc|
+2054|11|1|7|4|3644.04|0.00|0.08|R|F|1992-08-12|1992-08-31|1992-08-15|DELIVER IN PERSON|AIR|lyly careful requests wake fl|
+2055|45|6|1|15|14175.60|0.04|0.06|A|F|1993-09-15|1993-10-06|1993-10-07|NONE|REG AIR|furiously bold |
+2055|9|10|2|15|13635.00|0.06|0.05|R|F|1993-10-30|1993-11-21|1993-11-22|COLLECT COD|RAIL|gular foxes. b|
+2055|135|1|3|12|12421.56|0.00|0.02|A|F|1993-10-26|1993-11-23|1993-11-22|COLLECT COD|TRUCK|al pains. acco|
+2055|134|10|4|16|16546.08|0.02|0.02|A|F|1993-11-16|1993-11-12|1993-11-28|NONE|TRUCK|arefully daringly regular accounts.|
+2080|7|4|1|5|4535.00|0.08|0.05|R|F|1993-08-26|1993-08-07|1993-09-02|DELIVER IN PERSON|TRUCK|refully unusual theo|
+2080|197|9|2|39|42790.41|0.07|0.04|A|F|1993-08-22|1993-09-09|1993-08-23|COLLECT COD|FOB|ic deposits haggle slyly carefully eve|
+2081|89|10|1|26|25716.08|0.03|0.08|N|O|1997-10-21|1997-10-03|1997-11-10|NONE|FOB|among the slyly express accounts. silen|
+2081|149|2|2|13|13638.82|0.07|0.05|N|O|1997-08-23|1997-08-22|1997-09-09|TAKE BACK RETURN|MAIL|fter the even deposi|
+2081|13|10|3|32|29216.32|0.09|0.07|N|O|1997-09-05|1997-09-26|1997-10-03|TAKE BACK RETURN|SHIP|e. final, regular dependencies sleep slyly!|
+2081|85|6|4|23|22656.84|0.03|0.08|N|O|1997-07-06|1997-09-11|1997-07-21|TAKE BACK RETURN|MAIL|ual requests wake blithely above the|
+2081|113|7|5|19|19249.09|0.02|0.06|N|O|1997-10-01|1997-08-12|1997-10-18|COLLECT COD|SHIP|s affix sometimes express requests. quickly|
+2081|142|9|6|31|32306.34|0.03|0.06|N|O|1997-09-19|1997-09-13|1997-09-27|NONE|AIR| silent, spe|
+2082|75|3|1|36|35102.52|0.00|0.00|R|F|1995-01-20|1995-03-18|1995-01-31|COLLECT COD|MAIL|haggle furiously silent pinto beans|
+2082|105|10|2|12|12061.20|0.08|0.05|A|F|1995-01-27|1995-02-11|1995-02-07|NONE|FOB| ironic instructions. carefull|
+2083|24|3|1|37|34188.74|0.07|0.00|R|F|1993-09-07|1993-09-30|1993-09-18|TAKE BACK RETURN|MAIL|ng the special foxes wake packages. f|
+2084|182|3|1|42|45451.56|0.03|0.05|A|F|1993-03-29|1993-05-05|1993-04-22|COLLECT COD|REG AIR|y fluffily even foxes. |
+2084|180|10|2|23|24844.14|0.09|0.08|A|F|1993-06-05|1993-05-26|1993-06-06|DELIVER IN PERSON|AIR|es against |
+2084|136|2|3|37|38336.81|0.07|0.05|A|F|1993-07-16|1993-04-20|1993-08-06|NONE|AIR|y careful courts.|
+2084|94|8|4|9|8946.81|0.02|0.02|A|F|1993-03-18|1993-06-08|1993-03-30|NONE|TRUCK|heaves boost slyly after the pla|
+2084|27|10|5|28|25956.56|0.07|0.02|R|F|1993-05-04|1993-05-14|1993-05-31|COLLECT COD|TRUCK|cajole quickly carefu|
+2084|115|9|6|15|15226.65|0.09|0.04|A|F|1993-06-23|1993-04-25|1993-07-23|COLLECT COD|SHIP|tithes. bravely pendi|
+2084|194|6|7|34|37202.46|0.09|0.02|R|F|1993-06-20|1993-05-28|1993-06-25|DELIVER IN PERSON|RAIL| carefully ironic requests. fluffil|
+2085|41|8|1|45|42346.80|0.00|0.07|R|F|1994-02-27|1994-01-11|1994-03-29|TAKE BACK RETURN|MAIL|. carefully e|
+2086|60|1|1|22|21121.32|0.03|0.07|R|F|1994-12-04|1994-12-16|1994-12-20|DELIVER IN PERSON|RAIL|idly busy acc|
+2086|141|10|2|32|33316.48|0.04|0.06|A|F|1994-11-15|1995-01-05|1994-12-09|TAKE BACK RETURN|TRUCK|e carefully along th|
+2086|105|6|3|44|44224.40|0.02|0.01|A|F|1994-12-04|1994-11-30|1994-12-21|DELIVER IN PERSON|FOB|latelets s|
+2086|84|5|4|27|26570.16|0.02|0.00|A|F|1994-11-04|1995-01-14|1994-11-25|COLLECT COD|REG AIR|theodolites haggle blithely blithe p|
+2086|156|1|5|33|34852.95|0.04|0.00|A|F|1995-02-06|1994-11-25|1995-02-15|NONE|SHIP| slyly regular foxes. un|
+2086|200|3|6|20|22004.00|0.01|0.03|R|F|1994-11-30|1994-12-28|1994-12-07|COLLECT COD|FOB|lithely ironic acc|
+2086|156|8|7|7|7393.05|0.04|0.05|R|F|1994-12-27|1994-12-10|1995-01-05|COLLECT COD|RAIL| beans haggle car|
+2087|127|8|1|1|1027.12|0.05|0.04|N|O|1998-03-27|1998-03-24|1998-04-18|DELIVER IN PERSON|REG AIR|the quickly idle acco|
+2087|168|3|2|46|49135.36|0.10|0.03|N|O|1998-02-24|1998-04-02|1998-03-04|DELIVER IN PERSON|AIR|ter the dolphins.|
+2087|62|3|3|1|962.06|0.02|0.05|N|O|1998-05-27|1998-04-11|1998-06-12|COLLECT COD|REG AIR|hely final acc|
+2087|59|1|4|6|5754.30|0.03|0.08|N|O|1998-04-23|1998-03-27|1998-05-18|DELIVER IN PERSON|REG AIR|dazzle after the slyly si|
+2112|71|2|1|18|17479.26|0.02|0.05|N|O|1997-05-02|1997-03-16|1997-05-25|TAKE BACK RETURN|TRUCK|lphins solve ideas. even, special reque|
+2113|123|8|1|40|40924.80|0.04|0.06|N|O|1998-01-16|1997-12-11|1998-02-06|TAKE BACK RETURN|TRUCK|bout the quickly ironic t|
+2113|112|2|2|24|24290.64|0.03|0.02|N|O|1998-02-19|1998-01-08|1998-03-16|COLLECT COD|MAIL|kly regular accounts hinder about the|
+2114|168|9|1|50|53408.00|0.05|0.05|A|F|1995-02-05|1995-03-18|1995-02-13|COLLECT COD|RAIL|pecial pinto bean|
+2114|186|7|2|26|28240.68|0.02|0.02|A|F|1995-04-30|1995-04-16|1995-05-28|NONE|SHIP|ar asymptotes sleep |
+2114|162|1|3|25|26554.00|0.07|0.01|A|F|1995-02-15|1995-03-13|1995-02-22|COLLECT COD|AIR|unts. regular, express accounts wake. b|
+2115|196|8|1|27|29597.13|0.06|0.03|N|O|1998-09-01|1998-07-29|1998-09-04|NONE|AIR|de of the carefully bold accounts |
+2115|184|5|2|43|46619.74|0.06|0.02|N|O|1998-07-14|1998-07-25|1998-07-24|COLLECT COD|FOB| carefully pending requests alongs|
+2115|51|3|3|3|2853.15|0.03|0.04|N|O|1998-07-23|1998-07-30|1998-08-14|DELIVER IN PERSON|FOB|quickly ironic dolphin|
+2115|49|10|4|47|44604.88|0.06|0.07|N|O|1998-08-29|1998-07-30|1998-09-05|TAKE BACK RETURN|REG AIR|regular accounts integrate brav|
+2115|199|3|5|13|14289.47|0.04|0.00|N|O|1998-08-07|1998-08-06|1998-08-13|DELIVER IN PERSON|REG AIR|into beans. even accounts abou|
+2116|131|2|1|2|2062.26|0.00|0.02|R|F|1994-10-16|1994-11-24|1994-11-09|DELIVER IN PERSON|TRUCK|r theodolites use blithely about the ir|
+2116|140|1|2|47|48886.58|0.10|0.06|R|F|1994-09-01|1994-11-18|1994-09-25|COLLECT COD|MAIL|iously ironic dependencies around the iro|
+2116|184|5|3|11|11925.98|0.03|0.05|R|F|1994-09-15|1994-10-21|1994-09-21|NONE|FOB| pinto beans. final, final sauternes play |
+2117|165|2|1|36|38345.76|0.10|0.01|N|O|1997-08-06|1997-07-15|1997-08-07|DELIVER IN PERSON|SHIP|ronic accounts wake|
+2117|61|6|2|19|18260.14|0.04|0.00|N|O|1997-07-30|1997-06-18|1997-08-13|DELIVER IN PERSON|REG AIR|s between the slyly regula|
+2117|58|3|3|43|41196.15|0.04|0.03|N|O|1997-06-27|1997-06-12|1997-07-22|DELIVER IN PERSON|SHIP| foxes sleep furiously |
+2117|91|4|4|24|23786.16|0.00|0.07|N|O|1997-06-15|1997-05-27|1997-06-18|COLLECT COD|SHIP|thely slyly pending platelets. ironic, |
+2117|147|8|5|3|3141.42|0.02|0.05|N|O|1997-05-05|1997-07-20|1997-05-26|TAKE BACK RETURN|TRUCK|tes cajole|
+2117|1|4|6|27|24327.00|0.09|0.08|N|O|1997-06-30|1997-06-27|1997-07-11|TAKE BACK RETURN|REG AIR| the carefully ironic ideas|
+2118|160|1|1|24|25443.84|0.10|0.03|N|O|1997-01-06|1996-12-14|1997-01-14|TAKE BACK RETURN|RAIL|about the slyly bold depende|
+2118|184|5|2|4|4336.72|0.08|0.01|N|O|1996-10-25|1996-11-10|1996-11-22|COLLECT COD|AIR|theodolites affix according |
+2118|145|4|3|11|11496.54|0.05|0.04|N|O|1996-12-23|1996-12-20|1997-01-01|COLLECT COD|RAIL|y ironic accounts sleep upon the packages. |
+2119|102|7|1|36|36075.60|0.04|0.00|N|O|1996-11-10|1996-10-25|1996-12-03|TAKE BACK RETURN|RAIL|ly bold foxes. ironic accoun|
+2144|92|6|1|33|32738.97|0.00|0.07|R|F|1994-04-04|1994-06-20|1994-04-23|NONE|AIR| ironic excuses haggle final dependencies. |
+2144|51|9|2|46|43748.30|0.03|0.08|R|F|1994-04-08|1994-04-29|1994-05-07|COLLECT COD|SHIP| foxes haggle blithel|
+2144|4|9|3|29|26216.00|0.00|0.07|R|F|1994-05-03|1994-05-16|1994-06-01|DELIVER IN PERSON|FOB|ns wake carefully carefully ironic|
+2144|158|9|4|10|10581.50|0.00|0.04|R|F|1994-06-16|1994-05-03|1994-07-05|COLLECT COD|AIR| furiously unusual ideas. carefull|
+2145|78|8|1|13|12714.91|0.04|0.05|A|F|1992-11-12|1992-12-13|1992-12-07|TAKE BACK RETURN|MAIL|alongside of the slyly final|
+2145|154|6|2|6|6324.90|0.05|0.01|A|F|1992-10-10|1992-11-29|1992-10-14|NONE|AIR|s. fluffily express accounts sleep. slyl|
+2146|57|5|1|42|40196.10|0.10|0.01|A|F|1992-09-21|1992-11-02|1992-09-23|NONE|AIR|ns according to the doggedly |
+2146|157|5|2|6|6342.90|0.07|0.05|A|F|1993-01-03|1992-10-24|1993-01-24|DELIVER IN PERSON|RAIL|ing to the requests. dependencies boost |
+2146|25|8|3|14|12950.28|0.03|0.01|R|F|1992-09-16|1992-10-16|1992-09-20|COLLECT COD|SHIP|ecial, express a|
+2146|26|9|4|31|28706.62|0.02|0.00|A|F|1993-01-04|1992-10-24|1993-01-15|DELIVER IN PERSON|TRUCK|lly even deposit|
+2146|169|4|5|28|29936.48|0.02|0.05|R|F|1993-01-03|1992-10-17|1993-01-08|COLLECT COD|MAIL|r accounts sleep furio|
+2146|71|9|6|32|31074.24|0.07|0.03|R|F|1993-01-10|1992-10-19|1993-02-05|COLLECT COD|TRUCK|y regular foxes wake among the final|
+2146|25|6|7|39|36075.78|0.07|0.06|R|F|1993-01-05|1992-11-06|1993-01-14|DELIVER IN PERSON|TRUCK|uickly regular excuses detect. regular c|
+2147|29|8|1|50|46451.00|0.04|0.06|R|F|1992-11-18|1992-11-30|1992-11-30|NONE|RAIL|al accounts. even, even foxes wake|
+2147|101|2|2|4|4004.40|0.01|0.04|A|F|1992-09-27|1992-11-15|1992-10-22|NONE|AIR|mong the blithely special|
+2147|44|7|3|34|32097.36|0.10|0.04|R|F|1992-11-29|1992-11-08|1992-12-22|TAKE BACK RETURN|REG AIR|egular deposits hang car|
+2147|11|8|4|11|10021.11|0.06|0.07|A|F|1992-09-27|1992-11-16|1992-10-16|NONE|AIR| the fluffily|
+2148|116|6|1|21|21338.31|0.09|0.01|R|F|1995-05-28|1995-05-26|1995-06-15|NONE|FOB|deposits ag|
+2149|19|9|1|12|11028.12|0.05|0.07|R|F|1993-06-01|1993-05-06|1993-06-11|TAKE BACK RETURN|TRUCK|riously bl|
+2149|99|10|2|10|9990.90|0.06|0.01|R|F|1993-06-09|1993-04-17|1993-06-16|DELIVER IN PERSON|TRUCK|eposits sleep above|
+2149|49|2|3|47|44604.88|0.00|0.04|R|F|1993-06-27|1993-05-12|1993-07-11|COLLECT COD|AIR|hely final depo|
+2149|129|8|4|18|18524.16|0.06|0.00|A|F|1993-04-05|1993-05-11|1993-04-23|DELIVER IN PERSON|REG AIR|uriously final pac|
+2149|60|5|5|22|21121.32|0.06|0.04|R|F|1993-05-24|1993-04-23|1993-06-20|TAKE BACK RETURN|SHIP|ptotes sleep along the blithely ir|
+2150|78|7|1|26|25429.82|0.00|0.03|A|F|1994-06-21|1994-08-05|1994-06-23|NONE|TRUCK|. always unusual packages|
+2150|18|8|2|29|26622.29|0.04|0.03|A|F|1994-09-02|1994-08-04|1994-10-02|TAKE BACK RETURN|RAIL|y ironic theodolites. foxes ca|
+2150|107|2|3|29|29205.90|0.04|0.08|R|F|1994-06-10|1994-07-31|1994-06-26|COLLECT COD|RAIL|arefully final att|
+2150|54|6|4|39|37207.95|0.05|0.02|R|F|1994-07-31|1994-08-17|1994-08-11|TAKE BACK RETURN|TRUCK|ess accounts nag. unusual asymptotes haggl|
+2150|183|4|5|35|37911.30|0.01|0.01|A|F|1994-09-27|1994-08-17|1994-10-13|COLLECT COD|RAIL|refully pending dependen|
+2150|7|10|6|12|10884.00|0.09|0.03|A|F|1994-08-27|1994-08-22|1994-09-18|COLLECT COD|AIR|press platelets haggle until the slyly fi|
+2151|167|2|1|23|24544.68|0.06|0.02|N|O|1996-11-20|1996-12-17|1996-11-30|DELIVER IN PERSON|AIR| silent dependencies about the slyl|
+2151|15|9|2|29|26535.29|0.00|0.02|N|O|1997-03-04|1996-12-27|1997-03-21|TAKE BACK RETURN|SHIP| bold packages acro|
+2151|165|2|3|49|52192.84|0.07|0.01|N|O|1997-01-20|1997-02-09|1997-02-18|NONE|FOB| packages. f|
+2151|18|5|4|28|25704.28|0.10|0.08|N|O|1996-12-11|1996-12-26|1996-12-12|DELIVER IN PERSON|AIR|y special packages. carefully ironic instru|
+2176|191|4|1|38|41465.22|0.02|0.08|R|F|1992-11-29|1993-01-14|1992-12-22|DELIVER IN PERSON|REG AIR|lithely ironic pinto beans. furious|
+2176|95|8|2|14|13931.26|0.00|0.06|A|F|1992-11-17|1993-01-07|1992-12-03|DELIVER IN PERSON|SHIP|ely ironic platelets |
+2176|160|1|3|25|26504.00|0.02|0.02|R|F|1993-02-23|1993-01-05|1993-03-07|COLLECT COD|RAIL| ruthless deposits according to the ent|
+2176|143|6|4|2|2086.28|0.05|0.06|A|F|1993-02-26|1993-01-08|1993-03-23|DELIVER IN PERSON|AIR|s pinto beans|
+2177|129|10|1|45|46310.40|0.02|0.01|N|O|1997-02-11|1997-02-27|1997-02-17|NONE|SHIP|. theodolites haggle carefu|
+2177|139|5|2|27|28056.51|0.04|0.08|N|O|1997-01-29|1997-03-20|1997-02-04|DELIVER IN PERSON|SHIP|even, regula|
+2177|81|2|3|23|22564.84|0.07|0.05|N|O|1997-01-28|1997-03-02|1997-02-13|DELIVER IN PERSON|AIR|he silent foxes. iro|
+2177|55|3|4|34|32471.70|0.05|0.07|N|O|1997-02-03|1997-04-10|1997-02-21|COLLECT COD|REG AIR|tes are doggedly quickly|
+2177|57|9|5|46|44024.30|0.09|0.05|N|O|1997-05-10|1997-02-23|1997-05-28|COLLECT COD|RAIL|ending asymptotes.|
+2177|122|7|6|11|11243.32|0.02|0.04|N|O|1997-03-20|1997-03-07|1997-04-09|DELIVER IN PERSON|MAIL|gainst the ca|
+2178|157|2|1|15|15857.25|0.10|0.01|N|O|1997-03-27|1997-03-10|1997-04-18|NONE|REG AIR|l accounts. quickly expr|
+2178|16|10|2|27|24732.27|0.01|0.02|N|O|1997-02-26|1997-02-19|1997-03-25|NONE|MAIL| across the ironic reques|
+2178|5|2|3|40|36200.00|0.00|0.03|N|O|1997-03-17|1997-02-09|1997-04-15|COLLECT COD|RAIL|foxes are slowly regularly specia|
+2178|78|6|4|3|2934.21|0.07|0.07|N|O|1997-04-07|1997-01-23|1997-04-18|COLLECT COD|MAIL| permanentl|
+2179|130|9|1|22|22662.86|0.05|0.08|N|O|1996-11-16|1996-11-03|1996-11-25|DELIVER IN PERSON|FOB|lphins cajole acr|
+2179|139|5|2|20|20782.60|0.03|0.01|N|O|1996-09-30|1996-11-10|1996-10-30|NONE|REG AIR|ncies. fin|
+2179|104|9|3|5|5020.50|0.03|0.02|N|O|1996-11-09|1996-10-08|1996-11-11|DELIVER IN PERSON|REG AIR|ts haggle blithely. ironic, careful theodol|
+2179|6|3|4|24|21744.00|0.04|0.04|N|O|1996-10-26|1996-11-05|1996-11-16|COLLECT COD|RAIL| cajole carefully. |
+2179|108|5|5|7|7056.70|0.00|0.02|N|O|1996-10-24|1996-11-14|1996-11-21|TAKE BACK RETURN|RAIL|gular dependencies. ironic packages haggle|
+2180|16|3|1|31|28396.31|0.06|0.04|N|O|1996-10-20|1996-11-21|1996-11-06|COLLECT COD|REG AIR|n requests are furiously at the quickly|
+2180|193|7|2|39|42634.41|0.01|0.00|N|O|1997-01-03|1996-10-29|1997-01-25|NONE|RAIL|ep furiously furiously final request|
+2180|197|9|3|24|26332.56|0.03|0.00|N|O|1997-01-03|1996-10-24|1997-01-19|NONE|SHIP|uriously f|
+2180|111|5|4|47|47522.17|0.07|0.02|N|O|1996-09-23|1996-12-08|1996-10-12|NONE|FOB|pending, regular ideas. iron|
+2180|143|2|5|23|23992.22|0.02|0.06|N|O|1996-11-08|1996-10-25|1996-11-28|NONE|TRUCK|ggle alongside of the fluffily speci|
+2180|55|6|6|48|45842.40|0.09|0.03|N|O|1996-12-30|1996-11-22|1997-01-16|DELIVER IN PERSON|RAIL|nic instructions haggle careful|
+2181|178|9|1|4|4312.68|0.05|0.04|N|O|1995-09-25|1995-11-12|1995-09-28|COLLECT COD|FOB|tes. slyly silent packages use along th|
+2181|88|9|2|46|45451.68|0.00|0.02|N|O|1995-11-28|1995-10-17|1995-12-26|COLLECT COD|AIR|osits. final packages sleep|
+2181|91|2|3|15|14866.35|0.08|0.05|N|O|1995-10-05|1995-10-27|1995-11-03|DELIVER IN PERSON|FOB|e above the fluffily regul|
+2181|55|10|4|28|26741.40|0.04|0.05|N|O|1995-12-21|1995-10-23|1996-01-04|TAKE BACK RETURN|AIR|s excuses sleep car|
+2181|96|7|5|9|8964.81|0.06|0.07|N|O|1996-01-05|1995-12-05|1996-01-08|COLLECT COD|TRUCK|ward the quietly even requests. ir|
+2182|132|8|1|27|27867.51|0.02|0.07|R|F|1994-05-10|1994-07-04|1994-06-04|DELIVER IN PERSON|SHIP|en platele|
+2182|190|1|2|3|3270.57|0.05|0.03|R|F|1994-04-20|1994-07-04|1994-04-24|TAKE BACK RETURN|SHIP|y bold theodolites wi|
+2182|94|6|3|34|33799.06|0.02|0.03|R|F|1994-05-28|1994-06-02|1994-06-10|COLLECT COD|MAIL| slow tithes. ironi|
+2182|7|4|4|12|10884.00|0.04|0.07|A|F|1994-05-08|1994-06-02|1994-05-09|COLLECT COD|REG AIR|ments are fu|
+2182|179|9|5|37|39929.29|0.06|0.02|A|F|1994-04-08|1994-06-29|1994-04-18|TAKE BACK RETURN|TRUCK|ges. blithely ironic|
+2183|71|1|1|29|28161.03|0.05|0.01|N|O|1996-07-21|1996-08-24|1996-08-15|TAKE BACK RETURN|RAIL|ly unusual deposits sleep carefully|
+2183|52|3|2|25|23801.25|0.06|0.02|N|O|1996-07-06|1996-08-21|1996-08-05|NONE|RAIL|he quickly f|
+2208|58|3|1|48|45986.40|0.08|0.07|A|F|1995-05-13|1995-06-30|1995-05-20|COLLECT COD|MAIL|sits. idly permanent request|
+2208|97|1|2|11|10967.99|0.08|0.01|A|F|1995-05-06|1995-07-19|1995-05-22|COLLECT COD|TRUCK|ding waters lose. furiously regu|
+2208|74|4|3|41|39936.87|0.08|0.02|N|O|1995-08-18|1995-06-19|1995-09-05|COLLECT COD|RAIL|nd the furious, express dependencies.|
+2208|43|2|4|50|47152.00|0.07|0.07|N|F|1995-06-11|1995-05-31|1995-06-29|TAKE BACK RETURN|FOB|al foxes will hav|
+2208|30|5|5|43|39991.29|0.03|0.06|A|F|1995-05-10|1995-06-02|1995-06-09|TAKE BACK RETURN|MAIL|es. accounts cajole. fi|
+2208|167|2|6|18|19208.88|0.02|0.08|R|F|1995-06-06|1995-06-10|1995-06-11|TAKE BACK RETURN|TRUCK|packages are quickly bold de|
+2208|7|2|7|45|40815.00|0.00|0.08|A|F|1995-05-05|1995-06-10|1995-05-11|NONE|SHIP|e fluffily regular theodolites caj|
+2209|23|2|1|40|36920.80|0.05|0.01|R|F|1992-11-01|1992-09-25|1992-11-08|DELIVER IN PERSON|SHIP|ully special sheaves serve|
+2209|103|4|2|10|10031.00|0.00|0.02|R|F|1992-09-02|1992-09-24|1992-09-21|DELIVER IN PERSON|AIR|players. carefully reg|
+2209|64|1|3|11|10604.66|0.01|0.01|A|F|1992-07-12|1992-08-24|1992-08-10|DELIVER IN PERSON|REG AIR|express, regular pinto be|
+2209|181|2|4|39|42166.02|0.08|0.07|R|F|1992-11-04|1992-09-02|1992-11-11|TAKE BACK RETURN|MAIL|ly around the final packages. deposits ca|
+2209|124|7|5|24|24578.88|0.08|0.06|R|F|1992-08-09|1992-08-18|1992-08-25|COLLECT COD|AIR| along the bol|
+2209|178|7|6|7|7547.19|0.09|0.07|A|F|1992-08-18|1992-09-09|1992-09-12|DELIVER IN PERSON|AIR| quickly regular pack|
+2210|78|7|1|36|35210.52|0.10|0.00|A|F|1992-03-04|1992-03-24|1992-03-21|DELIVER IN PERSON|AIR| requests wake enticingly final|
+2211|48|1|1|25|23701.00|0.04|0.01|A|F|1994-10-09|1994-08-04|1994-11-03|TAKE BACK RETURN|RAIL|deas. carefully special theodolites along|
+2211|140|6|2|40|41605.60|0.09|0.06|A|F|1994-09-30|1994-09-10|1994-10-26|NONE|MAIL|posits among the express dolphins|
+2211|160|2|3|25|26504.00|0.00|0.07|A|F|1994-08-13|1994-08-17|1994-08-16|NONE|AIR|ly regular, express|
+2211|85|6|4|23|22656.84|0.03|0.02|R|F|1994-10-05|1994-09-13|1994-10-17|DELIVER IN PERSON|AIR|ependencies |
+2211|135|1|5|3|3105.39|0.02|0.04|A|F|1994-08-28|1994-09-10|1994-09-06|TAKE BACK RETURN|SHIP|pendencies after the regular f|
+2211|187|8|6|18|19569.24|0.05|0.08|A|F|1994-08-31|1994-09-07|1994-09-22|NONE|TRUCK|c grouches. slyly express pinto |
+2211|79|9|7|3|2937.21|0.06|0.05|R|F|1994-09-21|1994-08-10|1994-10-19|TAKE BACK RETURN|RAIL|y slyly final|
+2212|71|10|1|18|17479.26|0.07|0.06|R|F|1994-06-22|1994-06-18|1994-06-25|TAKE BACK RETURN|FOB| cajole. final, pending ideas should are bl|
+2213|118|8|1|20|20362.20|0.01|0.00|A|F|1993-01-21|1993-04-14|1993-01-29|COLLECT COD|REG AIR|iously express accounts; |
+2213|60|1|2|4|3840.24|0.09|0.05|R|F|1993-04-15|1993-04-15|1993-05-05|COLLECT COD|SHIP| affix carefully furiously |
+2213|70|5|3|1|970.07|0.05|0.05|A|F|1993-04-25|1993-04-06|1993-04-28|TAKE BACK RETURN|AIR|s along the ironic reques|
+2213|174|3|4|39|41892.63|0.09|0.05|A|F|1993-05-12|1993-04-07|1993-05-23|TAKE BACK RETURN|SHIP|the blithely |
+2213|38|9|5|43|40335.29|0.04|0.03|A|F|1993-04-18|1993-03-11|1993-05-11|TAKE BACK RETURN|RAIL|r packages are along the carefully bol|
+2213|48|5|6|41|38869.64|0.01|0.00|R|F|1993-01-31|1993-03-31|1993-02-28|COLLECT COD|FOB| carefully pend|
+2213|64|9|7|3|2892.18|0.02|0.04|A|F|1993-03-09|1993-03-17|1993-04-07|TAKE BACK RETURN|AIR|o wake. ironic platel|
+2214|76|5|1|27|26353.89|0.04|0.04|N|O|1998-05-31|1998-06-07|1998-06-19|DELIVER IN PERSON|REG AIR|x fluffily along the even packages-- |
+2214|194|5|2|50|54709.50|0.00|0.02|N|O|1998-07-06|1998-06-16|1998-07-16|TAKE BACK RETURN|MAIL|accounts. blith|
+2214|113|7|3|42|42550.62|0.04|0.08|N|O|1998-05-26|1998-07-13|1998-06-22|COLLECT COD|FOB|ons. deposi|
+2214|196|9|4|22|24116.18|0.01|0.01|N|O|1998-05-30|1998-07-02|1998-06-09|DELIVER IN PERSON|RAIL|t the blithely|
+2215|73|1|1|33|32111.31|0.00|0.00|N|O|1996-07-19|1996-08-10|1996-07-30|COLLECT COD|RAIL|dolites cajole b|
+2215|33|9|2|30|27990.90|0.01|0.00|N|O|1996-08-15|1996-09-10|1996-08-25|NONE|FOB|ckages caj|
+2215|57|5|3|30|28711.50|0.07|0.03|N|O|1996-09-09|1996-07-20|1996-09-28|COLLECT COD|TRUCK|against the carefu|
+2215|146|3|4|20|20922.80|0.02|0.02|N|O|1996-09-09|1996-08-10|1996-09-19|NONE|MAIL| unusual deposits haggle carefully. ide|
+2240|164|3|1|6|6384.96|0.01|0.00|A|F|1992-06-23|1992-05-17|1992-07-20|COLLECT COD|AIR|ymptotes boost. furiously bold p|
+2240|28|1|2|37|34336.74|0.03|0.07|R|F|1992-03-16|1992-05-31|1992-04-05|COLLECT COD|FOB| quickly after the packages? blithely si|
+2240|53|5|3|39|37168.95|0.08|0.06|A|F|1992-05-22|1992-05-10|1992-06-08|NONE|FOB|y orbits. final depos|
+2240|86|7|4|10|9860.80|0.09|0.00|A|F|1992-05-25|1992-04-14|1992-06-23|DELIVER IN PERSON|REG AIR|are across the ironic packages.|
+2240|161|10|5|29|30773.64|0.02|0.06|A|F|1992-03-29|1992-05-08|1992-04-09|COLLECT COD|MAIL|lyly even ideas w|
+2240|81|2|6|32|31394.56|0.06|0.06|R|F|1992-04-11|1992-04-18|1992-04-22|NONE|MAIL|ss thinly deposits. blithely bold package|
+2240|78|7|7|24|23473.68|0.04|0.05|R|F|1992-05-13|1992-04-09|1992-05-14|DELIVER IN PERSON|FOB|ng the silent accounts. slyly ironic t|
+2241|5|6|1|25|22625.00|0.00|0.08|R|F|1993-08-11|1993-07-23|1993-09-01|DELIVER IN PERSON|MAIL| final deposits use fluffily. even f|
+2241|195|8|2|38|41617.22|0.04|0.06|A|F|1993-08-04|1993-07-31|1993-08-06|TAKE BACK RETURN|TRUCK| silent, unusual d|
+2241|97|10|3|48|47860.32|0.08|0.04|A|F|1993-05-14|1993-07-30|1993-05-26|TAKE BACK RETURN|RAIL|ss accounts engage furiously. slyly even re|
+2241|167|4|4|19|20276.04|0.10|0.00|A|F|1993-06-01|1993-08-05|1993-06-07|TAKE BACK RETURN|TRUCK| are furiously quickl|
+2241|82|3|5|2|1964.16|0.04|0.03|A|F|1993-08-16|1993-08-02|1993-08-24|NONE|REG AIR|, express deposits. pear|
+2241|116|3|6|22|22354.42|0.02|0.08|R|F|1993-08-13|1993-06-15|1993-08-16|DELIVER IN PERSON|TRUCK|, ironic depen|
+2241|142|3|7|9|9379.26|0.09|0.03|A|F|1993-05-14|1993-07-12|1993-05-29|NONE|AIR|lyly final |
+2242|123|4|1|15|15346.80|0.09|0.08|N|O|1997-08-04|1997-09-21|1997-08-11|COLLECT COD|FOB|its. carefully express packages cajole. bli|
+2243|127|8|1|10|10271.20|0.04|0.06|N|O|1995-07-26|1995-07-18|1995-08-03|NONE|RAIL|express, daring foxes affix fur|
+2244|51|6|1|3|2853.15|0.02|0.02|A|F|1993-04-30|1993-03-15|1993-05-19|TAKE BACK RETURN|FOB| beans for the regular platel|
+2244|193|6|2|16|17491.04|0.01|0.06|R|F|1993-02-12|1993-03-09|1993-02-28|COLLECT COD|FOB|rate around the reques|
+2245|76|7|1|44|42947.08|0.03|0.03|A|F|1993-06-12|1993-06-10|1993-06-16|NONE|TRUCK|refully even sheaves|
+2245|74|3|2|28|27273.96|0.05|0.03|R|F|1993-08-19|1993-07-27|1993-09-04|COLLECT COD|TRUCK|e requests sleep furiou|
+2245|86|7|3|33|32540.64|0.03|0.01|R|F|1993-06-26|1993-06-11|1993-07-17|TAKE BACK RETURN|AIR|ing to the carefully ruthless accounts|
+2245|189|10|4|14|15248.52|0.02|0.04|R|F|1993-05-06|1993-07-21|1993-05-19|DELIVER IN PERSON|RAIL|nts. always unusual dep|
+2245|80|8|5|33|32342.64|0.03|0.07|R|F|1993-06-16|1993-06-05|1993-07-07|NONE|MAIL| across the express reques|
+2246|53|4|1|22|20967.10|0.02|0.01|N|O|1996-07-25|1996-08-03|1996-08-24|DELIVER IN PERSON|SHIP|ructions wake carefully fina|
+2246|104|5|2|43|43176.30|0.07|0.06|N|O|1996-08-25|1996-08-23|1996-09-19|DELIVER IN PERSON|AIR|ainst the ironic theodolites haggle fi|
+2246|18|8|3|11|10098.11|0.10|0.00|N|O|1996-06-21|1996-07-24|1996-07-18|TAKE BACK RETURN|TRUCK|quests alongside o|
+2246|163|8|4|13|13821.08|0.08|0.05|N|O|1996-09-15|1996-07-21|1996-10-08|DELIVER IN PERSON|AIR|equests. fluffily special epitaphs use|
+2247|172|2|1|12|12866.04|0.02|0.07|A|F|1992-09-06|1992-09-18|1992-09-26|NONE|MAIL|final accounts. requests across the furiou|
+2272|90|1|1|18|17821.62|0.04|0.00|R|F|1993-08-01|1993-07-06|1993-08-25|NONE|MAIL|ons along the blithely e|
+2272|34|10|2|40|37361.20|0.07|0.00|A|F|1993-04-25|1993-07-12|1993-05-15|DELIVER IN PERSON|FOB|lithely ir|
+2272|56|4|3|36|34417.80|0.03|0.02|A|F|1993-05-25|1993-05-23|1993-06-09|TAKE BACK RETURN|RAIL|about the ironic packages; quickly iron|
+2272|138|4|4|30|31143.90|0.09|0.07|A|F|1993-07-27|1993-05-15|1993-08-13|NONE|RAIL|quests at the foxes haggle evenly pack|
+2272|76|4|5|12|11712.84|0.03|0.03|A|F|1993-04-19|1993-05-14|1993-04-23|NONE|RAIL| accounts cajole. quickly b|
+2273|184|5|1|34|36862.12|0.02|0.03|N|O|1997-01-08|1997-02-02|1997-01-23|COLLECT COD|MAIL| furiously carefully bold de|
+2273|85|6|2|35|34477.80|0.00|0.05|N|O|1997-01-02|1997-01-19|1997-01-14|NONE|REG AIR|arefully f|
+2273|95|8|3|8|7960.72|0.00|0.04|N|O|1996-12-15|1997-02-27|1997-01-10|NONE|FOB|dependencies. slyly ir|
+2273|161|6|4|20|21223.20|0.06|0.04|N|O|1997-03-05|1997-02-25|1997-04-01|NONE|RAIL|cuses. quickly enticing requests wake |
+2273|162|7|5|18|19118.88|0.07|0.00|N|O|1996-12-16|1997-01-21|1997-01-03|COLLECT COD|TRUCK| beans. doggedly final packages wake|
+2273|155|7|6|16|16882.40|0.10|0.03|N|O|1997-01-10|1997-02-03|1997-02-01|TAKE BACK RETURN|RAIL|furiously above the ironic requests. |
+2273|20|1|7|7|6440.14|0.05|0.05|N|O|1997-02-19|1997-01-22|1997-02-21|TAKE BACK RETURN|TRUCK|ts. furiou|
+2274|12|6|1|18|16416.18|0.04|0.03|R|F|1993-09-06|1993-12-03|1993-09-22|COLLECT COD|SHIP|usly final re|
+2274|111|8|2|23|23255.53|0.04|0.03|R|F|1993-10-28|1993-11-03|1993-11-05|NONE|MAIL|kly special warhorse|
+2274|129|10|3|18|18524.16|0.03|0.06|R|F|1993-09-28|1993-11-22|1993-10-12|DELIVER IN PERSON|SHIP| express packages. even accounts hagg|
+2275|34|5|1|30|28020.90|0.08|0.05|R|F|1993-01-10|1992-11-21|1993-01-22|NONE|REG AIR|re slyly slyly special idea|
+2275|91|4|2|11|10901.99|0.08|0.03|A|F|1993-01-16|1992-12-10|1993-01-25|COLLECT COD|REG AIR|ost across the never express instruction|
+2276|119|9|1|5|5095.55|0.07|0.08|N|O|1996-05-09|1996-06-18|1996-05-13|DELIVER IN PERSON|FOB|ias instea|
+2276|135|1|2|13|13456.69|0.08|0.04|N|O|1996-07-24|1996-06-18|1996-08-16|COLLECT COD|RAIL|arefully ironic foxes cajole q|
+2276|171|2|3|27|28921.59|0.07|0.08|N|O|1996-07-30|1996-06-10|1996-07-31|DELIVER IN PERSON|RAIL|the carefully unusual accoun|
+2276|109|6|4|38|38345.80|0.06|0.03|N|O|1996-07-07|1996-06-28|1996-07-17|COLLECT COD|RAIL|ans. pinto beans boost c|
+2276|153|5|5|50|52657.50|0.03|0.05|N|O|1996-07-13|1996-06-25|1996-07-22|DELIVER IN PERSON|REG AIR| accounts dete|
+2276|6|9|6|4|3624.00|0.10|0.03|N|O|1996-07-05|1996-06-30|1996-08-04|COLLECT COD|FOB|s. deposits |
+2277|137|8|1|38|39410.94|0.03|0.07|R|F|1995-04-23|1995-03-25|1995-05-20|TAKE BACK RETURN|TRUCK|fully bold|
+2277|8|1|2|2|1816.00|0.10|0.08|A|F|1995-02-01|1995-02-04|1995-03-02|TAKE BACK RETURN|AIR|endencies sleep idly pending p|
+2277|198|10|3|4|4392.76|0.05|0.06|R|F|1995-04-27|1995-03-16|1995-04-29|TAKE BACK RETURN|SHIP|. quickly unusual deposi|
+2277|159|4|4|31|32833.65|0.02|0.00|R|F|1995-03-07|1995-03-19|1995-03-26|TAKE BACK RETURN|MAIL|ic instructions detect ru|
+2278|45|2|1|36|34021.44|0.04|0.05|N|O|1998-06-04|1998-06-06|1998-06-30|NONE|TRUCK|y ironic pinto beans br|
+2278|45|2|2|50|47252.00|0.02|0.00|N|O|1998-08-09|1998-07-08|1998-09-05|DELIVER IN PERSON|RAIL|into beans. blit|
+2278|97|9|3|22|21935.98|0.03|0.00|N|O|1998-05-15|1998-07-14|1998-06-04|TAKE BACK RETURN|REG AIR|ep regular accounts. blithely even|
+2279|14|5|1|12|10968.12|0.07|0.08|A|F|1993-05-10|1993-03-25|1993-06-02|COLLECT COD|REG AIR|lets across the excuses nag quickl|
+2279|41|2|2|38|35759.52|0.08|0.07|R|F|1993-06-09|1993-04-06|1993-06-26|COLLECT COD|TRUCK|s above the furiously express dep|
+2279|4|7|3|3|2712.00|0.09|0.04|A|F|1993-05-31|1993-05-07|1993-06-05|COLLECT COD|REG AIR|ing foxes above the even accounts use slyly|
+2279|52|4|4|42|39986.10|0.02|0.00|R|F|1993-02-28|1993-04-25|1993-03-02|TAKE BACK RETURN|REG AIR| above the furiously ironic deposits. |
+2279|169|8|5|9|9622.44|0.05|0.04|R|F|1993-05-21|1993-03-29|1993-06-17|DELIVER IN PERSON|MAIL|ns cajole after the final platelets. s|
+2279|147|10|6|12|12565.68|0.02|0.00|R|F|1993-05-04|1993-04-26|1993-05-28|DELIVER IN PERSON|FOB|ccounts. slyl|
+2279|119|9|7|32|32611.52|0.05|0.05|A|F|1993-04-20|1993-05-22|1993-05-18|DELIVER IN PERSON|RAIL|re quickly. furiously ironic ide|
+2304|200|2|1|42|46208.40|0.00|0.01|A|F|1994-01-20|1994-03-04|1994-02-05|COLLECT COD|RAIL|quests are blithely alongside of|
+2304|19|9|2|48|44112.48|0.00|0.00|R|F|1994-02-12|1994-02-16|1994-03-10|COLLECT COD|REG AIR| deposits cajole blithely e|
+2304|48|9|3|3|2844.12|0.00|0.05|R|F|1994-03-19|1994-03-04|1994-03-20|DELIVER IN PERSON|AIR|l excuses after the ev|
+2305|174|4|1|3|3222.51|0.00|0.01|A|F|1993-03-24|1993-04-05|1993-03-29|NONE|AIR|kages haggle quickly across the blithely |
+2305|60|8|2|39|37442.34|0.07|0.00|R|F|1993-04-16|1993-04-17|1993-04-22|COLLECT COD|MAIL|ms after the foxes |
+2305|102|3|3|32|32067.20|0.03|0.06|A|F|1993-04-02|1993-03-18|1993-04-03|NONE|AIR| haggle caref|
+2305|112|3|4|17|17205.87|0.00|0.05|A|F|1993-02-21|1993-03-30|1993-03-19|TAKE BACK RETURN|MAIL| carefully alongside of |
+2305|155|7|5|26|27433.90|0.06|0.07|A|F|1993-05-14|1993-02-28|1993-06-04|NONE|SHIP|arefully final theodo|
+2305|51|3|6|7|6657.35|0.06|0.00|R|F|1993-05-15|1993-04-25|1993-06-09|DELIVER IN PERSON|RAIL|gular deposits boost about the foxe|
+2306|196|9|1|50|54809.50|0.09|0.01|N|O|1995-07-27|1995-09-26|1995-08-06|DELIVER IN PERSON|FOB|y quickly |
+2306|149|2|2|39|40916.46|0.04|0.00|N|O|1995-09-07|1995-09-13|1995-10-03|COLLECT COD|SHIP|f the slyly unusual accounts. furiousl|
+2306|178|6|3|35|37735.95|0.01|0.07|N|O|1995-08-18|1995-08-30|1995-08-20|TAKE BACK RETURN|RAIL|raids along the furiously unusual asympto|
+2306|119|3|4|21|21401.31|0.06|0.01|N|O|1995-10-07|1995-09-18|1995-10-17|COLLECT COD|MAIL| ironic pinto |
+2306|142|9|5|42|43769.88|0.04|0.07|N|O|1995-09-05|1995-08-25|1995-09-28|COLLECT COD|MAIL|furiously final acco|
+2306|124|5|6|29|29699.48|0.00|0.03|N|O|1995-11-01|1995-09-01|1995-11-22|DELIVER IN PERSON|REG AIR|uld have to mold. s|
+2306|176|4|7|19|20447.23|0.07|0.01|N|O|1995-11-17|1995-09-06|1995-11-30|DELIVER IN PERSON|AIR|tainments nag furiously carefull|
+2307|142|9|1|24|25011.36|0.10|0.05|R|F|1993-10-07|1993-08-05|1993-10-20|COLLECT COD|AIR|stealthily special packages nag a|
+2307|140|6|2|2|2080.28|0.01|0.00|A|F|1993-09-21|1993-08-22|1993-10-03|COLLECT COD|SHIP|ously. furiously furious requ|
+2307|34|10|3|7|6538.21|0.07|0.04|R|F|1993-08-03|1993-09-04|1993-08-28|DELIVER IN PERSON|AIR|ven instructions wake fluffily |
+2307|165|6|4|19|20238.04|0.08|0.06|R|F|1993-10-23|1993-09-09|1993-11-09|TAKE BACK RETURN|TRUCK|olites haggle furiously around the |
+2307|143|4|5|7|7301.98|0.01|0.06|R|F|1993-09-01|1993-08-08|1993-09-29|NONE|AIR| packages cajo|
+2308|118|9|1|24|24434.64|0.06|0.04|R|F|1993-02-23|1992-12-24|1993-03-10|NONE|MAIL|ts sleep. busy excuses along the s|
+2308|56|1|2|36|34417.80|0.05|0.06|A|F|1992-11-11|1992-11-27|1992-11-23|NONE|MAIL|ong the pending hockey players. blithe|
+2309|170|7|1|14|14982.38|0.10|0.03|N|O|1996-01-01|1995-10-22|1996-01-23|NONE|AIR|asymptotes. furiously pending acco|
+2309|169|8|2|1|1069.16|0.01|0.05|N|O|1995-12-08|1995-11-03|1995-12-31|COLLECT COD|RAIL|eposits alongside of the final re|
+2309|15|2|3|5|4575.05|0.01|0.00|N|O|1995-12-10|1995-10-29|1996-01-06|TAKE BACK RETURN|SHIP|s. requests wake blithely specia|
+2309|139|10|4|46|47799.98|0.08|0.04|N|O|1995-10-02|1995-10-30|1995-10-30|NONE|REG AIR|sly according to the carefully |
+2309|137|3|5|9|9334.17|0.00|0.07|N|O|1995-12-21|1995-10-10|1996-01-20|COLLECT COD|AIR|ding, unusual instructions. dep|
+2309|195|8|6|21|22998.99|0.09|0.00|N|O|1995-11-05|1995-11-07|1995-11-22|NONE|AIR|unts around the dolphins ar|
+2309|138|4|7|48|49830.24|0.03|0.05|N|O|1995-10-21|1995-11-21|1995-11-09|NONE|MAIL|ccounts. id|
+2310|58|6|1|36|34489.80|0.03|0.03|N|O|1996-10-09|1996-10-28|1996-10-29|TAKE BACK RETURN|RAIL|iously against the slyly special accounts|
+2310|171|2|2|6|6427.02|0.07|0.01|N|O|1996-11-08|1996-12-09|1996-12-07|COLLECT COD|REG AIR|e slyly about the quickly ironic theodo|
+2310|42|1|3|48|45217.92|0.08|0.02|N|O|1996-10-04|1996-11-20|1996-10-25|TAKE BACK RETURN|FOB|ep slyly alongside of the |
+2311|141|8|1|18|18740.52|0.01|0.01|N|F|1995-06-11|1995-06-18|1995-07-02|NONE|FOB| fluffily even patterns haggle blithely. re|
+2311|122|1|2|49|50083.88|0.09|0.02|R|F|1995-05-14|1995-07-11|1995-05-20|COLLECT COD|FOB|ideas sleep|
+2311|54|5|3|15|14310.75|0.08|0.04|N|O|1995-06-23|1995-06-06|1995-07-09|COLLECT COD|AIR|ve the blithely pending accounts. furio|
+2311|90|1|4|42|41583.78|0.01|0.06|R|F|1995-06-03|1995-06-27|1995-06-11|DELIVER IN PERSON|MAIL|gle furiously. bold |
+2311|47|10|5|1|947.04|0.05|0.02|A|F|1995-06-07|1995-06-20|1995-06-10|NONE|AIR|ptotes. furiously regular theodolite|
+2311|12|9|6|32|29184.32|0.01|0.03|N|O|1995-07-19|1995-06-26|1995-07-26|NONE|RAIL|sts along the slyly|
+2336|193|5|1|20|21863.80|0.01|0.03|N|O|1996-03-12|1996-02-25|1996-03-18|NONE|REG AIR|across the fi|
+2337|45|2|1|49|46306.96|0.06|0.05|N|O|1997-08-08|1997-08-15|1997-08-31|TAKE BACK RETURN|FOB| along the packages. furiously p|
+2338|52|7|1|30|28561.50|0.07|0.06|N|O|1997-12-10|1997-10-15|1997-12-11|TAKE BACK RETURN|REG AIR|ould have to nag quickly|
+2339|192|3|1|22|24028.18|0.03|0.03|A|F|1994-01-06|1994-03-06|1994-01-10|NONE|FOB| furiously above |
+2339|30|5|2|28|26040.84|0.00|0.00|R|F|1994-01-25|1994-01-22|1994-01-28|DELIVER IN PERSON|RAIL|e bold, even packag|
+2339|117|4|3|13|13222.43|0.06|0.08|R|F|1994-03-10|1994-02-18|1994-03-20|TAKE BACK RETURN|REG AIR|ges. blithely special depend|
+2340|138|4|1|9|9343.17|0.08|0.02|N|O|1996-05-01|1996-02-24|1996-05-16|COLLECT COD|RAIL|. carefully ironic|
+2340|193|5|2|21|22956.99|0.06|0.02|N|O|1996-01-17|1996-03-04|1996-01-29|DELIVER IN PERSON|SHIP| asymptotes. unusual theo|
+2341|47|10|1|12|11364.48|0.08|0.03|R|F|1993-06-06|1993-07-08|1993-06-17|DELIVER IN PERSON|FOB|. quickly final deposits sl|
+2341|71|10|2|37|35929.59|0.07|0.08|A|F|1993-09-23|1993-07-25|1993-10-14|DELIVER IN PERSON|RAIL|was blithel|
+2341|195|8|3|8|8761.52|0.03|0.07|R|F|1993-06-08|1993-07-09|1993-06-10|COLLECT COD|FOB|ns affix above the iron|
+2342|42|1|1|12|11304.48|0.00|0.08|N|O|1996-07-31|1996-07-26|1996-08-14|NONE|TRUCK|print blithely even deposits. carefull|
+2342|117|1|2|24|24410.64|0.10|0.06|N|O|1996-09-30|1996-07-22|1996-10-28|TAKE BACK RETURN|AIR|nstructions c|
+2342|170|1|3|50|53508.50|0.10|0.01|N|O|1996-08-28|1996-07-18|1996-09-22|COLLECT COD|RAIL|cial asymptotes pr|
+2342|36|7|4|1|936.03|0.04|0.06|N|O|1996-08-31|1996-08-09|1996-09-07|DELIVER IN PERSON|REG AIR|ffily. unusual pinto beans wake c|
+2342|27|2|5|22|20394.44|0.08|0.01|N|O|1996-08-10|1996-08-02|1996-08-31|DELIVER IN PERSON|AIR|s. ironic |
+2343|110|1|1|27|27272.97|0.00|0.00|N|O|1995-11-10|1995-11-17|1995-12-10|TAKE BACK RETURN|SHIP|old theodolites.|
+2343|66|1|2|35|33812.10|0.03|0.06|N|O|1995-10-24|1995-11-09|1995-10-26|COLLECT COD|TRUCK|ges haggle furiously carefully regular req|
+2343|179|7|3|21|22662.57|0.00|0.03|N|O|1995-09-07|1995-10-26|1995-10-07|TAKE BACK RETURN|RAIL|osits. unusual theodolites boost furio|
+2368|152|3|1|16|16834.40|0.04|0.03|R|F|1993-10-31|1993-10-22|1993-11-06|NONE|REG AIR|telets wake carefully iro|
+2368|14|5|2|32|29248.32|0.03|0.00|R|F|1993-09-23|1993-10-07|1993-09-27|COLLECT COD|TRUCK|gular courts use blithely around the|
+2368|149|6|3|39|40916.46|0.08|0.03|R|F|1993-09-03|1993-09-20|1993-09-28|COLLECT COD|RAIL|ng the doggedly ironic requests are blithe|
+2368|156|8|4|17|17954.55|0.10|0.08|A|F|1993-10-03|1993-09-27|1993-10-05|NONE|FOB|fily. slyly final ideas alongside o|
+2369|24|7|1|30|27720.60|0.05|0.04|N|O|1997-04-23|1997-02-12|1997-05-21|COLLECT COD|REG AIR|pecial deposits sleep. blithely unusual w|
+2369|169|10|2|47|50250.52|0.10|0.02|N|O|1997-01-02|1997-02-18|1997-01-13|COLLECT COD|RAIL| to the regular dep|
+2370|46|3|1|3|2838.12|0.03|0.07|R|F|1994-03-24|1994-03-26|1994-04-15|COLLECT COD|SHIP|ly regular Tiresia|
+2370|2|5|2|24|21648.00|0.00|0.05|A|F|1994-05-15|1994-04-09|1994-06-12|NONE|REG AIR|final depen|
+2370|61|2|3|32|30753.92|0.05|0.02|A|F|1994-04-24|1994-03-03|1994-05-15|DELIVER IN PERSON|MAIL|ies since the final deposits|
+2370|6|3|4|21|19026.00|0.04|0.01|R|F|1994-02-01|1994-02-19|1994-02-09|TAKE BACK RETURN|MAIL|ecial dependencies must have to |
+2371|159|4|1|37|39188.55|0.05|0.05|N|O|1998-02-11|1998-03-24|1998-02-27|DELIVER IN PERSON|TRUCK|s boost fluffil|
+2371|35|1|2|21|19635.63|0.00|0.05|N|O|1998-04-14|1998-02-14|1998-04-18|COLLECT COD|AIR|gle furiously regu|
+2371|101|4|3|11|11012.10|0.05|0.02|N|O|1998-02-25|1998-04-06|1998-03-23|TAKE BACK RETURN|TRUCK|requests. regular pinto beans wake. car|
+2371|43|6|4|33|31120.32|0.05|0.08|N|O|1998-03-30|1998-02-06|1998-04-05|DELIVER IN PERSON|AIR|deas are. express r|
+2371|165|2|5|22|23433.52|0.02|0.05|N|O|1998-03-26|1998-03-19|1998-04-16|DELIVER IN PERSON|REG AIR|y daring accounts. regular ins|
+2371|86|7|6|39|38457.12|0.05|0.03|N|O|1998-04-01|1998-03-13|1998-04-27|NONE|REG AIR|tructions. regular, stealthy packages wak|
+2371|36|2|7|32|29952.96|0.07|0.07|N|O|1998-02-15|1998-04-03|1998-02-23|NONE|REG AIR|the ruthless accounts. |
+2372|43|4|1|42|39607.68|0.08|0.02|N|O|1998-01-04|1998-01-02|1998-02-02|COLLECT COD|REG AIR|lar packages. regular|
+2372|3|10|2|17|15351.00|0.07|0.01|N|O|1997-12-17|1998-01-17|1997-12-25|NONE|RAIL|xcuses. slyly ironic theod|
+2372|164|1|3|12|12769.92|0.04|0.04|N|O|1998-03-21|1997-12-21|1998-04-12|DELIVER IN PERSON|SHIP|lyly according to|
+2372|122|1|4|4|4088.48|0.00|0.07|N|O|1997-12-14|1997-12-28|1997-12-16|TAKE BACK RETURN|REG AIR|e carefully blithely even epitaphs. r|
+2372|20|7|5|5|4600.10|0.02|0.04|N|O|1998-02-08|1998-01-18|1998-03-02|TAKE BACK RETURN|RAIL|ets against the |
+2372|189|10|6|11|11980.98|0.02|0.06|N|O|1998-02-14|1998-01-18|1998-03-10|TAKE BACK RETURN|FOB| silent, pending de|
+2372|57|8|7|19|18183.95|0.01|0.06|N|O|1997-12-26|1998-02-19|1998-01-02|COLLECT COD|SHIP| beans haggle sometimes|
+2373|191|5|1|17|18550.23|0.02|0.01|R|F|1994-03-29|1994-05-19|1994-04-20|COLLECT COD|AIR|auternes. blithely even pinto bea|
+2373|136|2|2|3|3108.39|0.08|0.08|R|F|1994-05-15|1994-06-10|1994-06-04|COLLECT COD|TRUCK|dependencies wake ironical|
+2373|141|8|3|29|30193.06|0.05|0.02|A|F|1994-06-01|1994-05-14|1994-06-17|NONE|TRUCK|yly silent ideas affix furiousl|
+2373|91|5|4|5|4955.45|0.10|0.01|R|F|1994-06-02|1994-05-03|1994-06-21|NONE|REG AIR|uffily blithely ironic requests|
+2374|118|2|1|41|41742.51|0.07|0.00|A|F|1994-01-27|1993-12-11|1994-02-12|TAKE BACK RETURN|RAIL|heodolites. requests|
+2374|160|2|2|24|25443.84|0.07|0.08|A|F|1994-02-02|1994-01-12|1994-02-04|DELIVER IN PERSON|TRUCK|. requests are above t|
+2374|61|8|3|2|1922.12|0.06|0.02|R|F|1993-12-30|1994-01-24|1994-01-02|COLLECT COD|FOB|, unusual ideas. deposits cajole quietl|
+2374|74|5|4|28|27273.96|0.04|0.08|R|F|1994-02-19|1993-12-16|1994-03-15|COLLECT COD|MAIL|ets cajole fu|
+2374|1|2|5|25|22525.00|0.08|0.00|A|F|1993-11-26|1993-12-15|1993-12-10|COLLECT COD|RAIL|refully pending d|
+2375|168|9|1|3|3204.48|0.02|0.08|N|O|1997-02-14|1996-12-25|1997-02-22|COLLECT COD|RAIL|slyly across the furiously e|
+2375|132|8|2|9|9289.17|0.09|0.02|N|O|1997-02-17|1996-12-27|1997-02-27|DELIVER IN PERSON|MAIL|ly against the packages. bold pinto bean|
+2375|47|4|3|26|24623.04|0.02|0.06|N|O|1997-03-18|1997-02-02|1997-03-29|TAKE BACK RETURN|TRUCK|rate across the|
+2375|5|8|4|5|4525.00|0.01|0.00|N|O|1997-01-31|1997-01-25|1997-02-22|COLLECT COD|REG AIR|final packages cajole according to the furi|
+2375|88|9|5|42|41499.36|0.01|0.08|N|O|1997-01-24|1997-02-15|1997-02-07|DELIVER IN PERSON|FOB|apades. idea|
+2375|126|7|6|20|20522.40|0.09|0.08|N|O|1996-12-01|1996-12-26|1996-12-19|TAKE BACK RETURN|SHIP|ckages! blithely enticing deposi|
+2400|103|6|1|48|48148.80|0.01|0.02|N|O|1998-10-07|1998-08-30|1998-11-03|DELIVER IN PERSON|MAIL|fore the car|
+2400|90|1|2|1|990.09|0.04|0.07|N|O|1998-08-18|1998-09-12|1998-09-11|NONE|MAIL|silent deposits serve furious|
+2400|53|5|3|23|21920.15|0.02|0.08|N|O|1998-08-05|1998-08-28|1998-08-30|NONE|SHIP|tions. fluffily ironic platelets cajole c|
+2400|17|7|4|23|21091.23|0.09|0.04|N|O|1998-10-04|1998-10-04|1998-10-31|NONE|RAIL|ages lose carefully around the regula|
+2401|182|3|1|39|42205.02|0.00|0.03|N|O|1997-09-29|1997-10-21|1997-10-17|DELIVER IN PERSON|FOB|ould affix |
+2401|3|8|2|49|44247.00|0.05|0.07|N|O|1997-09-02|1997-09-11|1997-09-13|TAKE BACK RETURN|AIR|lites cajole carefully |
+2402|86|7|1|43|42401.44|0.03|0.08|N|O|1996-09-17|1996-11-20|1996-09-22|DELIVER IN PERSON|RAIL|slyly slyly blithe sheaves|
+2402|152|4|2|24|25251.60|0.02|0.05|N|O|1996-11-21|1996-10-19|1996-11-29|DELIVER IN PERSON|SHIP|as; blithely ironic requ|
+2403|83|4|1|34|33424.72|0.04|0.07|N|O|1998-05-30|1998-06-19|1998-06-05|NONE|REG AIR| slyly bold re|
+2403|152|4|2|19|19990.85|0.08|0.07|N|O|1998-04-20|1998-07-02|1998-05-13|DELIVER IN PERSON|FOB|sits. ironic in|
+2403|193|4|3|27|29516.13|0.05|0.03|N|O|1998-07-27|1998-07-08|1998-08-03|NONE|SHIP|deposits sleep slyly special theodolit|
+2403|31|2|4|30|27930.90|0.05|0.06|N|O|1998-08-08|1998-06-17|1998-08-20|NONE|TRUCK|ackages sleep furiously pendin|
+2404|147|10|1|36|37697.04|0.07|0.00|N|O|1997-03-27|1997-05-16|1997-04-06|COLLECT COD|REG AIR|s nag furi|
+2404|36|2|2|1|936.03|0.02|0.04|N|O|1997-05-22|1997-06-06|1997-05-28|DELIVER IN PERSON|MAIL|from the final orbits? even pinto beans hag|
+2404|18|5|3|41|37638.41|0.02|0.06|N|O|1997-06-12|1997-05-03|1997-07-12|NONE|AIR| dolphins are|
+2404|57|8|4|19|18183.95|0.09|0.03|N|O|1997-05-07|1997-05-24|1997-05-24|TAKE BACK RETURN|SHIP|cuses. quickly even in|
+2404|4|9|5|18|16272.00|0.00|0.04|N|O|1997-06-25|1997-05-06|1997-07-02|NONE|RAIL|packages. even requests according to |
+2405|89|10|1|18|17803.44|0.09|0.07|N|O|1997-01-23|1997-03-10|1997-02-03|COLLECT COD|REG AIR|carefully ironic accounts. slyly |
+2405|27|10|2|30|27810.60|0.10|0.08|N|O|1997-03-24|1997-03-10|1997-04-14|TAKE BACK RETURN|AIR|y final deposits are slyly caref|
+2405|17|8|3|49|44933.49|0.00|0.06|N|O|1996-12-24|1997-03-23|1997-01-01|TAKE BACK RETURN|FOB|cial requests. ironic, regu|
+2405|177|7|4|23|24774.91|0.08|0.05|N|O|1996-12-28|1997-01-29|1997-01-07|NONE|AIR|t wake blithely blithely regular idea|
+2406|170|5|1|18|19263.06|0.07|0.05|N|O|1997-02-17|1996-12-25|1997-02-19|COLLECT COD|MAIL|azzle furiously careful|
+2406|41|8|2|40|37641.60|0.02|0.07|N|O|1997-01-09|1996-12-02|1997-01-16|NONE|SHIP|gular accounts caj|
+2406|50|1|3|16|15200.80|0.07|0.03|N|O|1996-10-31|1996-11-28|1996-11-08|TAKE BACK RETURN|SHIP| special accou|
+2406|146|9|4|34|35568.76|0.07|0.06|N|O|1996-12-01|1996-12-07|1996-12-16|NONE|AIR|hinly even accounts are slyly q|
+2406|187|8|5|25|27179.50|0.08|0.02|N|O|1996-12-03|1996-12-14|1996-12-26|COLLECT COD|MAIL|al, regular in|
+2406|59|4|6|22|21099.10|0.05|0.02|N|O|1996-11-22|1997-01-17|1996-12-15|NONE|TRUCK|hely even foxes unwind furiously aga|
+2406|60|2|7|30|28801.80|0.07|0.07|N|O|1997-01-17|1997-01-12|1997-01-22|TAKE BACK RETURN|TRUCK| final pinto beans han|
+2407|64|3|1|14|13496.84|0.04|0.02|N|O|1998-10-10|1998-08-25|1998-10-27|NONE|FOB|l dependencies s|
+2407|166|7|2|9|9595.44|0.07|0.05|N|O|1998-08-06|1998-08-11|1998-08-20|TAKE BACK RETURN|TRUCK|ts. special deposits are closely.|
+2407|131|2|3|39|40214.07|0.02|0.02|N|O|1998-08-20|1998-09-12|1998-08-22|DELIVER IN PERSON|MAIL|iously final deposits solv|
+2407|91|4|4|10|9910.90|0.01|0.07|N|O|1998-08-14|1998-09-10|1998-08-29|COLLECT COD|FOB| pending instructions. theodolites x-|
+2407|198|1|5|14|15374.66|0.04|0.05|N|O|1998-09-24|1998-08-18|1998-10-06|DELIVER IN PERSON|FOB|tructions wake stealt|
+2407|71|9|6|18|17479.26|0.04|0.01|N|O|1998-10-03|1998-08-30|1998-10-19|TAKE BACK RETURN|MAIL| wake carefully. fluffily |
+2407|161|8|7|7|7428.12|0.07|0.03|N|O|1998-09-11|1998-08-15|1998-09-30|TAKE BACK RETURN|MAIL|totes are carefully accordin|
+2432|50|3|1|30|28501.50|0.03|0.02|N|O|1996-09-05|1996-10-10|1996-10-05|TAKE BACK RETURN|TRUCK| requests wake alongside of|
+2432|162|3|2|8|8497.28|0.07|0.01|N|O|1996-10-16|1996-10-01|1996-11-13|COLLECT COD|RAIL|s about the bold, close deposit|
+2432|109|2|3|13|13118.30|0.07|0.06|N|O|1996-09-03|1996-10-10|1996-10-03|NONE|RAIL|arefully about the caref|
+2432|13|4|4|14|12782.14|0.00|0.06|N|O|1996-08-18|1996-09-04|1996-08-27|TAKE BACK RETURN|RAIL|riously regular packages. p|
+2433|87|8|1|39|38496.12|0.01|0.04|R|F|1994-11-20|1994-09-23|1994-12-10|DELIVER IN PERSON|SHIP|ly final asy|
+2433|134|5|2|20|20682.60|0.05|0.06|A|F|1994-12-09|1994-10-20|1994-12-15|COLLECT COD|REG AIR|lithely blithely final ide|
+2433|157|2|3|38|40171.70|0.08|0.03|A|F|1994-10-15|1994-10-23|1994-11-06|DELIVER IN PERSON|SHIP|. slyly regular requests sle|
+2433|121|6|4|43|43908.16|0.01|0.05|A|F|1994-10-16|1994-10-23|1994-11-08|DELIVER IN PERSON|RAIL|ular requests. slyly even pa|
+2433|108|1|5|3|3024.30|0.06|0.02|A|F|1994-11-08|1994-09-24|1994-11-17|COLLECT COD|AIR|usly pending depos|
+2434|95|6|1|1|995.09|0.01|0.06|N|O|1997-08-02|1997-05-28|1997-08-19|TAKE BACK RETURN|MAIL| furiously express packages. ironic, pend|
+2434|127|10|2|39|40057.68|0.09|0.05|N|O|1997-06-10|1997-06-08|1997-07-03|COLLECT COD|RAIL|r deposits sleep furiou|
+2434|130|3|3|28|28843.64|0.02|0.05|N|O|1997-06-28|1997-06-26|1997-07-15|COLLECT COD|RAIL|ven theodolites around the slyly|
+2434|168|9|4|49|52339.84|0.00|0.05|N|O|1997-08-08|1997-07-23|1997-08-27|DELIVER IN PERSON|FOB| after the requests haggle bold, fina|
+2435|39|10|1|8|7512.24|0.08|0.03|A|F|1993-06-08|1993-04-04|1993-06-29|COLLECT COD|SHIP|e fluffily quickly final accounts. care|
+2435|49|2|2|43|40808.72|0.03|0.08|A|F|1993-03-27|1993-05-20|1993-04-18|DELIVER IN PERSON|TRUCK|alongside of the s|
+2435|12|9|3|24|21888.24|0.07|0.08|R|F|1993-03-14|1993-05-20|1993-03-26|DELIVER IN PERSON|SHIP|s. carefully regular d|
+2435|156|4|4|22|23235.30|0.02|0.05|R|F|1993-05-23|1993-04-14|1993-06-04|NONE|SHIP|e final, final deposits. carefully regular|
+2435|72|2|5|3|2916.21|0.07|0.07|R|F|1993-06-01|1993-03-25|1993-06-27|DELIVER IN PERSON|FOB| final accounts ar|
+2435|46|9|6|17|16082.68|0.02|0.02|A|F|1993-06-05|1993-05-05|1993-06-14|NONE|TRUCK|cajole aft|
+2435|121|10|7|8|8168.96|0.07|0.02|R|F|1993-05-03|1993-04-02|1993-05-17|COLLECT COD|SHIP|ng the fluffily special foxes nag |
+2436|155|6|1|48|50647.20|0.04|0.02|N|O|1995-10-22|1995-10-22|1995-11-16|DELIVER IN PERSON|FOB|he furiously |
+2436|117|7|2|18|18307.98|0.05|0.03|N|O|1995-10-14|1995-11-21|1995-11-12|TAKE BACK RETURN|TRUCK|y ironic accounts. furiously even packa|
+2436|164|3|3|6|6384.96|0.06|0.08|N|O|1995-10-25|1995-11-30|1995-11-24|DELIVER IN PERSON|RAIL|odolites. ep|
+2437|94|6|1|46|45728.14|0.07|0.04|A|F|1993-08-12|1993-06-16|1993-08-29|NONE|RAIL|e of the bold, dogged requests|
+2437|190|1|2|26|28344.94|0.00|0.04|A|F|1993-06-25|1993-05-22|1993-07-07|DELIVER IN PERSON|REG AIR|lyly regular accounts.|
+2437|2|7|3|23|20746.00|0.01|0.00|A|F|1993-08-15|1993-06-28|1993-08-23|TAKE BACK RETURN|SHIP|s deposits. pendi|
+2437|116|10|4|12|12193.32|0.03|0.08|A|F|1993-04-27|1993-07-01|1993-05-18|TAKE BACK RETURN|FOB|thely regular deposits. ironic fray|
+2437|17|7|5|29|26593.29|0.02|0.06|A|F|1993-05-12|1993-06-10|1993-05-25|NONE|FOB|ress dolphins. furiously fin|
+2437|19|3|6|10|9190.10|0.10|0.06|A|F|1993-05-20|1993-06-23|1993-05-22|TAKE BACK RETURN|MAIL|unts. even, ironic pl|
+2438|165|2|1|45|47932.20|0.01|0.00|A|F|1993-10-27|1993-09-24|1993-11-02|COLLECT COD|REG AIR|en theodolites w|
+2438|13|4|2|31|28303.31|0.08|0.01|R|F|1993-10-16|1993-08-31|1993-11-10|COLLECT COD|REG AIR|t. slyly ironic sh|
+2438|68|7|3|10|9680.60|0.10|0.00|R|F|1993-08-18|1993-08-28|1993-09-08|NONE|SHIP|engage car|
+2438|161|8|4|27|28651.32|0.01|0.02|R|F|1993-07-27|1993-10-01|1993-08-06|TAKE BACK RETURN|FOB|inal accounts. slyly final reques|
+2438|166|3|5|28|29852.48|0.07|0.06|R|F|1993-11-05|1993-08-22|1993-11-22|TAKE BACK RETURN|TRUCK|ctions. bli|
+2438|149|6|6|23|24130.22|0.09|0.02|R|F|1993-10-06|1993-08-17|1993-10-16|DELIVER IN PERSON|MAIL|ely; blithely special pinto beans breach|
+2438|183|4|7|46|49826.28|0.02|0.05|R|F|1993-10-27|1993-08-30|1993-11-14|COLLECT COD|SHIP| ironic requests cajole f|
+2439|164|1|1|2|2128.32|0.09|0.03|N|O|1997-04-14|1997-06-11|1997-05-09|COLLECT COD|MAIL|courts boos|
+2439|144|5|2|5|5220.70|0.07|0.01|N|O|1997-04-23|1997-04-26|1997-04-28|DELIVER IN PERSON|FOB|ites. furiously|
+2439|195|7|3|33|36141.27|0.08|0.05|N|O|1997-06-01|1997-05-15|1997-06-07|TAKE BACK RETURN|FOB|asymptotes wake packages-- furiously|
+2464|49|8|1|10|9490.40|0.05|0.03|N|O|1998-02-04|1997-12-29|1998-02-16|TAKE BACK RETURN|RAIL|slyly final pinto bean|
+2464|101|6|2|20|20022.00|0.01|0.07|N|O|1997-12-26|1998-01-02|1998-01-24|DELIVER IN PERSON|FOB|sts. slyly close ideas shall h|
+2465|68|5|1|27|26137.62|0.05|0.02|N|O|1995-09-05|1995-09-07|1995-09-17|DELIVER IN PERSON|FOB|posits boost carefully unusual instructio|
+2465|51|3|2|34|32335.70|0.02|0.05|N|O|1995-10-02|1995-08-04|1995-10-09|COLLECT COD|RAIL|posits wake. regular package|
+2465|32|3|3|8|7456.24|0.10|0.00|N|O|1995-10-16|1995-08-26|1995-11-07|TAKE BACK RETURN|FOB|s across the express deposits wak|
+2465|148|7|4|45|47166.30|0.03|0.01|N|O|1995-09-27|1995-08-25|1995-10-06|NONE|TRUCK|y silent foxes. final pinto beans above |
+2465|47|4|5|50|47352.00|0.01|0.04|N|O|1995-09-01|1995-09-06|1995-09-18|TAKE BACK RETURN|TRUCK|the pending th|
+2465|124|5|6|20|20482.40|0.03|0.03|N|O|1995-08-16|1995-08-13|1995-09-02|COLLECT COD|FOB|uriously? furiously ironic excu|
+2466|186|7|1|16|17378.88|0.00|0.02|R|F|1994-04-20|1994-04-20|1994-05-09|COLLECT COD|FOB|to beans sl|
+2466|105|8|2|10|10051.00|0.00|0.00|A|F|1994-05-08|1994-04-06|1994-06-05|DELIVER IN PERSON|AIR|sly regular deposits. regular, regula|
+2466|14|1|3|29|26506.29|0.10|0.07|A|F|1994-06-11|1994-04-27|1994-07-10|DELIVER IN PERSON|FOB|ckages. bold requests nag carefully.|
+2466|11|8|4|29|26419.29|0.04|0.04|A|F|1994-04-01|1994-04-20|1994-04-23|DELIVER IN PERSON|MAIL|es boost fluffily ab|
+2466|79|10|5|30|29372.10|0.02|0.01|A|F|1994-04-11|1994-05-02|1994-05-02|DELIVER IN PERSON|REG AIR|. fluffily even pinto beans are idly. f|
+2466|173|2|6|19|20390.23|0.10|0.07|R|F|1994-06-12|1994-04-18|1994-07-12|NONE|MAIL|ccounts cajole a|
+2466|155|7|7|35|36930.25|0.10|0.00|A|F|1994-06-01|1994-05-27|1994-06-21|COLLECT COD|AIR| packages detect carefully: ironically sl|
+2467|133|9|1|7|7231.91|0.00|0.00|N|O|1995-07-28|1995-10-04|1995-08-27|NONE|REG AIR|gular packages cajole |
+2468|94|7|1|46|45728.14|0.00|0.04|N|O|1997-07-16|1997-08-09|1997-08-07|COLLECT COD|SHIP|unusual theodolites su|
+2468|21|10|2|43|39603.86|0.00|0.04|N|O|1997-08-17|1997-08-21|1997-08-30|DELIVER IN PERSON|FOB|uriously eve|
+2468|195|6|3|44|48188.36|0.00|0.03|N|O|1997-10-01|1997-08-02|1997-10-09|TAKE BACK RETURN|RAIL|egular, silent sheave|
+2468|82|3|4|5|4910.40|0.08|0.00|N|O|1997-06-28|1997-08-02|1997-07-22|NONE|MAIL| sleep fluffily acc|
+2468|159|7|5|18|19064.70|0.07|0.00|N|O|1997-07-25|1997-08-26|1997-08-14|DELIVER IN PERSON|REG AIR|cies. fluffily r|
+2469|166|1|1|11|11727.76|0.00|0.04|N|O|1997-02-09|1997-01-26|1997-02-16|NONE|TRUCK|ies wake carefully b|
+2469|114|1|2|16|16225.76|0.07|0.06|N|O|1997-02-19|1997-02-04|1997-03-18|NONE|MAIL|ing asymptotes |
+2469|11|5|3|48|43728.48|0.05|0.06|N|O|1997-01-11|1997-01-03|1997-01-15|TAKE BACK RETURN|AIR|riously even theodolites u|
+2469|88|9|4|35|34582.80|0.06|0.06|N|O|1997-02-04|1997-02-02|1997-02-17|DELIVER IN PERSON|RAIL|ld packages haggle regular frets. fluffily |
+2469|121|4|5|30|30633.60|0.09|0.01|N|O|1996-12-21|1997-01-29|1997-01-02|COLLECT COD|SHIP| accounts. regular theodolites affix fu|
+2469|104|5|6|49|49200.90|0.02|0.02|N|O|1997-03-03|1996-12-26|1997-03-13|NONE|AIR| requests are car|
+2469|127|10|7|8|8216.96|0.02|0.00|N|O|1997-03-15|1997-01-20|1997-04-13|NONE|TRUCK|s. regular|
+2470|110|5|1|12|12121.32|0.06|0.06|N|O|1997-07-12|1997-05-24|1997-07-17|TAKE BACK RETURN|FOB|l accounts. deposits nag daringly. express,|
+2470|100|4|2|50|50005.00|0.03|0.03|N|O|1997-06-02|1997-06-01|1997-06-09|COLLECT COD|AIR| packages |
+2470|64|3|3|10|9640.60|0.05|0.08|N|O|1997-06-20|1997-06-19|1997-06-24|TAKE BACK RETURN|FOB| ironic requests a|
+2470|162|3|4|30|31864.80|0.04|0.08|N|O|1997-08-04|1997-07-13|1997-08-14|DELIVER IN PERSON|AIR|s across the furiously fina|
+2471|84|5|1|37|36410.96|0.05|0.01|N|O|1998-05-28|1998-04-17|1998-06-08|COLLECT COD|TRUCK|ounts mold blithely carefully express depo|
+2496|141|8|1|38|39563.32|0.02|0.07|R|F|1994-03-26|1994-04-06|1994-04-23|COLLECT COD|RAIL| bold accounts. furi|
+2496|23|4|2|39|35997.78|0.03|0.00|R|F|1994-03-23|1994-02-18|1994-04-10|TAKE BACK RETURN|FOB|arefully special dependencies abo|
+2496|189|10|3|36|39210.48|0.09|0.04|R|F|1994-03-27|1994-03-15|1994-04-17|TAKE BACK RETURN|SHIP|ully ironic f|
+2496|24|9|4|30|27720.60|0.04|0.01|A|F|1994-01-27|1994-03-11|1994-01-31|DELIVER IN PERSON|RAIL|ake. ironic foxes cajole quickly. fu|
+2497|12|2|1|34|31008.34|0.02|0.03|R|F|1992-09-02|1992-10-19|1992-09-12|COLLECT COD|AIR|ronic accounts. p|
+2497|77|7|2|15|14656.05|0.09|0.02|A|F|1992-12-23|1992-11-20|1993-01-18|DELIVER IN PERSON|SHIP|sly against the|
+2497|34|5|3|28|26152.84|0.02|0.08|A|F|1992-12-02|1992-11-21|1992-12-04|DELIVER IN PERSON|REG AIR|ouches. special, regular requests|
+2497|144|5|4|48|50118.72|0.06|0.05|A|F|1992-09-29|1992-11-13|1992-10-19|TAKE BACK RETURN|AIR| even, regular requests across |
+2497|175|5|5|28|30104.76|0.04|0.05|A|F|1992-11-10|1992-09-30|1992-11-18|DELIVER IN PERSON|MAIL|hely bold ideas. unusual instructions ac|
+2497|71|2|6|19|18450.33|0.05|0.08|A|F|1992-11-10|1992-11-20|1992-12-05|TAKE BACK RETURN|TRUCK| instructions? carefully daring accounts|
+2498|143|2|1|48|50070.72|0.10|0.01|R|F|1993-11-25|1994-01-09|1993-12-24|DELIVER IN PERSON|RAIL|onic requests wake|
+2499|150|3|1|15|15752.25|0.04|0.06|N|O|1995-12-21|1995-12-06|1996-01-19|DELIVER IN PERSON|FOB| slyly across the slyly|
+2499|46|3|2|48|45409.92|0.09|0.03|N|O|1995-10-14|1995-12-12|1995-11-11|DELIVER IN PERSON|AIR|ronic ideas cajole quickly requests. caref|
+2499|133|9|3|31|32027.03|0.09|0.05|N|O|1995-12-09|1995-10-28|1996-01-05|COLLECT COD|AIR|to beans across the carefully ironic theodo|
+2499|159|7|4|39|41306.85|0.06|0.02|N|O|1995-10-26|1995-10-27|1995-11-07|TAKE BACK RETURN|SHIP|otes sublat|
+2499|130|9|5|6|6180.78|0.02|0.01|N|O|1995-11-19|1995-12-14|1995-12-08|NONE|SHIP|cording to the|
+2499|119|3|6|12|12229.32|0.04|0.05|N|O|1995-11-18|1995-12-13|1995-11-23|COLLECT COD|REG AIR|le furiously along the r|
+2500|192|3|1|40|43687.60|0.00|0.02|A|F|1992-09-02|1992-09-30|1992-09-06|DELIVER IN PERSON|SHIP|efully unusual dolphins s|
+2500|37|8|2|34|31859.02|0.06|0.02|R|F|1992-10-03|1992-11-11|1992-10-29|DELIVER IN PERSON|TRUCK| stealthy a|
+2500|80|10|3|41|40183.28|0.02|0.00|R|F|1992-09-02|1992-11-11|1992-09-06|DELIVER IN PERSON|RAIL|s could have to integrate after the |
+2500|69|8|4|17|16474.02|0.01|0.02|A|F|1992-09-30|1992-10-16|1992-10-05|DELIVER IN PERSON|REG AIR|encies-- ironic, even packages|
+2501|84|5|1|4|3936.32|0.10|0.06|N|O|1997-07-17|1997-07-27|1997-07-22|COLLECT COD|RAIL|quests. furiously final|
+2501|106|1|2|33|33201.30|0.01|0.04|N|O|1997-07-14|1997-08-09|1997-07-26|NONE|MAIL|leep furiously packages. even sauternes |
+2501|72|2|3|20|19441.40|0.10|0.06|N|O|1997-09-23|1997-07-01|1997-10-03|DELIVER IN PERSON|RAIL|equests. furiou|
+2501|58|10|4|26|24909.30|0.09|0.01|N|O|1997-07-15|1997-08-15|1997-07-28|DELIVER IN PERSON|SHIP|c accounts. express, iron|
+2502|163|4|1|33|35084.28|0.10|0.06|R|F|1993-08-12|1993-07-22|1993-09-04|COLLECT COD|REG AIR|have to print|
+2503|123|2|1|33|33762.96|0.06|0.01|R|F|1993-07-06|1993-08-14|1993-08-02|NONE|SHIP|nal courts integrate according to the|
+2503|65|10|2|28|27021.68|0.06|0.01|R|F|1993-08-08|1993-08-31|1993-08-10|NONE|SHIP|s wake quickly slyly |
+2503|46|7|3|50|47302.00|0.09|0.01|A|F|1993-09-22|1993-08-17|1993-09-29|DELIVER IN PERSON|TRUCK|s around the slyly |
+2503|91|5|4|27|26759.43|0.09|0.00|A|F|1993-07-12|1993-07-24|1993-07-22|DELIVER IN PERSON|TRUCK|lly even p|
+2503|48|5|5|3|2844.12|0.04|0.02|A|F|1993-07-10|1993-09-17|1993-07-19|TAKE BACK RETURN|TRUCK|s cajole. slyly close courts nod f|
+2503|128|7|6|39|40096.68|0.05|0.05|R|F|1993-10-11|1993-09-09|1993-10-16|NONE|MAIL|d carefully fluffily|
+2503|19|6|7|17|15623.17|0.09|0.08|R|F|1993-09-04|1993-07-31|1993-09-23|DELIVER IN PERSON|SHIP|c accounts haggle blithel|
+2528|1|2|1|10|9010.00|0.02|0.03|R|F|1994-12-12|1994-12-29|1994-12-28|COLLECT COD|REG AIR|ely. fluffily even re|
+2528|74|3|2|13|12662.91|0.00|0.03|A|F|1994-11-27|1995-01-20|1994-12-03|TAKE BACK RETURN|REG AIR|ggle furiously. slyly final asympt|
+2528|175|6|3|35|37630.95|0.10|0.00|R|F|1994-12-19|1995-02-04|1995-01-15|NONE|MAIL|, even excuses. even,|
+2528|65|4|4|37|35707.22|0.00|0.01|A|F|1994-12-25|1995-02-02|1994-12-31|COLLECT COD|AIR|ng the pending excuses haggle after the bl|
+2529|131|7|1|4|4124.52|0.07|0.07|N|O|1996-10-19|1996-11-18|1996-10-24|DELIVER IN PERSON|SHIP|al dependencies haggle slyly alongsi|
+2530|21|2|1|9|8289.18|0.09|0.03|R|F|1994-05-10|1994-04-30|1994-05-24|TAKE BACK RETURN|REG AIR|lyly ironic|
+2530|93|7|2|42|41709.78|0.04|0.08|R|F|1994-03-27|1994-05-20|1994-03-29|NONE|RAIL|ng platelets wake s|
+2530|108|1|3|8|8064.80|0.10|0.08|A|F|1994-05-02|1994-05-08|1994-05-24|DELIVER IN PERSON|MAIL|ial asymptotes snooze slyly regular |
+2531|148|7|1|9|9433.26|0.03|0.07|N|O|1996-07-27|1996-07-03|1996-08-01|DELIVER IN PERSON|AIR|t the dogged, un|
+2531|157|2|2|3|3171.45|0.07|0.06|N|O|1996-07-20|1996-06-20|1996-08-10|NONE|MAIL|he quickly ev|
+2531|86|7|3|20|19721.60|0.06|0.04|N|O|1996-07-18|1996-06-25|1996-07-29|TAKE BACK RETURN|TRUCK|into beans. furious|
+2531|191|5|4|36|39282.84|0.08|0.01|N|O|1996-06-11|1996-07-26|1996-06-27|NONE|MAIL|y ironic, bold packages. blithely e|
+2531|56|4|5|28|26769.40|0.03|0.07|N|O|1996-07-06|1996-07-31|1996-07-19|TAKE BACK RETURN|REG AIR|its. busily|
+2531|145|4|6|46|48076.44|0.10|0.08|N|O|1996-07-03|1996-06-27|1996-07-12|TAKE BACK RETURN|REG AIR|e final, bold pains. ir|
+2532|53|4|1|3|2859.15|0.06|0.07|N|O|1995-12-14|1995-11-28|1995-12-15|COLLECT COD|FOB|unusual sentiments. even pinto|
+2532|160|2|2|33|34985.28|0.06|0.05|N|O|1995-11-23|1996-01-04|1995-12-16|DELIVER IN PERSON|TRUCK|rve carefully slyly ironic accounts! fluf|
+2532|135|1|3|1|1035.13|0.00|0.06|N|O|1996-01-27|1995-11-23|1996-01-29|DELIVER IN PERSON|REG AIR|ely final ideas cajole despite the ca|
+2532|78|8|4|50|48903.50|0.02|0.02|N|O|1995-11-13|1996-01-01|1995-11-26|NONE|TRUCK|yly after the fluffily regul|
+2532|114|1|5|9|9126.99|0.09|0.04|N|O|1995-11-30|1995-11-23|1995-12-12|DELIVER IN PERSON|TRUCK|cial ideas haggle slyly pending request|
+2532|150|1|6|20|21003.00|0.09|0.05|N|O|1995-12-02|1995-11-26|1995-12-08|TAKE BACK RETURN|AIR|er the slyly pending|
+2533|54|9|1|36|34345.80|0.06|0.04|N|O|1997-06-10|1997-04-28|1997-07-01|NONE|REG AIR|ss requests sleep neve|
+2533|198|10|2|5|5490.95|0.10|0.04|N|O|1997-05-26|1997-06-02|1997-06-24|NONE|FOB|ccounts. ironic, special accounts boo|
+2533|183|4|3|37|40077.66|0.00|0.08|N|O|1997-05-10|1997-04-26|1997-05-28|COLLECT COD|SHIP| haggle carefully |
+2533|30|5|4|17|15810.51|0.06|0.02|N|O|1997-05-23|1997-05-10|1997-06-18|NONE|FOB|ackages. blith|
+2533|126|1|5|38|38992.56|0.09|0.00|N|O|1997-05-10|1997-06-02|1997-05-28|TAKE BACK RETURN|REG AIR|of the regular accounts. even packages caj|
+2533|184|5|6|20|21683.60|0.05|0.08|N|O|1997-07-04|1997-04-30|1997-07-05|COLLECT COD|FOB|thless excuses are b|
+2533|94|7|7|14|13917.26|0.06|0.04|N|O|1997-07-06|1997-05-08|1997-08-03|COLLECT COD|FOB|ut the pending, special depos|
+2534|139|5|1|29|30134.77|0.07|0.07|N|O|1996-08-09|1996-09-29|1996-08-11|COLLECT COD|TRUCK|ugouts haggle slyly. final|
+2534|27|6|2|49|45423.98|0.08|0.08|N|O|1996-09-01|1996-08-20|1996-09-06|NONE|SHIP|sometimes regular requests. blithely unus|
+2534|1|4|3|50|45050.00|0.10|0.06|N|O|1996-09-25|1996-10-07|1996-10-09|TAKE BACK RETURN|AIR|ideas. deposits use. slyly regular pa|
+2534|75|3|4|43|41928.01|0.09|0.02|N|O|1996-10-25|1996-09-30|1996-11-05|TAKE BACK RETURN|REG AIR|ngly final depos|
+2534|165|2|5|14|14912.24|0.05|0.02|N|O|1996-08-12|1996-09-26|1996-08-28|COLLECT COD|MAIL|eposits doze quickly final|
+2534|116|10|6|12|12193.32|0.02|0.02|N|O|1996-07-29|1996-10-12|1996-08-14|TAKE BACK RETURN|AIR|sual depos|
+2534|173|3|7|17|18243.89|0.02|0.07|N|O|1996-07-22|1996-09-15|1996-08-03|NONE|SHIP|riously regular |
+2535|199|2|1|5|5495.95|0.06|0.01|A|F|1993-09-07|1993-07-25|1993-09-29|DELIVER IN PERSON|REG AIR|, unusual reque|
+2535|39|5|2|12|11268.36|0.08|0.05|A|F|1993-07-17|1993-08-17|1993-07-31|TAKE BACK RETURN|FOB|uses sleep among the packages. excuses |
+2535|54|5|3|5|4770.25|0.09|0.06|R|F|1993-07-28|1993-08-14|1993-08-11|DELIVER IN PERSON|SHIP| across the express requests. silent, eve|
+2535|160|5|4|19|20143.04|0.01|0.02|A|F|1993-06-01|1993-08-01|1993-06-19|DELIVER IN PERSON|FOB|ructions. final requests|
+2535|174|3|5|25|26854.25|0.07|0.04|A|F|1993-07-19|1993-08-07|1993-07-27|NONE|REG AIR|ions believe ab|
+2560|169|10|1|41|43835.56|0.07|0.01|R|F|1992-10-23|1992-11-11|1992-11-22|NONE|SHIP| after the accounts. regular foxes are be|
+2560|4|9|2|27|24408.00|0.00|0.01|R|F|1992-12-03|1992-11-16|1992-12-30|NONE|MAIL| against the carefully|
+2560|46|5|3|31|29327.24|0.01|0.05|A|F|1992-11-14|1992-10-14|1992-12-11|DELIVER IN PERSON|AIR|to beans. blithely regular Tiresias int|
+2560|72|1|4|36|34994.52|0.01|0.02|A|F|1992-10-18|1992-10-30|1992-11-05|TAKE BACK RETURN|MAIL|accounts alongside of the excuses are |
+2560|42|1|5|9|8478.36|0.04|0.02|A|F|1992-10-23|1992-10-29|1992-11-02|COLLECT COD|REG AIR| deposits affix quickly. unusual, eve|
+2560|108|9|6|13|13105.30|0.03|0.06|A|F|1992-09-07|1992-10-21|1992-09-24|COLLECT COD|FOB|slyly final accoun|
+2561|25|4|1|32|29600.64|0.02|0.01|N|O|1998-01-05|1997-12-28|1998-01-26|DELIVER IN PERSON|REG AIR|bold packages wake slyly. slyly|
+2561|98|1|2|5|4990.45|0.07|0.04|N|O|1997-12-27|1998-01-23|1998-01-13|TAKE BACK RETURN|AIR|p ironic, regular pinto beans.|
+2561|173|4|3|47|50438.99|0.04|0.02|N|O|1997-11-19|1998-01-21|1997-12-03|DELIVER IN PERSON|REG AIR|larly pending t|
+2561|108|9|4|39|39315.90|0.08|0.06|N|O|1998-01-20|1997-12-16|1998-02-05|TAKE BACK RETURN|MAIL|equests are furiously against the|
+2561|150|3|5|2|2100.30|0.04|0.08|N|O|1998-03-14|1998-01-21|1998-03-27|DELIVER IN PERSON|TRUCK|s are. silently silent foxes sleep about|
+2561|51|6|6|14|13314.70|0.02|0.03|N|O|1998-03-07|1998-02-04|1998-03-21|COLLECT COD|RAIL|ep unusual, ironic accounts|
+2562|53|5|1|28|26685.40|0.04|0.03|R|F|1992-10-04|1992-09-24|1992-10-09|COLLECT COD|MAIL|ans haggle special, special packages. |
+2562|148|9|2|1|1048.14|0.01|0.06|R|F|1992-10-16|1992-09-18|1992-10-17|NONE|TRUCK| slyly final ideas haggle car|
+2562|66|7|3|25|24151.50|0.05|0.03|A|F|1992-11-23|1992-10-08|1992-12-19|DELIVER IN PERSON|REG AIR| accounts-- silent, unusual ideas a|
+2562|148|1|4|37|38781.18|0.08|0.03|R|F|1992-10-29|1992-10-06|1992-11-09|COLLECT COD|FOB|. slyly regular ideas according to the fl|
+2562|160|8|5|29|30744.64|0.05|0.08|A|F|1992-11-01|1992-09-29|1992-11-13|TAKE BACK RETURN|MAIL|eep against the furiously r|
+2562|50|7|6|17|16150.85|0.01|0.06|A|F|1992-10-15|1992-10-08|1992-10-26|DELIVER IN PERSON|TRUCK|lar pinto beans. blithely ev|
+2563|65|4|1|10|9650.60|0.07|0.04|A|F|1994-01-26|1993-12-19|1994-01-28|DELIVER IN PERSON|AIR|tealthily abo|
+2563|167|4|2|28|29880.48|0.04|0.03|R|F|1994-03-17|1994-02-04|1994-04-13|TAKE BACK RETURN|RAIL|hely regular depe|
+2563|119|9|3|39|39745.29|0.07|0.00|R|F|1994-02-10|1993-12-31|1994-02-19|COLLECT COD|FOB|lent requests should integrate; carefully e|
+2563|90|1|4|50|49504.50|0.01|0.01|A|F|1994-01-26|1994-01-03|1994-02-09|DELIVER IN PERSON|SHIP|ly regular, regular excuses. bold plate|
+2563|15|6|5|42|38430.42|0.06|0.08|R|F|1994-02-21|1994-02-14|1994-03-04|DELIVER IN PERSON|AIR|ymptotes nag furiously slyly even inst|
+2563|121|2|6|5|5105.60|0.10|0.00|R|F|1993-12-27|1993-12-19|1994-01-02|DELIVER IN PERSON|REG AIR| the quickly final theodolite|
+2564|112|3|1|4|4048.44|0.02|0.00|R|F|1994-11-12|1994-10-29|1994-12-04|NONE|MAIL|y express requests sleep furi|
+2565|144|5|1|42|43853.88|0.04|0.08|N|O|1998-04-07|1998-04-02|1998-05-04|NONE|AIR|ngly silent |
+2565|189|10|2|26|28318.68|0.05|0.08|N|O|1998-05-07|1998-04-09|1998-05-15|DELIVER IN PERSON|TRUCK| pinto beans about the slyly regula|
+2565|115|5|3|34|34513.74|0.06|0.06|N|O|1998-03-19|1998-04-12|1998-04-17|DELIVER IN PERSON|SHIP|nstructions was carefu|
+2565|17|7|4|25|22925.25|0.10|0.08|N|O|1998-06-27|1998-05-20|1998-07-13|DELIVER IN PERSON|RAIL|, express accounts. final id|
+2565|76|7|5|26|25377.82|0.08|0.03|N|O|1998-03-05|1998-04-11|1998-03-11|TAKE BACK RETURN|AIR|ites wake. ironic acco|
+2565|141|4|6|48|49974.72|0.08|0.07|N|O|1998-06-18|1998-05-06|1998-07-13|DELIVER IN PERSON|TRUCK|r instructions sleep qui|
+2566|148|5|1|19|19914.66|0.06|0.07|R|F|1992-12-21|1992-11-24|1992-12-22|DELIVER IN PERSON|MAIL|ests. silent|
+2566|181|2|2|42|45409.56|0.08|0.02|R|F|1992-12-20|1992-12-22|1992-12-29|COLLECT COD|MAIL|ously ironic accounts|
+2566|23|8|3|18|16614.36|0.09|0.02|A|F|1992-11-16|1992-12-24|1992-12-16|COLLECT COD|FOB| braids according t|
+2566|42|9|4|3|2826.12|0.05|0.02|A|F|1992-11-04|1992-12-30|1992-12-04|TAKE BACK RETURN|FOB|ckages are ironic Tiresias. furious|
+2566|22|3|5|9|8298.18|0.04|0.03|R|F|1992-12-14|1992-12-28|1992-12-16|NONE|FOB|blithely bold accounts? quickl|
+2566|128|3|6|1|1028.12|0.07|0.03|A|F|1992-10-28|1992-11-20|1992-11-22|TAKE BACK RETURN|AIR|theodolites wake pending|
+2567|26|9|1|39|36114.78|0.03|0.04|N|O|1998-05-10|1998-05-10|1998-05-21|NONE|SHIP|ns. furiously final dependencies cajo|
+2567|112|3|2|50|50605.50|0.06|0.05|N|O|1998-05-05|1998-04-18|1998-05-09|DELIVER IN PERSON|TRUCK|. carefully pending foxes are furi|
+2567|52|10|3|6|5712.30|0.03|0.06|N|O|1998-04-21|1998-04-14|1998-05-11|NONE|RAIL|s cajole regular, final acco|
+2567|158|6|4|50|52907.50|0.05|0.03|N|O|1998-03-27|1998-05-25|1998-04-23|DELIVER IN PERSON|FOB|pinto beans? r|
+2567|81|2|5|46|45129.68|0.07|0.02|N|O|1998-06-02|1998-04-30|1998-06-13|COLLECT COD|AIR|efully pending epitaphs. carefully reg|
+2567|100|3|6|32|32003.20|0.01|0.07|N|O|1998-05-24|1998-04-30|1998-06-14|NONE|RAIL| the even, iro|
+2567|135|6|7|43|44510.59|0.06|0.02|N|O|1998-05-11|1998-04-15|1998-05-29|NONE|RAIL|requests. final courts cajole |
+2592|90|1|1|7|6930.63|0.10|0.04|R|F|1993-03-13|1993-04-25|1993-04-01|NONE|REG AIR| carefully special theodolites integrate |
+2592|66|1|2|2|1932.12|0.10|0.00|A|F|1993-03-24|1993-04-05|1993-04-16|DELIVER IN PERSON|RAIL|side of the b|
+2593|105|2|1|37|37188.70|0.08|0.06|R|F|1993-12-14|1993-10-08|1994-01-04|NONE|SHIP|s wake bravel|
+2593|90|1|2|28|27722.52|0.08|0.03|A|F|1993-10-30|1993-10-18|1993-11-06|DELIVER IN PERSON|SHIP|y even escapades shall|
+2593|128|3|3|6|6168.72|0.04|0.05|A|F|1993-11-28|1993-10-04|1993-12-28|TAKE BACK RETURN|REG AIR|ular packages. re|
+2593|161|10|4|44|46691.04|0.02|0.08|A|F|1993-09-05|1993-10-23|1993-09-29|NONE|RAIL|ents impress furiously; unusual theodoli|
+2593|4|5|5|3|2712.00|0.03|0.00|A|F|1993-12-16|1993-11-01|1993-12-29|COLLECT COD|SHIP|the furiously |
+2593|175|6|6|1|1075.17|0.08|0.08|A|F|1993-11-23|1993-10-25|1993-12-04|DELIVER IN PERSON|RAIL| accounts wake slyly |
+2593|192|5|7|11|12014.09|0.00|0.07|R|F|1993-11-01|1993-11-19|1993-11-28|TAKE BACK RETURN|RAIL|express packages sleep bold re|
+2594|72|3|1|7|6804.49|0.06|0.02|R|F|1993-03-26|1993-03-05|1993-04-24|DELIVER IN PERSON|FOB|arls cajole |
+2594|124|9|2|13|13313.56|0.10|0.05|R|F|1993-02-06|1993-03-01|1993-02-23|TAKE BACK RETURN|TRUCK|fully special accounts use courts|
+2594|126|1|3|24|24626.88|0.03|0.00|A|F|1993-01-31|1993-03-10|1993-02-04|COLLECT COD|REG AIR|lar accounts sleep fur|
+2594|144|7|4|46|48030.44|0.00|0.08|R|F|1993-04-17|1993-03-06|1993-04-21|TAKE BACK RETURN|SHIP|beans. instructions across t|
+2595|61|2|1|42|40364.52|0.08|0.02|N|O|1996-03-24|1996-01-28|1996-04-10|DELIVER IN PERSON|MAIL|ggle furiou|
+2595|88|9|2|30|29642.40|0.05|0.01|N|O|1996-03-05|1996-02-23|1996-03-19|NONE|AIR|ctions. regula|
+2595|24|3|3|19|17556.38|0.01|0.05|N|O|1995-12-23|1996-03-02|1996-01-17|COLLECT COD|MAIL|ns are neve|
+2595|159|1|4|29|30715.35|0.07|0.05|N|O|1996-01-01|1996-02-13|1996-01-18|TAKE BACK RETURN|RAIL|ronic accounts haggle carefully fin|
+2595|86|7|5|30|29582.40|0.09|0.07|N|O|1996-03-16|1996-01-31|1996-04-05|TAKE BACK RETURN|FOB|. final orbits cajole |
+2595|82|3|6|31|30444.48|0.06|0.04|N|O|1996-02-07|1996-02-10|1996-03-05|DELIVER IN PERSON|AIR|tipliers w|
+2596|170|5|1|6|6421.02|0.05|0.01|N|O|1996-12-15|1996-11-02|1996-12-29|TAKE BACK RETURN|TRUCK|ily special re|
+2596|139|10|2|43|44682.59|0.07|0.03|N|O|1996-09-03|1996-10-26|1996-09-15|NONE|FOB|ial packages haggl|
+2596|39|5|3|19|17841.57|0.10|0.00|N|O|1996-09-02|1996-11-03|1996-09-06|COLLECT COD|AIR|ias mold! sp|
+2596|105|6|4|10|10051.00|0.06|0.05|N|O|1996-08-25|1996-11-05|1996-09-13|DELIVER IN PERSON|REG AIR| instructions shall have|
+2597|84|5|1|24|23617.92|0.07|0.00|A|F|1993-05-15|1993-03-06|1993-05-25|TAKE BACK RETURN|FOB|pending packages. enticingly fi|
+2598|7|4|1|12|10884.00|0.00|0.01|N|O|1996-06-17|1996-04-12|1996-06-24|COLLECT COD|TRUCK|express packages nag sly|
+2598|148|7|2|40|41925.60|0.07|0.02|N|O|1996-05-11|1996-05-19|1996-06-08|TAKE BACK RETURN|AIR|the enticing|
+2598|104|9|3|4|4016.40|0.03|0.03|N|O|1996-05-23|1996-05-13|1996-05-25|COLLECT COD|AIR| across the furiously fi|
+2598|23|2|4|19|17537.38|0.02|0.00|N|O|1996-04-09|1996-05-30|1996-04-17|TAKE BACK RETURN|RAIL|nic packages. even accounts|
+2598|106|3|5|12|12073.20|0.01|0.08|N|O|1996-04-14|1996-04-24|1996-04-21|TAKE BACK RETURN|REG AIR|eposits cajol|
+2599|101|4|1|11|11012.10|0.08|0.08|N|O|1997-02-01|1996-12-14|1997-02-27|TAKE BACK RETURN|FOB| express accoun|
+2599|42|5|2|26|24493.04|0.03|0.04|N|O|1996-11-08|1996-12-21|1996-11-24|TAKE BACK RETURN|AIR|nag carefully |
+2599|99|10|3|29|28973.61|0.09|0.03|N|O|1997-01-10|1996-12-10|1997-02-02|COLLECT COD|RAIL|ly express dolphins. special, |
+2624|63|10|1|15|14445.90|0.03|0.07|N|O|1997-02-28|1997-02-19|1997-03-21|DELIVER IN PERSON|AIR|le. quickly pending requests|
+2624|189|10|2|12|13070.16|0.07|0.00|N|O|1997-02-24|1997-02-22|1997-02-27|DELIVER IN PERSON|SHIP|er the quickly unu|
+2625|20|1|1|42|38640.84|0.02|0.04|R|F|1992-10-18|1992-11-17|1992-10-23|DELIVER IN PERSON|AIR| even accounts haggle furiously|
+2626|22|5|1|45|41490.90|0.09|0.04|N|O|1995-11-22|1995-11-01|1995-11-23|NONE|AIR|deposits wake blithely according to |
+2626|175|3|2|2|2150.34|0.05|0.07|N|O|1995-10-19|1995-11-09|1995-10-24|TAKE BACK RETURN|FOB|uffy accounts haggle furiously above|
+2626|154|2|3|40|42166.00|0.05|0.07|N|O|1995-09-28|1995-12-03|1995-10-10|NONE|REG AIR|eans. ironic deposits haggle. depo|
+2627|131|7|1|28|28871.64|0.09|0.02|R|F|1992-05-14|1992-05-09|1992-05-31|COLLECT COD|SHIP|ggedly final excuses nag packages. f|
+2628|106|9|1|44|44268.40|0.07|0.03|R|F|1994-01-11|1994-01-14|1994-01-13|DELIVER IN PERSON|SHIP|lyly final, pending ide|
+2628|106|9|2|14|14085.40|0.01|0.03|A|F|1994-01-28|1993-11-30|1994-02-20|TAKE BACK RETURN|SHIP|g the furiously unusual pi|
+2628|64|9|3|42|40490.52|0.00|0.00|A|F|1993-11-20|1994-01-04|1993-12-19|DELIVER IN PERSON|TRUCK|ld notornis alongside |
+2628|95|7|4|23|22887.07|0.08|0.04|A|F|1993-10-27|1994-01-08|1993-11-12|DELIVER IN PERSON|TRUCK|usual packages sleep about the fina|
+2628|90|1|5|50|49504.50|0.07|0.01|A|F|1994-01-13|1993-12-11|1994-01-14|NONE|AIR|posits serve carefully toward |
+2629|118|9|1|6|6108.66|0.06|0.05|N|O|1998-06-10|1998-05-29|1998-06-13|DELIVER IN PERSON|SHIP|dolites hinder bli|
+2629|124|7|2|31|31747.72|0.08|0.03|N|O|1998-05-24|1998-05-26|1998-06-10|COLLECT COD|AIR|ate blithely bold, regular deposits. bold|
+2629|128|9|3|29|29815.48|0.08|0.07|N|O|1998-07-09|1998-06-17|1998-07-12|TAKE BACK RETURN|AIR|eposits serve unusual, express i|
+2629|70|5|4|33|32012.31|0.06|0.03|N|O|1998-05-29|1998-05-14|1998-05-30|NONE|TRUCK|es. slowly express accounts are along the|
+2630|29|8|1|46|42734.92|0.05|0.03|R|F|1992-11-05|1992-12-17|1992-12-05|TAKE BACK RETURN|MAIL|uests cajole. e|
+2630|57|2|2|8|7656.40|0.09|0.07|A|F|1992-11-16|1993-01-01|1992-12-07|DELIVER IN PERSON|TRUCK|indle fluffily silent, ironic pi|
+2630|173|2|3|45|48292.65|0.08|0.07|A|F|1993-01-04|1993-01-11|1993-01-09|NONE|FOB|edly express ideas. carefully final |
+2630|162|9|4|29|30802.64|0.08|0.07|A|F|1992-12-03|1993-01-04|1992-12-12|DELIVER IN PERSON|SHIP|efully unusual dependencies. even i|
+2631|122|7|1|42|42929.04|0.00|0.03|A|F|1994-01-04|1993-12-01|1994-01-16|TAKE BACK RETURN|SHIP|ect carefully at the furiously final the|
+2631|67|4|2|4|3868.24|0.07|0.06|R|F|1993-11-03|1993-12-17|1993-11-05|COLLECT COD|AIR|special theodolites. a|
+2631|118|8|3|15|15271.65|0.06|0.05|A|F|1993-09-30|1993-11-06|1993-10-13|DELIVER IN PERSON|SHIP|y. furiously even pinto be|
+2656|181|2|1|10|10811.80|0.02|0.06|R|F|1993-06-28|1993-07-04|1993-07-12|TAKE BACK RETURN|TRUCK|s nag regularly about the deposits. slyly|
+2656|137|8|2|38|39410.94|0.07|0.02|A|F|1993-06-25|1993-06-04|1993-07-24|NONE|RAIL|structions wake along the furio|
+2656|2|5|3|19|17138.00|0.03|0.02|R|F|1993-08-03|1993-07-25|1993-08-20|TAKE BACK RETURN|MAIL|ts serve deposi|
+2656|110|3|4|40|40404.40|0.05|0.04|R|F|1993-06-09|1993-07-24|1993-06-21|DELIVER IN PERSON|RAIL|refully final pearls. final ideas wake. qu|
+2657|115|9|1|22|22332.42|0.02|0.03|N|O|1995-12-08|1995-12-28|1995-12-21|TAKE BACK RETURN|MAIL|r ideas. furiously special dolphins|
+2657|165|2|2|15|15977.40|0.08|0.05|N|O|1995-12-09|1995-12-16|1995-12-18|NONE|RAIL|ole carefully above the ironic ideas. b|
+2657|79|9|3|25|24476.75|0.02|0.04|N|O|1995-10-21|1995-12-12|1995-11-09|COLLECT COD|FOB|lly pinto beans. final |
+2657|55|7|4|11|10505.55|0.04|0.08|N|O|1995-11-19|1995-12-11|1995-11-24|COLLECT COD|TRUCK|ckly enticing requests. fur|
+2657|78|9|5|42|41078.94|0.06|0.03|N|O|1996-01-23|1995-11-22|1996-01-25|COLLECT COD|RAIL|ckly slyly even accounts. platelets x-ray|
+2657|194|7|6|31|33919.89|0.01|0.03|N|O|1995-11-10|1995-11-27|1995-12-06|COLLECT COD|RAIL|re blithely |
+2658|132|3|1|41|42317.33|0.05|0.04|N|O|1995-11-07|1995-11-04|1995-12-04|NONE|MAIL|eposits. furiously final theodolite|
+2658|29|4|2|22|20438.44|0.08|0.05|N|O|1995-11-12|1995-11-18|1995-11-14|DELIVER IN PERSON|TRUCK|ts cajole. pending packages affix|
+2658|18|5|3|13|11934.13|0.07|0.06|N|O|1995-10-24|1995-12-12|1995-11-14|COLLECT COD|FOB|s kindle blithely regular accounts.|
+2658|92|5|4|22|21825.98|0.04|0.04|N|O|1995-12-02|1995-11-03|1995-12-26|DELIVER IN PERSON|SHIP| dependencies. blithely pending foxes abou|
+2658|7|8|5|45|40815.00|0.03|0.01|N|O|1995-11-02|1995-11-08|1995-11-29|DELIVER IN PERSON|MAIL|e special requests. quickly ex|
+2658|147|4|6|27|28272.78|0.05|0.07|N|O|1995-09-26|1995-12-08|1995-09-30|NONE|AIR|ecial packages use abov|
+2659|42|1|1|28|26377.12|0.08|0.05|A|F|1994-03-17|1994-01-24|1994-03-19|NONE|FOB|idle tithes|
+2659|43|2|2|21|19803.84|0.00|0.00|A|F|1993-12-23|1994-02-10|1994-01-17|DELIVER IN PERSON|RAIL|y beyond the furiously even co|
+2659|135|1|3|24|24843.12|0.04|0.03|R|F|1994-03-28|1994-02-20|1994-04-05|DELIVER IN PERSON|REG AIR| haggle carefully |
+2659|119|6|4|2|2038.22|0.00|0.08|R|F|1994-02-19|1994-03-12|1994-02-21|NONE|MAIL|sts above the fluffily express fo|
+2659|7|4|5|9|8163.00|0.08|0.03|A|F|1994-02-07|1994-03-17|1994-03-04|DELIVER IN PERSON|AIR|ly final packages sleep ac|
+2660|48|7|1|17|16116.68|0.00|0.05|N|O|1995-08-18|1995-09-13|1995-09-17|NONE|SHIP|al pinto beans wake after the furious|
+2661|178|9|1|31|33423.27|0.03|0.02|N|O|1997-04-07|1997-03-10|1997-04-23|TAKE BACK RETURN|AIR|e ironicall|
+2661|103|8|2|22|22068.20|0.08|0.02|N|O|1997-03-14|1997-03-17|1997-04-08|COLLECT COD|REG AIR| foxes affix quickly ironic request|
+2661|67|6|3|11|10637.66|0.00|0.08|N|O|1997-04-14|1997-02-11|1997-05-05|TAKE BACK RETURN|FOB|equests are a|
+2661|137|8|4|41|42522.33|0.06|0.02|N|O|1997-03-06|1997-03-27|1997-03-15|DELIVER IN PERSON|AIR|iously ironically ironic requests. |
+2662|102|5|1|43|43090.30|0.09|0.07|N|O|1996-11-24|1996-11-04|1996-12-08|NONE|RAIL|. slyly specia|
+2662|128|9|2|8|8224.96|0.02|0.07|N|O|1996-09-10|1996-10-09|1996-09-21|TAKE BACK RETURN|REG AIR|ajole carefully. sp|
+2662|2|5|3|6|5412.00|0.02|0.00|N|O|1996-11-30|1996-09-20|1996-12-03|DELIVER IN PERSON|REG AIR|olites cajole quickly along the b|
+2662|30|1|4|34|31621.02|0.06|0.07|N|O|1996-10-04|1996-11-05|1996-10-19|NONE|SHIP|ding theodolites use carefully. p|
+2663|114|4|1|35|35493.85|0.02|0.01|N|O|1995-12-11|1995-10-16|1996-01-07|TAKE BACK RETURN|REG AIR|tect. slyly fina|
+2688|18|5|1|45|41310.45|0.08|0.08|R|F|1992-05-21|1992-04-14|1992-05-28|NONE|FOB|sits run carefully|
+2688|15|6|2|46|42090.46|0.01|0.01|R|F|1992-05-24|1992-04-01|1992-05-26|COLLECT COD|TRUCK|elets. regular reque|
+2688|89|10|3|30|29672.40|0.05|0.04|A|F|1992-04-18|1992-03-18|1992-05-18|TAKE BACK RETURN|RAIL|ithely final |
+2688|25|10|4|3|2775.06|0.00|0.03|R|F|1992-02-04|1992-03-18|1992-02-24|DELIVER IN PERSON|RAIL|e fluffily |
+2688|59|10|5|22|21099.10|0.02|0.05|R|F|1992-02-09|1992-04-09|1992-02-11|DELIVER IN PERSON|RAIL|press, ironic excuses wake carefully id|
+2688|149|10|6|42|44063.88|0.01|0.01|R|F|1992-04-29|1992-04-04|1992-05-17|TAKE BACK RETURN|FOB|lly even account|
+2689|6|1|1|45|40770.00|0.02|0.04|R|F|1992-04-29|1992-06-22|1992-04-30|COLLECT COD|SHIP|e quickly. carefully silent|
+2690|140|1|1|44|45766.16|0.05|0.06|N|O|1996-05-30|1996-05-19|1996-06-26|NONE|REG AIR|ly alongside of th|
+2690|51|2|2|50|47552.50|0.03|0.03|N|O|1996-06-13|1996-05-22|1996-06-14|DELIVER IN PERSON|MAIL| doubt careful|
+2690|125|6|3|45|46130.40|0.02|0.07|N|O|1996-05-23|1996-06-02|1996-05-29|DELIVER IN PERSON|MAIL|ounts. slyly regular dependencies wa|
+2690|195|6|4|12|13142.28|0.04|0.07|N|O|1996-07-18|1996-06-03|1996-07-25|NONE|AIR|nal, regular atta|
+2690|86|7|5|30|29582.40|0.01|0.08|N|O|1996-05-20|1996-06-01|1996-06-04|TAKE BACK RETURN|SHIP|d accounts above the express req|
+2690|189|10|6|3|3267.54|0.07|0.01|N|O|1996-07-04|1996-05-28|1996-07-06|TAKE BACK RETURN|RAIL|. final reques|
+2690|79|7|7|35|34267.45|0.05|0.06|N|O|1996-07-25|1996-05-14|1996-08-03|COLLECT COD|FOB|y silent pinto be|
+2691|91|3|1|11|10901.99|0.04|0.07|R|F|1992-06-21|1992-06-08|1992-07-09|COLLECT COD|FOB|leep alongside of the accounts. slyly ironi|
+2691|48|7|2|2|1896.08|0.00|0.07|R|F|1992-05-10|1992-06-04|1992-05-11|TAKE BACK RETURN|TRUCK|s cajole at the blithely ironic warthog|
+2691|162|3|3|16|16994.56|0.09|0.03|R|F|1992-06-11|1992-07-29|1992-06-29|NONE|RAIL|bove the even foxes. unusual theodoli|
+2691|166|3|4|1|1066.16|0.08|0.00|A|F|1992-08-11|1992-06-07|1992-08-16|NONE|SHIP|egular instructions b|
+2692|17|1|1|3|2751.03|0.10|0.04|N|O|1998-02-25|1998-01-29|1998-03-27|TAKE BACK RETURN|MAIL|equests. bold, even foxes haggle slyl|
+2692|114|1|2|21|21296.31|0.03|0.05|N|O|1998-03-11|1998-02-11|1998-03-19|NONE|SHIP|posits. final, express requests nag furi|
+2693|9|10|1|26|23634.00|0.04|0.00|N|O|1996-09-14|1996-10-07|1996-10-03|COLLECT COD|MAIL|cajole alo|
+2693|102|3|2|43|43090.30|0.03|0.04|N|O|1996-10-24|1996-10-24|1996-11-03|TAKE BACK RETURN|TRUCK|as are according to th|
+2694|153|1|1|30|31594.50|0.02|0.06|N|O|1996-06-20|1996-06-01|1996-07-15|NONE|TRUCK|oxes. never iro|
+2694|157|2|2|35|37000.25|0.07|0.03|N|O|1996-05-24|1996-06-01|1996-05-25|NONE|RAIL|atelets past the furiously final deposits |
+2694|19|3|3|15|13785.15|0.08|0.02|N|O|1996-06-30|1996-05-01|1996-07-25|TAKE BACK RETURN|REG AIR|e blithely even platelets. special wa|
+2694|20|10|4|12|11040.24|0.00|0.05|N|O|1996-04-24|1996-04-22|1996-05-14|DELIVER IN PERSON|RAIL|foxes atop the hockey pla|
+2694|108|9|5|10|10081.00|0.08|0.08|N|O|1996-06-23|1996-05-28|1996-06-27|COLLECT COD|REG AIR|fluffily fluffy accounts. even packages hi|
+2695|184|5|1|21|22767.78|0.07|0.00|N|O|1996-10-04|1996-11-02|1996-10-21|NONE|MAIL|y regular pinto beans. evenly regular packa|
+2695|19|9|2|44|40436.44|0.09|0.07|N|O|1996-10-05|1996-10-10|1996-11-01|NONE|MAIL|ts. busy platelets boost|
+2695|144|7|3|21|21926.94|0.02|0.07|N|O|1996-09-13|1996-09-25|1996-10-13|NONE|TRUCK|s. furiously ironic platelets ar|
+2695|58|6|4|16|15328.80|0.08|0.08|N|O|1996-11-16|1996-10-05|1996-11-22|NONE|TRUCK|its. theodolites sleep slyly|
+2695|86|7|5|40|39443.20|0.02|0.03|N|O|1996-11-02|1996-10-26|1996-11-14|NONE|FOB|ructions. pending|
+2720|45|6|1|5|4725.20|0.10|0.06|A|F|1993-06-24|1993-08-08|1993-07-08|NONE|FOB|ously ironic foxes thrash|
+2720|17|8|2|42|38514.42|0.09|0.03|R|F|1993-07-25|1993-07-23|1993-08-23|COLLECT COD|REG AIR|fter the inst|
+2720|120|1|3|50|51006.00|0.10|0.02|A|F|1993-08-10|1993-07-29|1993-09-06|NONE|SHIP|l requests. deposits nag furiously|
+2720|109|2|4|49|49445.90|0.06|0.02|A|F|1993-07-09|1993-07-14|1993-07-13|NONE|REG AIR| accounts. fluffily bold pack|
+2720|121|6|5|27|27570.24|0.04|0.00|R|F|1993-06-29|1993-08-06|1993-07-28|NONE|TRUCK|eas. carefully regular |
+2721|183|4|1|49|53075.82|0.00|0.08|N|O|1996-02-14|1996-04-26|1996-03-02|DELIVER IN PERSON|AIR|ounts poach carefu|
+2721|3|4|2|2|1806.00|0.02|0.05|N|O|1996-02-13|1996-03-14|1996-02-28|TAKE BACK RETURN|TRUCK| slyly final requests against |
+2722|124|7|1|21|21506.52|0.09|0.01|A|F|1994-07-29|1994-06-26|1994-08-09|NONE|RAIL|e carefully around the furiously ironic pac|
+2722|146|7|2|15|15692.10|0.05|0.03|R|F|1994-07-02|1994-06-01|1994-07-13|COLLECT COD|AIR|refully final asympt|
+2722|34|10|3|16|14944.48|0.04|0.06|R|F|1994-05-25|1994-06-09|1994-05-26|NONE|MAIL|ts besides the fluffy,|
+2723|13|7|1|47|42911.47|0.09|0.07|N|O|1995-12-05|1995-11-19|1995-12-11|TAKE BACK RETURN|AIR|furiously r|
+2723|32|3|2|10|9320.30|0.06|0.08|N|O|1995-11-27|1995-11-29|1995-12-12|DELIVER IN PERSON|MAIL|al, special r|
+2723|162|1|3|2|2124.32|0.10|0.01|N|O|1995-11-09|1995-11-10|1995-11-14|TAKE BACK RETURN|FOB| courts boost quickly about th|
+2723|82|3|4|12|11784.96|0.01|0.05|N|O|1995-12-24|1995-11-15|1996-01-17|DELIVER IN PERSON|RAIL|bold foxes are bold packages. regular, fin|
+2723|129|10|5|40|41164.80|0.09|0.05|N|O|1995-11-17|1995-11-22|1995-11-18|TAKE BACK RETURN|MAIL|unwind fluffily carefully regular realms.|
+2724|92|4|1|47|46628.23|0.09|0.01|A|F|1994-11-23|1994-11-13|1994-12-03|COLLECT COD|TRUCK|unusual patterns nag. special p|
+2724|147|8|2|21|21989.94|0.09|0.02|A|F|1994-11-25|1994-10-15|1994-12-07|COLLECT COD|RAIL|as. carefully regular dependencies wak|
+2724|50|3|3|22|20901.10|0.04|0.06|A|F|1994-09-19|1994-11-18|1994-10-17|TAKE BACK RETURN|TRUCK|express fo|
+2724|35|6|4|1|935.03|0.07|0.03|A|F|1994-12-26|1994-11-27|1995-01-07|NONE|MAIL|lyly carefully blithe theodolites-- pl|
+2724|149|2|5|29|30425.06|0.05|0.06|A|F|1995-01-10|1994-11-17|1995-02-04|COLLECT COD|MAIL|l requests hagg|
+2725|118|2|1|23|23416.53|0.10|0.08|R|F|1994-08-25|1994-06-22|1994-08-28|TAKE BACK RETURN|REG AIR|y regular deposits. brave foxes |
+2725|5|8|2|41|37105.00|0.01|0.00|R|F|1994-07-05|1994-06-29|1994-08-02|DELIVER IN PERSON|TRUCK|ns sleep furiously c|
+2725|189|10|3|15|16337.70|0.07|0.03|R|F|1994-08-06|1994-08-09|1994-08-15|TAKE BACK RETURN|AIR|? furiously regular a|
+2726|1|6|1|50|45050.00|0.00|0.06|R|F|1993-03-04|1993-01-29|1993-03-28|COLLECT COD|TRUCK| furiously bold theodolites|
+2727|151|6|1|3|3153.45|0.03|0.01|N|O|1998-06-18|1998-06-06|1998-06-23|NONE|RAIL| the carefully regular foxes u|
+2752|31|2|1|41|38172.23|0.02|0.05|A|F|1994-03-02|1994-01-31|1994-03-06|DELIVER IN PERSON|AIR|tructions hag|
+2752|7|2|2|29|26303.00|0.02|0.04|R|F|1994-01-22|1994-01-08|1994-01-28|COLLECT COD|TRUCK|gly blithely re|
+2752|56|7|3|4|3824.20|0.08|0.00|A|F|1993-12-14|1994-02-13|1994-01-05|DELIVER IN PERSON|TRUCK|telets haggle. regular, final |
+2752|24|7|4|40|36960.80|0.09|0.06|A|F|1994-01-24|1994-01-18|1994-02-22|DELIVER IN PERSON|MAIL|into beans are after the sly|
+2752|126|5|5|22|22574.64|0.03|0.04|A|F|1994-03-20|1994-02-08|1994-04-01|TAKE BACK RETURN|TRUCK|equests nag. regular dependencies are furio|
+2752|170|5|6|21|22473.57|0.09|0.05|R|F|1994-01-01|1994-01-24|1994-01-24|COLLECT COD|SHIP| along the quickly |
+2752|199|10|7|38|41769.22|0.08|0.00|R|F|1994-02-23|1993-12-23|1994-03-24|DELIVER IN PERSON|SHIP|es boost. slyly silent ideas|
+2753|13|3|1|6|5478.06|0.10|0.04|A|F|1993-12-30|1994-01-28|1994-01-29|COLLECT COD|TRUCK|s accounts|
+2753|48|7|2|40|37921.60|0.03|0.05|A|F|1994-01-06|1994-02-13|1994-02-03|DELIVER IN PERSON|SHIP|latelets kindle slyly final depos|
+2753|89|10|3|30|29672.40|0.00|0.07|A|F|1994-01-26|1994-01-29|1994-02-02|NONE|RAIL|ans wake fluffily blithely iro|
+2753|31|7|4|7|6517.21|0.07|0.03|R|F|1994-02-11|1994-01-22|1994-03-10|DELIVER IN PERSON|AIR|xpress ideas detect b|
+2753|137|8|5|36|37336.68|0.04|0.08|R|F|1994-03-15|1994-01-03|1994-04-03|DELIVER IN PERSON|SHIP|gle slyly final c|
+2753|50|1|6|17|16150.85|0.01|0.08|A|F|1994-03-08|1994-01-17|1994-03-11|TAKE BACK RETURN|REG AIR| carefully bold deposits sublate s|
+2753|148|9|7|20|20962.80|0.01|0.06|R|F|1994-02-24|1994-02-04|1994-03-23|DELIVER IN PERSON|FOB| express pack|
+2754|149|6|1|4|4196.56|0.05|0.08|A|F|1994-07-13|1994-05-15|1994-08-02|NONE|REG AIR|blithely silent requests. regular depo|
+2754|177|5|2|19|20466.23|0.01|0.07|A|F|1994-06-27|1994-05-06|1994-06-28|NONE|FOB|latelets hag|
+2755|92|4|1|19|18849.71|0.10|0.00|R|F|1992-02-11|1992-03-15|1992-02-14|TAKE BACK RETURN|MAIL|furiously special deposits|
+2755|24|3|2|11|10164.22|0.03|0.08|A|F|1992-04-12|1992-05-07|1992-04-21|COLLECT COD|RAIL|egular excuses sleep carefully.|
+2755|64|3|3|21|20245.26|0.08|0.04|R|F|1992-02-13|1992-04-20|1992-03-02|NONE|AIR|furious re|
+2755|131|7|4|5|5155.65|0.01|0.00|A|F|1992-02-27|1992-04-07|1992-03-09|TAKE BACK RETURN|AIR|e the furi|
+2755|116|7|5|48|48773.28|0.05|0.06|R|F|1992-03-22|1992-03-10|1992-04-14|DELIVER IN PERSON|MAIL|yly even epitaphs for the |
+2756|118|9|1|35|35633.85|0.03|0.02|R|F|1994-06-08|1994-06-01|1994-06-21|TAKE BACK RETURN|AIR| deposits grow bold sheaves; iro|
+2756|80|9|2|47|46063.76|0.06|0.01|R|F|1994-05-10|1994-05-25|1994-05-13|NONE|AIR|e final, f|
+2756|105|8|3|31|31158.10|0.01|0.07|A|F|1994-07-27|1994-07-06|1994-08-22|TAKE BACK RETURN|TRUCK|en instructions use quickly.|
+2756|72|2|4|30|29162.10|0.00|0.04|A|F|1994-06-05|1994-06-30|1994-06-14|DELIVER IN PERSON|TRUCK|ular packages. regular deposi|
+2757|148|5|1|26|27251.64|0.07|0.00|N|O|1995-08-19|1995-10-02|1995-09-06|DELIVER IN PERSON|MAIL|around the blithely|
+2757|22|7|2|12|11064.24|0.07|0.08|N|O|1995-08-01|1995-09-04|1995-08-08|TAKE BACK RETURN|SHIP| regular, eve|
+2757|73|3|3|17|16542.19|0.10|0.04|N|O|1995-09-06|1995-09-27|1995-09-22|DELIVER IN PERSON|AIR|er the furiously silent |
+2757|140|1|4|25|26003.50|0.08|0.01|N|O|1995-11-09|1995-09-12|1995-11-23|NONE|AIR|uickly regular |
+2757|70|7|5|14|13580.98|0.04|0.05|N|O|1995-09-01|1995-08-24|1995-09-03|TAKE BACK RETURN|SHIP|special deposits u|
+2758|121|10|1|20|20422.40|0.02|0.04|N|O|1998-07-27|1998-09-10|1998-08-21|TAKE BACK RETURN|AIR|ptotes sleep furiously|
+2758|23|8|2|17|15691.34|0.10|0.06|N|O|1998-09-25|1998-10-03|1998-10-25|NONE|MAIL| accounts! qui|
+2758|26|5|3|1|926.02|0.06|0.02|N|O|1998-10-09|1998-09-15|1998-10-16|NONE|TRUCK|ake furious|
+2759|59|1|1|10|9590.50|0.10|0.03|R|F|1993-12-14|1994-01-08|1994-01-01|COLLECT COD|FOB|s. busily ironic theodo|
+2759|113|10|2|37|37485.07|0.00|0.06|R|F|1994-03-05|1994-02-22|1994-03-18|DELIVER IN PERSON|REG AIR|lar Tiresias affix ironically carefully sp|
+2759|112|9|3|11|11133.21|0.03|0.08|A|F|1994-01-24|1994-01-16|1994-02-21|DELIVER IN PERSON|TRUCK|hely regular |
+2759|23|2|4|31|28613.62|0.02|0.05|A|F|1994-01-11|1994-01-15|1994-01-23|NONE|SHIP|ithely aft|
+2784|33|4|1|45|41986.35|0.03|0.01|N|O|1998-02-15|1998-04-07|1998-02-26|COLLECT COD|AIR|yly along the asymptotes. reque|
+2784|54|5|2|23|21943.15|0.03|0.05|N|O|1998-03-28|1998-02-07|1998-04-17|DELIVER IN PERSON|AIR|uests lose after |
+2784|175|4|3|40|43006.80|0.07|0.01|N|O|1998-04-28|1998-03-19|1998-05-03|DELIVER IN PERSON|TRUCK|deas nag furiously never unusual |
+2784|29|10|4|3|2787.06|0.04|0.03|N|O|1998-01-19|1998-04-05|1998-02-05|TAKE BACK RETURN|AIR|n packages. foxes haggle quickly sile|
+2785|100|3|1|34|34003.40|0.08|0.06|N|O|1995-08-07|1995-09-09|1995-09-05|NONE|RAIL|ly final packages haggl|
+2785|110|7|2|37|37374.07|0.08|0.04|N|O|1995-07-25|1995-09-12|1995-08-06|DELIVER IN PERSON|TRUCK|tructions. furiously |
+2785|65|10|3|33|31846.98|0.08|0.06|N|O|1995-10-16|1995-08-24|1995-11-02|DELIVER IN PERSON|MAIL|fter the furiously final p|
+2785|48|1|4|34|32233.36|0.00|0.02|N|O|1995-09-16|1995-09-09|1995-10-11|COLLECT COD|SHIP|kages wake carefully silent |
+2786|136|2|1|15|15541.95|0.03|0.04|A|F|1992-05-19|1992-05-08|1992-05-28|COLLECT COD|TRUCK|low deposits are ironic|
+2786|51|3|2|42|39944.10|0.10|0.04|R|F|1992-05-15|1992-04-22|1992-05-30|DELIVER IN PERSON|AIR|unts are against the furious|
+2786|156|1|3|41|43302.15|0.04|0.05|R|F|1992-07-01|1992-06-04|1992-07-13|COLLECT COD|RAIL|ix requests. bold requests a|
+2786|23|4|4|24|22152.48|0.05|0.02|A|F|1992-04-04|1992-06-09|1992-05-02|DELIVER IN PERSON|MAIL|ans. slyly unusual platelets detect. unus|
+2786|50|3|5|43|40852.15|0.06|0.03|R|F|1992-04-22|1992-05-13|1992-04-29|NONE|RAIL|ons. theodolites after|
+2786|162|1|6|21|22305.36|0.08|0.00|A|F|1992-05-03|1992-05-01|1992-05-14|COLLECT COD|AIR|slow instructi|
+2787|33|9|1|4|3732.12|0.04|0.04|N|O|1996-01-26|1995-11-26|1996-02-20|TAKE BACK RETURN|SHIP|ts. instructions nag furiously according |
+2788|177|8|1|16|17234.72|0.06|0.06|A|F|1994-10-04|1994-11-25|1994-10-18|DELIVER IN PERSON|AIR| requests wake carefully. carefully si|
+2789|163|8|1|16|17010.56|0.03|0.02|N|O|1998-04-18|1998-05-25|1998-05-12|DELIVER IN PERSON|REG AIR|o beans use carefully|
+2789|23|4|2|41|37843.82|0.02|0.05|N|O|1998-03-20|1998-05-15|1998-03-21|COLLECT COD|MAIL|d packages-- fluffily specia|
+2789|176|5|3|33|35513.61|0.06|0.02|N|O|1998-04-21|1998-05-02|1998-04-30|COLLECT COD|TRUCK|deposits. ironic |
+2789|16|3|4|47|43052.47|0.02|0.04|N|O|1998-03-29|1998-05-05|1998-04-07|NONE|RAIL|usly busy packages wake against the unusual|
+2789|197|1|5|23|25235.37|0.02|0.07|N|O|1998-03-25|1998-05-10|1998-04-24|COLLECT COD|RAIL|cording to the careful de|
+2789|144|5|6|16|16706.24|0.07|0.03|N|O|1998-05-11|1998-05-08|1998-05-24|TAKE BACK RETURN|RAIL|d the carefully iron|
+2789|133|4|7|42|43391.46|0.01|0.00|N|O|1998-04-28|1998-05-17|1998-05-24|TAKE BACK RETURN|AIR|ending packages shoul|
+2790|185|6|1|27|29299.86|0.06|0.08|R|F|1994-09-04|1994-09-27|1994-09-16|TAKE BACK RETURN|MAIL|ilent packages cajole. quickly ironic requ|
+2790|117|1|2|50|50855.50|0.00|0.06|A|F|1994-12-08|1994-11-17|1994-12-19|NONE|RAIL|fter the regular ideas. f|
+2790|184|5|3|19|20599.42|0.06|0.00|R|F|1994-10-23|1994-10-03|1994-10-26|TAKE BACK RETURN|RAIL|uffily even excuses. furiously thin|
+2790|197|8|4|24|26332.56|0.07|0.01|A|F|1994-12-04|1994-10-10|1994-12-25|NONE|MAIL|ments. slyly f|
+2790|148|9|5|11|11529.54|0.08|0.03|A|F|1994-09-28|1994-11-14|1994-10-04|TAKE BACK RETURN|AIR|lar requests poach slyly foxes|
+2790|73|3|6|13|12649.91|0.08|0.00|R|F|1994-09-20|1994-10-10|1994-10-20|COLLECT COD|SHIP|n deposits according to the regul|
+2790|4|1|7|32|28928.00|0.08|0.02|A|F|1994-09-25|1994-10-26|1994-10-01|NONE|SHIP|ully pending|
+2791|59|10|1|49|46993.45|0.10|0.04|A|F|1995-01-11|1994-11-10|1995-02-08|COLLECT COD|MAIL| accounts sleep at the bold, regular pinto |
+2791|63|4|2|4|3852.24|0.10|0.08|A|F|1995-01-02|1994-12-28|1995-01-29|NONE|SHIP|slyly bold packages boost. slyly|
+2791|133|9|3|44|45457.72|0.08|0.06|R|F|1994-11-17|1994-11-12|1994-12-14|NONE|FOB|heodolites use furio|
+2791|156|8|4|24|25347.60|0.04|0.02|R|F|1995-01-30|1994-11-20|1995-02-08|DELIVER IN PERSON|TRUCK|ilent forges. quickly special pinto beans |
+2791|105|2|5|8|8040.80|0.02|0.04|R|F|1995-01-30|1994-11-24|1995-02-13|NONE|FOB|se. close ideas alongs|
+2791|75|3|6|9|8775.63|0.08|0.02|R|F|1994-11-19|1994-12-14|1994-12-10|TAKE BACK RETURN|AIR|pendencies. blithely bold patterns acr|
+2791|29|2|7|26|24154.52|0.06|0.03|R|F|1995-02-06|1994-12-07|1995-02-23|DELIVER IN PERSON|AIR|uriously special instructio|
+2816|59|10|1|33|31648.65|0.00|0.07|R|F|1994-10-19|1994-11-10|1994-11-09|NONE|REG AIR|s; slyly even theodo|
+2816|142|3|2|4|4168.56|0.05|0.04|R|F|1994-12-11|1994-12-07|1995-01-03|NONE|FOB|. blithely pending id|
+2816|121|6|3|4|4084.48|0.02|0.06|R|F|1994-12-12|1994-12-05|1994-12-30|NONE|RAIL| requests print above the final deposits|
+2817|60|8|1|25|24001.50|0.07|0.01|R|F|1994-04-21|1994-06-20|1994-05-07|DELIVER IN PERSON|FOB|doze blithely.|
+2817|32|8|2|5|4660.15|0.03|0.04|A|F|1994-05-07|1994-05-31|1994-05-12|TAKE BACK RETURN|AIR|furiously unusual theodolites use furiou|
+2817|172|10|3|35|37525.95|0.01|0.07|A|F|1994-05-20|1994-06-03|1994-05-22|COLLECT COD|FOB|gular foxes|
+2817|161|2|4|4|4244.64|0.00|0.05|R|F|1994-06-04|1994-06-11|1994-06-10|NONE|TRUCK|n accounts wake across the fluf|
+2818|121|4|1|12|12253.44|0.10|0.03|A|F|1995-02-01|1995-03-10|1995-02-16|NONE|AIR|lms. quickly bold asymp|
+2818|199|2|2|22|24182.18|0.06|0.07|R|F|1995-02-28|1995-03-10|1995-03-06|TAKE BACK RETURN|RAIL|egrate toward the carefully iron|
+2818|45|6|3|11|10395.44|0.01|0.06|R|F|1995-02-18|1995-02-11|1995-03-19|TAKE BACK RETURN|TRUCK|ggle across the carefully blithe|
+2818|40|6|4|32|30081.28|0.08|0.08|R|F|1995-02-04|1995-03-05|1995-02-18|COLLECT COD|REG AIR|arefully! ac|
+2818|18|8|5|42|38556.42|0.08|0.04|A|F|1995-02-12|1995-02-19|1995-03-13|COLLECT COD|MAIL|ar accounts wake carefully a|
+2818|91|5|6|7|6937.63|0.06|0.03|R|F|1995-03-24|1995-03-09|1995-04-06|TAKE BACK RETURN|TRUCK|ly according to the r|
+2819|70|1|1|17|16491.19|0.08|0.08|A|F|1994-07-16|1994-07-15|1994-07-17|TAKE BACK RETURN|RAIL|en deposits above the f|
+2819|67|2|2|12|11604.72|0.03|0.08|R|F|1994-07-18|1994-06-24|1994-07-28|NONE|MAIL| regular, regular a|
+2819|5|2|3|28|25340.00|0.03|0.08|R|F|1994-05-09|1994-07-02|1994-05-15|NONE|RAIL|ckages sublate carefully closely regular |
+2819|153|4|4|5|5265.75|0.00|0.02|R|F|1994-05-29|1994-06-12|1994-06-28|NONE|TRUCK| fluffily unusual foxes sleep caref|
+2819|200|3|5|6|6601.20|0.03|0.01|A|F|1994-07-22|1994-08-02|1994-07-29|NONE|REG AIR|eas after the carefully express pack|
+2820|174|2|1|23|24705.91|0.04|0.08|R|F|1994-07-10|1994-08-08|1994-07-21|NONE|MAIL| was furiously. deposits among the ironic|
+2820|126|9|2|33|33861.96|0.08|0.06|A|F|1994-07-07|1994-08-17|1994-08-02|DELIVER IN PERSON|AIR|carefully even pinto beans. |
+2820|141|10|3|38|39563.32|0.03|0.08|A|F|1994-09-10|1994-08-07|1994-10-07|TAKE BACK RETURN|MAIL|ests despite the carefully unusual a|
+2820|197|9|4|40|43887.60|0.06|0.06|A|F|1994-08-08|1994-07-30|1994-08-21|TAKE BACK RETURN|REG AIR|g multipliers. final c|
+2821|181|2|1|4|4324.72|0.00|0.00|A|F|1993-09-15|1993-10-02|1993-09-17|TAKE BACK RETURN|TRUCK|nding foxes.|
+2821|72|1|2|4|3888.28|0.09|0.00|A|F|1993-11-19|1993-09-20|1993-11-27|TAKE BACK RETURN|TRUCK|ual multipliers. final deposits cajol|
+2821|164|1|3|27|28732.32|0.01|0.01|A|F|1993-11-27|1993-10-11|1993-12-08|COLLECT COD|TRUCK|requests. blit|
+2822|151|9|1|39|40994.85|0.04|0.02|R|F|1993-09-11|1993-08-29|1993-09-18|NONE|MAIL|kly about the sly|
+2823|86|7|1|45|44373.60|0.03|0.04|N|O|1995-12-28|1995-11-27|1996-01-02|DELIVER IN PERSON|SHIP|furiously special idea|
+2823|160|5|2|18|19082.88|0.00|0.03|N|O|1995-11-11|1995-10-30|1995-12-08|TAKE BACK RETURN|TRUCK| final deposits. furiously regular foxes u|
+2823|186|7|3|11|11947.98|0.07|0.02|N|O|1995-12-10|1995-11-24|1995-12-21|DELIVER IN PERSON|SHIP|bold requests nag blithely s|
+2823|139|10|4|48|49878.24|0.09|0.03|N|O|1995-11-21|1995-10-30|1995-11-27|NONE|SHIP|ously busily slow excus|
+2823|99|2|5|18|17983.62|0.04|0.06|N|O|1995-11-09|1995-10-30|1995-11-19|NONE|AIR|eas. decoys cajole deposi|
+2823|123|2|6|20|20462.40|0.07|0.00|N|O|1995-11-13|1995-12-06|1995-12-07|NONE|MAIL|its sleep between the unusual, ironic pac|
+2823|86|7|7|12|11832.96|0.02|0.04|N|O|1995-12-22|1995-11-20|1996-01-13|NONE|REG AIR|the slyly ironic dolphins; fin|
+2848|65|4|1|44|42462.64|0.01|0.05|R|F|1992-04-14|1992-05-09|1992-04-19|DELIVER IN PERSON|MAIL|ions. slyly express instructions n|
+2848|165|6|2|8|8521.28|0.07|0.01|A|F|1992-03-21|1992-05-18|1992-04-07|DELIVER IN PERSON|TRUCK|. silent, final ideas sublate packages. ir|
+2848|138|4|3|8|8305.04|0.07|0.08|A|F|1992-06-20|1992-04-12|1992-07-09|NONE|SHIP|sly regular foxes. |
+2848|125|6|4|34|34854.08|0.02|0.08|A|F|1992-03-15|1992-04-24|1992-04-12|TAKE BACK RETURN|RAIL|ts along the blithely regu|
+2848|195|7|5|18|19713.42|0.07|0.03|R|F|1992-04-10|1992-06-01|1992-05-05|DELIVER IN PERSON|TRUCK|osits haggle. stealthily ironic packa|
+2849|154|2|1|16|16866.40|0.09|0.08|N|O|1996-05-20|1996-07-23|1996-06-18|NONE|TRUCK|. furiously regular requ|
+2849|187|8|2|39|42400.02|0.10|0.03|N|O|1996-05-22|1996-07-18|1996-06-05|TAKE BACK RETURN|SHIP|s sleep furiously silently regul|
+2849|60|1|3|24|23041.44|0.01|0.05|N|O|1996-06-12|1996-07-10|1996-06-27|TAKE BACK RETURN|AIR|e slyly even asymptotes. slo|
+2849|55|7|4|48|45842.40|0.05|0.02|N|O|1996-05-03|1996-06-05|1996-05-28|NONE|AIR|mong the carefully regular theodol|
+2849|28|7|5|30|27840.60|0.10|0.06|N|O|1996-08-24|1996-07-08|1996-09-03|TAKE BACK RETURN|SHIP|ly. carefully silent|
+2849|69|4|6|30|29071.80|0.06|0.07|N|O|1996-06-20|1996-07-23|1996-07-06|NONE|FOB|yly furiously even id|
+2850|97|1|1|43|42874.87|0.02|0.05|N|O|1997-01-11|1996-11-03|1997-02-01|COLLECT COD|REG AIR|unusual accounts|
+2850|110|7|2|30|30303.30|0.09|0.01|N|O|1996-12-14|1996-11-29|1997-01-03|COLLECT COD|AIR|even ideas. busy pinto beans sleep above t|
+2850|105|6|3|49|49249.90|0.09|0.04|N|O|1996-10-07|1996-12-12|1996-10-12|TAKE BACK RETURN|MAIL| slyly unusual req|
+2850|199|3|4|4|4396.76|0.04|0.04|N|O|1996-10-28|1996-12-26|1996-11-07|COLLECT COD|RAIL|al deposits cajole carefully quickly |
+2851|148|5|1|8|8385.12|0.09|0.03|N|O|1997-11-12|1997-11-22|1997-12-11|NONE|REG AIR|y special theodolites. carefully|
+2852|177|6|1|6|6463.02|0.01|0.01|R|F|1993-03-02|1993-04-11|1993-03-11|TAKE BACK RETURN|RAIL| accounts above the furiously un|
+2852|41|10|2|24|22584.96|0.05|0.07|R|F|1993-01-18|1993-03-13|1993-02-14|DELIVER IN PERSON|MAIL| the blithe|
+2852|164|9|3|29|30860.64|0.09|0.05|R|F|1993-04-21|1993-03-22|1993-05-02|COLLECT COD|SHIP|lyly ironi|
+2852|100|3|4|12|12001.20|0.08|0.02|A|F|1993-02-25|1993-03-24|1993-03-07|TAKE BACK RETURN|TRUCK|le. request|
+2852|154|2|5|28|29516.20|0.05|0.03|R|F|1993-02-08|1993-03-30|1993-02-11|NONE|MAIL|e accounts. caref|
+2853|139|5|1|14|14547.82|0.07|0.05|R|F|1994-05-16|1994-07-01|1994-05-27|NONE|TRUCK|oach slyly along t|
+2853|134|10|2|26|26887.38|0.06|0.01|R|F|1994-06-26|1994-06-05|1994-07-02|TAKE BACK RETURN|MAIL|dolphins wake slyly. blith|
+2853|173|3|3|40|42926.80|0.06|0.04|A|F|1994-08-06|1994-06-24|1994-08-29|NONE|RAIL|lyly. pearls cajole. final accounts ca|
+2853|132|8|4|20|20642.60|0.02|0.04|A|F|1994-08-30|1994-06-16|1994-09-06|TAKE BACK RETURN|TRUCK|e slyly silent foxes. express deposits sno|
+2853|36|7|5|1|936.03|0.08|0.05|R|F|1994-09-01|1994-06-27|1994-09-12|TAKE BACK RETURN|FOB|refully slyly quick packages. final c|
+2854|181|2|1|46|49734.28|0.00|0.04|A|F|1994-09-22|1994-08-02|1994-09-30|COLLECT COD|AIR|. furiously regular deposits across th|
+2854|88|9|2|29|28654.32|0.09|0.07|R|F|1994-07-06|1994-08-26|1994-07-09|COLLECT COD|SHIP|y slyly ironic accounts. foxes haggle slyl|
+2854|160|8|3|20|21203.20|0.08|0.01|R|F|1994-09-18|1994-08-03|1994-10-12|COLLECT COD|AIR|rs impress after the deposits. |
+2854|170|1|4|34|36385.78|0.06|0.03|A|F|1994-09-06|1994-08-07|1994-09-22|NONE|REG AIR|age carefully|
+2854|102|3|5|7|7014.70|0.03|0.06|A|F|1994-09-23|1994-08-14|1994-10-10|DELIVER IN PERSON|REG AIR| the pending|
+2854|18|2|6|13|11934.13|0.04|0.03|R|F|1994-09-15|1994-08-18|1994-09-19|DELIVER IN PERSON|SHIP| excuses wak|
+2855|33|4|1|50|46651.50|0.03|0.07|A|F|1993-05-20|1993-06-28|1993-06-16|TAKE BACK RETURN|TRUCK|beans. deposits |
+2880|35|6|1|40|37401.20|0.09|0.00|A|F|1992-05-26|1992-06-01|1992-05-31|COLLECT COD|TRUCK|even requests. quick|
+2880|139|5|2|26|27017.38|0.07|0.07|R|F|1992-04-12|1992-04-15|1992-04-28|NONE|RAIL|ully among the regular warthogs|
+2880|115|9|3|42|42634.62|0.01|0.01|R|F|1992-06-17|1992-05-29|1992-07-11|NONE|REG AIR|ions. carefully final accounts are unusual,|
+2880|18|2|4|46|42228.46|0.02|0.02|A|F|1992-04-21|1992-06-05|1992-05-16|COLLECT COD|RAIL|eep quickly according to t|
+2881|180|10|1|16|17282.88|0.02|0.06|A|F|1992-06-21|1992-06-27|1992-07-03|TAKE BACK RETURN|TRUCK|usly bold |
+2881|10|1|2|1|910.01|0.09|0.03|A|F|1992-05-13|1992-07-21|1992-05-18|COLLECT COD|MAIL|final theodolites. quickly|
+2881|93|6|3|21|20854.89|0.07|0.03|A|F|1992-05-28|1992-07-03|1992-06-02|TAKE BACK RETURN|SHIP|hely express Tiresias. final dependencies |
+2881|140|6|4|7|7280.98|0.06|0.01|R|F|1992-08-03|1992-07-10|1992-08-27|NONE|REG AIR|ironic packages are carefully final ac|
+2882|4|7|1|14|12656.00|0.09|0.02|N|O|1995-09-28|1995-11-11|1995-10-18|TAKE BACK RETURN|MAIL|kly. even requests w|
+2882|42|1|2|30|28261.20|0.00|0.00|N|O|1995-10-15|1995-10-13|1995-10-25|NONE|REG AIR|among the furiously even theodolites. regu|
+2882|197|9|3|29|31818.51|0.10|0.08|N|O|1995-09-10|1995-11-01|1995-10-02|NONE|TRUCK|kages. furiously ironic|
+2882|78|6|4|27|26407.89|0.06|0.02|N|O|1995-09-04|1995-11-11|1995-09-12|DELIVER IN PERSON|MAIL|rding to the regu|
+2882|134|5|5|32|33092.16|0.07|0.03|N|O|1995-10-21|1995-11-10|1995-11-01|COLLECT COD|RAIL|sts. quickly regular e|
+2882|87|8|6|47|46392.76|0.06|0.03|N|O|1995-09-13|1995-09-21|1995-09-14|NONE|REG AIR|l, special|
+2883|1|4|1|33|29733.00|0.08|0.07|R|F|1995-02-26|1995-03-04|1995-03-01|NONE|RAIL|s. final i|
+2883|125|6|2|27|27678.24|0.00|0.02|A|F|1995-03-12|1995-03-10|1995-04-04|TAKE BACK RETURN|REG AIR|s. brave pinto beans nag furiously|
+2883|189|10|3|47|51191.46|0.05|0.04|R|F|1995-01-29|1995-04-19|1995-02-05|DELIVER IN PERSON|SHIP|ep carefully ironic|
+2883|98|2|4|23|22956.07|0.00|0.02|R|F|1995-02-03|1995-03-17|1995-02-19|TAKE BACK RETURN|AIR| even requests cajole. special, regular |
+2883|195|8|5|36|39426.84|0.07|0.06|A|F|1995-05-02|1995-03-14|1995-05-30|COLLECT COD|MAIL|ests detect slyly special packages|
+2884|71|2|1|41|39813.87|0.03|0.00|N|O|1998-01-02|1997-12-17|1998-01-20|DELIVER IN PERSON|TRUCK|ep. slyly even accounts a|
+2884|146|5|2|25|26153.50|0.09|0.08|N|O|1998-01-18|1997-12-06|1998-02-16|TAKE BACK RETURN|MAIL|onic theodolites with the instructi|
+2884|26|7|3|8|7408.16|0.08|0.08|N|O|1997-11-30|1997-11-28|1997-12-14|COLLECT COD|TRUCK|pending accounts about |
+2885|4|9|1|6|5424.00|0.10|0.01|A|F|1993-01-05|1992-12-12|1993-01-19|COLLECT COD|FOB|ctions solve. slyly regular requests n|
+2885|112|3|2|4|4048.44|0.07|0.00|A|F|1992-10-09|1992-12-17|1992-11-04|TAKE BACK RETURN|SHIP| pending packages wake. |
+2885|1|6|3|45|40545.00|0.10|0.04|A|F|1992-12-24|1992-10-30|1993-01-04|NONE|SHIP|ess ideas. regular, silen|
+2885|32|3|4|15|13980.45|0.03|0.04|R|F|1992-10-31|1992-11-24|1992-11-21|DELIVER IN PERSON|MAIL|odolites. boldly pending packages han|
+2885|175|5|5|43|46232.31|0.06|0.00|R|F|1992-11-17|1992-10-30|1992-12-04|DELIVER IN PERSON|SHIP|cial deposits use bold|
+2885|190|1|6|5|5450.95|0.01|0.02|R|F|1993-01-06|1992-11-13|1993-02-05|TAKE BACK RETURN|TRUCK|s. slyly express th|
+2885|50|9|7|40|38002.00|0.05|0.03|A|F|1992-09-23|1992-11-15|1992-10-07|TAKE BACK RETURN|AIR| express depos|
+2886|60|1|1|1|960.06|0.09|0.05|A|F|1995-02-01|1994-12-18|1995-02-28|COLLECT COD|REG AIR|eposits fr|
+2886|184|5|2|38|41198.84|0.02|0.04|A|F|1995-01-21|1995-01-08|1995-01-30|NONE|SHIP|old requests along the fur|
+2886|63|8|3|2|1926.12|0.04|0.07|A|F|1994-11-18|1995-01-31|1994-12-05|COLLECT COD|REG AIR|ar theodolites. e|
+2886|130|3|4|46|47385.98|0.03|0.08|A|F|1995-02-02|1995-01-26|1995-02-15|TAKE BACK RETURN|SHIP|ously final packages sleep blithely regular|
+2887|66|3|1|11|10626.66|0.06|0.00|N|O|1997-07-08|1997-07-17|1997-07-15|COLLECT COD|SHIP|ackages. unusual, speci|
+2887|112|6|2|17|17205.87|0.00|0.08|N|O|1997-08-31|1997-07-04|1997-09-17|DELIVER IN PERSON|SHIP|fily final packages. regula|
+2912|122|1|1|8|8176.96|0.06|0.04|A|F|1992-04-09|1992-04-19|1992-04-26|NONE|RAIL|hs cajole over the slyl|
+2912|115|9|2|18|18271.98|0.00|0.08|R|F|1992-03-13|1992-04-19|1992-03-30|TAKE BACK RETURN|RAIL|unts cajole reg|
+2913|123|6|1|39|39901.68|0.06|0.04|N|O|1997-08-28|1997-09-27|1997-09-02|TAKE BACK RETURN|AIR|. final packages a|
+2913|22|5|2|22|20284.44|0.10|0.07|N|O|1997-09-18|1997-08-11|1997-10-02|COLLECT COD|MAIL|riously pending realms. blithely even pac|
+2913|166|1|3|17|18124.72|0.07|0.04|N|O|1997-10-21|1997-09-25|1997-11-20|NONE|FOB|requests doze quickly. furious|
+2913|143|4|4|5|5215.70|0.10|0.07|N|O|1997-10-07|1997-08-25|1997-10-09|TAKE BACK RETURN|RAIL|haggle. even, bold instructi|
+2913|15|9|5|13|11895.13|0.03|0.01|N|O|1997-10-02|1997-08-20|1997-10-26|COLLECT COD|MAIL|inos are carefully alongside of the bol|
+2913|168|5|6|35|37385.60|0.06|0.08|N|O|1997-08-30|1997-08-21|1997-09-03|COLLECT COD|MAIL|es. quickly even braids against|
+2914|66|7|1|22|21253.32|0.05|0.06|R|F|1993-05-11|1993-04-09|1993-05-22|DELIVER IN PERSON|FOB| carefully about the fluffily ironic gifts|
+2914|163|10|2|25|26579.00|0.03|0.04|A|F|1993-05-14|1993-04-04|1993-05-22|NONE|SHIP|cross the carefully even accounts.|
+2914|35|1|3|4|3740.12|0.00|0.05|R|F|1993-06-11|1993-04-09|1993-06-14|TAKE BACK RETURN|SHIP|s integrate. bold deposits sleep req|
+2914|121|2|4|9|9190.08|0.06|0.01|R|F|1993-06-17|1993-05-26|1993-06-19|NONE|REG AIR|s. carefully final foxes ar|
+2915|175|5|1|28|30104.76|0.10|0.02|R|F|1994-04-17|1994-06-09|1994-05-10|NONE|MAIL|yly special |
+2915|94|7|2|12|11929.08|0.00|0.03|A|F|1994-07-18|1994-06-11|1994-07-27|TAKE BACK RETURN|RAIL|accounts. slyly final|
+2915|136|2|3|15|15541.95|0.07|0.00|A|F|1994-05-01|1994-06-12|1994-05-15|DELIVER IN PERSON|TRUCK|al requests haggle furiousl|
+2915|81|2|4|43|42186.44|0.06|0.05|R|F|1994-06-02|1994-05-24|1994-06-06|DELIVER IN PERSON|SHIP|into beans dazzle alongside of|
+2916|83|4|1|21|20644.68|0.06|0.04|N|O|1996-03-11|1996-02-21|1996-03-30|NONE|REG AIR|uickly express ideas over the slyly even |
+2917|93|4|1|36|35751.24|0.10|0.01|N|O|1998-04-07|1998-02-23|1998-05-01|DELIVER IN PERSON|RAIL|usly ironic d|
+2917|21|2|2|20|18420.40|0.06|0.03|N|O|1997-12-31|1998-01-22|1998-01-12|NONE|MAIL|slyly even ideas wa|
+2917|90|1|3|4|3960.36|0.02|0.07|N|O|1998-01-10|1998-01-18|1998-02-08|TAKE BACK RETURN|REG AIR|s. unusual instruct|
+2917|167|2|4|5|5335.80|0.05|0.01|N|O|1997-12-16|1998-01-26|1998-01-07|NONE|RAIL|bove the furiously silent packages. pend|
+2917|41|10|5|37|34818.48|0.04|0.01|N|O|1997-12-12|1998-02-03|1997-12-23|COLLECT COD|RAIL|dependencies. express |
+2917|194|8|6|7|7659.33|0.05|0.01|N|O|1998-03-21|1998-03-03|1998-03-25|NONE|REG AIR|ly about the regular accounts. carefully pe|
+2918|78|7|1|24|23473.68|0.10|0.03|N|O|1996-12-20|1996-10-28|1996-12-26|DELIVER IN PERSON|FOB| quickly. express requests haggle careful|
+2919|102|5|1|2|2004.20|0.03|0.05|R|F|1993-12-28|1994-02-23|1994-01-18|COLLECT COD|TRUCK|re slyly. regular ideas detect furiousl|
+2919|121|4|2|49|50034.88|0.07|0.02|R|F|1993-12-16|1994-02-28|1993-12-19|COLLECT COD|FOB|hely final inst|
+2919|46|5|3|44|41625.76|0.07|0.07|A|F|1994-04-01|1994-01-12|1994-04-07|TAKE BACK RETURN|TRUCK|final ideas haggle carefully fluff|
+2919|102|5|4|44|44092.40|0.00|0.05|R|F|1994-02-04|1994-02-03|1994-03-02|TAKE BACK RETURN|AIR|es doze around the furiously |
+2944|120|1|1|44|44885.28|0.08|0.05|N|O|1997-12-25|1997-10-28|1998-01-21|COLLECT COD|AIR|ickly special theodolit|
+2944|42|9|2|44|41449.76|0.06|0.02|N|O|1997-10-28|1997-11-22|1997-11-10|NONE|SHIP|ickly. regular requests haggle. idea|
+2944|170|5|3|2|2140.34|0.06|0.07|N|O|1997-12-13|1997-12-01|1998-01-08|DELIVER IN PERSON|REG AIR|luffily expr|
+2944|17|7|4|23|21091.23|0.02|0.03|N|O|1998-01-12|1997-12-03|1998-01-17|TAKE BACK RETURN|MAIL| excuses? regular platelets e|
+2944|75|4|5|18|17551.26|0.10|0.01|N|O|1998-01-07|1997-10-26|1998-01-27|TAKE BACK RETURN|FOB| furiously slyl|
+2944|60|2|6|17|16321.02|0.00|0.03|N|O|1997-10-18|1997-11-27|1997-10-29|TAKE BACK RETURN|SHIP|slyly final dolphins sleep silent the|
+2944|90|1|7|7|6930.63|0.01|0.06|N|O|1997-10-30|1997-11-03|1997-11-03|DELIVER IN PERSON|FOB|fluffily blithely express pea|
+2945|59|10|1|37|35484.85|0.00|0.02|N|O|1996-02-10|1996-03-20|1996-02-12|COLLECT COD|SHIP|l instructions. regular, regular |
+2945|72|2|2|30|29162.10|0.05|0.01|N|O|1996-01-19|1996-02-11|1996-01-26|NONE|TRUCK|ular instructions|
+2945|127|8|3|28|28759.36|0.06|0.02|N|O|1996-03-17|1996-03-13|1996-04-15|COLLECT COD|FOB|le slyly along the eve|
+2945|188|9|4|34|36998.12|0.08|0.06|N|O|1996-02-03|1996-03-17|1996-02-29|COLLECT COD|REG AIR|at the unusual theodolite|
+2945|173|1|5|10|10731.70|0.09|0.05|N|O|1996-03-13|1996-03-10|1996-04-06|COLLECT COD|FOB|thely. final courts could hang qu|
+2945|97|9|6|45|44869.05|0.07|0.00|N|O|1996-03-01|1996-03-25|1996-03-08|TAKE BACK RETURN|MAIL|ainst the final packages|
+2945|52|10|7|47|44746.35|0.07|0.05|N|O|1996-01-05|1996-02-11|1996-01-12|DELIVER IN PERSON|MAIL|quests use|
+2946|10|5|1|25|22750.25|0.05|0.02|N|O|1996-05-06|1996-04-23|1996-05-16|DELIVER IN PERSON|SHIP|ic deposits. furiously|
+2946|94|5|2|48|47716.32|0.03|0.07|N|O|1996-06-02|1996-03-31|1996-06-16|COLLECT COD|TRUCK|oss the platelets. furi|
+2946|3|6|3|35|31605.00|0.03|0.00|N|O|1996-03-15|1996-04-02|1996-03-26|NONE|REG AIR| sublate along the fluffily iron|
+2947|10|1|1|37|33670.37|0.09|0.07|N|O|1995-08-09|1995-07-05|1995-08-20|DELIVER IN PERSON|RAIL|e accounts: expres|
+2947|186|7|2|10|10861.80|0.09|0.07|A|F|1995-06-07|1995-06-26|1995-06-08|NONE|MAIL|lly special |
+2948|118|9|1|48|48869.28|0.00|0.04|R|F|1994-08-29|1994-10-23|1994-09-23|NONE|TRUCK|unusual excuses use about the |
+2948|92|3|2|49|48612.41|0.04|0.07|R|F|1994-12-16|1994-11-08|1995-01-07|DELIVER IN PERSON|MAIL|ress requests. furiously blithe foxes |
+2949|21|6|1|4|3684.08|0.06|0.06|A|F|1994-06-07|1994-06-17|1994-07-04|TAKE BACK RETURN|REG AIR|gular pinto beans wake alongside of the reg|
+2949|70|5|2|50|48503.50|0.05|0.04|A|F|1994-08-04|1994-06-23|1994-08-17|TAKE BACK RETURN|FOB|gular courts cajole across t|
+2949|180|9|3|38|41046.84|0.02|0.06|R|F|1994-05-22|1994-05-25|1994-05-27|COLLECT COD|REG AIR|se slyly requests. carefull|
+2950|130|1|1|32|32964.16|0.01|0.05|N|O|1997-09-21|1997-08-25|1997-10-08|DELIVER IN PERSON|REG AIR|its wake carefully slyly final ideas.|
+2950|66|7|2|18|17389.08|0.10|0.01|N|O|1997-07-19|1997-08-29|1997-08-17|COLLECT COD|TRUCK|uests cajole furio|
+2950|53|4|3|14|13342.70|0.01|0.02|N|O|1997-07-29|1997-08-05|1997-07-31|TAKE BACK RETURN|MAIL|ccounts haggle carefully according |
+2950|187|8|4|45|48923.10|0.08|0.00|N|O|1997-09-05|1997-09-23|1997-09-11|NONE|FOB|ides the b|
+2950|61|2|5|46|44208.76|0.02|0.05|N|O|1997-07-15|1997-09-30|1997-07-25|COLLECT COD|RAIL|to the regular accounts are slyly carefu|
+2950|174|5|6|27|29002.59|0.01|0.03|N|O|1997-10-01|1997-09-13|1997-10-08|NONE|TRUCK|are alongside of the carefully silent |
+2951|3|8|1|5|4515.00|0.03|0.03|N|O|1996-03-27|1996-04-16|1996-03-30|NONE|REG AIR|to beans wake ac|
+2951|136|2|2|24|24867.12|0.07|0.03|N|O|1996-03-24|1996-04-16|1996-04-08|NONE|SHIP| ironic multipliers. express, regular|
+2951|187|8|3|40|43487.20|0.02|0.07|N|O|1996-05-03|1996-04-20|1996-05-22|COLLECT COD|REG AIR|ial deposits wake fluffily about th|
+2951|73|3|4|21|20434.47|0.06|0.08|N|O|1996-04-12|1996-04-27|1996-04-14|DELIVER IN PERSON|REG AIR|nt instructions toward the f|
+2951|51|6|5|15|14265.75|0.07|0.00|N|O|1996-03-25|1996-04-23|1996-03-27|COLLECT COD|REG AIR|inal account|
+2951|138|4|6|18|18686.34|0.06|0.00|N|O|1996-04-04|1996-04-27|1996-04-06|COLLECT COD|FOB|ep about the final, even package|
+2976|9|4|1|32|29088.00|0.06|0.00|A|F|1994-01-26|1994-02-13|1994-02-10|NONE|MAIL|nding, ironic deposits sleep f|
+2976|4|5|2|24|21696.00|0.00|0.03|A|F|1994-03-19|1994-01-26|1994-04-18|COLLECT COD|TRUCK|ronic pinto beans. slyly bol|
+2976|10|5|3|35|31850.35|0.10|0.07|R|F|1993-12-19|1994-02-14|1994-01-11|NONE|RAIL|boost slyly about the regular, regular re|
+2976|82|3|4|22|21605.76|0.00|0.04|A|F|1994-02-08|1994-03-03|1994-02-12|TAKE BACK RETURN|FOB|ncies kindle furiously. carefull|
+2976|134|5|5|13|13443.69|0.00|0.06|A|F|1994-02-06|1994-02-02|1994-02-19|NONE|FOB| furiously final courts boost |
+2976|109|2|6|30|30273.00|0.08|0.03|R|F|1994-03-27|1994-02-01|1994-04-26|TAKE BACK RETURN|RAIL|c ideas! unusual|
+2977|70|5|1|25|24251.75|0.03|0.07|N|O|1996-09-21|1996-10-06|1996-10-13|TAKE BACK RETURN|RAIL|furiously pe|
+2978|90|1|1|29|28712.61|0.00|0.08|A|F|1995-06-03|1995-07-25|1995-06-06|NONE|SHIP|ecial ideas promise slyly|
+2978|127|2|2|42|43139.04|0.01|0.06|N|O|1995-08-19|1995-07-18|1995-09-07|DELIVER IN PERSON|MAIL|ial requests nag blithely alongside of th|
+2978|43|2|3|26|24519.04|0.07|0.05|N|O|1995-07-29|1995-07-22|1995-08-20|COLLECT COD|REG AIR|as haggle against the carefully express dep|
+2978|28|1|4|7|6496.14|0.00|0.00|N|O|1995-07-18|1995-07-03|1995-07-23|NONE|FOB|. final ideas are blithe|
+2978|29|2|5|33|30657.66|0.09|0.03|R|F|1995-05-06|1995-07-23|1995-05-16|COLLECT COD|FOB|s. blithely unusual pack|
+2978|168|7|6|4|4272.64|0.08|0.04|N|O|1995-07-06|1995-07-31|1995-07-19|COLLECT COD|AIR|ffily unusual |
+2979|9|6|1|8|7272.00|0.00|0.08|N|O|1996-06-18|1996-05-21|1996-07-06|COLLECT COD|REG AIR|st blithely; blithely regular gifts dazz|
+2979|11|2|2|47|42817.47|0.05|0.00|N|O|1996-03-25|1996-05-13|1996-04-04|TAKE BACK RETURN|SHIP|iously unusual dependencies wake across|
+2979|188|9|3|35|38086.30|0.04|0.03|N|O|1996-05-25|1996-06-11|1996-06-24|DELIVER IN PERSON|MAIL|old ideas beneath the blit|
+2979|165|4|4|28|29824.48|0.05|0.08|N|O|1996-06-04|1996-04-23|1996-06-24|DELIVER IN PERSON|FOB|ing, regular pinto beans. blithel|
+2980|37|3|1|2|1874.06|0.09|0.03|N|O|1996-11-18|1996-10-22|1996-11-27|TAKE BACK RETURN|SHIP|enly across the special, pending packag|
+2980|10|7|2|48|43680.48|0.04|0.05|N|O|1996-09-25|1996-12-09|1996-10-12|NONE|REG AIR|totes. regular pinto |
+2980|133|9|3|27|27894.51|0.08|0.08|N|O|1996-12-08|1996-12-03|1996-12-14|NONE|REG AIR| theodolites cajole blithely sl|
+2980|25|10|4|49|45325.98|0.03|0.02|N|O|1996-10-04|1996-12-04|1996-10-06|NONE|RAIL|hy packages sleep quic|
+2980|187|8|5|24|26092.32|0.05|0.04|N|O|1997-01-12|1996-10-27|1997-01-14|NONE|MAIL|elets. fluffily regular in|
+2980|109|4|6|43|43391.30|0.01|0.01|N|O|1996-12-07|1996-11-10|1997-01-02|COLLECT COD|AIR|sts. slyly regu|
+2981|14|4|1|17|15538.17|0.03|0.05|N|O|1998-10-17|1998-10-02|1998-10-21|DELIVER IN PERSON|RAIL|, unusual packages x-ray. furious|
+2981|176|4|2|8|8609.36|0.06|0.03|N|O|1998-08-21|1998-09-28|1998-09-05|DELIVER IN PERSON|MAIL|ng to the f|
+2981|37|3|3|14|13118.42|0.03|0.07|N|O|1998-08-30|1998-10-04|1998-09-04|DELIVER IN PERSON|MAIL|kages detect furiously express requests.|
+2982|112|6|1|21|21254.31|0.00|0.01|A|F|1995-04-03|1995-06-08|1995-04-18|DELIVER IN PERSON|AIR|ironic deposits. furiously ex|
+2982|99|2|2|13|12988.17|0.02|0.08|R|F|1995-03-31|1995-05-07|1995-04-18|TAKE BACK RETURN|RAIL|regular deposits unwind alongside |
+2982|70|5|3|21|20371.47|0.01|0.01|R|F|1995-04-19|1995-06-03|1995-04-28|COLLECT COD|SHIP|egular ideas use furiously? bl|
+2983|163|4|1|44|46779.04|0.03|0.06|R|F|1992-02-09|1992-03-07|1992-03-09|TAKE BACK RETURN|AIR|ly regular instruct|
+2983|49|8|2|11|10439.44|0.09|0.06|A|F|1992-04-29|1992-02-27|1992-05-26|NONE|MAIL|aids integrate s|
+3008|132|3|1|8|8257.04|0.10|0.04|N|O|1995-12-06|1996-01-12|1995-12-22|TAKE BACK RETURN|FOB|yly ironic foxes. regular requests h|
+3008|200|3|2|31|34106.20|0.05|0.06|N|O|1995-12-14|1995-12-11|1995-12-31|TAKE BACK RETURN|AIR| bold packages. quic|
+3008|24|5|3|40|36960.80|0.01|0.03|N|O|1995-12-18|1996-01-06|1996-01-11|COLLECT COD|AIR|esias. theodolites detect blithely |
+3008|60|1|4|48|46082.88|0.07|0.06|N|O|1996-01-23|1996-01-07|1996-02-09|COLLECT COD|SHIP|ld theodolites. fluffily bold theodolit|
+3008|105|10|5|31|31158.10|0.03|0.02|N|O|1995-12-01|1996-01-20|1995-12-28|COLLECT COD|RAIL|nts use thinly around the carefully iro|
+3009|45|8|1|48|45361.92|0.10|0.02|N|O|1997-03-19|1997-05-13|1997-04-11|TAKE BACK RETURN|TRUCK| dependencies sleep quickly a|
+3009|185|6|2|38|41236.84|0.00|0.01|N|O|1997-05-01|1997-04-10|1997-05-17|TAKE BACK RETURN|AIR|nal packages should haggle slyly. quickl|
+3009|130|3|3|26|26783.38|0.08|0.02|N|O|1997-05-15|1997-05-10|1997-06-13|TAKE BACK RETURN|SHIP|uriously specia|
+3010|138|4|1|23|23876.99|0.04|0.00|N|O|1996-03-08|1996-02-29|1996-03-27|NONE|TRUCK|ounts. pendin|
+3010|174|4|2|22|23631.74|0.09|0.06|N|O|1996-03-06|1996-04-06|1996-03-18|COLLECT COD|REG AIR| final deposit|
+3010|58|6|3|24|22993.20|0.04|0.07|N|O|1996-05-09|1996-03-14|1996-05-15|DELIVER IN PERSON|RAIL|ar, even reques|
+3010|24|7|4|28|25872.56|0.09|0.06|N|O|1996-03-05|1996-03-28|1996-04-03|DELIVER IN PERSON|FOB|ake carefully carefully even request|
+3010|104|5|5|9|9036.90|0.02|0.02|N|O|1996-04-28|1996-03-17|1996-05-18|NONE|SHIP|inal packages. quickly even pinto|
+3010|92|3|6|38|37699.42|0.05|0.07|N|O|1996-04-15|1996-03-16|1996-04-21|DELIVER IN PERSON|RAIL|accounts ar|
+3011|198|10|1|5|5490.95|0.02|0.04|R|F|1992-04-21|1992-02-23|1992-05-15|NONE|TRUCK|nusual sentiments. carefully bold idea|
+3011|123|4|2|42|42971.04|0.05|0.00|A|F|1992-02-01|1992-03-18|1992-02-29|NONE|TRUCK|osits haggle quickly pending, |
+3012|195|7|1|49|53664.31|0.00|0.00|A|F|1993-08-07|1993-07-01|1993-08-08|NONE|MAIL| quickly furious packages. silently unusua|
+3012|161|2|2|37|39262.92|0.06|0.03|A|F|1993-08-16|1993-06-07|1993-08-24|TAKE BACK RETURN|REG AIR|uickly permanent packages sleep caref|
+3013|94|6|1|31|30816.79|0.08|0.08|N|O|1997-05-03|1997-04-05|1997-05-25|NONE|AIR|y furious depen|
+3013|139|5|2|30|31173.90|0.05|0.06|N|O|1997-05-02|1997-03-09|1997-05-12|TAKE BACK RETURN|MAIL|ronic packages. slyly even|
+3013|120|10|3|35|35704.20|0.00|0.03|N|O|1997-04-02|1997-05-04|1997-04-16|COLLECT COD|MAIL|ely accord|
+3013|181|2|4|17|18380.06|0.01|0.07|N|O|1997-02-26|1997-05-02|1997-03-27|DELIVER IN PERSON|SHIP|fully unusual account|
+3013|60|5|5|20|19201.20|0.00|0.04|N|O|1997-05-06|1997-03-18|1997-05-12|COLLECT COD|RAIL|unts boost regular ideas. slyly pe|
+3013|72|2|6|19|18469.33|0.08|0.07|N|O|1997-05-11|1997-04-18|1997-05-15|COLLECT COD|REG AIR|fluffily pending packages nag furiously al|
+3014|163|4|1|36|38273.76|0.05|0.03|A|F|1992-11-16|1993-01-20|1992-11-28|TAKE BACK RETURN|FOB|ding accounts boost fu|
+3014|106|1|2|36|36219.60|0.00|0.08|R|F|1992-12-28|1992-12-29|1993-01-24|COLLECT COD|MAIL|iously ironic r|
+3014|151|9|3|48|50455.20|0.06|0.02|A|F|1992-12-19|1993-01-08|1992-12-25|DELIVER IN PERSON|REG AIR|y pending theodolites wake. reg|
+3014|114|1|4|14|14197.54|0.10|0.02|R|F|1992-11-19|1993-01-01|1992-12-17|DELIVER IN PERSON|SHIP|. slyly brave platelets nag. careful,|
+3014|75|5|5|28|27301.96|0.02|0.08|R|F|1993-01-09|1992-12-18|1993-01-10|TAKE BACK RETURN|FOB|es are. final braids nag slyly. fluff|
+3014|38|4|6|30|28140.90|0.04|0.01|R|F|1993-02-28|1993-01-02|1993-03-20|TAKE BACK RETURN|AIR| final foxes.|
+3015|3|8|1|5|4515.00|0.09|0.00|A|F|1993-01-10|1992-12-02|1993-01-19|TAKE BACK RETURN|RAIL| the furiously pendi|
+3015|18|2|2|17|15606.17|0.03|0.01|R|F|1992-10-16|1992-11-20|1992-10-28|COLLECT COD|AIR|s above the fluffily final t|
+3015|91|4|3|23|22795.07|0.03|0.05|A|F|1992-12-03|1992-11-19|1992-12-23|DELIVER IN PERSON|FOB|s are slyly carefully special pinto bea|
+3015|156|7|4|7|7393.05|0.10|0.03|A|F|1992-12-07|1992-12-17|1992-12-30|DELIVER IN PERSON|REG AIR| after the evenly special packages ca|
+3015|165|4|5|42|44736.72|0.04|0.02|R|F|1993-01-21|1992-11-07|1993-02-11|DELIVER IN PERSON|AIR|encies haggle furious|
+3015|66|7|6|18|17389.08|0.02|0.03|R|F|1992-10-10|1992-11-19|1992-10-18|TAKE BACK RETURN|MAIL|equests wake fluffil|
+3040|16|6|1|18|16488.18|0.08|0.04|R|F|1993-06-25|1993-07-06|1993-07-19|TAKE BACK RETURN|SHIP|ly thin accou|
+3040|133|9|2|9|9298.17|0.00|0.01|A|F|1993-06-12|1993-05-16|1993-06-14|NONE|RAIL|ges. pending packages wake. requests|
+3040|126|5|3|30|30783.60|0.01|0.01|A|F|1993-08-06|1993-05-18|1993-08-19|NONE|MAIL|x furiously bold packages. expres|
+3040|83|4|4|14|13763.12|0.05|0.04|A|F|1993-05-13|1993-05-18|1993-05-19|TAKE BACK RETURN|REG AIR| haggle carefully. express hocke|
+3040|52|3|5|43|40938.15|0.04|0.04|R|F|1993-05-21|1993-05-25|1993-05-26|NONE|MAIL|sts nag slyly alongside of the depos|
+3040|18|5|6|10|9180.10|0.08|0.04|R|F|1993-05-16|1993-06-24|1993-06-11|DELIVER IN PERSON|MAIL|ely regular foxes haggle dari|
+3041|181|2|1|5|5405.90|0.07|0.04|N|O|1997-07-20|1997-07-15|1997-08-17|COLLECT COD|FOB|posits dazzle special p|
+3041|146|9|2|9|9415.26|0.03|0.03|N|O|1997-06-29|1997-08-14|1997-07-19|COLLECT COD|AIR|iously across the silent pinto beans. furi|
+3041|68|5|3|9|8712.54|0.09|0.06|N|O|1997-08-28|1997-07-23|1997-09-16|TAKE BACK RETURN|FOB|scapades after the special|
+3042|105|2|1|30|30153.00|0.08|0.06|A|F|1995-01-12|1995-02-15|1995-01-24|DELIVER IN PERSON|SHIP|the requests detect fu|
+3042|102|3|2|28|28058.80|0.05|0.03|A|F|1994-11-24|1995-01-02|1994-12-06|TAKE BACK RETURN|MAIL|ng the furiously r|
+3042|14|8|3|34|31076.34|0.04|0.00|R|F|1994-12-11|1995-02-03|1994-12-21|TAKE BACK RETURN|TRUCK|can wake after the enticingly stealthy i|
+3042|48|1|4|19|18012.76|0.02|0.01|A|F|1995-03-05|1995-01-24|1995-03-17|COLLECT COD|TRUCK|e carefully. regul|
+3043|46|9|1|23|21758.92|0.07|0.04|R|F|1992-05-08|1992-07-22|1992-05-18|COLLECT COD|TRUCK|uickly above the pending,|
+3043|6|3|2|15|13590.00|0.03|0.05|A|F|1992-05-27|1992-06-03|1992-06-09|COLLECT COD|FOB|usly furiously|
+3043|60|1|3|42|40322.52|0.10|0.07|R|F|1992-07-15|1992-06-19|1992-07-23|NONE|MAIL|ide of the un|
+3043|91|2|4|5|4955.45|0.10|0.01|A|F|1992-05-22|1992-07-02|1992-06-20|TAKE BACK RETURN|TRUCK|ake blithely re|
+3044|101|2|1|10|10011.00|0.07|0.08|N|O|1996-07-13|1996-05-06|1996-07-21|TAKE BACK RETURN|REG AIR| slyly ironic requests. s|
+3044|168|7|2|3|3204.48|0.06|0.02|N|O|1996-07-27|1996-05-26|1996-08-15|TAKE BACK RETURN|AIR|ecoys haggle furiously pending requests.|
+3044|19|3|3|47|43193.47|0.09|0.00|N|O|1996-05-24|1996-06-22|1996-05-30|NONE|REG AIR|ly around the car|
+3045|88|9|1|41|40511.28|0.05|0.01|N|O|1995-09-30|1995-11-24|1995-10-03|TAKE BACK RETURN|MAIL|ely final foxes. carefully ironic pinto b|
+3045|69|6|2|48|46514.88|0.02|0.03|N|O|1995-10-01|1995-12-16|1995-10-10|TAKE BACK RETURN|MAIL|ole quickly outside th|
+3046|74|5|1|44|42859.08|0.03|0.03|N|O|1996-03-03|1996-02-25|1996-04-01|NONE|AIR| are quickly. blithe|
+3046|54|5|2|46|43886.30|0.03|0.08|N|O|1996-03-22|1996-02-28|1996-04-07|TAKE BACK RETURN|AIR|sits sleep furious|
+3046|2|9|3|31|27962.00|0.03|0.07|N|O|1996-03-24|1996-01-30|1996-03-26|NONE|RAIL|y pending somas alongside of the slyly iro|
+3047|104|5|1|17|17069.70|0.08|0.02|N|O|1997-06-14|1997-04-20|1997-06-23|COLLECT COD|FOB|onic instruction|
+3047|14|1|2|23|21022.23|0.00|0.04|N|O|1997-05-20|1997-06-14|1997-05-28|TAKE BACK RETURN|REG AIR| slyly ironi|
+3072|57|9|1|6|5742.30|0.09|0.05|R|F|1994-02-09|1994-03-24|1994-02-28|DELIVER IN PERSON|REG AIR|gular requests abov|
+3072|108|3|2|36|36291.60|0.07|0.02|R|F|1994-04-14|1994-04-22|1994-05-06|COLLECT COD|AIR| theodolites. blithely e|
+3072|97|8|3|7|6979.63|0.04|0.07|R|F|1994-05-09|1994-03-31|1994-05-19|COLLECT COD|TRUCK|uests. ironic, ironic depos|
+3072|83|4|4|39|38340.12|0.05|0.08|A|F|1994-05-27|1994-04-20|1994-06-14|COLLECT COD|MAIL|es; slyly spe|
+3072|88|9|5|1|988.08|0.01|0.08|R|F|1994-02-26|1994-03-14|1994-03-19|NONE|AIR| slyly ironic attainments. car|
+3073|194|7|1|16|17507.04|0.07|0.01|R|F|1994-03-02|1994-03-23|1994-03-31|DELIVER IN PERSON|AIR|n requests. ironi|
+3073|22|5|2|47|43334.94|0.09|0.00|R|F|1994-03-26|1994-02-12|1994-04-21|NONE|REG AIR|eposits. fluffily|
+3073|87|8|3|10|9870.80|0.03|0.00|R|F|1994-02-11|1994-03-24|1994-02-26|COLLECT COD|FOB| furiously caref|
+3073|29|4|4|14|13006.28|0.09|0.07|R|F|1994-03-24|1994-04-01|1994-04-07|NONE|RAIL|ilently quiet epitaphs.|
+3073|41|10|5|25|23526.00|0.00|0.07|R|F|1994-04-14|1994-03-07|1994-04-22|NONE|TRUCK|nag asymptotes. pinto beans sleep |
+3073|147|8|6|39|40838.46|0.09|0.02|R|F|1994-05-01|1994-02-16|1994-05-12|DELIVER IN PERSON|AIR|lar excuses across the furiously even |
+3073|44|5|7|11|10384.44|0.08|0.07|A|F|1994-05-01|1994-03-06|1994-05-08|COLLECT COD|SHIP|instructions sleep according to the |
+3074|37|8|1|50|46851.50|0.08|0.08|A|F|1993-01-31|1992-12-15|1993-02-20|NONE|AIR|furiously pending requests haggle s|
+3074|139|5|2|39|40526.07|0.03|0.00|R|F|1992-12-08|1993-01-28|1992-12-09|DELIVER IN PERSON|TRUCK|iously throu|
+3075|9|6|1|39|35451.00|0.02|0.03|A|F|1994-06-10|1994-06-21|1994-06-20|NONE|FOB|ing deposits nag |
+3075|52|10|2|2|1904.10|0.07|0.08|R|F|1994-06-14|1994-06-10|1994-06-25|TAKE BACK RETURN|AIR|. unusual, unusual accounts haggle furious|
+3076|85|6|1|44|43343.52|0.00|0.05|A|F|1993-09-14|1993-10-04|1993-09-17|TAKE BACK RETURN|FOB| instructions h|
+3076|106|1|2|22|22134.20|0.08|0.00|A|F|1993-09-05|1993-09-10|1993-09-27|NONE|REG AIR|packages wake furiou|
+3076|5|8|3|31|28055.00|0.06|0.06|A|F|1993-08-10|1993-09-17|1993-08-17|TAKE BACK RETURN|SHIP|regular depos|
+3077|72|2|1|25|24301.75|0.06|0.01|N|O|1997-09-14|1997-10-16|1997-10-06|NONE|TRUCK|lent account|
+3077|91|3|2|40|39643.60|0.05|0.06|N|O|1997-10-22|1997-09-19|1997-11-19|DELIVER IN PERSON|AIR|to the enticing packag|
+3077|78|7|3|13|12714.91|0.03|0.07|N|O|1997-09-09|1997-10-15|1997-09-19|NONE|TRUCK|luffily close depende|
+3077|115|5|4|23|23347.53|0.03|0.02|N|O|1997-11-05|1997-09-16|1997-11-20|NONE|MAIL|lly. fluffily pending dinos across|
+3078|132|3|1|25|25803.25|0.01|0.03|A|F|1993-04-22|1993-05-01|1993-04-28|TAKE BACK RETURN|AIR|express dinos. carefully ironic|
+3078|78|8|2|21|20539.47|0.09|0.07|A|F|1993-03-20|1993-03-21|1993-04-01|COLLECT COD|AIR|e fluffily. |
+3079|70|5|1|20|19401.40|0.05|0.00|N|O|1997-10-18|1997-10-26|1997-11-14|NONE|RAIL|ets are according to the quickly dari|
+3079|117|1|2|38|38650.18|0.08|0.07|N|O|1997-11-07|1997-11-25|1997-12-06|NONE|RAIL|e carefully regular realms|
+3079|17|8|3|40|36680.40|0.02|0.08|N|O|1997-09-26|1997-12-11|1997-10-09|NONE|RAIL|ide of the pending, special deposi|
+3079|24|5|4|2|1848.04|0.00|0.08|N|O|1998-01-05|1997-11-17|1998-01-28|NONE|FOB|ly busy requests believ|
+3079|188|9|5|2|2176.36|0.10|0.00|N|O|1997-12-27|1997-10-25|1998-01-08|COLLECT COD|SHIP|y regular asymptotes doz|
+3079|166|1|6|46|49043.36|0.00|0.00|N|O|1997-11-19|1997-11-04|1997-11-25|DELIVER IN PERSON|REG AIR|es. final, regula|
+3104|51|6|1|20|19021.00|0.01|0.08|A|F|1993-12-31|1993-11-24|1994-01-12|DELIVER IN PERSON|REG AIR|s are. furiously s|
+3104|48|1|2|47|44557.88|0.02|0.05|A|F|1993-12-25|1993-11-02|1994-01-12|COLLECT COD|RAIL|ily daring acc|
+3104|63|4|3|11|10593.66|0.02|0.03|A|F|1993-10-05|1993-11-30|1993-10-27|NONE|TRUCK| special deposits u|
+3104|38|9|4|26|24388.78|0.02|0.08|R|F|1994-01-02|1993-12-05|1994-01-31|TAKE BACK RETURN|TRUCK|es boost carefully. slyly |
+3105|184|5|1|11|11925.98|0.01|0.06|N|O|1997-02-07|1997-02-09|1997-03-01|NONE|FOB|kly bold depths caj|
+3105|45|6|2|9|8505.36|0.08|0.08|N|O|1996-12-25|1997-02-04|1997-01-09|COLLECT COD|SHIP|es wake among t|
+3105|25|4|3|48|44400.96|0.02|0.05|N|O|1997-02-28|1997-01-31|1997-03-18|DELIVER IN PERSON|REG AIR|ending platelets wake carefully ironic inst|
+3105|91|5|4|23|22795.07|0.04|0.07|N|O|1997-03-08|1996-12-14|1997-03-18|COLLECT COD|REG AIR| detect slyly. blithely unusual requests ar|
+3105|90|1|5|8|7920.72|0.07|0.07|N|O|1996-12-28|1996-12-28|1997-01-25|NONE|FOB|s. blithely unusual ideas was after|
+3105|47|6|6|30|28411.20|0.08|0.05|N|O|1997-03-03|1997-02-03|1997-03-05|NONE|FOB|ess accounts boost among t|
+3106|86|7|1|22|21693.76|0.03|0.02|N|O|1997-02-28|1997-02-12|1997-03-03|DELIVER IN PERSON|FOB|structions atop the blithely|
+3106|136|2|2|49|50770.37|0.06|0.06|N|O|1997-02-27|1997-03-11|1997-03-12|NONE|TRUCK|lets. quietly regular courts |
+3106|52|7|3|42|39986.10|0.09|0.07|N|O|1997-04-05|1997-03-17|1997-04-22|COLLECT COD|REG AIR|nstructions wake. furiously |
+3106|196|10|4|6|6577.14|0.10|0.07|N|O|1997-02-02|1997-04-11|1997-02-27|COLLECT COD|REG AIR|symptotes. slyly bold platelets cajol|
+3106|65|2|5|16|15440.96|0.09|0.08|N|O|1997-02-25|1997-04-10|1997-03-16|NONE|AIR|sits wake slyl|
+3107|149|6|1|16|16786.24|0.05|0.04|N|O|1997-08-30|1997-10-20|1997-09-20|TAKE BACK RETURN|REG AIR|regular pinto beans. ironic ideas haggle|
+3107|142|3|2|35|36474.90|0.05|0.06|N|O|1997-08-27|1997-11-19|1997-09-14|COLLECT COD|TRUCK|ets doubt furiously final ideas. final|
+3107|170|9|3|23|24613.91|0.03|0.06|N|O|1997-12-10|1997-11-11|1997-12-14|TAKE BACK RETURN|SHIP|atelets must ha|
+3107|87|8|4|27|26651.16|0.00|0.08|N|O|1997-11-15|1997-10-31|1997-11-28|DELIVER IN PERSON|FOB|furiously final |
+3108|109|2|1|37|37336.70|0.06|0.04|A|F|1993-10-16|1993-10-01|1993-11-09|DELIVER IN PERSON|RAIL| final requests. |
+3108|166|1|2|26|27720.16|0.08|0.05|A|F|1993-11-12|1993-10-05|1993-12-09|COLLECT COD|TRUCK| slyly slow foxes wake furious|
+3109|18|2|1|32|29376.32|0.08|0.03|A|F|1993-09-05|1993-10-06|1993-09-18|DELIVER IN PERSON|FOB|ecial orbits are furiou|
+3109|145|4|2|49|51211.86|0.08|0.06|R|F|1993-10-24|1993-09-30|1993-11-21|TAKE BACK RETURN|AIR| even pearls. furiously pending |
+3109|176|4|3|43|46275.31|0.04|0.07|R|F|1993-09-29|1993-09-06|1993-10-13|COLLECT COD|MAIL|ding to the foxes. |
+3109|79|10|4|26|25455.82|0.01|0.05|R|F|1993-11-16|1993-10-18|1993-12-06|TAKE BACK RETURN|TRUCK| sleep slyly according to t|
+3109|143|2|5|50|52157.00|0.01|0.08|A|F|1993-09-17|1993-10-16|1993-10-11|NONE|FOB| regular packages boost blithely even, re|
+3109|15|9|6|10|9150.10|0.10|0.04|A|F|1993-10-26|1993-10-03|1993-11-09|NONE|TRUCK|sits haggle carefully. regular, unusual ac|
+3110|89|10|1|1|989.08|0.02|0.07|A|F|1995-01-15|1995-01-20|1995-01-30|DELIVER IN PERSON|REG AIR|c theodolites a|
+3110|57|2|2|31|29668.55|0.01|0.06|R|F|1995-03-31|1995-03-07|1995-04-21|TAKE BACK RETURN|REG AIR|en deposits. ironic|
+3110|3|10|3|34|30702.00|0.02|0.02|A|F|1995-02-23|1995-01-27|1995-03-09|TAKE BACK RETURN|FOB|ly pending requests ha|
+3110|40|1|4|16|15040.64|0.04|0.04|A|F|1995-01-10|1995-02-06|1995-01-26|NONE|MAIL|across the regular acco|
+3110|140|6|5|39|40565.46|0.09|0.01|A|F|1995-02-09|1995-01-21|1995-02-21|NONE|MAIL|side of the blithely unusual courts. slyly |
+3111|137|8|1|22|22816.86|0.06|0.05|N|O|1995-09-21|1995-11-09|1995-10-17|COLLECT COD|REG AIR|quests. regular dolphins against the |
+3111|58|10|2|30|28741.50|0.06|0.05|N|O|1995-10-05|1995-11-15|1995-11-01|TAKE BACK RETURN|TRUCK|eas are furiously slyly special deposits.|
+3111|52|3|3|10|9520.50|0.02|0.02|N|O|1995-11-10|1995-11-02|1995-12-04|NONE|FOB|ng the slyly ironic inst|
+3111|132|3|4|31|31996.03|0.00|0.08|N|O|1995-10-26|1995-09-26|1995-11-02|TAKE BACK RETURN|MAIL|kages detect express attainments|
+3111|54|6|5|14|13356.70|0.05|0.04|N|O|1995-10-17|1995-10-19|1995-10-19|TAKE BACK RETURN|SHIP|re. pinto |
+3111|86|7|6|5|4930.40|0.03|0.08|N|O|1995-08-30|1995-10-16|1995-09-04|DELIVER IN PERSON|TRUCK|. carefully even ideas|
+3111|148|9|7|41|42973.74|0.09|0.05|N|O|1995-11-22|1995-11-01|1995-12-01|TAKE BACK RETURN|FOB|fily slow ideas. |
+3136|142|5|1|30|31264.20|0.02|0.08|R|F|1994-08-13|1994-10-02|1994-09-02|TAKE BACK RETURN|RAIL|leep blithel|
+3136|103|4|2|7|7021.70|0.05|0.07|A|F|1994-10-08|1994-09-14|1994-10-11|TAKE BACK RETURN|SHIP|ic pinto beans are slyly. f|
+3136|158|3|3|43|45500.45|0.00|0.07|A|F|1994-09-05|1994-09-25|1994-09-11|NONE|RAIL|. special theodolites ha|
+3136|116|6|4|26|26418.86|0.04|0.05|A|F|1994-10-13|1994-11-07|1994-11-05|TAKE BACK RETURN|AIR|eep fluffily. daringly silent attainments d|
+3136|67|8|5|2|1934.12|0.08|0.07|R|F|1994-11-21|1994-11-03|1994-11-26|DELIVER IN PERSON|TRUCK|? special, silent |
+3136|80|1|6|29|28422.32|0.08|0.07|A|F|1994-11-16|1994-10-03|1994-12-14|NONE|FOB|latelets. final |
+3137|3|4|1|6|5418.00|0.02|0.02|N|O|1995-09-19|1995-10-23|1995-10-16|NONE|SHIP|ly express as|
+3137|6|3|2|4|3624.00|0.06|0.04|N|O|1995-10-01|1995-09-11|1995-10-30|COLLECT COD|RAIL|posits wake. silent excuses boost about|
+3138|93|5|1|7|6951.63|0.05|0.05|R|F|1994-03-04|1994-03-14|1994-03-20|NONE|AIR|lithely quickly even packages. packages|
+3138|44|5|2|27|25489.08|0.09|0.01|R|F|1994-03-24|1994-03-23|1994-04-18|DELIVER IN PERSON|FOB|counts cajole fluffily carefully special i|
+3138|197|8|3|32|35110.08|0.00|0.01|R|F|1994-02-24|1994-05-07|1994-02-28|TAKE BACK RETURN|MAIL|inal foxes affix slyly. fluffily regul|
+3138|172|3|4|38|40742.46|0.07|0.04|R|F|1994-02-21|1994-03-21|1994-03-13|COLLECT COD|FOB|lithely fluffily un|
+3138|10|1|5|12|10920.12|0.09|0.02|A|F|1994-03-04|1994-04-11|1994-03-21|COLLECT COD|FOB|. bold pinto beans haggl|
+3138|44|7|6|25|23601.00|0.05|0.08|A|F|1994-05-19|1994-04-07|1994-06-17|TAKE BACK RETURN|AIR|dolites around the carefully busy the|
+3139|40|6|1|46|43241.84|0.08|0.03|R|F|1992-04-28|1992-03-04|1992-05-19|TAKE BACK RETURN|FOB|of the unusual, unusual re|
+3140|7|4|1|21|19047.00|0.08|0.02|R|F|1992-04-12|1992-05-31|1992-04-21|NONE|REG AIR| furiously sly excuses according to the|
+3140|89|10|2|10|9890.80|0.07|0.01|A|F|1992-05-30|1992-05-09|1992-06-09|COLLECT COD|RAIL|accounts. expres|
+3140|133|4|3|28|28927.64|0.06|0.00|R|F|1992-06-08|1992-07-07|1992-07-08|TAKE BACK RETURN|SHIP|lar ideas. slyly ironic d|
+3141|177|6|1|32|34469.44|0.06|0.00|N|O|1995-11-21|1995-12-18|1995-11-26|DELIVER IN PERSON|FOB|oxes are quickly about t|
+3141|10|7|2|37|33670.37|0.10|0.05|N|O|1996-01-24|1995-12-16|1996-01-27|DELIVER IN PERSON|AIR|press pinto beans. bold accounts boost b|
+3141|79|7|3|9|8811.63|0.09|0.02|N|O|1995-11-11|1995-12-10|1995-12-02|DELIVER IN PERSON|MAIL|uickly ironic, pendi|
+3141|46|9|4|47|44463.88|0.03|0.01|N|O|1995-11-29|1996-01-13|1995-12-10|TAKE BACK RETURN|TRUCK| are slyly pi|
+3142|120|7|1|15|15301.80|0.03|0.08|R|F|1992-08-15|1992-08-18|1992-08-22|DELIVER IN PERSON|AIR|instructions are. ironic packages doz|
+3143|90|1|1|22|21781.98|0.02|0.00|A|F|1993-05-11|1993-03-26|1993-05-20|TAKE BACK RETURN|MAIL|l, special instructions nag |
+3143|183|4|2|40|43327.20|0.03|0.08|A|F|1993-05-07|1993-03-29|1993-05-17|COLLECT COD|FOB|sly unusual theodolites. slyly ev|
+3143|183|4|3|22|23829.96|0.05|0.03|A|F|1993-03-18|1993-05-09|1993-04-14|DELIVER IN PERSON|MAIL|beans. fluf|
+3143|66|7|4|46|44438.76|0.05|0.08|R|F|1993-04-19|1993-03-21|1993-05-05|COLLECT COD|REG AIR|low forges haggle. even packages use bli|
+3168|60|8|1|46|44162.76|0.08|0.08|R|F|1992-02-14|1992-03-02|1992-03-02|TAKE BACK RETURN|SHIP|y across the express accounts. fluff|
+3168|154|5|2|1|1054.15|0.06|0.08|A|F|1992-05-27|1992-03-12|1992-06-09|TAKE BACK RETURN|SHIP|pinto beans. slyly regular courts haggle |
+3168|128|3|3|13|13365.56|0.09|0.02|A|F|1992-03-05|1992-04-29|1992-03-15|NONE|SHIP|ironic somas haggle quick|
+3168|165|10|4|11|11716.76|0.02|0.05|R|F|1992-04-12|1992-03-17|1992-05-12|COLLECT COD|SHIP|ously furious dependenc|
+3169|192|4|1|12|13106.28|0.01|0.04|R|F|1994-01-05|1994-03-18|1994-01-21|COLLECT COD|REG AIR| regular d|
+3169|200|3|2|17|18703.40|0.05|0.04|R|F|1994-03-02|1994-01-21|1994-03-03|DELIVER IN PERSON|TRUCK|usly regular packages. ironi|
+3169|188|9|3|12|13058.16|0.08|0.07|A|F|1994-04-18|1994-03-12|1994-05-08|TAKE BACK RETURN|FOB|atelets. pac|
+3169|105|6|4|26|26132.60|0.10|0.04|R|F|1994-04-08|1994-03-21|1994-04-29|NONE|TRUCK|ter the regular ideas. slyly iro|
+3169|108|9|5|6|6048.60|0.09|0.01|A|F|1994-03-24|1994-02-22|1994-04-04|TAKE BACK RETURN|AIR|ular instructions. ca|
+3169|177|8|6|46|49549.82|0.02|0.07|A|F|1994-02-01|1994-01-22|1994-02-24|DELIVER IN PERSON|RAIL|thely bold theodolites are fl|
+3170|40|6|1|12|11280.48|0.03|0.03|N|O|1998-02-12|1998-01-17|1998-02-24|NONE|TRUCK|ing accounts along the speci|
+3170|100|2|2|21|21002.10|0.01|0.00|N|O|1997-12-09|1998-01-31|1997-12-21|DELIVER IN PERSON|MAIL|o beans. carefully final requests dou|
+3170|89|10|3|27|26705.16|0.00|0.05|N|O|1998-02-25|1998-01-29|1998-02-27|COLLECT COD|AIR|efully bold foxes. regular, ev|
+3170|41|2|4|34|31995.36|0.05|0.04|N|O|1998-02-01|1998-01-11|1998-02-20|TAKE BACK RETURN|TRUCK|s about the fluffily final de|
+3170|90|1|5|32|31682.88|0.02|0.04|N|O|1997-11-24|1997-12-12|1997-12-15|COLLECT COD|SHIP|ggle about the furiously r|
+3170|110|5|6|43|43434.73|0.08|0.05|N|O|1998-01-05|1998-01-04|1998-01-14|NONE|REG AIR|. express dolphins use sly|
+3170|84|5|7|26|25586.08|0.10|0.05|N|O|1998-02-12|1997-12-22|1998-02-28|COLLECT COD|TRUCK|s engage furiously. |
+3171|47|4|1|34|32199.36|0.04|0.00|A|F|1993-05-30|1993-05-27|1993-06-06|DELIVER IN PERSON|REG AIR|r the final, even packages. quickly|
+3171|139|10|2|50|51956.50|0.01|0.04|A|F|1993-07-19|1993-05-15|1993-07-31|TAKE BACK RETURN|REG AIR|riously final foxes about the ca|
+3172|96|9|1|4|3984.36|0.06|0.07|A|F|1992-09-26|1992-08-15|1992-10-20|DELIVER IN PERSON|TRUCK|s are slyly thin package|
+3172|148|7|2|43|45070.02|0.05|0.07|R|F|1992-08-22|1992-07-07|1992-08-26|COLLECT COD|MAIL| final packages. |
+3172|132|3|3|13|13417.69|0.03|0.01|R|F|1992-07-06|1992-08-06|1992-08-05|DELIVER IN PERSON|MAIL|inal deposits haggle along the|
+3172|135|6|4|28|28983.64|0.08|0.04|R|F|1992-07-09|1992-07-14|1992-07-16|NONE|MAIL|regular ideas. packages are furi|
+3172|64|5|5|31|29885.86|0.05|0.08|A|F|1992-09-01|1992-08-27|1992-09-23|NONE|SHIP|. slyly regular dependencies haggle quiet|
+3173|195|6|1|35|38331.65|0.01|0.08|N|O|1996-09-09|1996-10-15|1996-10-04|TAKE BACK RETURN|RAIL| across the slyly even requests.|
+3173|178|7|2|5|5390.85|0.09|0.07|N|O|1996-12-06|1996-09-17|1996-12-07|DELIVER IN PERSON|REG AIR|express depo|
+3173|46|9|3|16|15136.64|0.06|0.01|N|O|1996-08-12|1996-09-21|1996-08-22|NONE|SHIP|e special,|
+3173|94|5|4|2|1988.18|0.00|0.00|N|O|1996-10-15|1996-11-06|1996-10-18|COLLECT COD|MAIL|ular pearls|
+3173|185|6|5|2|2170.36|0.00|0.06|N|O|1996-08-18|1996-09-21|1996-09-07|DELIVER IN PERSON|MAIL|fluffily above t|
+3174|186|7|1|6|6517.08|0.04|0.08|N|O|1996-03-13|1996-02-09|1996-03-22|DELIVER IN PERSON|AIR| furiously ironic|
+3174|194|7|2|4|4376.76|0.01|0.05|N|O|1995-11-17|1996-01-08|1995-11-27|DELIVER IN PERSON|RAIL|deas sleep thi|
+3174|92|4|3|21|20833.89|0.08|0.05|N|O|1996-02-20|1995-12-28|1996-03-17|NONE|MAIL|iously. idly bold theodolites a|
+3174|192|6|4|13|14198.47|0.08|0.06|N|O|1996-01-11|1996-01-26|1996-02-01|DELIVER IN PERSON|SHIP|leep quickly? slyly special platelets|
+3174|72|2|5|39|37910.73|0.02|0.06|N|O|1995-12-02|1996-02-08|1995-12-12|TAKE BACK RETURN|TRUCK| wake slyly foxes. bold requests p|
+3174|120|7|6|8|8160.96|0.07|0.08|N|O|1995-12-07|1996-01-08|1995-12-29|DELIVER IN PERSON|TRUCK|nic deposits among t|
+3175|120|10|1|28|28563.36|0.10|0.01|R|F|1994-09-27|1994-10-05|1994-10-04|NONE|FOB|ore the even, silent foxes. b|
+3175|1|4|2|38|34238.00|0.01|0.07|R|F|1994-10-10|1994-08-25|1994-10-28|NONE|MAIL|the quickly even dolph|
+3175|129|4|3|12|12349.44|0.09|0.07|R|F|1994-10-16|1994-09-15|1994-10-18|NONE|AIR|ter the pending deposits. slyly e|
+3175|85|6|4|14|13791.12|0.02|0.05|R|F|1994-10-21|1994-09-05|1994-11-15|NONE|MAIL|nt dependencies are quietly even |
+3175|18|8|5|47|43146.47|0.08|0.03|R|F|1994-08-08|1994-09-10|1994-08-21|COLLECT COD|REG AIR| final requests x-r|
+3175|175|6|6|44|47307.48|0.01|0.00|R|F|1994-09-26|1994-08-30|1994-10-24|TAKE BACK RETURN|MAIL|are carefully furiously ironic accounts. e|
+3175|1|4|7|32|28832.00|0.01|0.02|R|F|1994-09-29|1994-09-20|1994-10-10|TAKE BACK RETURN|SHIP|lites sleep|
+3200|116|6|1|17|17273.87|0.10|0.00|N|O|1996-06-06|1996-04-21|1996-06-14|DELIVER IN PERSON|AIR|side of the furiously pendin|
+3200|166|1|2|27|28786.32|0.03|0.00|N|O|1996-05-07|1996-05-01|1996-05-09|TAKE BACK RETURN|REG AIR|as haggle furiously against the fluff|
+3200|131|2|3|36|37120.68|0.01|0.01|N|O|1996-03-22|1996-03-19|1996-03-30|DELIVER IN PERSON|FOB|f the carefu|
+3200|30|9|4|11|10230.33|0.10|0.02|N|O|1996-03-18|1996-03-21|1996-04-14|COLLECT COD|RAIL|osits sleep fur|
+3200|198|9|5|16|17571.04|0.05|0.00|N|O|1996-02-28|1996-03-13|1996-03-11|NONE|RAIL|ly against the quiet packages. blith|
+3200|175|3|6|25|26879.25|0.10|0.01|N|O|1996-02-08|1996-04-11|1996-03-06|COLLECT COD|FOB| slyly regular hockey players! pinto beans |
+3201|46|7|1|11|10406.44|0.10|0.06|A|F|1993-09-27|1993-08-29|1993-10-18|NONE|TRUCK|ing to the furiously expr|
+3201|118|5|2|27|27488.97|0.08|0.02|R|F|1993-08-31|1993-08-24|1993-09-08|TAKE BACK RETURN|FOB|deposits are slyly along|
+3201|119|6|3|50|50955.50|0.00|0.08|R|F|1993-10-27|1993-09-30|1993-11-16|COLLECT COD|TRUCK| deposits. express, ir|
+3202|183|4|1|30|32495.40|0.09|0.02|A|F|1993-03-18|1993-03-10|1993-03-23|COLLECT COD|SHIP|ven platelets. furiously final|
+3202|20|4|2|22|20240.44|0.01|0.02|R|F|1993-02-16|1993-02-16|1993-03-16|TAKE BACK RETURN|MAIL|the express packages. fu|
+3203|144|5|1|23|24015.22|0.01|0.07|N|O|1998-01-04|1998-01-12|1998-01-24|COLLECT COD|SHIP|uses. fluffily ironic pinto bea|
+3203|188|9|2|22|23939.96|0.03|0.03|N|O|1998-02-12|1998-01-01|1998-02-18|TAKE BACK RETURN|REG AIR|e the blithely regular accounts boost f|
+3204|12|2|1|10|9120.10|0.10|0.07|R|F|1993-01-27|1993-03-08|1993-01-29|COLLECT COD|SHIP|counts. bold |
+3204|7|10|2|39|35373.00|0.10|0.03|R|F|1993-02-11|1993-03-19|1993-02-28|TAKE BACK RETURN|MAIL|sits sleep theodolites. slyly bo|
+3205|68|5|1|7|6776.42|0.09|0.00|R|F|1992-07-05|1992-06-17|1992-07-07|NONE|SHIP|ly alongsi|
+3205|29|10|2|32|29728.64|0.08|0.03|A|F|1992-06-01|1992-07-10|1992-06-06|TAKE BACK RETURN|RAIL|lar accoun|
+3205|103|6|3|38|38117.80|0.10|0.08|A|F|1992-07-31|1992-06-03|1992-08-20|DELIVER IN PERSON|AIR|usly quiet accounts. slyly pending pinto |
+3205|56|7|4|10|9560.50|0.01|0.07|A|F|1992-06-18|1992-07-04|1992-07-16|COLLECT COD|RAIL| deposits cajole careful|
+3205|70|9|5|18|17461.26|0.03|0.03|A|F|1992-07-04|1992-06-14|1992-08-03|TAKE BACK RETURN|RAIL|symptotes. slyly even deposits ar|
+3205|195|8|6|19|20808.61|0.07|0.08|R|F|1992-05-28|1992-05-30|1992-06-05|COLLECT COD|AIR|yly pending packages snooz|
+3205|69|8|7|36|34886.16|0.06|0.03|A|F|1992-05-31|1992-06-19|1992-06-03|TAKE BACK RETURN|SHIP|s. ironic platelets above the s|
+3206|176|4|1|1|1076.17|0.07|0.05|N|O|1996-11-22|1996-10-16|1996-12-07|TAKE BACK RETURN|FOB|y unusual foxes cajole ab|
+3206|111|5|2|37|37411.07|0.07|0.01|N|O|1996-09-06|1996-10-31|1996-09-25|COLLECT COD|SHIP| quick theodolites hagg|
+3206|186|7|3|24|26068.32|0.00|0.08|N|O|1996-08-25|1996-10-01|1996-09-04|COLLECT COD|TRUCK|encies sleep deposits--|
+3207|113|3|1|2|2026.22|0.10|0.03|N|O|1998-06-15|1998-04-20|1998-06-21|COLLECT COD|MAIL|among the ironic, even packages |
+3207|71|9|2|42|40784.94|0.00|0.00|N|O|1998-05-02|1998-05-10|1998-06-01|NONE|SHIP|to the quickly special accounts? ironically|
+3207|152|7|3|17|17886.55|0.03|0.04|N|O|1998-03-27|1998-04-06|1998-03-28|COLLECT COD|RAIL|eep against the instructions. gifts hag|
+3207|19|6|4|32|29408.32|0.00|0.03|N|O|1998-06-17|1998-04-26|1998-07-07|TAKE BACK RETURN|SHIP|y across the slyly express foxes. bl|
+3207|83|4|5|8|7864.64|0.00|0.06|N|O|1998-06-13|1998-04-26|1998-07-11|COLLECT COD|SHIP|y. final pint|
+3207|134|5|6|32|33092.16|0.03|0.05|N|O|1998-04-19|1998-05-01|1998-05-08|COLLECT COD|FOB|l deposits wake beyond the carefully|
+3232|14|5|1|22|20108.22|0.10|0.01|A|F|1992-11-30|1992-12-09|1992-12-04|NONE|RAIL|thely. furio|
+3232|135|1|2|34|35194.42|0.07|0.04|R|F|1993-01-09|1992-11-14|1993-02-03|NONE|SHIP|old packages integrate quickly |
+3232|181|2|3|3|3243.54|0.04|0.06|R|F|1992-12-14|1992-12-11|1992-12-29|DELIVER IN PERSON|FOB|ily blithely ironic acco|
+3233|51|2|1|23|21874.15|0.04|0.05|A|F|1994-12-07|1995-01-11|1994-12-26|NONE|AIR|pending instructions use after the carefu|
+3233|154|6|2|6|6324.90|0.02|0.08|A|F|1994-12-06|1994-12-05|1994-12-07|TAKE BACK RETURN|REG AIR|requests are quickly above the slyly p|
+3233|100|4|3|2|2000.20|0.04|0.06|R|F|1995-01-03|1995-01-02|1995-01-21|TAKE BACK RETURN|AIR| across the bold packages|
+3233|9|2|4|25|22725.00|0.04|0.07|A|F|1994-11-24|1995-01-07|1994-12-11|NONE|RAIL|oss the pl|
+3234|79|10|1|45|44058.15|0.01|0.04|N|O|1996-05-15|1996-05-09|1996-06-02|DELIVER IN PERSON|TRUCK| express packages are carefully. f|
+3234|84|5|2|23|22633.84|0.03|0.00|N|O|1996-05-29|1996-05-15|1996-06-17|DELIVER IN PERSON|AIR|d-- fluffily special packag|
+3234|75|4|3|16|15601.12|0.06|0.05|N|O|1996-06-10|1996-05-30|1996-06-18|COLLECT COD|RAIL|ithely ironic accounts wake along t|
+3234|122|1|4|50|51106.00|0.09|0.05|N|O|1996-06-11|1996-05-19|1996-06-18|NONE|MAIL|ly regular ideas according to the regula|
+3234|165|2|5|14|14912.24|0.01|0.07|N|O|1996-04-06|1996-05-30|1996-04-13|NONE|REG AIR|lithely regular f|
+3235|109|2|1|9|9081.90|0.07|0.00|N|O|1995-11-17|1995-12-24|1995-11-30|COLLECT COD|AIR|l courts sleep quickly slyly |
+3235|95|6|2|43|42788.87|0.10|0.07|N|O|1995-12-25|1996-01-23|1996-01-09|COLLECT COD|MAIL|ckly final instru|
+3235|138|9|3|29|30105.77|0.06|0.06|N|O|1996-01-28|1995-12-26|1996-02-12|DELIVER IN PERSON|RAIL|e fluffy pinto bea|
+3235|178|9|4|23|24797.91|0.00|0.01|N|O|1996-02-16|1996-01-05|1996-03-07|DELIVER IN PERSON|SHIP|ldly ironic pinto beans|
+3236|117|4|1|10|10171.10|0.06|0.05|N|O|1996-11-15|1996-12-14|1996-11-29|TAKE BACK RETURN|AIR|arefully. fluffily reg|
+3236|122|7|2|21|21464.52|0.01|0.07|N|O|1996-12-23|1996-12-12|1997-01-21|NONE|AIR| final pinto |
+3236|118|2|3|7|7126.77|0.07|0.01|N|O|1996-12-27|1996-12-18|1997-01-24|DELIVER IN PERSON|SHIP|dolites. slyly unus|
+3237|11|5|1|11|10021.11|0.02|0.07|A|F|1992-08-03|1992-07-31|1992-08-13|TAKE BACK RETURN|AIR|es. permanently express platelets besid|
+3238|72|3|1|12|11664.84|0.06|0.01|R|F|1993-03-06|1993-05-08|1993-04-01|DELIVER IN PERSON|AIR|ackages affix furiously. furiously bol|
+3238|173|2|2|26|27902.42|0.01|0.06|A|F|1993-02-25|1993-04-04|1993-03-20|TAKE BACK RETURN|REG AIR|g accounts sleep furiously ironic attai|
+3238|81|2|3|1|981.08|0.00|0.04|R|F|1993-05-17|1993-04-18|1993-05-27|NONE|SHIP|wake alongs|
+3239|45|8|1|50|47252.00|0.05|0.01|N|O|1998-02-09|1998-04-02|1998-02-22|NONE|FOB|d blithely stea|
+3239|45|8|2|43|40636.72|0.01|0.06|N|O|1998-01-15|1998-03-12|1998-01-29|COLLECT COD|REG AIR|y. bold pinto beans use |
+3239|13|7|3|13|11869.13|0.01|0.05|N|O|1998-02-10|1998-02-19|1998-02-25|DELIVER IN PERSON|MAIL|r deposits solve fluf|
+3239|195|6|4|26|28474.94|0.03|0.05|N|O|1998-01-21|1998-03-21|1998-02-08|DELIVER IN PERSON|SHIP|ngly pending platelets are fluff|
+3239|12|9|5|31|28272.31|0.10|0.08|N|O|1998-04-14|1998-03-24|1998-04-17|DELIVER IN PERSON|FOB|foxes. pendin|
+3264|200|1|1|39|42907.80|0.06|0.06|N|O|1996-11-07|1996-12-12|1996-11-20|TAKE BACK RETURN|REG AIR|sleep carefully after the slyly final|
+3264|131|2|2|34|35058.42|0.00|0.01|N|O|1997-01-03|1997-01-06|1997-01-29|TAKE BACK RETURN|REG AIR|rns haggle carefully. blit|
+3264|125|8|3|11|11276.32|0.09|0.03|N|O|1996-12-11|1996-12-19|1996-12-15|DELIVER IN PERSON|SHIP|regular packages|
+3264|109|10|4|24|24218.40|0.09|0.07|N|O|1997-01-07|1996-12-13|1997-01-11|TAKE BACK RETURN|RAIL|ctions. quick|
+3264|63|4|5|6|5778.36|0.04|0.03|N|O|1996-11-10|1996-12-05|1996-11-22|TAKE BACK RETURN|SHIP|press packages. ironical|
+3264|141|2|6|43|44769.02|0.06|0.06|N|O|1997-01-17|1997-01-24|1997-02-01|TAKE BACK RETURN|TRUCK|leep at the blithely bold|
+3265|25|4|1|8|7400.16|0.06|0.02|A|F|1992-09-01|1992-09-12|1992-09-27|DELIVER IN PERSON|TRUCK|thely ironic requests sleep slyly-- i|
+3265|72|2|2|7|6804.49|0.09|0.00|R|F|1992-09-16|1992-09-04|1992-10-14|DELIVER IN PERSON|MAIL|he forges. fluffily regular asym|
+3265|191|4|3|28|30553.32|0.09|0.08|A|F|1992-10-22|1992-08-23|1992-10-25|NONE|RAIL|n requests. quickly final dinos|
+3266|64|1|1|31|29885.86|0.09|0.02|N|O|1995-06-19|1995-05-04|1995-07-06|COLLECT COD|MAIL|grate among the quickly express deposits|
+3266|38|4|2|43|40335.29|0.06|0.07|R|F|1995-05-04|1995-05-30|1995-05-11|COLLECT COD|AIR|ular asymptotes use careful|
+3267|185|6|1|33|35810.94|0.06|0.01|N|O|1997-03-30|1997-03-25|1997-04-23|TAKE BACK RETURN|AIR|es boost. |
+3268|96|7|1|1|996.09|0.06|0.08|A|F|1994-09-12|1994-08-31|1994-09-16|NONE|TRUCK|. ironic, bold requests use carefull|
+3268|42|9|2|40|37681.60|0.08|0.01|R|F|1994-06-30|1994-08-22|1994-07-25|COLLECT COD|FOB|ly. bold, eve|
+3269|161|10|1|40|42446.40|0.02|0.07|N|O|1996-06-11|1996-05-06|1996-06-15|DELIVER IN PERSON|TRUCK|es. pending d|
+3269|38|4|2|46|43149.38|0.00|0.02|N|O|1996-04-21|1996-04-12|1996-05-10|DELIVER IN PERSON|MAIL|final asymptotes nag|
+3269|44|3|3|39|36817.56|0.02|0.03|N|O|1996-03-13|1996-05-26|1996-03-19|COLLECT COD|MAIL|he express packages?|
+3269|83|4|4|37|36373.96|0.07|0.05|N|O|1996-06-14|1996-04-27|1996-07-07|NONE|MAIL|egular requests. carefully un|
+3269|93|7|5|42|41709.78|0.09|0.05|N|O|1996-03-19|1996-04-24|1996-04-18|COLLECT COD|TRUCK| the special packages. |
+3269|131|7|6|16|16498.08|0.01|0.08|N|O|1996-03-03|1996-04-06|1996-03-06|NONE|RAIL|s cajole. silent deposits are f|
+3270|35|1|1|11|10285.33|0.07|0.06|N|O|1997-07-29|1997-08-11|1997-08-05|TAKE BACK RETURN|AIR| solve at the regular deposits. |
+3270|38|4|2|44|41273.32|0.10|0.05|N|O|1997-07-20|1997-08-15|1997-08-04|DELIVER IN PERSON|SHIP| accounts. carefully even |
+3270|65|4|3|20|19301.20|0.01|0.02|N|O|1997-08-26|1997-07-31|1997-08-30|DELIVER IN PERSON|FOB|en accounts among the c|
+3270|189|10|4|29|31586.22|0.06|0.05|N|O|1997-07-01|1997-07-23|1997-07-10|TAKE BACK RETURN|MAIL|sly regular asymptotes. slyly dog|
+3270|34|10|5|32|29888.96|0.03|0.00|N|O|1997-09-23|1997-08-17|1997-09-27|NONE|REG AIR|promise carefully.|
+3270|57|5|6|29|27754.45|0.01|0.04|N|O|1997-08-22|1997-08-17|1997-09-06|COLLECT COD|RAIL|ptotes nag above the quickly bold deposits|
+3270|117|1|7|9|9153.99|0.06|0.08|N|O|1997-08-14|1997-08-11|1997-09-09|DELIVER IN PERSON|SHIP|ual packages|
+3271|57|9|1|30|28711.50|0.01|0.04|A|F|1992-01-16|1992-03-20|1992-01-17|DELIVER IN PERSON|AIR|r the unusual Tiresia|
+3271|54|5|2|18|17172.90|0.09|0.06|R|F|1992-05-01|1992-03-28|1992-05-29|DELIVER IN PERSON|FOB| packages eat around the furiously regul|
+3271|95|6|3|14|13931.26|0.05|0.01|A|F|1992-02-24|1992-02-14|1992-03-23|NONE|AIR|ending, even packa|
+3271|64|1|4|29|27957.74|0.07|0.04|A|F|1992-03-10|1992-02-05|1992-03-14|COLLECT COD|MAIL|lar instructions. carefully regular|
+3296|84|5|1|12|11808.96|0.06|0.07|R|F|1994-12-08|1994-12-14|1994-12-24|COLLECT COD|AIR|y about the slyly bold pinto bea|
+3296|149|8|2|31|32523.34|0.08|0.00|R|F|1995-01-26|1994-12-25|1995-02-16|NONE|REG AIR|ainst the furi|
+3296|185|6|3|29|31470.22|0.02|0.04|A|F|1995-01-12|1994-11-26|1995-02-06|DELIVER IN PERSON|SHIP|ss ideas are reg|
+3296|140|1|4|47|48886.58|0.06|0.00|A|F|1994-11-08|1994-12-20|1994-11-30|NONE|FOB|egular deposits. quic|
+3296|177|6|5|16|17234.72|0.06|0.02|R|F|1995-01-11|1994-12-27|1995-01-12|DELIVER IN PERSON|SHIP|kages cajole carefully |
+3296|197|1|6|40|43887.60|0.00|0.04|A|F|1994-12-28|1994-12-08|1995-01-13|COLLECT COD|REG AIR|ronic ideas across the|
+3296|36|2|7|6|5616.18|0.02|0.01|R|F|1995-01-03|1994-12-23|1995-01-27|TAKE BACK RETURN|AIR|carefully fur|
+3297|134|10|1|10|10341.30|0.10|0.04|A|F|1992-12-14|1993-01-21|1992-12-26|NONE|SHIP|ironic idea|
+3298|149|6|1|9|9442.26|0.01|0.06|N|O|1996-08-15|1996-05-24|1996-09-12|COLLECT COD|REG AIR|ly final accou|
+3298|186|7|2|27|29326.86|0.06|0.06|N|O|1996-07-10|1996-05-21|1996-07-15|DELIVER IN PERSON|FOB|lar packages. regular deposit|
+3298|29|2|3|25|23225.50|0.10|0.08|N|O|1996-06-30|1996-05-31|1996-07-23|COLLECT COD|SHIP|ly express f|
+3298|191|5|4|1|1091.19|0.10|0.03|N|O|1996-07-31|1996-05-23|1996-08-24|TAKE BACK RETURN|FOB|refully regular requ|
+3299|183|4|1|40|43327.20|0.03|0.02|A|F|1994-03-21|1994-03-23|1994-04-12|COLLECT COD|AIR|lyly even request|
+3300|129|4|1|3|3087.36|0.07|0.02|N|O|1995-11-01|1995-10-02|1995-11-20|NONE|REG AIR|g according to the dugouts. caref|
+3300|149|10|2|23|24130.22|0.02|0.02|N|O|1995-08-17|1995-09-03|1995-09-04|COLLECT COD|TRUCK|he fluffily final a|
+3301|169|8|1|45|48112.20|0.04|0.05|A|F|1994-11-19|1994-10-27|1994-11-24|TAKE BACK RETURN|FOB|nusual, final excuses after the entici|
+3302|36|2|1|45|42121.35|0.09|0.00|N|O|1996-01-24|1995-12-16|1996-02-13|COLLECT COD|FOB|counts use quickl|
+3303|184|5|1|25|27104.50|0.06|0.01|N|O|1998-03-25|1998-01-31|1998-04-12|NONE|SHIP|lly regular pi|
+3303|21|2|2|15|13815.30|0.04|0.06|N|O|1998-01-29|1998-01-22|1998-02-21|COLLECT COD|SHIP| detect sly|
+3303|99|10|3|37|36966.33|0.05|0.02|N|O|1998-02-16|1998-03-07|1998-02-18|TAKE BACK RETURN|TRUCK| carefully ironic asympt|
+3303|36|2|4|26|24336.78|0.09|0.00|N|O|1998-01-18|1998-03-11|1998-02-11|DELIVER IN PERSON|REG AIR|ickly permanent requests w|
+3328|113|7|1|6|6078.66|0.03|0.08|A|F|1993-03-07|1993-01-25|1993-03-29|COLLECT COD|TRUCK|ffily even instructions detect b|
+3328|5|2|2|23|20815.00|0.01|0.06|R|F|1993-01-12|1993-02-07|1993-01-30|TAKE BACK RETURN|MAIL|y. careful|
+3328|139|10|3|44|45721.72|0.05|0.00|R|F|1992-12-03|1992-12-19|1992-12-09|TAKE BACK RETURN|FOB|dly quickly final foxes? re|
+3328|95|9|4|42|41793.78|0.01|0.05|R|F|1992-11-24|1992-12-20|1992-12-06|DELIVER IN PERSON|AIR|ronic requests|
+3328|131|7|5|25|25778.25|0.05|0.00|R|F|1993-01-28|1993-01-04|1993-01-31|NONE|RAIL|e unusual, r|
+3329|138|4|1|36|37372.68|0.09|0.08|N|O|1995-08-06|1995-08-03|1995-08-14|DELIVER IN PERSON|TRUCK|ts at the re|
+3329|6|3|2|9|8154.00|0.00|0.02|N|O|1995-07-24|1995-08-02|1995-08-01|COLLECT COD|MAIL|lly final depo|
+3329|123|4|3|1|1023.12|0.04|0.08|N|O|1995-08-22|1995-09-28|1995-09-09|COLLECT COD|REG AIR|regular packages are carefull|
+3330|20|7|1|49|45080.98|0.05|0.01|R|F|1995-03-02|1995-03-03|1995-03-16|DELIVER IN PERSON|TRUCK|haggle carefully alongside of the bold r|
+3331|64|9|1|9|8676.54|0.08|0.07|A|F|1993-07-18|1993-07-03|1993-08-16|TAKE BACK RETURN|AIR|odolites. bold accounts|
+3331|21|2|2|38|34998.76|0.06|0.04|R|F|1993-07-24|1993-06-22|1993-08-23|NONE|AIR|ymptotes haggle across the ca|
+3331|3|10|3|26|23478.00|0.09|0.05|A|F|1993-08-05|1993-07-17|1993-08-29|DELIVER IN PERSON|MAIL|p asymptotes. carefully unusual in|
+3332|84|5|1|28|27554.24|0.10|0.02|R|F|1994-12-30|1995-01-16|1995-01-16|COLLECT COD|FOB|s against the carefully special multipl|
+3332|136|2|2|21|21758.73|0.08|0.04|R|F|1995-02-04|1995-01-08|1995-02-06|COLLECT COD|MAIL| quick packages sle|
+3332|134|5|3|27|27921.51|0.03|0.02|A|F|1994-12-10|1995-01-14|1994-12-11|TAKE BACK RETURN|FOB|ording to the slyly regula|
+3333|150|9|1|27|28354.05|0.06|0.08|A|F|1992-12-06|1992-10-26|1992-12-07|COLLECT COD|SHIP|s dazzle fluffil|
+3333|199|3|2|36|39570.84|0.08|0.07|R|F|1992-11-20|1992-11-06|1992-12-16|TAKE BACK RETURN|FOB|foxes sleep neve|
+3333|108|1|3|38|38307.80|0.05|0.05|A|F|1992-10-30|1992-11-03|1992-11-04|NONE|MAIL|ccounts promise bl|
+3333|113|4|4|49|49642.39|0.07|0.07|R|F|1992-10-02|1992-11-30|1992-10-12|DELIVER IN PERSON|MAIL|riously ironic r|
+3333|43|2|5|45|42436.80|0.07|0.08|A|F|1992-10-04|1992-11-08|1992-10-27|COLLECT COD|SHIP|dolites. quickly r|
+3334|187|8|1|20|21743.60|0.04|0.03|N|O|1996-05-21|1996-04-08|1996-05-26|TAKE BACK RETURN|AIR|uses nag furiously. instructions are ca|
+3334|190|1|2|7|7631.33|0.09|0.07|N|O|1996-04-28|1996-04-08|1996-05-25|NONE|SHIP|nts sublate slyly express pack|
+3335|105|10|1|13|13066.30|0.06|0.07|N|O|1996-01-20|1995-12-20|1996-02-09|COLLECT COD|REG AIR|out the special asymptotes|
+3335|31|2|2|44|40965.32|0.07|0.02|N|O|1996-01-05|1995-12-25|1996-01-18|DELIVER IN PERSON|SHIP|r packages cajole ac|
+3335|140|6|3|16|16642.24|0.01|0.06|N|O|1995-10-18|1995-12-08|1995-11-03|DELIVER IN PERSON|SHIP|g packages. carefully regular reque|
+3335|90|1|4|47|46534.23|0.10|0.03|N|O|1995-12-02|1995-11-19|1995-12-27|NONE|MAIL| quickly special ideas.|
+3360|174|4|1|31|33299.27|0.08|0.04|N|O|1998-04-24|1998-04-12|1998-05-23|COLLECT COD|REG AIR|quests. carefully even deposits wake acros|
+3360|91|3|2|29|28741.61|0.00|0.06|N|O|1998-04-15|1998-02-25|1998-05-13|TAKE BACK RETURN|FOB|press asymptotes. furiously final |
+3360|82|3|3|39|38301.12|0.08|0.03|N|O|1998-04-09|1998-04-20|1998-05-05|DELIVER IN PERSON|REG AIR|s. blithely express pinto bean|
+3360|117|7|4|29|29496.19|0.10|0.01|N|O|1998-05-19|1998-03-03|1998-06-09|TAKE BACK RETURN|FOB|hely gifts. spe|
+3360|58|6|5|4|3832.20|0.08|0.07|N|O|1998-02-27|1998-03-23|1998-03-28|COLLECT COD|SHIP|ly busy inst|
+3360|71|1|6|42|40784.94|0.04|0.01|N|O|1998-05-07|1998-04-18|1998-06-04|DELIVER IN PERSON|FOB|ages cajole. pending, |
+3361|144|5|1|6|6264.84|0.02|0.02|R|F|1992-10-02|1992-10-25|1992-10-05|DELIVER IN PERSON|FOB| packages sleep. furiously unus|
+3361|171|10|2|33|35348.61|0.01|0.02|R|F|1992-11-09|1992-10-15|1992-11-11|TAKE BACK RETURN|MAIL|uriously ironic accounts. ironic, ir|
+3361|191|5|3|31|33826.89|0.06|0.04|R|F|1992-08-29|1992-10-13|1992-09-08|NONE|FOB|ts. pending, regular accounts sleep fur|
+3362|22|5|1|14|12908.28|0.06|0.05|N|O|1995-08-01|1995-09-06|1995-08-22|NONE|FOB|even Tires|
+3362|195|6|2|41|44902.79|0.05|0.03|N|O|1995-10-31|1995-09-04|1995-11-17|COLLECT COD|REG AIR|ake alongside of the |
+3362|115|9|3|40|40604.40|0.05|0.06|N|O|1995-08-19|1995-10-17|1995-09-05|TAKE BACK RETURN|FOB|packages haggle furi|
+3362|2|7|4|3|2706.00|0.03|0.01|N|O|1995-08-26|1995-09-02|1995-09-17|NONE|SHIP|its cajole blithely excuses. de|
+3362|138|9|5|36|37372.68|0.06|0.00|N|O|1995-10-05|1995-08-28|1995-11-03|TAKE BACK RETURN|RAIL|es against the quickly permanent pint|
+3362|188|9|6|46|50056.28|0.09|0.05|N|O|1995-08-02|1995-10-12|1995-08-28|COLLECT COD|REG AIR|ly bold packages. regular deposits cajol|
+3363|10|3|1|42|38220.42|0.00|0.08|N|O|1995-11-09|1995-11-25|1995-11-15|TAKE BACK RETURN|RAIL| blithely final ideas nag after|
+3363|191|4|2|21|22914.99|0.08|0.08|N|O|1995-12-10|1995-10-28|1995-12-28|COLLECT COD|RAIL|he regular, brave deposits. f|
+3363|159|7|3|2|2118.30|0.01|0.07|N|O|1996-01-22|1995-12-01|1996-02-18|TAKE BACK RETURN|SHIP|uickly bold ide|
+3363|113|3|4|20|20262.20|0.07|0.06|N|O|1995-12-11|1995-11-15|1995-12-21|COLLECT COD|MAIL|carefully quiet excuses wake. sl|
+3363|200|4|5|4|4400.80|0.00|0.08|N|O|1995-10-30|1995-11-17|1995-11-22|COLLECT COD|FOB| ironic dependencie|
+3364|90|1|1|49|48514.41|0.03|0.05|N|O|1997-09-17|1997-08-23|1997-10-06|NONE|SHIP|d accounts? caref|
+3364|111|2|2|38|38422.18|0.02|0.02|N|O|1997-08-30|1997-09-12|1997-09-27|COLLECT COD|REG AIR| slyly express|
+3364|156|4|3|10|10561.50|0.00|0.01|N|O|1997-08-10|1997-08-24|1997-08-15|TAKE BACK RETURN|SHIP|g the accounts. final, busy accounts wi|
+3364|160|5|4|7|7421.12|0.10|0.05|N|O|1997-07-09|1997-08-01|1997-07-16|NONE|TRUCK|furiously regular ideas haggle furiously b|
+3364|81|2|5|3|2943.24|0.01|0.00|N|O|1997-10-19|1997-08-15|1997-10-28|TAKE BACK RETURN|TRUCK|c theodolites. blithely ir|
+3365|151|6|1|37|38892.55|0.02|0.08|R|F|1994-12-22|1995-02-07|1995-01-20|TAKE BACK RETURN|SHIP|requests. quickly pending instructions a|
+3365|167|2|2|37|39484.92|0.07|0.08|A|F|1994-11-24|1995-01-09|1994-11-27|NONE|REG AIR|oze blithely. furiously ironic theodolit|
+3365|115|6|3|13|13196.43|0.09|0.02|R|F|1995-02-25|1995-01-31|1995-03-16|NONE|RAIL|pths wake r|
+3365|176|4|4|49|52732.33|0.02|0.07|R|F|1995-01-03|1995-01-01|1995-01-18|COLLECT COD|MAIL|lyly unusual asymptotes. final|
+3365|16|3|5|2|1832.02|0.00|0.03|R|F|1995-02-04|1994-12-30|1995-03-06|TAKE BACK RETURN|FOB|es cajole fluffily pe|
+3365|126|5|6|24|24626.88|0.01|0.00|R|F|1995-02-27|1995-01-09|1995-03-27|DELIVER IN PERSON|REG AIR|into beans? carefully regula|
+3366|40|1|1|4|3760.16|0.07|0.01|N|O|1997-05-20|1997-06-25|1997-06-03|DELIVER IN PERSON|AIR| carefully about |
+3366|136|2|2|9|9325.17|0.00|0.08|N|O|1997-06-02|1997-07-05|1997-06-26|COLLECT COD|REG AIR|ackages sleep carefully across the bli|
+3367|41|10|1|27|25408.08|0.01|0.03|A|F|1993-04-13|1993-03-16|1993-04-26|NONE|RAIL|kly even instructions caj|
+3367|141|10|2|34|35398.76|0.04|0.08|A|F|1993-03-30|1993-02-23|1993-04-11|COLLECT COD|MAIL| accounts wake slyly |
+3367|120|7|3|38|38764.56|0.03|0.03|R|F|1993-03-13|1993-02-12|1993-03-31|NONE|RAIL|even packages sleep blithely slyly expr|
+3392|171|10|1|40|42846.80|0.01|0.01|N|O|1996-02-18|1995-12-16|1996-02-26|COLLECT COD|MAIL|ress instructions affix carefully. fur|
+3392|123|2|2|13|13300.56|0.09|0.02|N|O|1995-11-26|1996-01-17|1995-12-01|NONE|MAIL|across the fluffily bold deposits.|
+3392|127|10|3|34|34922.08|0.10|0.08|N|O|1996-01-20|1996-01-21|1996-01-24|DELIVER IN PERSON|MAIL|e carefully even braids. |
+3392|124|3|4|7|7168.84|0.08|0.05|N|O|1995-12-07|1996-01-09|1995-12-29|TAKE BACK RETURN|RAIL|as. express, final accounts dou|
+3393|117|7|1|16|16273.76|0.01|0.00|N|O|1995-07-17|1995-08-19|1995-08-04|COLLECT COD|TRUCK|uses. instructions after the blithely |
+3393|125|4|2|44|45105.28|0.08|0.04|N|O|1995-10-16|1995-08-05|1995-11-01|NONE|AIR|ld requests hag|
+3393|97|1|3|25|24927.25|0.07|0.02|N|O|1995-10-17|1995-08-12|1995-11-11|DELIVER IN PERSON|MAIL|ng excuses|
+3393|72|2|4|48|46659.36|0.06|0.06|N|O|1995-07-12|1995-09-15|1995-08-02|NONE|FOB| blithely final reques|
+3393|178|7|5|37|39892.29|0.07|0.02|N|O|1995-10-16|1995-08-19|1995-10-19|COLLECT COD|AIR|ss the slyly ironic pinto beans. ironic,|
+3393|62|7|6|17|16355.02|0.04|0.01|N|O|1995-08-15|1995-09-07|1995-09-10|COLLECT COD|MAIL|kly ironic deposits could|
+3394|155|6|1|33|34819.95|0.07|0.08|N|O|1996-08-07|1996-07-17|1996-09-02|TAKE BACK RETURN|SHIP|ideas alongside of th|
+3394|146|3|2|43|44984.02|0.08|0.03|N|O|1996-08-23|1996-07-20|1996-08-25|COLLECT COD|RAIL|hockey players. slyly regular requests afte|
+3394|88|9|3|26|25690.08|0.01|0.00|N|O|1996-08-08|1996-06-12|1996-09-05|TAKE BACK RETURN|RAIL|its use furiously. even, even account|
+3394|81|2|4|14|13735.12|0.08|0.00|N|O|1996-06-02|1996-07-02|1996-06-19|COLLECT COD|MAIL|e furiously final theodolites. furio|
+3394|127|8|5|30|30813.60|0.04|0.06|N|O|1996-05-12|1996-07-24|1996-05-19|COLLECT COD|REG AIR|t ideas according to the fluffily iro|
+3394|184|5|6|14|15178.52|0.05|0.05|N|O|1996-06-18|1996-06-24|1996-07-17|NONE|REG AIR|arefully regular do|
+3395|142|3|1|21|21884.94|0.03|0.06|R|F|1994-12-19|1995-01-13|1994-12-25|TAKE BACK RETURN|SHIP| careful dep|
+3395|36|2|2|38|35569.14|0.01|0.07|R|F|1995-01-13|1995-01-13|1995-01-25|COLLECT COD|SHIP| silent accounts are blithely|
+3395|43|4|3|43|40550.72|0.06|0.07|A|F|1994-12-13|1995-01-07|1994-12-14|COLLECT COD|AIR|ckages above the furiously regu|
+3395|122|1|4|39|39862.68|0.05|0.07|R|F|1994-12-03|1995-01-17|1994-12-10|NONE|AIR|riously unusual theodolites. fur|
+3396|128|7|1|34|34956.08|0.00|0.06|A|F|1994-05-30|1994-08-16|1994-06-11|NONE|AIR|. slyly unusual packages wak|
+3396|49|6|2|43|40808.72|0.03|0.08|A|F|1994-07-03|1994-08-09|1994-07-14|TAKE BACK RETURN|MAIL|cial packages cajole blithely around the |
+3396|138|4|3|9|9343.17|0.01|0.06|R|F|1994-07-01|1994-08-18|1994-07-21|DELIVER IN PERSON|AIR|usly special foxes. accounts wake careful|
+3396|75|3|4|32|31202.24|0.06|0.02|R|F|1994-08-07|1994-08-10|1994-09-05|COLLECT COD|TRUCK|osits are slyly. final, bold foxes s|
+3396|126|5|5|27|27705.24|0.02|0.01|A|F|1994-09-14|1994-07-26|1994-09-28|DELIVER IN PERSON|FOB| theodolites |
+3396|39|10|6|18|16902.54|0.10|0.00|A|F|1994-07-27|1994-06-26|1994-08-25|TAKE BACK RETURN|REG AIR|l requests haggle furiously along the fur|
+3396|198|2|7|31|34043.89|0.05|0.06|A|F|1994-06-07|1994-06-23|1994-06-19|TAKE BACK RETURN|REG AIR|l, express pinto beans. quic|
+3397|195|8|1|8|8761.52|0.07|0.01|A|F|1994-08-05|1994-08-11|1994-08-08|DELIVER IN PERSON|RAIL|y final foxes|
+3397|13|3|2|11|10043.11|0.00|0.07|A|F|1994-07-29|1994-09-18|1994-08-12|DELIVER IN PERSON|REG AIR|iously careful packages. s|
+3397|184|5|3|1|1084.18|0.07|0.05|R|F|1994-08-03|1994-07-30|1994-08-28|NONE|RAIL| regular packag|
+3397|86|7|4|33|32540.64|0.05|0.01|R|F|1994-09-04|1994-08-06|1994-09-22|COLLECT COD|RAIL|gular accounts. blithely re|
+3397|132|3|5|28|28899.64|0.05|0.05|R|F|1994-07-13|1994-08-26|1994-07-17|NONE|TRUCK|counts around the final reques|
+3398|173|4|1|1|1073.17|0.01|0.08|N|O|1996-11-22|1996-11-16|1996-12-09|COLLECT COD|MAIL| blithely final deposits.|
+3399|134|5|1|28|28955.64|0.09|0.05|N|O|1995-06-29|1995-05-19|1995-07-12|COLLECT COD|AIR|oggedly final theodolites grow. fi|
+3399|55|6|2|8|7640.40|0.01|0.05|A|F|1995-05-15|1995-04-19|1995-06-05|COLLECT COD|TRUCK|s use carefully carefully ir|
+3399|67|4|3|3|2901.18|0.03|0.00|N|F|1995-06-16|1995-04-04|1995-06-23|NONE|SHIP|hely pending dugouts |
+3399|14|5|4|21|19194.21|0.09|0.06|A|F|1995-03-12|1995-05-18|1995-03-28|TAKE BACK RETURN|MAIL|se final courts. exc|
+3424|181|2|1|39|42166.02|0.06|0.07|N|O|1996-11-03|1996-11-08|1996-11-23|DELIVER IN PERSON|MAIL|bits boost closely slyly p|
+3425|120|1|1|11|11221.32|0.03|0.08|N|O|1996-04-24|1996-05-29|1996-05-23|DELIVER IN PERSON|FOB|ckly final deposits use quickly?|
+3425|79|7|2|37|36225.59|0.06|0.03|N|O|1996-06-04|1996-05-09|1996-06-12|NONE|SHIP|as sleep carefully into the caref|
+3425|14|4|3|8|7312.08|0.06|0.08|N|O|1996-07-22|1996-06-07|1996-07-26|TAKE BACK RETURN|AIR|iously regular theodolites wake. s|
+3425|19|10|4|37|34003.37|0.04|0.01|N|O|1996-07-10|1996-05-10|1996-08-02|NONE|SHIP|ngside of the furiously thin dol|
+3425|79|9|5|48|46995.36|0.08|0.04|N|O|1996-04-14|1996-05-25|1996-04-23|TAKE BACK RETURN|AIR|uctions wake fluffily. care|
+3425|148|9|6|24|25155.36|0.05|0.04|N|O|1996-04-22|1996-06-24|1996-04-25|TAKE BACK RETURN|AIR|ajole blithely sl|
+3426|110|5|1|20|20202.20|0.05|0.04|N|O|1996-11-10|1996-12-24|1996-12-01|COLLECT COD|FOB|sits cajole blit|
+3426|14|4|2|19|17366.19|0.10|0.08|N|O|1996-11-02|1997-01-13|1996-11-15|DELIVER IN PERSON|RAIL|slyly special packages oug|
+3426|67|6|3|19|18374.14|0.08|0.05|N|O|1996-12-07|1996-12-15|1996-12-14|DELIVER IN PERSON|FOB|c accounts cajole carefu|
+3426|6|7|4|9|8154.00|0.09|0.05|N|O|1996-12-24|1997-01-14|1997-01-13|NONE|FOB|pecial theodolites haggle fluf|
+3426|49|6|5|31|29420.24|0.07|0.08|N|O|1996-11-11|1996-12-10|1996-12-10|DELIVER IN PERSON|SHIP| even sentiment|
+3427|54|5|1|41|39116.05|0.10|0.01|N|O|1997-09-11|1997-07-03|1997-10-04|COLLECT COD|RAIL|s the carefully|
+3427|189|10|2|24|26140.32|0.02|0.04|N|O|1997-07-01|1997-07-28|1997-07-30|NONE|SHIP|y bold, sly deposits. pendi|
+3427|139|5|3|40|41565.20|0.06|0.05|N|O|1997-06-12|1997-08-19|1997-06-23|COLLECT COD|MAIL|patterns cajole ca|
+3427|119|6|4|31|31592.41|0.08|0.04|N|O|1997-08-12|1997-07-26|1997-08-25|COLLECT COD|RAIL|s are carefull|
+3428|198|9|1|4|4392.76|0.00|0.03|N|O|1996-05-09|1996-06-13|1996-06-02|NONE|REG AIR|sly pending requests int|
+3428|118|9|2|35|35633.85|0.02|0.03|N|O|1996-05-01|1996-06-07|1996-05-20|COLLECT COD|TRUCK|ly regular pinto beans sleep|
+3428|136|7|3|47|48698.11|0.07|0.05|N|O|1996-04-16|1996-06-08|1996-05-05|NONE|REG AIR|y final pinto |
+3429|137|8|1|48|49782.24|0.06|0.02|N|O|1997-04-08|1997-03-09|1997-04-25|TAKE BACK RETURN|SHIP| haggle furiously ir|
+3429|59|7|2|15|14385.75|0.03|0.04|N|O|1997-02-04|1997-03-09|1997-03-01|TAKE BACK RETURN|TRUCK|beans are fu|
+3429|69|4|3|10|9690.60|0.05|0.07|N|O|1997-01-19|1997-02-22|1997-01-25|TAKE BACK RETURN|REG AIR|ackages. quickly e|
+3429|89|10|4|28|27694.24|0.10|0.07|N|O|1997-01-30|1997-03-18|1997-02-17|TAKE BACK RETURN|AIR|nstructions boost. thin|
+3429|165|6|5|45|47932.20|0.10|0.00|N|O|1997-04-21|1997-03-08|1997-05-05|COLLECT COD|REG AIR|ites poach a|
+3430|189|10|1|2|2178.36|0.07|0.06|R|F|1995-03-07|1995-01-28|1995-03-30|TAKE BACK RETURN|MAIL|sh furiously according to the evenly e|
+3430|81|2|2|32|31394.56|0.08|0.00|R|F|1995-01-17|1995-01-28|1995-02-06|NONE|TRUCK|egular instruction|
+3430|97|8|3|41|40880.69|0.06|0.04|R|F|1995-02-18|1995-02-21|1995-03-11|TAKE BACK RETURN|AIR|cuses. silent excuses h|
+3430|65|2|4|50|48253.00|0.01|0.00|R|F|1994-12-15|1995-03-03|1994-12-24|COLLECT COD|REG AIR|ironic theodolites. carefully regular pac|
+3430|95|9|5|5|4975.45|0.05|0.05|A|F|1995-04-02|1995-02-12|1995-04-08|DELIVER IN PERSON|FOB|even accounts haggle slyly bol|
+3430|171|10|6|15|16067.55|0.08|0.07|A|F|1995-02-01|1995-03-12|1995-02-04|COLLECT COD|SHIP|cajole around the accounts. qui|
+3430|52|7|7|23|21897.15|0.09|0.08|A|F|1995-03-06|1995-03-01|1995-03-10|COLLECT COD|MAIL|eas according to the|
+3431|180|8|1|41|44287.38|0.03|0.06|A|F|1993-09-26|1993-10-13|1993-10-22|NONE|AIR| sleep carefully ironically special|
+3456|111|8|1|34|34377.74|0.10|0.06|A|F|1993-08-29|1993-08-26|1993-09-07|TAKE BACK RETURN|SHIP|usy pinto beans b|
+3457|182|3|1|29|31383.22|0.03|0.02|R|F|1995-05-12|1995-07-13|1995-06-05|NONE|TRUCK|refully final excuses wake|
+3457|106|7|2|22|22134.20|0.06|0.01|N|O|1995-06-23|1995-06-16|1995-06-29|NONE|SHIP|packages nag furiously against|
+3457|109|2|3|7|7063.70|0.07|0.08|N|O|1995-08-14|1995-07-06|1995-08-18|COLLECT COD|SHIP| pending accounts along the|
+3457|1|2|4|24|21624.00|0.07|0.07|N|O|1995-08-03|1995-05-30|1995-08-14|TAKE BACK RETURN|REG AIR|tructions haggle alongsid|
+3457|109|4|5|42|42382.20|0.05|0.01|A|F|1995-06-12|1995-06-14|1995-06-14|COLLECT COD|MAIL|riously final instruc|
+3457|144|1|6|45|46986.30|0.08|0.01|N|O|1995-08-12|1995-07-18|1995-08-23|TAKE BACK RETURN|SHIP| packages. care|
+3457|167|4|7|9|9604.44|0.04|0.00|R|F|1995-05-29|1995-06-30|1995-06-12|DELIVER IN PERSON|FOB|quests. foxes sleep quickly|
+3458|133|4|1|48|49590.24|0.06|0.04|R|F|1995-03-17|1995-01-25|1995-03-28|TAKE BACK RETURN|AIR|iously pending dep|
+3458|50|3|2|46|43702.30|0.06|0.06|R|F|1995-03-08|1995-01-21|1995-03-10|TAKE BACK RETURN|SHIP|nod across the boldly even instruct|
+3458|143|4|3|36|37553.04|0.01|0.06|R|F|1995-04-20|1995-02-14|1995-05-09|TAKE BACK RETURN|REG AIR|s lose. blithely ironic requests boost|
+3458|16|10|4|16|14656.16|0.09|0.03|R|F|1995-03-01|1995-02-25|1995-03-06|TAKE BACK RETURN|AIR|s grow carefully. express, final grouc|
+3458|157|5|5|2|2114.30|0.09|0.03|A|F|1995-02-05|1995-02-01|1995-03-07|COLLECT COD|FOB|ironic packages haggle past the furiously |
+3458|142|1|6|6|6252.84|0.09|0.04|A|F|1995-03-10|1995-02-02|1995-03-23|TAKE BACK RETURN|AIR|dolites; regular theodolites cajole |
+3459|179|7|1|31|33454.27|0.06|0.01|A|F|1994-09-05|1994-10-20|1994-10-03|NONE|REG AIR|y regular pain|
+3459|130|9|2|30|30903.90|0.04|0.08|R|F|1994-11-22|1994-09-12|1994-12-11|NONE|REG AIR|nic theodolites; evenly i|
+3459|41|8|3|45|42346.80|0.04|0.05|A|F|1994-07-31|1994-09-09|1994-08-02|TAKE BACK RETURN|REG AIR|ntly speci|
+3459|69|10|4|10|9690.60|0.05|0.06|A|F|1994-10-06|1994-09-16|1994-11-03|TAKE BACK RETURN|REG AIR| furiously silent dolphi|
+3459|189|10|5|10|10891.80|0.02|0.02|R|F|1994-08-01|1994-10-17|1994-08-11|TAKE BACK RETURN|FOB|. blithely ironic pinto beans above|
+3460|11|1|1|40|36440.40|0.10|0.06|N|O|1995-12-28|1995-12-14|1996-01-02|NONE|REG AIR|odolites are slyly bold deposits|
+3460|74|4|2|3|2922.21|0.06|0.00|N|O|1996-01-19|1995-12-28|1996-01-31|COLLECT COD|AIR|er quickly |
+3460|35|1|3|40|37401.20|0.08|0.07|N|O|1995-10-29|1995-11-10|1995-11-24|TAKE BACK RETURN|REG AIR|o the even deposits|
+3460|95|8|4|50|49754.50|0.02|0.07|N|O|1996-01-30|1995-12-10|1996-02-06|DELIVER IN PERSON|SHIP|e slyly about the sly|
+3460|130|1|5|47|48416.11|0.08|0.05|N|O|1995-12-09|1995-11-12|1995-12-22|TAKE BACK RETURN|SHIP|es haggle slyly regular accounts. fi|
+3460|63|10|6|46|44300.76|0.03|0.07|N|O|1996-01-27|1996-01-01|1996-02-01|NONE|TRUCK|uses run among the carefully even deposits|
+3460|45|2|7|28|26461.12|0.00|0.01|N|O|1995-10-28|1995-11-13|1995-11-17|COLLECT COD|SHIP|inal, ironic instructions. carefully|
+3461|100|4|1|49|49004.90|0.06|0.06|A|F|1993-03-09|1993-04-16|1993-03-13|DELIVER IN PERSON|RAIL|ual request|
+3461|63|4|2|27|26002.62|0.06|0.06|A|F|1993-02-10|1993-03-02|1993-03-04|COLLECT COD|SHIP|ely unusual deposits. quickly ir|
+3461|39|5|3|44|41317.32|0.09|0.06|A|F|1993-05-20|1993-04-03|1993-05-27|COLLECT COD|RAIL| haggle quickly even ideas. fin|
+3461|95|7|4|41|40798.69|0.09|0.02|R|F|1993-02-19|1993-04-20|1993-02-21|NONE|TRUCK|heodolites. blithely ironi|
+3461|90|1|5|16|15841.44|0.08|0.06|A|F|1993-05-09|1993-04-29|1993-05-26|TAKE BACK RETURN|TRUCK| pending deposi|
+3461|167|2|6|24|25611.84|0.10|0.00|A|F|1993-06-01|1993-03-12|1993-06-20|TAKE BACK RETURN|MAIL|thely. carefully re|
+3462|151|3|1|4|4204.60|0.09|0.04|N|O|1997-06-12|1997-07-31|1997-06-16|COLLECT COD|RAIL|ackages. fu|
+3462|40|1|2|43|40421.72|0.08|0.03|N|O|1997-08-01|1997-07-18|1997-08-29|NONE|RAIL| carefully. final, final ideas sleep slyly|
+3462|129|4|3|6|6174.72|0.05|0.04|N|O|1997-06-02|1997-08-09|1997-06-30|NONE|RAIL|iously regular fo|
+3462|99|3|4|2|1998.18|0.09|0.07|N|O|1997-09-10|1997-08-08|1997-09-19|NONE|AIR|nic packages. even accounts alongside |
+3462|38|4|5|14|13132.42|0.01|0.02|N|O|1997-05-31|1997-07-05|1997-06-24|COLLECT COD|MAIL|yly. blithely bold theodolites wa|
+3463|61|10|1|45|43247.70|0.02|0.02|A|F|1993-10-30|1993-11-04|1993-11-08|DELIVER IN PERSON|FOB|nts are slyly |
+3463|98|1|2|43|42917.87|0.04|0.02|A|F|1993-10-28|1993-09-24|1993-11-03|DELIVER IN PERSON|FOB| across the |
+3488|160|5|1|1|1060.16|0.04|0.01|A|F|1995-03-06|1995-02-16|1995-03-23|DELIVER IN PERSON|FOB| final excuses. carefully even waters hagg|
+3488|104|9|2|48|48196.80|0.00|0.03|A|F|1995-03-29|1995-03-26|1995-04-28|COLLECT COD|SHIP|sly? final requests |
+3488|160|1|3|11|11661.76|0.03|0.08|R|F|1995-03-25|1995-02-08|1995-04-16|COLLECT COD|TRUCK|unusual re|
+3488|42|9|4|12|11304.48|0.05|0.07|R|F|1995-04-27|1995-02-16|1995-05-09|DELIVER IN PERSON|RAIL|e slyly; furiously final packages wak|
+3488|156|1|5|18|19010.70|0.09|0.06|A|F|1995-03-18|1995-03-19|1995-03-29|DELIVER IN PERSON|FOB|s the carefully r|
+3489|186|7|1|19|20637.42|0.09|0.05|A|F|1993-07-31|1993-10-26|1993-08-15|NONE|SHIP|c deposits alongside of the pending, fu|
+3489|29|4|2|46|42734.92|0.00|0.00|A|F|1993-08-02|1993-10-09|1993-08-10|TAKE BACK RETURN|TRUCK|xcuses? quickly stealthy dependenci|
+3490|92|6|1|43|42659.87|0.05|0.05|N|O|1997-08-04|1997-08-06|1997-08-14|TAKE BACK RETURN|SHIP|. even requests cajol|
+3490|86|7|2|50|49304.00|0.05|0.07|N|O|1997-06-27|1997-08-15|1997-06-28|NONE|RAIL| haggle carefu|
+3490|93|7|3|8|7944.72|0.10|0.04|N|O|1997-08-11|1997-07-25|1997-08-28|COLLECT COD|MAIL|inal deposits use furiousl|
+3491|154|2|1|28|29516.20|0.04|0.03|N|O|1998-09-29|1998-09-08|1998-10-23|COLLECT COD|FOB|ccounts. sly|
+3491|122|3|2|22|22486.64|0.08|0.02|N|O|1998-08-19|1998-08-22|1998-09-03|TAKE BACK RETURN|REG AIR| grow against the boldly pending pinto bea|
+3492|156|7|1|3|3168.45|0.02|0.08|R|F|1994-11-26|1994-12-28|1994-12-19|COLLECT COD|REG AIR|the deposits. carefully |
+3492|126|9|2|7|7182.84|0.04|0.00|R|F|1995-03-10|1995-01-03|1995-03-16|COLLECT COD|FOB|thely regular dolphi|
+3492|109|10|3|34|34309.40|0.05|0.06|A|F|1994-12-07|1994-12-29|1994-12-24|COLLECT COD|AIR| unusual requests. ir|
+3492|147|6|4|30|31414.20|0.02|0.06|A|F|1995-01-29|1995-01-02|1995-02-13|DELIVER IN PERSON|MAIL| detect furiously permanent, unusual accou|
+3492|122|1|5|47|48039.64|0.09|0.07|R|F|1995-03-24|1994-12-28|1995-03-29|NONE|REG AIR|deposits. quickly express |
+3492|22|7|6|47|43334.94|0.04|0.07|R|F|1994-12-12|1995-01-18|1994-12-26|COLLECT COD|RAIL|ronic instructions u|
+3493|93|6|1|31|30785.79|0.06|0.07|R|F|1993-10-22|1993-10-12|1993-11-07|DELIVER IN PERSON|REG AIR|ructions. slyly regular accounts across the|
+3493|132|3|2|10|10321.30|0.02|0.06|R|F|1993-08-27|1993-10-07|1993-09-23|COLLECT COD|TRUCK|hall have to integ|
+3494|117|1|1|40|40684.40|0.05|0.04|R|F|1993-07-10|1993-06-01|1993-07-25|TAKE BACK RETURN|TRUCK|lites haggle furiously about the fin|
+3494|75|6|2|23|22426.61|0.10|0.01|A|F|1993-06-19|1993-06-04|1993-07-14|NONE|FOB|osits nag |
+3494|198|2|3|40|43927.60|0.02|0.08|A|F|1993-05-30|1993-07-02|1993-06-20|TAKE BACK RETURN|MAIL|uests cajole blithely|
+3494|77|8|4|30|29312.10|0.04|0.03|R|F|1993-07-01|1993-06-08|1993-07-15|TAKE BACK RETURN|TRUCK|ns are quickly regular, |
+3495|28|3|1|20|18560.40|0.10|0.03|N|O|1996-04-24|1996-05-18|1996-05-01|TAKE BACK RETURN|RAIL|posits are carefully; forges cajole qui|
+3495|173|1|2|24|25756.08|0.05|0.02|N|O|1996-03-22|1996-04-10|1996-04-07|DELIVER IN PERSON|RAIL|ic, final pains along the even request|
+3495|199|10|3|16|17587.04|0.08|0.02|N|O|1996-03-30|1996-04-02|1996-04-12|TAKE BACK RETURN|AIR|y bold dependencies; blithely idle sautern|
+3520|28|1|1|30|27840.60|0.04|0.02|N|O|1997-11-11|1997-10-02|1997-12-06|COLLECT COD|SHIP|deas should solve blithely among the ironi|
+3520|167|4|2|38|40552.08|0.00|0.04|N|O|1997-08-14|1997-10-26|1997-09-09|NONE|RAIL|yly final packages according to the quickl|
+3520|106|9|3|5|5030.50|0.01|0.02|N|O|1997-11-13|1997-09-22|1997-12-09|NONE|MAIL|ly even ideas haggle |
+3520|64|5|4|41|39526.46|0.01|0.01|N|O|1997-08-06|1997-09-20|1997-08-20|TAKE BACK RETURN|AIR| carefully pendi|
+3520|163|10|5|35|37210.60|0.02|0.02|N|O|1997-09-16|1997-09-03|1997-09-24|DELIVER IN PERSON|FOB|s nag carefully. sometimes unusual account|
+3521|59|4|1|48|46034.40|0.09|0.03|A|F|1993-01-03|1992-12-31|1993-01-22|NONE|AIR|ses use. furiously express ideas wake f|
+3521|131|2|2|2|2062.26|0.05|0.06|R|F|1993-01-29|1992-12-20|1993-02-23|NONE|MAIL|refully duri|
+3521|178|8|3|38|40970.46|0.00|0.08|A|F|1993-02-15|1992-12-10|1993-03-10|COLLECT COD|FOB|ges hang q|
+3521|144|7|4|26|27147.64|0.02|0.08|R|F|1993-01-04|1993-01-20|1993-01-17|DELIVER IN PERSON|AIR|onic dependencies haggle. fur|
+3521|36|7|5|28|26208.84|0.10|0.01|A|F|1993-01-06|1993-01-22|1993-02-02|TAKE BACK RETURN|FOB|e slyly above the slyly final|
+3522|4|9|1|6|5424.00|0.08|0.03|A|F|1995-01-21|1994-12-09|1995-01-23|NONE|SHIP|tes snooze |
+3522|87|8|2|48|47379.84|0.00|0.03|R|F|1994-12-05|1994-10-30|1994-12-26|TAKE BACK RETURN|SHIP|ve the quickly special packages|
+3522|157|2|3|46|48628.90|0.09|0.02|A|F|1994-11-12|1994-11-30|1994-11-20|NONE|AIR|d the express, silent foxes. blit|
+3522|130|9|4|7|7210.91|0.10|0.02|A|F|1994-10-31|1994-11-19|1994-11-28|NONE|TRUCK|e stealthil|
+3522|50|9|5|27|25651.35|0.02|0.05|R|F|1994-11-29|1994-12-15|1994-12-08|COLLECT COD|REG AIR|ic tithes. car|
+3522|158|10|6|18|19046.70|0.01|0.03|A|F|1994-11-16|1994-10-29|1994-11-29|COLLECT COD|RAIL|sits wake carefully pen|
+3523|25|6|1|15|13875.30|0.06|0.02|N|O|1998-06-26|1998-05-22|1998-07-24|COLLECT COD|REG AIR|se slyly pending, sp|
+3523|133|9|2|4|4132.52|0.03|0.06|N|O|1998-05-08|1998-05-18|1998-05-25|TAKE BACK RETURN|MAIL|ts. final accounts detect furiously along |
+3523|50|7|3|24|22801.20|0.07|0.04|N|O|1998-08-02|1998-06-22|1998-08-27|COLLECT COD|FOB|ke according to the doggedly re|
+3523|192|4|4|36|39318.84|0.06|0.08|N|O|1998-05-26|1998-06-04|1998-06-25|DELIVER IN PERSON|SHIP|accounts. fluffily regu|
+3523|134|5|5|48|49638.24|0.00|0.01|N|O|1998-07-22|1998-06-25|1998-08-19|DELIVER IN PERSON|AIR| regular requests|
+3524|137|8|1|5|5185.65|0.01|0.04|R|F|1992-05-23|1992-07-25|1992-06-19|DELIVER IN PERSON|RAIL|ts whithout the bold depende|
+3524|143|6|2|17|17733.38|0.09|0.08|A|F|1992-09-01|1992-07-17|1992-09-05|DELIVER IN PERSON|FOB|g, final epitaphs about the pinto |
+3525|46|7|1|12|11352.48|0.01|0.03|N|O|1996-03-08|1996-03-18|1996-03-16|NONE|TRUCK|lar excuses wake carefull|
+3525|138|9|2|27|28029.51|0.03|0.03|N|O|1995-12-30|1996-01-23|1996-01-02|DELIVER IN PERSON|SHIP|y slyly special asymptotes|
+3525|75|5|3|31|30227.17|0.00|0.03|N|O|1996-03-08|1996-02-27|1996-03-13|COLLECT COD|TRUCK|he careful|
+3525|184|5|4|28|30357.04|0.03|0.02|N|O|1996-01-22|1996-02-08|1996-01-27|COLLECT COD|FOB| nag according |
+3526|98|9|1|11|10978.99|0.02|0.03|R|F|1995-05-23|1995-05-28|1995-05-24|NONE|TRUCK|ges. furiously regular d|
+3526|117|7|2|23|23393.53|0.03|0.04|A|F|1995-05-01|1995-05-31|1995-05-25|DELIVER IN PERSON|FOB|special, regular packages cajole. |
+3526|33|9|3|20|18660.60|0.05|0.08|N|F|1995-06-16|1995-04-26|1995-06-22|DELIVER IN PERSON|REG AIR|kages. bold, special requests detect sl|
+3527|102|7|1|47|47098.70|0.07|0.02|N|O|1997-07-14|1997-07-29|1997-07-21|DELIVER IN PERSON|RAIL|unts. express re|
+3527|26|9|2|33|30558.66|0.01|0.02|N|O|1997-09-25|1997-09-17|1997-10-12|NONE|FOB|kly alongside of |
+3527|162|7|3|50|53108.00|0.09|0.07|N|O|1997-07-17|1997-08-03|1997-07-29|DELIVER IN PERSON|SHIP|e even accounts was about th|
+3527|128|3|4|17|17478.04|0.02|0.05|N|O|1997-07-30|1997-09-01|1997-08-17|COLLECT COD|MAIL|ular instruction|
+3552|197|8|1|18|19749.42|0.01|0.07|N|O|1997-08-11|1997-07-14|1997-08-15|DELIVER IN PERSON|TRUCK|s deposits against the blithely unusual pin|
+3552|90|1|2|44|43563.96|0.01|0.00|N|O|1997-08-08|1997-06-15|1997-08-29|COLLECT COD|FOB|ns after the blithely reg|
+3552|161|6|3|36|38201.76|0.04|0.08|N|O|1997-06-29|1997-06-24|1997-07-21|COLLECT COD|TRUCK|ly regular theodolites. fin|
+3553|143|10|1|4|4172.56|0.05|0.01|R|F|1994-06-13|1994-07-10|1994-07-03|COLLECT COD|RAIL|olites boost bli|
+3553|65|4|2|26|25091.56|0.05|0.08|A|F|1994-08-06|1994-07-30|1994-08-23|DELIVER IN PERSON|MAIL|fily special p|
+3553|22|5|3|18|16596.36|0.04|0.03|A|F|1994-07-03|1994-06-30|1994-07-07|COLLECT COD|RAIL|. quickly ironic|
+3553|32|8|4|40|37281.20|0.06|0.00|A|F|1994-09-14|1994-06-26|1994-09-25|NONE|RAIL| slyly pending asymptotes against the furi|
+3553|157|2|5|36|38057.40|0.06|0.08|R|F|1994-08-12|1994-06-25|1994-09-06|DELIVER IN PERSON|TRUCK| realms. pending, bold theodolites |
+3554|175|5|1|32|34405.44|0.01|0.05|N|O|1995-09-28|1995-09-01|1995-10-07|NONE|RAIL|. blithely ironic t|
+3554|145|6|2|18|18812.52|0.03|0.00|N|O|1995-09-11|1995-08-12|1995-10-04|DELIVER IN PERSON|REG AIR| haggle. furiously fluffy requests ac|
+3554|192|3|3|41|44779.79|0.02|0.01|N|O|1995-07-13|1995-08-28|1995-07-27|DELIVER IN PERSON|MAIL|ent dependencies. sly|
+3555|166|3|1|11|11727.76|0.05|0.02|N|O|1996-09-25|1996-10-01|1996-10-03|NONE|FOB|oost caref|
+3555|79|10|2|15|14686.05|0.03|0.08|N|O|1996-07-13|1996-09-01|1996-08-02|TAKE BACK RETURN|RAIL|y across the pending a|
+3555|43|2|3|25|23576.00|0.09|0.07|N|O|1996-10-01|1996-08-23|1996-10-24|TAKE BACK RETURN|MAIL|sual packages. quickly |
+3555|5|6|4|19|17195.00|0.00|0.05|N|O|1996-09-08|1996-09-14|1996-10-01|COLLECT COD|REG AIR|leep special theodolit|
+3555|33|4|5|29|27057.87|0.07|0.04|N|O|1996-08-02|1996-09-04|1996-08-08|DELIVER IN PERSON|TRUCK|deas. carefully s|
+3555|28|3|6|33|30624.66|0.04|0.08|N|O|1996-09-20|1996-09-23|1996-10-05|TAKE BACK RETURN|AIR|fluffily regular a|
+3555|126|5|7|9|9235.08|0.07|0.02|N|O|1996-10-13|1996-10-02|1996-10-22|NONE|SHIP|are. slyly final foxes acro|
+3556|142|9|1|45|46896.30|0.05|0.06|A|F|1992-10-14|1992-12-21|1992-10-16|NONE|TRUCK|ckages boost quickl|
+3556|31|2|2|43|40034.29|0.02|0.06|R|F|1993-01-18|1992-11-09|1993-02-04|NONE|FOB|wake carefull|
+3556|87|8|3|28|27638.24|0.10|0.04|A|F|1993-01-06|1992-11-27|1993-01-16|NONE|MAIL|refully final instructions? ironic packa|
+3557|175|3|1|41|44081.97|0.01|0.07|R|F|1993-01-30|1992-12-31|1993-02-18|COLLECT COD|FOB|ideas breach c|
+3557|129|10|2|37|38077.44|0.03|0.05|R|F|1993-02-16|1993-01-05|1993-03-15|DELIVER IN PERSON|RAIL|gside of the ca|
+3558|87|8|1|8|7896.64|0.01|0.03|N|O|1996-05-31|1996-05-26|1996-06-25|COLLECT COD|AIR|? even requests sle|
+3558|10|7|2|28|25480.28|0.02|0.08|N|O|1996-06-02|1996-04-18|1996-06-24|COLLECT COD|TRUCK|l deposits |
+3558|187|8|3|3|3261.54|0.03|0.06|N|O|1996-05-19|1996-04-28|1996-05-26|DELIVER IN PERSON|RAIL|l, final deposits haggle. fina|
+3558|91|5|4|22|21803.98|0.06|0.03|N|O|1996-04-27|1996-04-19|1996-04-30|DELIVER IN PERSON|SHIP|refully ironic theodolites are fu|
+3558|29|8|5|38|35302.76|0.03|0.08|N|O|1996-05-29|1996-05-02|1996-06-09|COLLECT COD|RAIL|refully permanently iron|
+3558|72|1|6|17|16525.19|0.07|0.07|N|O|1996-03-14|1996-05-04|1996-04-05|NONE|RAIL|ithely unusual packa|
+3559|90|1|1|29|28712.61|0.00|0.07|R|F|1992-12-10|1992-12-03|1992-12-20|COLLECT COD|REG AIR|l, regular accounts wake flu|
+3584|11|8|1|4|3644.04|0.04|0.08|N|O|1997-08-16|1997-10-31|1997-08-28|DELIVER IN PERSON|TRUCK|nal packag|
+3584|160|8|2|23|24383.68|0.00|0.03|N|O|1997-09-10|1997-10-15|1997-09-30|COLLECT COD|TRUCK|l platelets until the asymptotes |
+3584|24|5|3|6|5544.12|0.03|0.06|N|O|1997-10-28|1997-11-09|1997-11-24|TAKE BACK RETURN|MAIL|deposits across the|
+3584|146|5|4|11|11507.54|0.06|0.02|N|O|1997-11-27|1997-10-15|1997-12-08|NONE|REG AIR|lithely slyly |
+3584|18|5|5|39|35802.39|0.09|0.07|N|O|1997-09-20|1997-10-31|1997-10-06|COLLECT COD|AIR|eposits. carefu|
+3585|122|1|1|21|21464.52|0.05|0.04|A|F|1994-12-04|1994-12-25|1995-01-01|TAKE BACK RETURN|TRUCK|ounts use. express, final platelets us|
+3585|19|10|2|40|36760.40|0.03|0.00|R|F|1995-01-22|1995-01-17|1995-02-07|TAKE BACK RETURN|RAIL|elets affix. even asymptotes play care|
+3585|112|2|3|11|11133.21|0.01|0.04|R|F|1995-01-04|1995-02-14|1995-01-15|NONE|MAIL|even packages|
+3585|48|1|4|33|31285.32|0.08|0.08|A|F|1994-12-14|1995-01-19|1994-12-22|NONE|RAIL|ironic dependencies serve furi|
+3585|25|8|5|13|12025.26|0.06|0.07|R|F|1995-03-15|1995-01-22|1995-03-17|DELIVER IN PERSON|AIR|ccording to the foxes. slyly iro|
+3585|94|7|6|7|6958.63|0.10|0.02|A|F|1994-12-13|1995-01-20|1995-01-05|TAKE BACK RETURN|TRUCK|dependencies sleep un|
+3585|42|1|7|45|42391.80|0.03|0.00|A|F|1995-01-20|1995-02-19|1995-02-11|DELIVER IN PERSON|MAIL|are blithely c|
+3586|194|7|1|2|2188.38|0.03|0.08|R|F|1994-02-10|1994-01-07|1994-03-03|DELIVER IN PERSON|RAIL|he even, unusual decoy|
+3586|84|5|2|29|28538.32|0.04|0.07|R|F|1994-03-06|1994-03-02|1994-03-13|DELIVER IN PERSON|RAIL| slyly unusual i|
+3586|58|3|3|2|1916.10|0.03|0.06|R|F|1994-03-22|1994-02-20|1994-04-08|NONE|REG AIR|unts. slyly final ideas agai|
+3586|84|5|4|33|32474.64|0.06|0.01|R|F|1994-01-24|1994-02-09|1994-02-07|NONE|TRUCK|refully across the fur|
+3586|108|1|5|8|8064.80|0.06|0.02|A|F|1994-03-29|1994-02-26|1994-04-02|NONE|FOB|theodolites hagg|
+3586|99|1|6|8|7992.72|0.09|0.01|A|F|1994-03-18|1994-01-17|1994-04-06|DELIVER IN PERSON|RAIL| ironic pinto beans cajole carefully theo|
+3586|123|4|7|33|33762.96|0.05|0.04|A|F|1994-02-11|1994-01-15|1994-03-03|NONE|REG AIR|iously regular pinto beans integrate|
+3587|197|10|1|5|5485.95|0.09|0.07|N|O|1996-09-03|1996-07-05|1996-09-11|DELIVER IN PERSON|SHIP|ithely regular decoys above the |
+3587|132|8|2|48|49542.24|0.00|0.03|N|O|1996-08-02|1996-07-02|1996-08-05|TAKE BACK RETURN|MAIL|beans. blithely final depe|
+3587|151|3|3|36|37841.40|0.05|0.05|N|O|1996-07-26|1996-06-16|1996-08-23|TAKE BACK RETURN|MAIL|ully regular excuse|
+3587|124|9|4|31|31747.72|0.03|0.01|N|O|1996-07-21|1996-07-01|1996-07-23|COLLECT COD|SHIP|press fluffily regul|
+3587|70|7|5|12|11640.84|0.06|0.03|N|O|1996-08-30|1996-07-04|1996-09-22|DELIVER IN PERSON|RAIL|g the even pinto beans. special,|
+3587|107|2|6|16|16113.60|0.01|0.03|N|O|1996-05-11|1996-06-19|1996-06-04|COLLECT COD|FOB|y ruthless dolphins to |
+3587|74|2|7|23|22403.61|0.07|0.05|N|O|1996-08-30|1996-07-01|1996-09-10|COLLECT COD|FOB|l multipliers sleep theodolites-- slyly |
+3588|91|5|1|28|27750.52|0.04|0.08|R|F|1995-05-03|1995-05-03|1995-05-14|DELIVER IN PERSON|TRUCK|special pinto beans cajole slyly. slyly |
+3588|88|9|2|6|5928.48|0.06|0.08|A|F|1995-04-09|1995-05-30|1995-04-10|TAKE BACK RETURN|MAIL|s. fluffily fluf|
+3588|159|10|3|45|47661.75|0.04|0.02|R|F|1995-05-07|1995-05-04|1995-05-28|TAKE BACK RETURN|TRUCK|ecial pains integrate blithely. reques|
+3588|127|10|4|22|22596.64|0.05|0.00|A|F|1995-04-08|1995-05-06|1995-04-27|NONE|RAIL|inal accounts. pending, bo|
+3588|55|3|5|28|26741.40|0.03|0.03|A|F|1995-04-23|1995-05-25|1995-04-28|DELIVER IN PERSON|TRUCK| express sheaves. unusual theodo|
+3588|110|3|6|37|37374.07|0.08|0.04|N|F|1995-06-17|1995-05-25|1995-06-24|TAKE BACK RETURN|RAIL|xcuses sleep quickly along th|
+3588|39|5|7|46|43195.38|0.08|0.07|A|F|1995-06-06|1995-05-08|1995-06-08|NONE|AIR| slyly ironic deposits sublate ab|
+3589|37|3|1|42|39355.26|0.08|0.08|R|F|1994-08-11|1994-07-17|1994-08-23|DELIVER IN PERSON|AIR|he blithely unusual pac|
+3590|176|6|1|10|10761.70|0.08|0.00|N|O|1995-07-17|1995-06-26|1995-08-12|TAKE BACK RETURN|SHIP|t the quickly ironic|
+3590|95|6|2|19|18906.71|0.03|0.03|N|O|1995-08-02|1995-06-20|1995-08-08|NONE|SHIP|special pinto beans. blithely reg|
+3590|96|9|3|43|42831.87|0.07|0.06|N|O|1995-07-12|1995-07-25|1995-07-16|DELIVER IN PERSON|SHIP|s could have to use|
+3590|56|8|4|26|24857.30|0.01|0.03|N|O|1995-07-08|1995-06-17|1995-08-02|DELIVER IN PERSON|SHIP|arefully along th|
+3590|191|2|5|37|40374.03|0.00|0.08|N|O|1995-09-01|1995-06-29|1995-09-10|NONE|SHIP|ccounts above the silent waters thrash f|
+3590|119|10|6|31|31592.41|0.03|0.01|N|O|1995-06-24|1995-07-12|1995-06-25|DELIVER IN PERSON|REG AIR|ve furiously final instructions. slyly regu|
+3590|194|7|7|44|48144.36|0.05|0.04|N|F|1995-06-07|1995-06-15|1995-06-27|NONE|MAIL|s sleep after the regular platelets. blit|
+3591|29|8|1|21|19509.42|0.03|0.03|A|F|1994-02-25|1994-02-02|1994-03-05|DELIVER IN PERSON|TRUCK|structions against |
+3591|69|6|2|24|23257.44|0.04|0.04|R|F|1993-12-26|1994-01-07|1994-01-25|COLLECT COD|FOB|ages. slyly regular dependencies cajo|
+3591|164|9|3|4|4256.64|0.01|0.03|A|F|1994-04-04|1994-02-19|1994-05-02|DELIVER IN PERSON|RAIL|he final packages. deposits serve quick|
+3591|153|4|4|49|51604.35|0.01|0.00|A|F|1994-03-21|1994-01-26|1994-03-28|COLLECT COD|AIR| mold slyly. bl|
+3616|197|9|1|30|32915.70|0.01|0.00|A|F|1994-05-05|1994-04-24|1994-05-12|TAKE BACK RETURN|FOB|ly ironic accounts unwind b|
+3616|138|9|2|28|29067.64|0.08|0.06|R|F|1994-02-20|1994-04-18|1994-03-05|DELIVER IN PERSON|REG AIR|ironic packages. furiously ev|
+3617|117|8|1|46|46787.06|0.03|0.02|N|O|1996-05-19|1996-05-14|1996-06-11|NONE|RAIL|ar theodolites. regu|
+3617|98|9|2|16|15969.44|0.05|0.02|N|O|1996-05-08|1996-06-03|1996-05-19|COLLECT COD|RAIL| slyly on th|
+3617|98|2|3|32|31938.88|0.00|0.06|N|O|1996-04-20|1996-06-07|1996-05-19|DELIVER IN PERSON|MAIL|uriously against the express accounts. ex|
+3617|41|10|4|22|20702.88|0.10|0.05|N|O|1996-07-11|1996-05-02|1996-07-25|NONE|REG AIR|uffily even accounts. packages sleep blithe|
+3617|137|8|5|11|11408.43|0.08|0.05|N|O|1996-07-16|1996-04-23|1996-07-28|COLLECT COD|MAIL|ly quickly even requests. final|
+3618|140|1|1|38|39525.32|0.08|0.00|N|O|1997-12-22|1998-02-23|1998-01-03|TAKE BACK RETURN|TRUCK|nts haggle fluffily above the regular |
+3618|144|5|2|48|50118.72|0.04|0.00|N|O|1998-03-12|1998-02-13|1998-03-29|DELIVER IN PERSON|TRUCK|tructions atop the ironi|
+3618|63|2|3|24|23113.44|0.01|0.04|N|O|1998-01-26|1998-01-15|1998-02-17|TAKE BACK RETURN|AIR|xpress acc|
+3618|161|2|4|26|27590.16|0.01|0.05|N|O|1998-03-23|1998-01-24|1998-04-15|DELIVER IN PERSON|AIR|iously regular deposits cajole ruthless|
+3619|96|7|1|49|48808.41|0.01|0.08|N|O|1997-01-22|1996-12-21|1997-02-17|TAKE BACK RETURN|MAIL| waters. furiously even deposits |
+3619|116|10|2|27|27434.97|0.08|0.04|N|O|1996-12-12|1997-01-18|1996-12-18|TAKE BACK RETURN|SHIP|pecial accounts haggle care|
+3619|48|7|3|46|43609.84|0.08|0.03|N|O|1997-01-31|1997-01-27|1997-02-11|NONE|SHIP|press, expres|
+3619|93|6|4|18|17875.62|0.04|0.02|N|O|1997-03-18|1996-12-24|1997-03-21|COLLECT COD|AIR|eodolites |
+3619|120|10|5|38|38764.56|0.05|0.08|N|O|1996-12-08|1997-02-03|1997-01-07|NONE|RAIL|theodolites detect abo|
+3619|152|3|6|43|45242.45|0.01|0.01|N|O|1997-01-25|1997-01-06|1997-02-07|COLLECT COD|RAIL| bold, even|
+3620|59|7|1|41|39321.05|0.03|0.08|N|O|1997-03-21|1997-04-20|1997-03-30|COLLECT COD|FOB|t attainments cajole qui|
+3620|167|4|2|16|17074.56|0.00|0.06|N|O|1997-05-17|1997-05-08|1997-06-03|COLLECT COD|SHIP|s. even, pending in|
+3621|17|8|1|29|26593.29|0.02|0.06|A|F|1993-08-03|1993-07-08|1993-08-10|DELIVER IN PERSON|FOB|al requests. fl|
+3621|93|5|2|13|12910.17|0.09|0.04|R|F|1993-08-30|1993-06-30|1993-09-01|NONE|REG AIR|r the unusual packages. brave theodoli|
+3621|164|9|3|45|47887.20|0.07|0.07|R|F|1993-08-09|1993-06-18|1993-09-05|DELIVER IN PERSON|AIR| doubt about the bold deposits. carefully|
+3621|44|3|4|20|18880.80|0.05|0.04|R|F|1993-05-27|1993-07-04|1993-06-22|TAKE BACK RETURN|SHIP|gular accounts use carefully with|
+3622|175|6|1|47|50532.99|0.09|0.00|N|O|1996-02-24|1996-02-22|1996-03-12|TAKE BACK RETURN|TRUCK|are careful|
+3622|89|10|2|4|3956.32|0.04|0.04|N|O|1996-02-03|1996-02-19|1996-02-16|TAKE BACK RETURN|TRUCK|lithely brave foxes. furi|
+3622|190|1|3|46|50148.74|0.07|0.07|N|O|1995-12-18|1996-01-23|1996-01-12|TAKE BACK RETURN|AIR|sits wake. blithe|
+3622|177|8|4|9|9694.53|0.08|0.05|N|O|1995-12-12|1996-02-09|1995-12-13|TAKE BACK RETURN|SHIP|arefully. furiously regular ideas n|
+3623|80|10|1|32|31362.56|0.05|0.00|N|O|1997-04-18|1997-03-15|1997-05-09|COLLECT COD|SHIP| courts. furiously regular ideas b|
+3623|117|4|2|33|33564.63|0.08|0.01|N|O|1997-03-17|1997-02-13|1997-04-02|TAKE BACK RETURN|TRUCK|odolites. blithely spe|
+3623|24|7|3|21|19404.42|0.02|0.02|N|O|1997-01-19|1997-03-18|1997-01-24|NONE|FOB|ress ideas are furio|
+3623|165|2|4|42|44736.72|0.05|0.06|N|O|1997-01-11|1997-03-24|1997-01-21|COLLECT COD|RAIL|g to the slyly regular packa|
+3623|88|9|5|30|29642.40|0.10|0.04|N|O|1997-04-04|1997-03-03|1997-05-01|NONE|RAIL| ironic somas sleep fluffily|
+3623|186|7|6|7|7603.26|0.01|0.02|N|O|1997-01-05|1997-03-26|1997-01-26|NONE|TRUCK|aves. slyly special packages cajole. fu|
+3623|140|6|7|13|13521.82|0.03|0.08|N|O|1997-01-02|1997-02-26|1997-01-26|DELIVER IN PERSON|SHIP|deas. furiously expres|
+3648|144|5|1|16|16706.24|0.02|0.06|A|F|1993-08-14|1993-08-14|1993-08-15|COLLECT COD|FOB|s nag packages.|
+3648|105|2|2|30|30153.00|0.00|0.01|R|F|1993-08-31|1993-09-06|1993-09-06|DELIVER IN PERSON|FOB| above the somas boost furious|
+3648|46|7|3|34|32165.36|0.10|0.00|A|F|1993-08-21|1993-07-25|1993-09-15|DELIVER IN PERSON|FOB| deposits are furiously. careful, |
+3648|13|10|4|16|14608.16|0.06|0.03|R|F|1993-07-27|1993-08-26|1993-08-24|DELIVER IN PERSON|FOB|uriously stealthy deposits haggle furi|
+3648|117|7|5|25|25427.75|0.06|0.03|R|F|1993-08-15|1993-08-25|1993-09-09|TAKE BACK RETURN|TRUCK|s requests. silent asymp|
+3648|169|10|6|14|14968.24|0.08|0.06|R|F|1993-10-02|1993-08-26|1993-10-09|COLLECT COD|AIR|sly pending excuses. carefully i|
+3648|195|6|7|49|53664.31|0.09|0.03|R|F|1993-06-27|1993-07-27|1993-07-24|TAKE BACK RETURN|FOB|egular instructions. slyly regular pinto|
+3649|5|6|1|25|22625.00|0.10|0.04|A|F|1994-10-27|1994-08-23|1994-11-05|TAKE BACK RETURN|TRUCK|special re|
+3649|89|10|2|23|22748.84|0.08|0.00|R|F|1994-09-26|1994-10-01|1994-09-28|NONE|REG AIR|rs promise blithe|
+3649|70|7|3|14|13580.98|0.02|0.04|A|F|1994-09-19|1994-08-17|1994-10-12|DELIVER IN PERSON|TRUCK|ithely bold accounts wake |
+3649|76|4|4|40|39042.80|0.00|0.08|R|F|1994-07-20|1994-08-30|1994-08-14|TAKE BACK RETURN|RAIL|luffy somas sleep quickly-- ironic de|
+3649|100|1|5|24|24002.40|0.05|0.03|A|F|1994-07-07|1994-08-20|1994-07-27|TAKE BACK RETURN|FOB|c accounts. quickly final theodo|
+3649|122|3|6|3|3066.36|0.10|0.04|A|F|1994-07-17|1994-08-10|1994-08-03|NONE|FOB|lly bold requests nag; |
+3650|136|2|1|30|31083.90|0.10|0.00|A|F|1992-08-26|1992-07-05|1992-09-01|DELIVER IN PERSON|SHIP|ckly special platelets. furiously sil|
+3650|128|9|2|43|44209.16|0.05|0.05|A|F|1992-09-07|1992-08-12|1992-09-10|COLLECT COD|TRUCK|gside of the quick|
+3650|2|9|3|1|902.00|0.04|0.06|A|F|1992-06-23|1992-07-18|1992-07-08|NONE|REG AIR|re about the pinto |
+3650|63|2|4|31|29854.86|0.10|0.08|R|F|1992-06-15|1992-07-01|1992-07-15|DELIVER IN PERSON|RAIL| against the ironic accounts cajol|
+3650|187|8|5|19|20656.42|0.05|0.04|R|F|1992-08-29|1992-08-09|1992-09-21|DELIVER IN PERSON|AIR|y even forges. fluffily furious accounts|
+3650|94|8|6|27|26840.43|0.07|0.08|A|F|1992-07-03|1992-07-23|1992-07-13|COLLECT COD|MAIL|ular requests snooze fluffily regular pi|
+3650|70|7|7|43|41713.01|0.10|0.07|A|F|1992-06-25|1992-07-09|1992-07-22|DELIVER IN PERSON|RAIL|structions use caref|
+3651|19|9|1|20|18380.20|0.01|0.04|N|O|1998-06-10|1998-06-06|1998-06-23|NONE|SHIP|tect quickly among the r|
+3651|155|7|2|24|25323.60|0.09|0.04|N|O|1998-06-22|1998-07-17|1998-07-10|DELIVER IN PERSON|RAIL|excuses haggle according to th|
+3651|113|10|3|41|41537.51|0.00|0.05|N|O|1998-05-10|1998-07-09|1998-05-13|NONE|RAIL|blithely. furiously |
+3651|110|5|4|27|27272.97|0.05|0.03|N|O|1998-05-03|1998-06-30|1998-05-05|DELIVER IN PERSON|RAIL| sleep blithely furiously do|
+3652|180|8|1|24|25924.32|0.05|0.03|N|O|1997-06-07|1997-04-07|1997-06-12|COLLECT COD|MAIL|the final p|
+3652|137|8|2|37|38373.81|0.02|0.05|N|O|1997-05-11|1997-04-06|1997-06-05|COLLECT COD|MAIL|osits haggle carefu|
+3652|163|8|3|39|41463.24|0.01|0.02|N|O|1997-03-10|1997-04-03|1997-03-21|NONE|REG AIR|y express instructions. un|
+3652|80|9|4|1|980.08|0.01|0.04|N|O|1997-04-20|1997-05-03|1997-05-18|DELIVER IN PERSON|SHIP| bold dependencies sublate. r|
+3653|145|4|1|38|39715.32|0.08|0.05|A|F|1994-06-26|1994-05-13|1994-07-13|NONE|REG AIR|ainst the |
+3653|64|1|2|29|27957.74|0.07|0.01|A|F|1994-04-11|1994-06-11|1994-04-29|COLLECT COD|RAIL|ording to the special, final|
+3653|181|2|3|17|18380.06|0.09|0.03|R|F|1994-06-24|1994-06-02|1994-07-17|DELIVER IN PERSON|RAIL|gle slyly regular|
+3653|186|7|4|9|9775.62|0.10|0.07|R|F|1994-04-03|1994-05-19|1994-04-10|COLLECT COD|FOB|slyly silent account|
+3653|188|9|5|41|44615.38|0.08|0.01|A|F|1994-06-18|1994-05-18|1994-06-20|COLLECT COD|RAIL|onic packages affix sly|
+3653|43|4|6|9|8487.36|0.05|0.03|A|F|1994-07-21|1994-05-31|1994-08-17|NONE|MAIL|tes: blithely bo|
+3653|49|6|7|2|1898.08|0.06|0.03|R|F|1994-06-02|1994-05-31|1994-06-29|NONE|FOB|n accounts. fina|
+3654|165|2|1|46|48997.36|0.08|0.05|A|F|1992-06-05|1992-08-19|1992-06-06|DELIVER IN PERSON|FOB|usly regular foxes. furio|
+3654|93|4|2|29|28799.61|0.07|0.06|A|F|1992-09-11|1992-07-20|1992-10-04|DELIVER IN PERSON|FOB|odolites detect. quickly r|
+3654|2|7|3|37|33374.00|0.07|0.05|A|F|1992-09-22|1992-07-20|1992-10-19|TAKE BACK RETURN|RAIL|unts doze bravely ab|
+3654|168|9|4|11|11749.76|0.08|0.00|A|F|1992-07-20|1992-07-30|1992-07-23|TAKE BACK RETURN|SHIP|quickly along the express, ironic req|
+3654|94|5|5|34|33799.06|0.04|0.00|R|F|1992-07-26|1992-08-26|1992-08-12|TAKE BACK RETURN|REG AIR| the quick|
+3654|107|4|6|20|20142.00|0.03|0.02|A|F|1992-07-30|1992-07-05|1992-08-05|COLLECT COD|SHIP|s sleep about the slyly |
+3654|173|1|7|45|48292.65|0.01|0.07|A|F|1992-09-15|1992-07-04|1992-09-20|DELIVER IN PERSON|FOB|sly ironic notornis nag slyly|
+3655|184|5|1|5|5420.90|0.03|0.04|R|F|1993-01-17|1992-12-31|1993-01-23|DELIVER IN PERSON|TRUCK|riously bold pinto be|
+3655|97|10|2|1|997.09|0.10|0.06|R|F|1992-10-24|1992-12-18|1992-11-07|DELIVER IN PERSON|AIR|arefully slow pinto beans are|
+3655|30|5|3|35|32551.05|0.01|0.04|R|F|1992-12-20|1992-11-16|1993-01-15|TAKE BACK RETURN|MAIL|blithely even accounts! furiously regular|
+3655|72|3|4|35|34022.45|0.04|0.07|R|F|1992-10-17|1992-12-23|1992-10-28|COLLECT COD|MAIL|ng foxes cajole fluffily slyly final fo|
+3680|177|6|1|48|51704.16|0.00|0.06|R|F|1993-01-16|1993-01-23|1993-01-19|COLLECT COD|FOB|packages. quickly fluff|
+3680|5|8|2|41|37105.00|0.00|0.04|A|F|1993-01-06|1993-03-02|1993-01-08|NONE|FOB|iously ironic platelets in|
+3680|56|4|3|33|31549.65|0.09|0.08|R|F|1993-03-16|1993-02-19|1993-04-05|NONE|FOB|ts. ironic, fina|
+3681|106|9|1|35|35213.50|0.03|0.08|R|F|1992-07-31|1992-05-18|1992-08-07|COLLECT COD|FOB|lyly special pinto |
+3682|61|10|1|6|5766.36|0.07|0.02|N|O|1997-05-06|1997-04-04|1997-05-11|NONE|AIR|ronic deposits wake slyly. ca|
+3682|116|7|2|18|18289.98|0.06|0.06|N|O|1997-04-30|1997-03-21|1997-05-10|NONE|FOB|regular dependencies|
+3682|47|10|3|17|16099.68|0.03|0.05|N|O|1997-02-12|1997-04-04|1997-02-22|COLLECT COD|FOB|, ironic packages wake a|
+3682|57|5|4|30|28711.50|0.09|0.05|N|O|1997-04-16|1997-04-16|1997-04-29|NONE|MAIL|he requests cajole quickly pending package|
+3683|101|4|1|35|35038.50|0.05|0.03|A|F|1993-05-31|1993-04-17|1993-06-14|NONE|SHIP| the furiously expr|
+3683|49|8|2|41|38910.64|0.01|0.06|A|F|1993-03-26|1993-05-06|1993-04-09|NONE|TRUCK|ress instructions. slyly express a|
+3683|100|3|3|23|23002.30|0.00|0.08|R|F|1993-07-02|1993-05-16|1993-07-30|NONE|TRUCK|xpress accounts sleep slyly re|
+3684|126|7|1|48|49253.76|0.04|0.06|A|F|1993-08-20|1993-09-02|1993-09-10|DELIVER IN PERSON|REG AIR|its boost alongside|
+3684|46|7|2|6|5676.24|0.06|0.08|R|F|1993-08-09|1993-10-05|1993-09-06|DELIVER IN PERSON|FOB|he silent requests. packages sleep fu|
+3684|163|8|3|19|20200.04|0.04|0.02|A|F|1993-10-19|1993-08-25|1993-11-02|COLLECT COD|FOB|e slyly carefully pending foxes. d|
+3684|135|1|4|13|13456.69|0.02|0.05|A|F|1993-07-23|1993-09-16|1993-08-06|NONE|TRUCK|ing, unusual pinto beans! thinly p|
+3685|47|4|1|37|35040.48|0.02|0.03|R|F|1992-03-11|1992-04-09|1992-04-05|DELIVER IN PERSON|TRUCK|ress attai|
+3685|58|6|2|7|6706.35|0.05|0.00|R|F|1992-05-16|1992-02-23|1992-05-17|DELIVER IN PERSON|FOB|sits. special asymptotes about the r|
+3685|134|5|3|38|39296.94|0.08|0.03|A|F|1992-05-17|1992-03-16|1992-06-06|TAKE BACK RETURN|TRUCK|thely unusual pack|
+3685|192|5|4|39|42595.41|0.10|0.05|R|F|1992-02-19|1992-04-06|1992-03-02|COLLECT COD|FOB|ic courts nag carefully after the |
+3685|56|7|5|37|35373.85|0.00|0.01|A|F|1992-03-02|1992-04-10|1992-03-04|NONE|FOB|. carefully sly requests are regular, regu|
+3686|122|5|1|7|7154.84|0.02|0.04|N|O|1998-07-15|1998-08-22|1998-07-30|DELIVER IN PERSON|TRUCK| furiously unusual accou|
+3686|200|2|2|38|41807.60|0.06|0.03|N|O|1998-09-04|1998-08-11|1998-09-19|DELIVER IN PERSON|AIR|y silent foxes! carefully ruthless cour|
+3686|45|6|3|31|29296.24|0.10|0.06|N|O|1998-09-09|1998-08-28|1998-10-09|COLLECT COD|MAIL|gle across the courts. furiously regu|
+3686|117|1|4|7|7119.77|0.10|0.01|N|O|1998-07-16|1998-09-02|1998-07-22|NONE|FOB|ake carefully carefully q|
+3687|145|4|1|32|33444.48|0.03|0.06|R|F|1993-05-07|1993-04-05|1993-05-25|DELIVER IN PERSON|AIR|deas cajole fo|
+3687|81|2|2|2|1962.16|0.00|0.08|R|F|1993-02-23|1993-03-25|1993-03-11|NONE|TRUCK| express requests. slyly regular depend|
+3687|174|4|3|10|10741.70|0.01|0.02|A|F|1993-02-11|1993-03-22|1993-03-09|NONE|FOB|ing pinto beans|
+3687|162|9|4|19|20181.04|0.02|0.05|A|F|1993-05-14|1993-04-24|1993-06-01|DELIVER IN PERSON|MAIL|ly final asymptotes according to t|
+3687|119|9|5|31|31592.41|0.07|0.08|A|F|1993-05-28|1993-03-20|1993-06-05|DELIVER IN PERSON|FOB|foxes cajole quickly about the furiously f|
+3712|141|4|1|27|28110.78|0.01|0.05|R|F|1992-02-01|1992-02-26|1992-03-02|TAKE BACK RETURN|SHIP|ctions. even accounts haggle alongside |
+3712|185|6|2|13|14107.34|0.03|0.03|R|F|1992-04-30|1992-02-11|1992-05-30|DELIVER IN PERSON|FOB|s around the furiously ironic account|
+3712|64|1|3|44|42418.64|0.01|0.01|A|F|1992-03-26|1992-02-19|1992-04-18|TAKE BACK RETURN|FOB|ously permanently regular req|
+3712|148|7|4|38|39829.32|0.01|0.06|A|F|1992-01-15|1992-03-24|1992-01-27|COLLECT COD|RAIL|s nag carefully-- even, reg|
+3713|112|6|1|41|41496.51|0.07|0.08|N|O|1998-05-11|1998-07-17|1998-05-22|COLLECT COD|RAIL|eposits wake blithely fina|
+3713|177|7|2|19|20466.23|0.04|0.04|N|O|1998-06-25|1998-07-24|1998-07-08|DELIVER IN PERSON|AIR|tructions serve blithely around the furi|
+3713|180|1|3|19|20523.42|0.03|0.02|N|O|1998-05-19|1998-07-06|1998-06-09|DELIVER IN PERSON|REG AIR|quests cajole careful|
+3713|169|10|4|45|48112.20|0.06|0.04|N|O|1998-06-15|1998-07-30|1998-07-14|DELIVER IN PERSON|MAIL|al pinto beans affix after the slyly |
+3713|90|1|5|46|45544.14|0.10|0.04|N|O|1998-08-22|1998-06-27|1998-08-31|NONE|MAIL|totes. carefully special theodolites s|
+3713|182|3|6|29|31383.22|0.09|0.03|N|O|1998-08-04|1998-06-13|1998-08-21|NONE|RAIL|the regular dugouts wake furiously sil|
+3713|130|1|7|14|14421.82|0.04|0.00|N|O|1998-07-19|1998-07-02|1998-07-28|DELIVER IN PERSON|SHIP|eposits impress according|
+3714|69|6|1|13|12597.78|0.07|0.03|N|O|1998-06-26|1998-06-17|1998-07-07|TAKE BACK RETURN|REG AIR| the furiously final|
+3714|146|3|2|14|14645.96|0.02|0.05|N|O|1998-05-30|1998-06-30|1998-05-31|DELIVER IN PERSON|RAIL|ending ideas. thinly unusual theodo|
+3714|159|10|3|16|16946.40|0.00|0.02|N|O|1998-05-25|1998-07-07|1998-06-17|TAKE BACK RETURN|AIR|ccounts cajole fu|
+3714|30|9|4|44|40921.32|0.04|0.02|N|O|1998-07-18|1998-07-10|1998-07-22|DELIVER IN PERSON|AIR|s. quickly ironic dugouts sublat|
+3715|97|1|1|13|12962.17|0.00|0.03|N|O|1996-05-11|1996-04-25|1996-06-09|TAKE BACK RETURN|SHIP|e quickly ironic|
+3715|169|6|2|16|17106.56|0.01|0.06|N|O|1996-06-28|1996-04-22|1996-06-30|TAKE BACK RETURN|AIR|usly regular pearls haggle final packages|
+3715|12|3|3|37|33744.37|0.05|0.02|N|O|1996-05-03|1996-04-30|1996-05-17|NONE|SHIP|ut the carefully expr|
+3716|32|8|1|10|9320.30|0.09|0.04|N|O|1997-12-02|1997-11-09|1997-12-14|TAKE BACK RETURN|SHIP|ts. quickly sly ideas slee|
+3716|194|5|2|39|42673.41|0.02|0.08|N|O|1997-11-27|1997-10-23|1997-12-24|COLLECT COD|REG AIR|even deposits.|
+3716|107|8|3|42|42298.20|0.02|0.08|N|O|1997-12-03|1997-10-12|1997-12-15|NONE|TRUCK| of the pend|
+3716|165|10|4|19|20238.04|0.05|0.08|N|O|1997-09-25|1997-10-18|1997-10-12|NONE|TRUCK|arefully unusual accounts. flu|
+3716|182|3|5|25|27054.50|0.06|0.05|N|O|1997-11-23|1997-10-24|1997-11-24|COLLECT COD|REG AIR|fully unusual accounts. carefu|
+3717|153|8|1|45|47391.75|0.07|0.04|N|O|1998-08-09|1998-08-18|1998-08-14|TAKE BACK RETURN|TRUCK|ests wake whithout the blithely final pl|
+3717|53|5|2|3|2859.15|0.01|0.07|N|O|1998-06-09|1998-07-31|1998-06-14|NONE|REG AIR|nside the regular packages sleep|
+3717|196|7|3|45|49328.55|0.05|0.08|N|O|1998-09-19|1998-07-22|1998-09-28|DELIVER IN PERSON|MAIL|s the blithely unu|
+3717|69|6|4|5|4845.30|0.06|0.03|N|O|1998-09-02|1998-08-20|1998-09-26|TAKE BACK RETURN|AIR|quickly among |
+3717|16|7|5|7|6412.07|0.09|0.02|N|O|1998-09-08|1998-07-18|1998-09-10|DELIVER IN PERSON|RAIL| after the packa|
+3717|64|1|6|38|36634.28|0.01|0.07|N|O|1998-07-10|1998-07-08|1998-07-29|COLLECT COD|RAIL|ly about the car|
+3717|106|7|7|28|28170.80|0.03|0.01|N|O|1998-07-25|1998-08-12|1998-08-16|COLLECT COD|RAIL|ts sleep q|
+3718|21|10|1|40|36840.80|0.01|0.04|N|O|1996-11-20|1996-12-17|1996-12-03|DELIVER IN PERSON|MAIL|out the express deposits|
+3718|163|8|2|16|17010.56|0.02|0.06|N|O|1996-11-11|1996-12-25|1996-11-12|COLLECT COD|TRUCK|slyly even accounts. blithely special acco|
+3718|70|5|3|8|7760.56|0.05|0.03|N|O|1996-12-06|1996-12-06|1996-12-15|TAKE BACK RETURN|AIR| the even deposits sleep carefully b|
+3719|22|5|1|35|32270.70|0.06|0.08|N|O|1997-06-11|1997-04-03|1997-06-15|TAKE BACK RETURN|TRUCK|ly foxes. pending braids haggle furio|
+3719|174|4|2|2|2148.34|0.02|0.08|N|O|1997-02-17|1997-04-25|1997-03-03|NONE|REG AIR|ccounts boost carefu|
+3719|182|3|3|12|12986.16|0.05|0.06|N|O|1997-06-10|1997-05-04|1997-07-09|TAKE BACK RETURN|REG AIR|grate according to the |
+3719|90|1|4|13|12871.17|0.02|0.00|N|O|1997-05-03|1997-04-16|1997-05-27|TAKE BACK RETURN|SHIP|iously. regular dep|
+3719|78|8|5|19|18583.33|0.06|0.08|N|O|1997-05-22|1997-03-20|1997-06-12|COLLECT COD|TRUCK|he regular ideas integrate acros|
+3719|142|5|6|43|44812.02|0.03|0.08|N|O|1997-05-08|1997-04-15|1997-06-06|COLLECT COD|RAIL|the furiously special pinto bean|
+3719|19|10|7|16|14704.16|0.10|0.01|N|O|1997-03-02|1997-03-18|1997-03-28|TAKE BACK RETURN|RAIL| express asymptotes. ir|
+3744|195|8|1|30|32855.70|0.05|0.06|A|F|1992-05-07|1992-02-12|1992-05-17|TAKE BACK RETURN|FOB|nts among |
+3745|137|8|1|18|18668.34|0.01|0.05|A|F|1993-10-17|1993-11-16|1993-11-13|DELIVER IN PERSON|SHIP| slyly bold pinto beans according to |
+3746|165|6|1|37|39410.92|0.07|0.00|A|F|1994-12-29|1994-10-25|1995-01-03|COLLECT COD|FOB|e of the careful|
+3746|144|7|2|28|29235.92|0.06|0.08|R|F|1994-09-20|1994-10-21|1994-09-27|DELIVER IN PERSON|FOB|s after the even, special requests|
+3746|188|9|3|3|3264.54|0.10|0.01|R|F|1994-11-03|1994-12-10|1994-11-12|NONE|MAIL| the silent ideas cajole carefully |
+3746|28|7|4|11|10208.22|0.00|0.05|R|F|1994-10-02|1994-11-19|1994-10-10|COLLECT COD|SHIP| ironic theodolites are among th|
+3747|141|10|1|42|43727.88|0.05|0.05|N|O|1996-11-10|1996-10-19|1996-11-19|TAKE BACK RETURN|REG AIR|y. blithely fina|
+3747|170|1|2|33|35315.61|0.01|0.03|N|O|1996-10-14|1996-11-12|1996-11-11|NONE|REG AIR| regular p|
+3747|139|10|3|30|31173.90|0.00|0.07|N|O|1996-12-16|1996-11-15|1996-12-17|NONE|RAIL|! furiously f|
+3747|33|9|4|21|19593.63|0.00|0.06|N|O|1996-11-18|1996-09-23|1996-11-26|TAKE BACK RETURN|AIR|ithely bold orbits mold furiously blit|
+3747|126|5|5|32|32835.84|0.08|0.05|N|O|1996-09-10|1996-11-04|1996-10-10|DELIVER IN PERSON|MAIL|quests shall h|
+3747|154|5|6|14|14758.10|0.08|0.07|N|O|1996-11-03|1996-10-29|1996-11-06|TAKE BACK RETURN|AIR|packages cajole carefu|
+3747|118|2|7|23|23416.53|0.00|0.04|N|O|1996-11-08|1996-11-10|1996-12-03|NONE|REG AIR|kages are ironic|
+3748|104|7|1|12|12049.20|0.06|0.01|N|O|1998-04-17|1998-04-15|1998-05-12|NONE|AIR|old reques|
+3748|165|4|2|24|25563.84|0.08|0.04|N|O|1998-06-07|1998-05-02|1998-06-21|DELIVER IN PERSON|TRUCK|al deposits. blithely|
+3748|197|1|3|19|20846.61|0.05|0.01|N|O|1998-04-23|1998-05-17|1998-05-23|COLLECT COD|RAIL|pinto beans run carefully quic|
+3748|187|8|4|5|5435.90|0.00|0.07|N|O|1998-06-29|1998-05-06|1998-07-12|DELIVER IN PERSON|MAIL| regular accounts sleep quickly-- furious|
+3748|147|4|5|21|21989.94|0.07|0.08|N|O|1998-03-30|1998-04-07|1998-04-05|TAKE BACK RETURN|MAIL|fix carefully furiously express ideas. furi|
+3749|173|3|1|11|11804.87|0.07|0.05|N|O|1995-06-25|1995-05-23|1995-07-10|TAKE BACK RETURN|RAIL|egular requests along the |
+3749|129|8|2|9|9262.08|0.08|0.05|A|F|1995-04-23|1995-04-18|1995-04-26|NONE|REG AIR|uses cajole blithely pla|
+3749|199|2|3|31|34074.89|0.00|0.05|N|F|1995-06-11|1995-05-20|1995-06-27|COLLECT COD|REG AIR|s. foxes sleep slyly unusual grouc|
+3749|131|2|4|7|7217.91|0.07|0.06|A|F|1995-03-31|1995-04-05|1995-04-11|NONE|TRUCK|he slyly ironic packages|
+3749|183|4|5|14|15164.52|0.02|0.00|N|F|1995-06-11|1995-05-19|1995-07-11|DELIVER IN PERSON|SHIP|press instruc|
+3749|54|6|6|10|9540.50|0.10|0.03|N|O|1995-06-24|1995-05-24|1995-07-18|COLLECT COD|SHIP|essly. regular pi|
+3750|134|10|1|37|38262.81|0.04|0.03|N|O|1995-07-08|1995-07-28|1995-07-28|DELIVER IN PERSON|REG AIR|usly busy account|
+3750|152|3|2|33|34720.95|0.05|0.03|N|O|1995-06-27|1995-06-20|1995-07-03|TAKE BACK RETURN|REG AIR|theodolites haggle. slyly pendin|
+3750|80|10|3|20|19601.60|0.09|0.05|N|F|1995-06-17|1995-06-06|1995-06-28|TAKE BACK RETURN|REG AIR|ss, ironic requests! fur|
+3750|166|1|4|33|35183.28|0.04|0.03|N|F|1995-06-15|1995-06-04|1995-06-29|COLLECT COD|RAIL|ep blithely according to the flu|
+3750|83|4|5|1|983.08|0.05|0.01|N|O|1995-07-24|1995-06-25|1995-08-21|DELIVER IN PERSON|REG AIR|l dolphins against the slyly|
+3750|113|7|6|47|47616.17|0.01|0.08|R|F|1995-05-11|1995-06-13|1995-06-02|TAKE BACK RETURN|FOB|slowly regular accounts. blithely ev|
+3751|172|2|1|37|39670.29|0.00|0.04|R|F|1994-04-30|1994-05-30|1994-05-30|NONE|REG AIR|ly express courts |
+3751|141|8|2|32|33316.48|0.03|0.05|R|F|1994-05-05|1994-07-02|1994-06-02|COLLECT COD|MAIL|rthogs could have to slee|
+3751|65|2|3|45|43427.70|0.08|0.06|R|F|1994-05-27|1994-06-19|1994-06-14|NONE|RAIL|according to |
+3751|14|4|4|39|35646.39|0.07|0.01|A|F|1994-08-16|1994-07-11|1994-09-12|COLLECT COD|TRUCK|refully according to the iro|
+3751|58|3|5|12|11496.60|0.02|0.03|A|F|1994-08-09|1994-06-30|1994-08-12|TAKE BACK RETURN|TRUCK|accounts wake furious|
+3751|76|5|6|39|38066.73|0.02|0.08|R|F|1994-08-01|1994-06-01|1994-08-26|COLLECT COD|SHIP|to beans. pending, express packages c|
+3776|3|10|1|39|35217.00|0.05|0.01|R|F|1993-01-03|1993-02-05|1993-01-08|COLLECT COD|FOB|yly blithely pending packages|
+3776|159|4|2|14|14828.10|0.06|0.08|R|F|1992-12-30|1993-02-12|1993-01-27|DELIVER IN PERSON|RAIL|y special ideas. express packages pr|
+3776|141|8|3|49|51015.86|0.01|0.08|R|F|1992-12-03|1993-02-16|1992-12-28|TAKE BACK RETURN|RAIL|equests. final, thin grouches |
+3776|92|6|4|49|48612.41|0.08|0.05|A|F|1993-02-11|1993-01-06|1993-02-27|COLLECT COD|MAIL|es: careful warthogs haggle fluffi|
+3777|100|4|1|11|11001.10|0.02|0.03|A|F|1994-04-09|1994-06-05|1994-04-14|NONE|FOB|ld ideas. even theodolites|
+3777|8|5|2|10|9080.00|0.03|0.01|R|F|1994-05-22|1994-05-29|1994-06-13|COLLECT COD|RAIL|le. ironic depths a|
+3777|166|7|3|18|19190.88|0.10|0.06|R|F|1994-05-04|1994-05-23|1994-05-22|COLLECT COD|REG AIR|eful packages use slyly: even deposits |
+3777|18|9|4|35|32130.35|0.10|0.04|A|F|1994-05-25|1994-05-26|1994-06-13|COLLECT COD|AIR|s. carefully express asymptotes accordi|
+3777|98|10|5|14|13973.26|0.04|0.05|R|F|1994-05-06|1994-06-24|1994-05-31|NONE|TRUCK|ording to the iro|
+3778|57|2|1|21|20098.05|0.01|0.06|R|F|1993-05-27|1993-07-10|1993-06-03|COLLECT COD|REG AIR|ts. blithely special theodoli|
+3778|29|10|2|32|29728.64|0.09|0.00|A|F|1993-06-22|1993-08-18|1993-07-03|TAKE BACK RETURN|MAIL|tes affix carefully above the |
+3778|94|6|3|41|40757.69|0.05|0.00|R|F|1993-06-21|1993-07-27|1993-07-15|COLLECT COD|FOB|e the furiously ironi|
+3778|169|4|4|28|29936.48|0.03|0.05|R|F|1993-08-18|1993-07-10|1993-09-06|TAKE BACK RETURN|REG AIR|y silent orbits print carefully against |
+3778|98|2|5|28|27946.52|0.01|0.06|R|F|1993-09-02|1993-08-08|1993-10-02|DELIVER IN PERSON|FOB|r deposits. theodol|
+3778|20|7|6|26|23920.52|0.00|0.01|A|F|1993-09-24|1993-07-06|1993-10-22|NONE|TRUCK| against the fluffily|
+3778|105|6|7|49|49249.90|0.02|0.04|A|F|1993-06-13|1993-08-08|1993-07-04|DELIVER IN PERSON|MAIL|ans. furiously |
+3779|46|5|1|28|26489.12|0.04|0.05|N|O|1997-05-06|1997-04-01|1997-05-18|TAKE BACK RETURN|AIR|s. close requests sleep|
+3779|110|3|2|5|5050.55|0.07|0.03|N|O|1997-01-07|1997-03-26|1997-02-05|DELIVER IN PERSON|AIR|heodolites. slyly regular a|
+3780|127|8|1|25|25678.00|0.08|0.04|N|O|1996-06-27|1996-07-02|1996-07-22|NONE|AIR|l, unusual |
+3780|190|1|2|40|43607.60|0.10|0.04|N|O|1996-06-06|1996-05-29|1996-07-01|COLLECT COD|SHIP|gular deposits-- furiously regular |
+3781|14|5|1|48|43872.48|0.02|0.06|N|O|1996-08-22|1996-08-13|1996-09-15|NONE|REG AIR|equests may cajole careful|
+3781|188|9|2|39|42439.02|0.10|0.00|N|O|1996-08-20|1996-08-16|1996-09-01|DELIVER IN PERSON|REG AIR|unts are carefully. ir|
+3781|30|1|3|17|15810.51|0.01|0.03|N|O|1996-06-23|1996-09-04|1996-07-19|TAKE BACK RETURN|REG AIR|. theodolite|
+3781|31|2|4|15|13965.45|0.05|0.00|N|O|1996-08-23|1996-08-08|1996-09-06|TAKE BACK RETURN|AIR| carefully blithe|
+3781|16|6|5|23|21068.23|0.09|0.08|N|O|1996-09-05|1996-08-18|1996-09-27|DELIVER IN PERSON|SHIP|pendencies are b|
+3782|27|10|1|29|26883.58|0.01|0.07|N|O|1996-09-17|1996-10-03|1996-10-07|DELIVER IN PERSON|REG AIR|quickly unusual pinto beans. carefully fina|
+3782|153|1|2|10|10531.50|0.03|0.05|N|O|1996-09-07|1996-11-19|1996-10-04|COLLECT COD|FOB|ven pinto b|
+3782|136|7|3|30|31083.90|0.06|0.06|N|O|1996-12-19|1996-10-31|1997-01-14|TAKE BACK RETURN|MAIL|slyly even pinto beans hag|
+3782|117|7|4|34|34581.74|0.02|0.06|N|O|1996-11-07|1996-10-22|1996-11-19|DELIVER IN PERSON|MAIL|gage after the even|
+3782|130|3|5|40|41205.20|0.09|0.04|N|O|1996-12-16|1996-11-22|1997-01-01|COLLECT COD|AIR|s instructions. regular accou|
+3783|167|4|1|36|38417.76|0.04|0.08|R|F|1993-12-17|1994-02-26|1994-01-03|DELIVER IN PERSON|SHIP|ites haggle among the carefully unusu|
+3783|73|3|2|36|35030.52|0.02|0.02|R|F|1994-03-02|1994-02-09|1994-03-15|COLLECT COD|TRUCK|egular accounts|
+3783|85|6|3|50|49254.00|0.04|0.01|R|F|1994-03-14|1994-01-09|1994-04-10|DELIVER IN PERSON|FOB|he furiously regular deposits. |
+3783|27|6|4|37|34299.74|0.10|0.05|R|F|1993-12-09|1994-02-17|1993-12-30|COLLECT COD|REG AIR|ing to the ideas. regular accounts de|
+3808|43|10|1|28|26405.12|0.02|0.01|R|F|1994-05-27|1994-06-18|1994-06-22|TAKE BACK RETURN|FOB|lly final accounts alo|
+3808|127|6|2|47|48274.64|0.04|0.08|R|F|1994-06-12|1994-06-03|1994-07-02|COLLECT COD|TRUCK|fully for the quickly final deposits: flu|
+3808|31|2|3|45|41896.35|0.00|0.03|R|F|1994-07-03|1994-05-29|1994-07-14|TAKE BACK RETURN|REG AIR| carefully special|
+3808|100|1|4|34|34003.40|0.07|0.04|R|F|1994-08-13|1994-07-22|1994-08-31|DELIVER IN PERSON|FOB| pearls will have to |
+3808|155|7|5|29|30599.35|0.08|0.03|A|F|1994-06-22|1994-05-26|1994-07-06|TAKE BACK RETURN|TRUCK| deposits across the pac|
+3808|168|5|6|44|46999.04|0.06|0.06|A|F|1994-06-07|1994-06-04|1994-06-25|NONE|REG AIR|the blithely regular foxes. even, final |
+3809|191|3|1|17|18550.23|0.10|0.04|N|O|1996-08-14|1996-07-05|1996-09-04|DELIVER IN PERSON|FOB|es detect furiously sil|
+3809|133|4|2|32|33060.16|0.01|0.02|N|O|1996-07-03|1996-06-01|1996-07-25|COLLECT COD|SHIP|xcuses would boost against the fluffily eve|
+3809|105|6|3|46|46234.60|0.10|0.06|N|O|1996-08-20|1996-06-01|1996-08-24|TAKE BACK RETURN|TRUCK|l asymptotes. special |
+3809|178|9|4|43|46361.31|0.00|0.04|N|O|1996-05-06|1996-06-22|1996-06-05|TAKE BACK RETURN|TRUCK|yly ironic decoys; regular, iron|
+3810|184|5|1|49|53124.82|0.05|0.01|R|F|1992-11-27|1992-10-30|1992-12-16|COLLECT COD|AIR|cajole. fur|
+3810|169|8|2|18|19244.88|0.01|0.04|A|F|1992-11-28|1992-11-15|1992-12-27|DELIVER IN PERSON|SHIP|s. furiously careful deposi|
+3810|137|3|3|41|42522.33|0.08|0.08|A|F|1992-10-26|1992-10-27|1992-11-05|COLLECT COD|SHIP|l requests boost slyly along the slyl|
+3810|182|3|4|11|11903.98|0.06|0.04|A|F|1992-12-18|1992-12-11|1993-01-15|DELIVER IN PERSON|MAIL| the pending pinto beans. expr|
+3811|164|3|1|24|25539.84|0.04|0.02|N|O|1998-07-13|1998-05-16|1998-08-12|TAKE BACK RETURN|TRUCK|deposits. slyly regular accounts cajo|
+3811|166|5|2|2|2132.32|0.01|0.08|N|O|1998-06-16|1998-06-16|1998-06-23|NONE|MAIL|slyly fluff|
+3811|43|6|3|19|17917.76|0.02|0.06|N|O|1998-07-20|1998-06-14|1998-07-29|NONE|MAIL|s boost blithely furiou|
+3811|171|1|4|50|53558.50|0.08|0.03|N|O|1998-07-28|1998-07-06|1998-08-16|COLLECT COD|FOB|ts are slyly fluffy ideas. furiou|
+3811|182|3|5|23|24890.14|0.00|0.04|N|O|1998-08-13|1998-07-09|1998-08-29|COLLECT COD|AIR|nstructions sleep quickly. slyly final |
+3811|2|7|6|35|31570.00|0.04|0.07|N|O|1998-04-17|1998-06-30|1998-04-25|NONE|REG AIR|yly final dolphins? quickly ironic frets|
+3812|145|4|1|33|34489.62|0.00|0.05|N|O|1996-10-10|1996-10-05|1996-10-15|TAKE BACK RETURN|MAIL|posits engage. ironic, regular p|
+3812|173|2|2|33|35414.61|0.06|0.03|N|O|1996-10-05|1996-10-13|1996-10-22|TAKE BACK RETURN|MAIL|inal excuses d|
+3813|176|7|1|37|39818.29|0.05|0.04|N|O|1998-10-13|1998-09-19|1998-10-28|NONE|REG AIR|ravely special packages haggle p|
+3813|123|2|2|39|39901.68|0.05|0.00|N|O|1998-08-30|1998-08-12|1998-09-29|COLLECT COD|FOB|y ideas. final ideas about the sp|
+3814|131|7|1|7|7217.91|0.02|0.02|R|F|1995-05-01|1995-05-09|1995-05-28|DELIVER IN PERSON|REG AIR|es sleep furiou|
+3814|173|3|2|14|15024.38|0.01|0.00|R|F|1995-03-17|1995-05-10|1995-04-16|DELIVER IN PERSON|AIR|sits along the final, ironic deposit|
+3814|168|7|3|36|38453.76|0.06|0.02|N|O|1995-06-19|1995-04-18|1995-06-28|COLLECT COD|SHIP|beans cajole quickly sl|
+3814|66|7|4|20|19321.20|0.04|0.07|R|F|1995-02-23|1995-03-26|1995-03-04|DELIVER IN PERSON|SHIP|. doggedly ironic deposits will have to wa|
+3814|107|2|5|15|15106.50|0.03|0.04|N|O|1995-06-23|1995-03-25|1995-07-09|COLLECT COD|SHIP| carefully final deposits haggle slyly|
+3814|83|4|6|47|46204.76|0.09|0.05|A|F|1995-04-16|1995-04-03|1995-05-14|DELIVER IN PERSON|AIR|nusual requests. bli|
+3814|132|8|7|12|12385.56|0.10|0.01|R|F|1995-03-18|1995-04-16|1995-03-20|TAKE BACK RETURN|REG AIR|ages cajole. packages haggle. final|
+3815|77|7|1|3|2931.21|0.07|0.00|N|O|1997-11-16|1997-11-15|1997-11-30|NONE|FOB|egular, express ideas. ironic, final dep|
+3815|130|5|2|11|11331.43|0.02|0.04|N|O|1997-11-01|1997-11-05|1997-11-27|COLLECT COD|TRUCK|sleep blithe|
+3840|187|8|1|45|48923.10|0.02|0.08|N|O|1998-10-31|1998-09-19|1998-11-30|DELIVER IN PERSON|TRUCK|o beans are. carefully final courts x|
+3840|46|9|2|12|11352.48|0.04|0.07|N|O|1998-10-02|1998-08-19|1998-10-20|TAKE BACK RETURN|RAIL|xpress pinto beans. accounts a|
+3840|73|4|3|45|43788.15|0.02|0.05|N|O|1998-10-12|1998-10-12|1998-10-28|TAKE BACK RETURN|FOB|onic, even packages are. pe|
+3840|148|9|4|41|42973.74|0.07|0.02|N|O|1998-07-21|1998-10-08|1998-08-01|TAKE BACK RETURN|MAIL| nag slyly? slyly pending accounts |
+3840|173|3|5|7|7512.19|0.09|0.08|N|O|1998-09-17|1998-09-20|1998-10-14|DELIVER IN PERSON|MAIL|. furiously final gifts sleep carefully pin|
+3840|107|8|6|33|33234.30|0.10|0.02|N|O|1998-07-29|1998-10-06|1998-08-04|DELIVER IN PERSON|SHIP|hely silent deposits w|
+3841|157|5|1|1|1057.15|0.06|0.03|A|F|1994-10-10|1994-11-12|1994-10-21|DELIVER IN PERSON|AIR| boost even re|
+3841|21|10|2|31|28551.62|0.09|0.03|A|F|1995-01-24|1994-11-25|1995-02-20|TAKE BACK RETURN|SHIP|n theodolites shall promise carefully. qui|
+3841|152|10|3|40|42086.00|0.06|0.02|A|F|1995-02-02|1994-11-30|1995-02-14|TAKE BACK RETURN|MAIL|its. quickly regular ideas nag carefully|
+3841|50|1|4|9|8550.45|0.10|0.07|A|F|1994-11-21|1994-12-26|1994-11-26|NONE|FOB|s according to the courts shall nag s|
+3841|176|7|5|3|3228.51|0.04|0.02|R|F|1994-10-24|1994-12-07|1994-11-09|COLLECT COD|FOB|foxes integrate |
+3841|163|8|6|48|51031.68|0.03|0.00|R|F|1994-11-23|1994-11-22|1994-12-01|DELIVER IN PERSON|FOB| according to the regular, |
+3842|162|7|1|28|29740.48|0.05|0.07|A|F|1992-06-17|1992-06-03|1992-06-24|DELIVER IN PERSON|TRUCK|s excuses thrash carefully.|
+3842|122|1|2|21|21464.52|0.07|0.05|R|F|1992-07-15|1992-06-02|1992-07-21|NONE|RAIL|r pinto be|
+3842|194|7|3|28|30637.32|0.00|0.00|A|F|1992-06-20|1992-05-22|1992-07-13|DELIVER IN PERSON|MAIL|lly alongside of the|
+3842|88|9|4|15|14821.20|0.07|0.01|A|F|1992-06-26|1992-06-23|1992-07-09|COLLECT COD|MAIL|ave packages are slyl|
+3842|68|3|5|13|12584.78|0.09|0.02|R|F|1992-04-13|1992-06-22|1992-05-11|COLLECT COD|RAIL|t blithely. busily regular accounts alon|
+3842|107|4|6|24|24170.40|0.08|0.08|R|F|1992-08-05|1992-06-29|1992-08-16|TAKE BACK RETURN|MAIL|phins are quickly|
+3843|15|6|1|7|6405.07|0.10|0.03|N|O|1997-02-13|1997-02-21|1997-02-20|TAKE BACK RETURN|SHIP|slyly even instructions. furiously eve|
+3843|1|4|2|30|27030.00|0.01|0.05|N|O|1997-02-14|1997-03-25|1997-03-13|DELIVER IN PERSON|AIR| wake. slyly even packages boost |
+3844|135|1|1|2|2070.26|0.03|0.07|R|F|1995-02-24|1995-02-03|1995-03-18|TAKE BACK RETURN|AIR|es haggle final acco|
+3844|102|7|2|5|5010.50|0.10|0.03|R|F|1995-04-29|1995-02-24|1995-05-05|TAKE BACK RETURN|RAIL| unwind quickly about the pending, i|
+3845|34|5|1|44|41097.32|0.01|0.08|A|F|1992-07-20|1992-07-15|1992-07-24|DELIVER IN PERSON|REG AIR|s haggle among the fluffily regula|
+3845|24|7|2|16|14784.32|0.09|0.05|A|F|1992-08-08|1992-06-08|1992-08-26|DELIVER IN PERSON|SHIP|ely bold ideas use. ex|
+3845|59|1|3|17|16303.85|0.08|0.01|A|F|1992-06-12|1992-07-05|1992-06-26|TAKE BACK RETURN|RAIL|counts haggle. reg|
+3845|46|9|4|1|946.04|0.04|0.05|R|F|1992-05-21|1992-06-07|1992-06-17|COLLECT COD|REG AIR| blithely ironic t|
+3845|196|7|5|27|29597.13|0.00|0.05|R|F|1992-08-20|1992-07-17|1992-09-02|COLLECT COD|REG AIR|kages. care|
+3845|105|8|6|30|30153.00|0.09|0.06|R|F|1992-08-21|1992-07-07|1992-08-25|COLLECT COD|FOB|counts do wake blithely. ironic requests |
+3846|61|10|1|15|14415.90|0.06|0.03|N|O|1998-02-17|1998-04-27|1998-02-21|NONE|REG AIR|uternes. carefully even|
+3846|171|2|2|30|32135.10|0.08|0.07|N|O|1998-05-01|1998-03-12|1998-05-20|TAKE BACK RETURN|FOB|deposits according to the fur|
+3846|15|5|3|49|44835.49|0.08|0.07|N|O|1998-02-14|1998-03-22|1998-02-17|DELIVER IN PERSON|RAIL|efully even packages against the blithe|
+3846|165|10|4|33|35150.28|0.05|0.00|N|O|1998-05-12|1998-03-14|1998-05-14|DELIVER IN PERSON|TRUCK|s instructions are. fu|
+3847|189|10|1|7|7624.26|0.08|0.00|A|F|1993-05-06|1993-06-06|1993-05-22|COLLECT COD|MAIL| about the blithely daring Tiresias. fl|
+3872|181|2|1|28|30273.04|0.10|0.04|N|O|1996-11-05|1996-11-10|1996-11-24|DELIVER IN PERSON|REG AIR|t after the carefully ironic excuses. f|
+3872|17|4|2|38|34846.38|0.04|0.05|N|O|1996-10-18|1996-12-03|1996-11-15|TAKE BACK RETURN|AIR|iously against the ironic, unusual a|
+3872|169|4|3|18|19244.88|0.07|0.07|N|O|1996-12-25|1996-10-24|1997-01-08|TAKE BACK RETURN|SHIP|s. regular, brave accounts sleep blith|
+3872|11|2|4|41|37351.41|0.07|0.03|N|O|1996-11-23|1996-11-12|1996-12-03|COLLECT COD|REG AIR|ly regular epitaphs boost|
+3872|70|7|5|42|40742.94|0.03|0.00|N|O|1997-01-03|1996-10-12|1997-01-16|COLLECT COD|MAIL|s the furio|
+3872|140|6|6|40|41605.60|0.07|0.05|N|O|1997-01-02|1996-10-29|1997-01-14|NONE|REG AIR|nts? regularly ironic ex|
+3873|68|3|1|19|18393.14|0.04|0.04|N|O|1998-05-15|1998-05-10|1998-05-17|NONE|FOB|y final ac|
+3873|145|8|2|44|45986.16|0.05|0.05|N|O|1998-07-23|1998-05-22|1998-08-14|COLLECT COD|AIR|yly even platelets wake. |
+3873|140|6|3|29|30164.06|0.01|0.04|N|O|1998-06-22|1998-05-20|1998-07-05|COLLECT COD|REG AIR|olphins af|
+3874|170|7|1|21|22473.57|0.09|0.08|R|F|1993-06-19|1993-07-20|1993-07-08|DELIVER IN PERSON|SHIP| requests cajole fluff|
+3874|19|6|2|48|44112.48|0.06|0.07|R|F|1993-06-13|1993-07-20|1993-06-20|NONE|RAIL| ideas throughout |
+3875|81|2|1|24|23545.92|0.02|0.08|N|O|1997-10-15|1997-11-27|1997-11-09|COLLECT COD|AIR|ecial packages. |
+3875|113|7|2|49|49642.39|0.04|0.04|N|O|1997-10-18|1997-10-13|1997-10-19|NONE|MAIL|sleep furiously about the deposits. quickl|
+3876|141|8|1|12|12493.68|0.06|0.07|N|O|1996-09-16|1996-10-23|1996-10-05|TAKE BACK RETURN|REG AIR|y above the pending tithes. blithely ironi|
+3876|140|6|2|37|38485.18|0.00|0.03|N|O|1996-11-30|1996-10-18|1996-12-18|DELIVER IN PERSON|AIR|t dependencies. blithely final packages u|
+3876|127|8|3|41|42111.92|0.02|0.04|N|O|1996-10-15|1996-10-17|1996-10-19|NONE|AIR| quickly blit|
+3877|50|7|1|12|11400.60|0.06|0.01|R|F|1993-05-30|1993-08-09|1993-06-24|TAKE BACK RETURN|FOB|nal requests. even requests are. pac|
+3877|145|4|2|47|49121.58|0.05|0.00|A|F|1993-08-01|1993-08-16|1993-08-04|NONE|FOB|furiously quick requests nag along the theo|
+3877|80|8|3|44|43123.52|0.09|0.00|A|F|1993-06-07|1993-07-15|1993-07-06|DELIVER IN PERSON|REG AIR|elets. quickly regular accounts caj|
+3877|148|9|4|36|37733.04|0.06|0.01|A|F|1993-07-27|1993-07-13|1993-08-11|DELIVER IN PERSON|AIR|lithely about the dogged ideas. ac|
+3877|5|6|5|41|37105.00|0.03|0.07|A|F|1993-06-30|1993-07-20|1993-07-01|DELIVER IN PERSON|FOB|integrate against the expres|
+3877|123|4|6|7|7161.84|0.04|0.08|R|F|1993-06-14|1993-07-09|1993-06-28|NONE|TRUCK|lar dolphins cajole silently |
+3878|200|1|1|6|6601.20|0.07|0.04|N|O|1997-06-21|1997-05-22|1997-07-01|COLLECT COD|FOB|s. regular instru|
+3878|88|9|2|13|12845.04|0.01|0.06|N|O|1997-06-08|1997-06-03|1997-06-25|TAKE BACK RETURN|TRUCK|leep ruthlessly about the carefu|
+3878|41|8|3|20|18820.80|0.08|0.03|N|O|1997-06-20|1997-05-24|1997-07-20|TAKE BACK RETURN|MAIL|the furiously careful ideas cajole slyly sl|
+3878|152|3|4|20|21043.00|0.01|0.07|N|O|1997-07-13|1997-05-22|1997-07-20|NONE|FOB|about the carefully ironic pa|
+3879|126|5|1|45|46175.40|0.10|0.08|N|O|1996-03-18|1996-01-03|1996-04-03|COLLECT COD|RAIL|ly according to the expr|
+3879|45|4|2|35|33076.40|0.00|0.07|N|O|1995-12-08|1996-01-23|1995-12-28|TAKE BACK RETURN|MAIL|o beans. accounts cajole furiously. re|
+3904|38|4|1|22|20636.66|0.04|0.03|N|O|1998-02-02|1998-02-09|1998-02-10|TAKE BACK RETURN|REG AIR|structions cajole carefully. carefully f|
+3904|184|5|2|19|20599.42|0.09|0.01|N|O|1998-02-10|1998-02-13|1998-02-20|TAKE BACK RETURN|AIR| excuses sleep slyly according to th|
+3905|101|8|1|43|43047.30|0.07|0.08|A|F|1994-03-30|1994-02-18|1994-04-09|DELIVER IN PERSON|REG AIR|uses are care|
+3905|116|10|2|7|7112.77|0.03|0.00|R|F|1994-03-01|1994-02-19|1994-03-11|DELIVER IN PERSON|AIR|ully furiously furious packag|
+3905|170|7|3|6|6421.02|0.07|0.02|R|F|1994-04-07|1994-03-07|1994-04-21|DELIVER IN PERSON|RAIL|ow furiously. deposits wake ironic |
+3906|153|1|1|42|44232.30|0.00|0.04|R|F|1992-09-03|1992-07-22|1992-09-04|COLLECT COD|RAIL|jole blithely after the furiously regular |
+3906|40|1|2|50|47002.00|0.01|0.07|R|F|1992-09-24|1992-08-24|1992-09-29|NONE|MAIL|ke slyly. stealt|
+3906|180|9|3|15|16202.70|0.06|0.02|R|F|1992-07-30|1992-08-26|1992-08-02|TAKE BACK RETURN|FOB|dependencies at the |
+3906|59|10|4|36|34525.80|0.08|0.08|A|F|1992-08-07|1992-08-08|1992-08-24|NONE|SHIP|y. ironic deposits haggle sl|
+3907|112|6|1|41|41496.51|0.06|0.02|A|F|1992-09-13|1992-10-23|1992-09-29|COLLECT COD|MAIL|ackages wake along the carefully regul|
+3907|145|4|2|41|42850.74|0.03|0.00|A|F|1992-10-25|1992-10-17|1992-11-01|TAKE BACK RETURN|RAIL|s above the unusual ideas sleep furiousl|
+3907|52|4|3|45|42842.25|0.02|0.07|R|F|1992-09-21|1992-09-19|1992-10-18|COLLECT COD|REG AIR| about the regular pac|
+3907|176|5|4|48|51656.16|0.05|0.07|A|F|1992-09-24|1992-10-16|1992-10-06|DELIVER IN PERSON|TRUCK|nt asymptotes lose across th|
+3907|62|3|5|22|21165.32|0.09|0.01|R|F|1992-09-20|1992-10-30|1992-09-29|TAKE BACK RETURN|TRUCK|ly. furiously unusual deposits use afte|
+3907|126|9|6|34|34888.08|0.02|0.02|R|F|1992-09-06|1992-10-08|1992-09-12|COLLECT COD|FOB| requests according to the slyly pending |
+3907|110|5|7|8|8080.88|0.10|0.01|A|F|1992-09-18|1992-10-29|1992-09-27|NONE|REG AIR|furiously final packages.|
+3908|92|4|1|50|49604.50|0.05|0.04|R|F|1993-06-19|1993-04-27|1993-07-05|DELIVER IN PERSON|MAIL| even accounts wake |
+3908|148|9|2|8|8385.12|0.06|0.03|A|F|1993-03-12|1993-04-13|1993-03-22|DELIVER IN PERSON|SHIP|r instructions was requests. ironically |
+3909|178|6|1|30|32345.10|0.03|0.07|N|O|1998-10-17|1998-10-14|1998-10-28|COLLECT COD|TRUCK|ly even deposits across the ironic notorni|
+3909|191|4|2|46|50194.74|0.03|0.01|N|O|1998-10-08|1998-10-15|1998-10-24|NONE|FOB|the blithely unusual ideas|
+3910|139|10|1|10|10391.30|0.00|0.08|N|O|1996-10-18|1996-10-31|1996-11-14|DELIVER IN PERSON|FOB|tions boost furiously unusual e|
+3910|71|10|2|31|30103.17|0.05|0.03|N|O|1996-12-22|1996-11-14|1997-01-01|TAKE BACK RETURN|SHIP|ess instructions. |
+3910|20|7|3|6|5520.12|0.04|0.04|N|O|1996-12-08|1996-10-30|1996-12-31|DELIVER IN PERSON|MAIL|ly sly platelets are fluffily slyly si|
+3910|153|1|4|1|1053.15|0.03|0.06|N|O|1996-09-12|1996-10-21|1996-09-19|DELIVER IN PERSON|FOB|s sleep neve|
+3911|113|7|1|10|10131.10|0.07|0.06|N|O|1995-06-22|1995-05-30|1995-06-28|COLLECT COD|FOB|ss theodolites are blithely along t|
+3911|119|9|2|14|14267.54|0.08|0.05|R|F|1995-04-28|1995-05-03|1995-05-22|NONE|RAIL|e blithely brave depo|
+3911|92|5|3|12|11905.08|0.10|0.05|R|F|1995-04-04|1995-04-16|1995-04-10|COLLECT COD|FOB|uctions. blithely regula|
+3936|137|8|1|25|25928.25|0.06|0.03|N|O|1996-12-03|1996-12-27|1997-01-01|DELIVER IN PERSON|RAIL|gular requests nag quic|
+3936|188|9|2|24|26116.32|0.10|0.07|N|O|1996-11-22|1997-01-01|1996-12-08|NONE|AIR|ns. accounts mold fl|
+3936|83|4|3|42|41289.36|0.00|0.07|N|O|1997-01-03|1997-01-29|1997-01-14|COLLECT COD|AIR|elets wake amo|
+3936|62|7|4|12|11544.72|0.06|0.05|N|O|1996-11-25|1997-01-09|1996-12-06|DELIVER IN PERSON|SHIP|ithely across the carefully brave req|
+3936|84|5|5|35|34442.80|0.02|0.08|N|O|1996-12-04|1997-01-06|1996-12-22|NONE|SHIP|lly ironic requ|
+3936|103|6|6|26|26080.60|0.01|0.02|N|O|1997-02-27|1997-01-16|1997-03-22|NONE|RAIL|quickly pen|
+3937|70|7|1|48|46563.36|0.10|0.02|N|O|1998-03-15|1998-02-22|1998-03-30|DELIVER IN PERSON|FOB|gainst the thinl|
+3937|48|1|2|30|28441.20|0.01|0.07|N|O|1998-01-17|1998-01-03|1998-02-08|COLLECT COD|TRUCK|al packages slee|
+3937|115|5|3|27|27407.97|0.03|0.00|N|O|1998-02-06|1998-01-12|1998-02-20|NONE|MAIL|ven ideas. slyly expr|
+3937|154|2|4|50|52707.50|0.01|0.02|N|O|1998-01-15|1998-01-09|1998-02-04|DELIVER IN PERSON|AIR|ong the carefully exp|
+3937|3|10|5|29|26187.00|0.03|0.07|N|O|1998-03-06|1998-02-22|1998-03-14|NONE|TRUCK|nt pinto beans above the pending instr|
+3937|193|6|6|6|6559.14|0.00|0.00|N|O|1998-01-24|1998-02-13|1998-01-27|DELIVER IN PERSON|FOB|into beans. slyly silent orbits alongside o|
+3937|164|9|7|1|1064.16|0.02|0.05|N|O|1998-03-29|1998-01-08|1998-04-27|TAKE BACK RETURN|TRUCK|refully agains|
+3938|159|4|1|46|48720.90|0.10|0.07|R|F|1993-05-20|1993-05-04|1993-06-12|DELIVER IN PERSON|FOB|ly even foxes are slyly fu|
+3939|160|8|1|8|8481.28|0.03|0.06|N|O|1996-01-29|1996-04-05|1996-02-26|COLLECT COD|REG AIR|e packages. express, pen|
+3940|178|7|1|33|35579.61|0.10|0.07|N|O|1996-05-19|1996-04-19|1996-05-23|TAKE BACK RETURN|RAIL|ly ironic packages about the pending accou|
+3940|69|4|2|40|38762.40|0.08|0.02|N|O|1996-02-29|1996-03-22|1996-03-04|NONE|MAIL|ts. regular fox|
+3940|89|10|3|8|7912.64|0.07|0.08|N|O|1996-04-04|1996-04-12|1996-04-18|DELIVER IN PERSON|RAIL|ions cajole furiously regular pinto beans. |
+3940|137|3|4|11|11408.43|0.09|0.05|N|O|1996-03-09|1996-05-13|1996-03-17|COLLECT COD|REG AIR|e of the special packages. furiously|
+3940|1|6|5|41|36941.00|0.00|0.07|N|O|1996-05-08|1996-05-03|1996-06-03|COLLECT COD|MAIL|thily. deposits cajole.|
+3941|41|2|1|47|44228.88|0.05|0.07|N|O|1996-11-24|1996-10-09|1996-12-22|DELIVER IN PERSON|RAIL| carefully pending|
+3941|123|6|2|19|19439.28|0.05|0.00|N|O|1996-11-10|1996-10-26|1996-12-05|COLLECT COD|RAIL|eposits haggle furiously even|
+3941|10|3|3|2|1820.02|0.01|0.03|N|O|1996-12-04|1996-10-01|1996-12-25|NONE|REG AIR|es wake after the|
+3941|110|7|4|29|29293.19|0.00|0.03|N|O|1996-09-14|1996-10-04|1996-09-19|NONE|MAIL|g the blithely|
+3942|183|4|1|6|6499.08|0.05|0.05|A|F|1993-07-01|1993-09-14|1993-07-23|DELIVER IN PERSON|SHIP|ep ruthlessly carefully final accounts: s|
+3942|194|7|2|5|5470.95|0.06|0.02|R|F|1993-09-27|1993-09-24|1993-10-07|DELIVER IN PERSON|MAIL|. fluffily pending deposits above the flu|
+3942|156|4|3|25|26403.75|0.04|0.06|R|F|1993-09-13|1993-08-01|1993-09-29|COLLECT COD|RAIL|d the quick packages|
+3943|198|2|1|15|16472.85|0.03|0.01|N|O|1997-01-13|1996-12-17|1997-02-02|COLLECT COD|REG AIR| grow fluffily according to the |
+3943|96|7|2|9|8964.81|0.00|0.06|N|O|1996-11-27|1997-01-03|1996-12-17|COLLECT COD|RAIL|refully ironic |
+3943|17|4|3|32|29344.32|0.00|0.02|N|O|1996-10-22|1996-12-17|1996-11-04|TAKE BACK RETURN|TRUCK| unusual ideas into the furiously even pack|
+3943|50|1|4|5|4750.25|0.04|0.04|N|O|1997-01-09|1996-11-10|1997-02-06|COLLECT COD|RAIL|arefully regular deposits accord|
+3968|54|2|1|27|25759.35|0.04|0.05|N|O|1997-04-25|1997-04-17|1997-05-11|TAKE BACK RETURN|MAIL|t silently.|
+3968|26|9|2|45|41670.90|0.00|0.07|N|O|1997-06-18|1997-04-24|1997-06-25|DELIVER IN PERSON|FOB|ully slyly fi|
+3968|156|7|3|43|45414.45|0.07|0.06|N|O|1997-04-30|1997-05-14|1997-05-18|TAKE BACK RETURN|SHIP|ly regular accounts|
+3968|61|8|4|7|6727.42|0.07|0.02|N|O|1997-03-30|1997-05-01|1997-04-12|DELIVER IN PERSON|SHIP|efully bold instructions. express|
+3969|52|4|1|39|37129.95|0.04|0.04|N|O|1997-06-12|1997-06-13|1997-07-05|NONE|MAIL|ly bold ideas s|
+3969|197|1|2|26|28526.94|0.05|0.03|N|O|1997-07-08|1997-07-30|1997-07-10|TAKE BACK RETURN|AIR|fluffily; braids detect.|
+3969|79|8|3|46|45037.22|0.04|0.02|N|O|1997-05-29|1997-06-15|1997-06-10|TAKE BACK RETURN|SHIP|fully final requests sleep stealthily. care|
+3969|151|9|4|21|22074.15|0.07|0.04|N|O|1997-08-31|1997-07-16|1997-09-02|TAKE BACK RETURN|MAIL|unts doze quickly final reque|
+3969|72|3|5|40|38882.80|0.09|0.00|N|O|1997-05-19|1997-08-02|1997-06-05|COLLECT COD|TRUCK|lar requests cajole furiously blithely regu|
+3969|105|8|6|4|4020.40|0.02|0.01|N|O|1997-06-04|1997-07-31|1997-06-13|COLLECT COD|REG AIR|dencies wake blithely? quickly even theodo|
+3970|88|9|1|2|1976.16|0.01|0.07|R|F|1992-04-24|1992-06-03|1992-05-16|TAKE BACK RETURN|RAIL|carefully pending foxes wake blithely |
+3970|109|6|2|18|18163.80|0.03|0.08|A|F|1992-06-06|1992-06-18|1992-07-05|DELIVER IN PERSON|TRUCK| maintain slyly. ir|
+3970|154|6|3|10|10541.50|0.10|0.04|A|F|1992-07-01|1992-05-31|1992-07-02|NONE|AIR| special packages wake after the final br|
+3970|22|5|4|34|31348.68|0.05|0.00|A|F|1992-06-25|1992-05-23|1992-07-12|COLLECT COD|SHIP|y final gifts are. carefully pe|
+3970|30|3|5|23|21390.69|0.05|0.04|A|F|1992-06-04|1992-06-14|1992-06-13|COLLECT COD|TRUCK| above the final braids. regular|
+3970|9|6|6|46|41814.00|0.07|0.04|R|F|1992-04-29|1992-05-14|1992-05-24|NONE|FOB|yly ironic|
+3970|5|8|7|46|41630.00|0.08|0.08|R|F|1992-05-02|1992-05-12|1992-05-10|COLLECT COD|MAIL|ix slyly. quickly silen|
+3971|96|8|1|47|46816.23|0.06|0.04|N|O|1996-07-07|1996-08-08|1996-08-01|TAKE BACK RETURN|RAIL|e slyly final dependencies x-ray |
+3971|191|5|2|2|2182.38|0.04|0.03|N|O|1996-07-15|1996-08-12|1996-07-26|NONE|SHIP|haggle abou|
+3972|51|3|1|2|1902.10|0.05|0.03|A|F|1994-07-24|1994-06-30|1994-08-13|TAKE BACK RETURN|SHIP|y final theodolite|
+3973|30|9|1|21|19530.63|0.02|0.06|R|F|1992-06-18|1992-06-03|1992-07-02|COLLECT COD|REG AIR|equests. furiously|
+3973|115|2|2|37|37559.07|0.07|0.00|A|F|1992-05-29|1992-05-04|1992-06-23|TAKE BACK RETURN|SHIP|inos wake fluffily. pending requests nag |
+3973|40|6|3|40|37601.60|0.08|0.05|R|F|1992-05-03|1992-06-09|1992-05-21|COLLECT COD|RAIL|g the carefully blithe f|
+3974|22|1|1|47|43334.94|0.10|0.03|N|O|1996-06-03|1996-05-08|1996-06-28|NONE|TRUCK|dencies above the re|
+3974|61|8|2|17|16338.02|0.05|0.07|N|O|1996-04-05|1996-05-21|1996-04-28|COLLECT COD|TRUCK|ions eat slyly after the blithely |
+3975|57|9|1|38|36367.90|0.01|0.05|N|O|1995-08-02|1995-06-18|1995-08-19|COLLECT COD|TRUCK|es are furiously: furi|
+4000|196|7|1|41|44943.79|0.06|0.01|A|F|1992-03-02|1992-03-14|1992-03-27|COLLECT COD|FOB|ve the even, fi|
+4000|75|5|2|44|42903.08|0.09|0.06|A|F|1992-03-27|1992-02-18|1992-03-31|COLLECT COD|AIR|equests use blithely blithely bold d|
+4001|106|1|1|26|26158.60|0.00|0.01|N|O|1997-07-26|1997-06-18|1997-08-08|DELIVER IN PERSON|RAIL|tegrate blithely|
+4001|41|10|2|19|17879.76|0.03|0.02|N|O|1997-08-23|1997-06-15|1997-09-18|COLLECT COD|SHIP|ackages. carefully ironi|
+4001|94|5|3|18|17893.62|0.07|0.00|N|O|1997-06-04|1997-06-22|1997-06-13|DELIVER IN PERSON|MAIL|lithely ironic d|
+4001|2|9|4|39|35178.00|0.00|0.00|N|O|1997-06-13|1997-06-17|1997-06-25|NONE|SHIP| dogged excuses. blithe|
+4002|111|5|1|35|35388.85|0.01|0.08|N|O|1997-05-16|1997-06-15|1997-06-02|DELIVER IN PERSON|TRUCK|eep. quickly|
+4002|198|9|2|20|21963.80|0.00|0.03|N|O|1997-06-15|1997-05-20|1997-07-11|NONE|MAIL|lly even ins|
+4002|40|1|3|6|5640.24|0.08|0.07|N|O|1997-05-02|1997-07-07|1997-05-16|TAKE BACK RETURN|RAIL| furiously furiously special theodoli|
+4002|199|3|4|6|6595.14|0.06|0.06|N|O|1997-07-01|1997-05-15|1997-07-31|NONE|MAIL|he slyly iro|
+4002|99|1|5|4|3996.36|0.08|0.07|N|O|1997-05-06|1997-06-15|1997-05-24|NONE|REG AIR|ccording to the careful|
+4003|52|4|1|18|17136.90|0.04|0.07|R|F|1993-02-02|1993-04-15|1993-02-28|TAKE BACK RETURN|AIR|ar grouches s|
+4004|121|2|1|23|23485.76|0.07|0.02|A|F|1993-08-12|1993-07-13|1993-08-16|TAKE BACK RETURN|TRUCK| bold theodolites? special packages accordi|
+4004|64|5|2|47|45310.82|0.07|0.04|R|F|1993-06-25|1993-08-03|1993-07-12|NONE|SHIP|thely instead of the even, unu|
+4004|114|5|3|39|39550.29|0.10|0.05|R|F|1993-07-12|1993-07-27|1993-07-18|NONE|MAIL|ccounts sleep furious|
+4004|74|4|4|46|44807.22|0.10|0.04|R|F|1993-09-04|1993-07-13|1993-09-28|COLLECT COD|FOB|ncies. slyly pending dolphins sleep furio|
+4004|155|3|5|9|9496.35|0.04|0.06|A|F|1993-08-25|1993-06-10|1993-09-24|COLLECT COD|MAIL|ly ironic requests. quickly pending ide|
+4004|161|10|6|44|46691.04|0.07|0.05|R|F|1993-07-25|1993-07-23|1993-08-16|TAKE BACK RETURN|REG AIR|ut the sauternes. bold, ironi|
+4004|126|9|7|20|20522.40|0.07|0.05|A|F|1993-06-19|1993-06-14|1993-07-04|COLLECT COD|REG AIR|. ironic deposits cajole blithely?|
+4005|4|1|1|26|23504.00|0.09|0.05|N|O|1996-12-01|1997-02-03|1996-12-15|NONE|REG AIR| to the quic|
+4005|17|8|2|28|25676.28|0.02|0.06|N|O|1996-12-11|1997-01-24|1996-12-17|DELIVER IN PERSON|REG AIR|ly carefully ironic deposits. slyly|
+4005|72|10|3|28|27217.96|0.03|0.01|N|O|1996-12-08|1997-01-14|1996-12-30|TAKE BACK RETURN|MAIL|y pending dependenc|
+4005|15|9|4|49|44835.49|0.09|0.00|N|O|1997-01-31|1996-12-24|1997-03-02|NONE|RAIL|tions sleep across the silent d|
+4005|6|7|5|14|12684.00|0.09|0.08|N|O|1996-11-27|1997-01-09|1996-12-25|NONE|TRUCK|ld requests. slyly final instructi|
+4006|55|7|1|11|10505.55|0.05|0.08|A|F|1995-04-29|1995-02-21|1995-05-20|TAKE BACK RETURN|RAIL|ress foxes cajole quick|
+4006|159|4|2|18|19064.70|0.05|0.03|A|F|1995-01-29|1995-03-08|1995-02-02|TAKE BACK RETURN|MAIL|gouts! slyly iron|
+4006|24|5|3|15|13860.30|0.01|0.02|R|F|1995-02-23|1995-04-02|1995-02-25|TAKE BACK RETURN|RAIL|n deposits cajole slyl|
+4006|114|5|4|25|25352.75|0.00|0.07|A|F|1995-02-23|1995-02-09|1995-02-24|DELIVER IN PERSON|SHIP| requests use depos|
+4007|57|2|1|32|30625.60|0.00|0.03|R|F|1993-09-30|1993-08-16|1993-10-03|DELIVER IN PERSON|RAIL|nal accounts across t|
+4007|116|10|2|41|41660.51|0.04|0.06|A|F|1993-10-11|1993-08-30|1993-11-04|DELIVER IN PERSON|TRUCK|eposits. regular epitaphs boost blithely.|
+4007|102|9|3|5|5010.50|0.09|0.06|A|F|1993-09-17|1993-08-29|1993-10-12|TAKE BACK RETURN|FOB|y unusual packa|
+4007|138|4|4|15|15571.95|0.05|0.02|A|F|1993-09-01|1993-07-19|1993-09-03|DELIVER IN PERSON|FOB|le furiously quickly |
+4007|26|7|5|23|21298.46|0.02|0.07|A|F|1993-10-08|1993-09-09|1993-10-23|COLLECT COD|MAIL|ter the accounts. expr|
+4032|102|3|1|8|8016.80|0.06|0.00|N|O|1998-06-04|1998-05-17|1998-07-03|TAKE BACK RETURN|RAIL|ometimes even cou|
+4032|2|9|2|27|24354.00|0.09|0.00|N|O|1998-05-31|1998-04-19|1998-06-24|COLLECT COD|REG AIR|le furiously according to|
+4032|154|2|3|23|24245.45|0.09|0.06|N|O|1998-06-12|1998-05-11|1998-06-24|COLLECT COD|MAIL|ording to the |
+4032|85|6|4|10|9850.80|0.09|0.05|N|O|1998-03-31|1998-04-22|1998-04-07|NONE|REG AIR| carefully bol|
+4033|110|1|1|27|27272.97|0.01|0.04|R|F|1993-08-08|1993-08-14|1993-08-09|NONE|AIR|pinto beans|
+4033|38|4|2|34|31893.02|0.07|0.00|R|F|1993-07-19|1993-08-05|1993-07-26|NONE|RAIL|t the blithely dogg|
+4034|190|1|1|48|52329.12|0.03|0.03|A|F|1994-03-01|1994-01-16|1994-03-16|NONE|RAIL| blithely regular requests play carefull|
+4034|57|5|2|47|44981.35|0.07|0.05|A|F|1994-01-27|1993-12-26|1994-02-04|NONE|TRUCK|eodolites was slyly ironic ideas. de|
+4034|54|5|3|43|41024.15|0.10|0.03|A|F|1993-11-29|1994-01-08|1993-12-10|DELIVER IN PERSON|FOB|posits wake carefully af|
+4034|28|9|4|46|42688.92|0.06|0.00|A|F|1994-02-22|1994-01-09|1994-03-04|DELIVER IN PERSON|AIR|uests. furiously unusual instructions wake|
+4034|196|10|5|7|7673.33|0.07|0.06|R|F|1994-03-04|1994-01-22|1994-04-01|NONE|AIR|y even theodolites. slyly regular instru|
+4034|50|9|6|5|4750.25|0.01|0.06|A|F|1994-02-12|1994-01-24|1994-02-13|COLLECT COD|AIR|fully around the furiously ironic re|
+4035|97|8|1|4|3988.36|0.08|0.03|R|F|1992-04-21|1992-04-23|1992-04-25|COLLECT COD|AIR|ilent, even pear|
+4035|136|7|2|4|4144.52|0.07|0.00|A|F|1992-05-21|1992-04-24|1992-05-24|DELIVER IN PERSON|FOB|en instructions sleep blith|
+4035|118|8|3|1|1018.11|0.03|0.01|R|F|1992-06-18|1992-05-19|1992-07-02|COLLECT COD|FOB| requests. quickly |
+4035|182|3|4|13|14068.34|0.00|0.01|R|F|1992-06-10|1992-05-16|1992-07-10|NONE|SHIP|s. furiously even courts wake slyly|
+4036|6|1|1|46|41676.00|0.09|0.00|N|O|1997-06-21|1997-05-29|1997-07-18|NONE|REG AIR|usly across the even th|
+4036|53|1|2|21|20014.05|0.09|0.07|N|O|1997-08-08|1997-06-28|1997-08-09|COLLECT COD|MAIL|e carefully. qui|
+4036|142|3|3|6|6252.84|0.07|0.01|N|O|1997-06-19|1997-06-16|1997-07-01|DELIVER IN PERSON|SHIP|equests wake about the bold id|
+4036|127|10|4|20|20542.40|0.08|0.02|N|O|1997-08-11|1997-07-11|1997-09-03|NONE|TRUCK|slyly bold deposits cajole pending, blithe|
+4037|64|9|1|32|30849.92|0.00|0.06|A|F|1993-05-06|1993-06-08|1993-05-31|DELIVER IN PERSON|AIR|e of the pending, iron|
+4037|47|8|2|4|3788.16|0.09|0.07|A|F|1993-07-05|1993-06-12|1993-08-03|DELIVER IN PERSON|RAIL|s around the blithely ironic ac|
+4038|196|10|1|40|43847.60|0.05|0.01|N|O|1996-01-15|1996-03-13|1996-01-25|COLLECT COD|TRUCK|t. slyly silent pinto beans amo|
+4038|12|9|2|37|33744.37|0.04|0.03|N|O|1996-03-17|1996-03-19|1996-04-07|DELIVER IN PERSON|REG AIR| packages |
+4038|32|3|3|24|22368.72|0.10|0.04|N|O|1996-04-06|1996-02-15|1996-04-18|TAKE BACK RETURN|RAIL|the furiously regu|
+4038|150|1|4|29|30454.35|0.07|0.06|N|O|1996-01-07|1996-03-08|1996-01-13|NONE|FOB|ffix. quietly ironic packages a|
+4038|79|7|5|24|23497.68|0.07|0.06|N|O|1996-04-01|1996-04-05|1996-04-28|DELIVER IN PERSON|TRUCK|ake quickly after the final, ironic ac|
+4038|36|2|6|6|5616.18|0.07|0.05|N|O|1996-02-09|1996-03-05|1996-03-10|COLLECT COD|SHIP| special instructions. packa|
+4039|94|5|1|38|37775.42|0.03|0.06|N|O|1998-03-09|1997-12-31|1998-03-21|DELIVER IN PERSON|REG AIR|sual asymptotes. ironic deposits nag aft|
+4039|122|5|2|17|17376.04|0.10|0.04|N|O|1998-01-15|1998-01-20|1998-01-28|TAKE BACK RETURN|MAIL| regular foxes haggle carefully bo|
+4039|64|1|3|9|8676.54|0.10|0.01|N|O|1998-03-08|1998-02-05|1998-04-05|TAKE BACK RETURN|FOB|t? pinto beans cajole across the thinly r|
+4039|28|3|4|43|39904.86|0.01|0.02|N|O|1998-01-02|1997-12-22|1998-01-15|NONE|FOB|beans believe bene|
+4039|134|5|5|43|44467.59|0.09|0.00|N|O|1998-01-20|1998-01-11|1998-02-05|COLLECT COD|SHIP|sts along the regular in|
+4064|199|1|1|3|3297.57|0.10|0.04|N|O|1997-01-04|1997-01-01|1997-01-23|NONE|SHIP|its! quickly sp|
+4064|40|6|2|15|14100.60|0.02|0.02|N|O|1996-11-09|1996-12-04|1996-11-18|DELIVER IN PERSON|MAIL|braids affix across the regular sheave|
+4064|197|10|3|32|35110.08|0.04|0.07|N|O|1997-01-14|1997-01-01|1997-01-21|COLLECT COD|REG AIR|es boost. careful|
+4064|163|8|4|24|25515.84|0.02|0.02|N|O|1997-01-01|1996-12-31|1997-01-23|DELIVER IN PERSON|SHIP|ly regular ideas.|
+4064|21|2|5|12|11052.24|0.08|0.08|N|O|1997-02-08|1996-12-18|1997-03-06|TAKE BACK RETURN|RAIL|ding to the requests|
+4064|184|5|6|46|49872.28|0.03|0.00|N|O|1996-10-13|1997-01-05|1996-11-06|DELIVER IN PERSON|REG AIR|alongside of the f|
+4064|200|2|7|9|9901.80|0.01|0.06|N|O|1996-12-17|1996-12-13|1997-01-12|NONE|AIR|furiously f|
+4065|138|9|1|14|14533.82|0.04|0.02|A|F|1994-08-22|1994-07-29|1994-09-19|DELIVER IN PERSON|TRUCK|e furiously outside |
+4065|15|6|2|46|42090.46|0.03|0.05|A|F|1994-06-29|1994-08-01|1994-07-03|TAKE BACK RETURN|SHIP|, regular requests may mold above the |
+4065|97|10|3|33|32903.97|0.00|0.03|A|F|1994-09-03|1994-08-16|1994-09-13|DELIVER IN PERSON|AIR|ain blithely |
+4065|107|2|4|8|8056.80|0.00|0.01|R|F|1994-10-04|1994-08-05|1994-10-25|TAKE BACK RETURN|SHIP|ages haggle carefully|
+4065|123|4|5|29|29670.48|0.02|0.07|A|F|1994-06-29|1994-08-19|1994-07-17|NONE|RAIL|equests. packages sleep slyl|
+4065|110|5|6|16|16161.76|0.05|0.00|R|F|1994-08-25|1994-08-06|1994-09-09|COLLECT COD|TRUCK|ncies use furiously. quickly un|
+4065|144|7|7|11|11485.54|0.10|0.04|A|F|1994-07-25|1994-08-02|1994-07-30|NONE|RAIL|hang silently about |
+4066|139|5|1|9|9352.17|0.01|0.05|N|O|1997-05-06|1997-03-25|1997-05-27|COLLECT COD|FOB|nal, ironic accounts. blithel|
+4066|93|5|2|19|18868.71|0.05|0.00|N|O|1997-05-13|1997-04-17|1997-06-08|NONE|TRUCK|quests. slyly regu|
+4066|76|5|3|8|7808.56|0.03|0.03|N|O|1997-04-24|1997-03-11|1997-05-20|NONE|REG AIR|accounts. special pinto beans|
+4066|179|9|4|49|52879.33|0.01|0.01|N|O|1997-02-17|1997-03-24|1997-02-19|NONE|TRUCK|ial braids. furiously final deposits sl|
+4066|171|2|5|43|46060.31|0.05|0.02|N|O|1997-02-16|1997-04-14|1997-02-18|DELIVER IN PERSON|MAIL|r instructions. slyly special |
+4066|109|2|6|44|44400.40|0.01|0.00|N|O|1997-03-01|1997-04-27|1997-03-29|DELIVER IN PERSON|MAIL|express accounts nag bli|
+4067|180|1|1|18|19443.24|0.03|0.08|A|F|1993-01-24|1992-12-23|1993-02-20|TAKE BACK RETURN|FOB|e the slyly final packages d|
+4067|96|10|2|14|13945.26|0.00|0.00|R|F|1993-02-03|1992-12-02|1993-02-07|TAKE BACK RETURN|TRUCK|ructions. quickly ironic accounts detect |
+4067|141|10|3|17|17699.38|0.03|0.05|A|F|1993-01-26|1992-11-23|1993-01-27|NONE|REG AIR|ts haggle slyly unusual, final|
+4067|90|1|4|40|39603.60|0.07|0.08|R|F|1993-01-09|1992-11-21|1993-01-16|DELIVER IN PERSON|TRUCK|lar theodolites nag blithely above the|
+4067|85|6|5|17|16746.36|0.08|0.03|A|F|1993-01-20|1992-12-29|1993-02-03|DELIVER IN PERSON|REG AIR|r accounts. slyly special pa|
+4067|96|8|6|12|11953.08|0.04|0.03|A|F|1992-12-12|1992-11-28|1992-12-15|DELIVER IN PERSON|AIR|lly slyly even theodol|
+4067|83|4|7|17|16712.36|0.01|0.01|R|F|1992-12-12|1992-12-23|1992-12-30|NONE|AIR|ts affix. regular, regular requests s|
+4068|110|1|1|43|43434.73|0.05|0.06|N|O|1996-11-28|1996-11-16|1996-12-22|NONE|AIR|ructions. regular, special packag|
+4068|57|5|2|31|29668.55|0.08|0.03|N|O|1996-12-11|1996-12-07|1996-12-30|NONE|SHIP|ds wake carefully amon|
+4069|129|2|1|39|40135.68|0.09|0.02|R|F|1992-09-06|1992-07-22|1992-09-25|COLLECT COD|SHIP|ven theodolites nag quickly. fluffi|
+4069|43|4|2|32|30177.28|0.10|0.08|A|F|1992-06-18|1992-07-20|1992-07-07|TAKE BACK RETURN|TRUCK|unts. deposit|
+4069|186|7|3|3|3258.54|0.06|0.01|R|F|1992-07-26|1992-07-07|1992-08-04|COLLECT COD|FOB|l packages. even, |
+4069|79|8|4|22|21539.54|0.10|0.05|A|F|1992-08-05|1992-08-04|1992-08-25|COLLECT COD|SHIP|ts. slyly special instruction|
+4069|157|5|5|50|52857.50|0.09|0.06|A|F|1992-07-26|1992-06-30|1992-08-01|TAKE BACK RETURN|REG AIR|even foxes among the express wate|
+4069|125|8|6|3|3075.36|0.02|0.01|A|F|1992-05-24|1992-06-18|1992-06-12|COLLECT COD|MAIL|y final deposits wake furiously! slyl|
+4069|184|5|7|50|54209.00|0.00|0.01|R|F|1992-09-03|1992-06-14|1992-10-01|NONE|REG AIR|ages. carefully regular |
+4070|183|4|1|2|2166.36|0.09|0.08|N|O|1995-08-03|1995-09-10|1995-08-17|TAKE BACK RETURN|REG AIR|ptotes affix|
+4070|155|3|2|40|42206.00|0.07|0.07|N|O|1995-07-13|1995-07-23|1995-08-06|COLLECT COD|MAIL|about the sentiments. quick|
+4070|62|3|3|11|10582.66|0.00|0.08|N|O|1995-08-23|1995-08-15|1995-08-31|TAKE BACK RETURN|MAIL| carefully final pack|
+4070|29|4|4|46|42734.92|0.02|0.02|N|O|1995-06-22|1995-07-14|1995-07-11|DELIVER IN PERSON|REG AIR|nticing ideas. boldly|
+4071|112|2|1|22|22266.42|0.02|0.07|N|O|1996-10-31|1996-12-14|1996-11-05|NONE|REG AIR|sits cajole carefully final instructio|
+4071|18|8|2|47|43146.47|0.00|0.03|N|O|1996-11-04|1996-12-09|1996-11-16|NONE|TRUCK|ts cajole furiously along the|
+4096|27|10|1|31|28737.62|0.10|0.02|A|F|1992-07-14|1992-09-03|1992-07-31|COLLECT COD|TRUCK|y final, even platelets. boldly|
+4096|57|9|2|17|16269.85|0.07|0.03|R|F|1992-09-30|1992-08-11|1992-10-11|TAKE BACK RETURN|REG AIR|platelets alongside of the |
+4096|9|10|3|21|19089.00|0.08|0.00|A|F|1992-08-24|1992-09-04|1992-09-11|DELIVER IN PERSON|MAIL|tes mold flu|
+4096|128|3|4|20|20562.40|0.02|0.07|R|F|1992-08-24|1992-09-13|1992-08-28|DELIVER IN PERSON|TRUCK|sual requests. furiously bold packages wake|
+4097|74|5|1|50|48703.50|0.04|0.04|N|O|1996-08-31|1996-08-14|1996-09-27|DELIVER IN PERSON|MAIL|egular deposits. blithely pending|
+4097|74|4|2|46|44807.22|0.10|0.01|N|O|1996-07-29|1996-08-19|1996-08-25|COLLECT COD|AIR| even depend|
+4097|174|2|3|42|45115.14|0.06|0.06|N|O|1996-08-11|1996-07-30|1996-08-15|NONE|FOB|carefully silent foxes are against the |
+4098|200|1|1|46|50609.20|0.07|0.03|N|O|1997-01-26|1997-01-27|1997-02-13|TAKE BACK RETURN|SHIP|e slyly blithely silent deposits. fluff|
+4099|4|7|1|29|26216.00|0.09|0.07|R|F|1992-11-21|1992-11-04|1992-11-30|NONE|FOB| slowly final warthogs sleep blithely. q|
+4099|137|3|2|3|3111.39|0.04|0.06|A|F|1992-09-12|1992-10-18|1992-10-01|NONE|RAIL|. special packages sleep|
+4099|51|3|3|36|34237.80|0.06|0.06|R|F|1992-11-06|1992-09-28|1992-12-02|NONE|FOB|beans cajole slyly quickly ironic |
+4099|139|5|4|7|7273.91|0.05|0.02|A|F|1992-09-12|1992-11-13|1992-09-14|TAKE BACK RETURN|AIR|onic foxes. quickly final fox|
+4099|163|10|5|48|51031.68|0.00|0.02|R|F|1992-10-18|1992-10-14|1992-11-01|NONE|REG AIR|ts haggle according to the slyly f|
+4099|59|10|6|39|37402.95|0.07|0.02|R|F|1992-12-13|1992-11-13|1992-12-26|DELIVER IN PERSON|REG AIR|fluffy accounts impress pending, iro|
+4099|180|8|7|46|49688.28|0.06|0.07|R|F|1992-10-29|1992-11-03|1992-11-10|DELIVER IN PERSON|REG AIR|ages nag requests.|
+4100|74|5|1|4|3896.28|0.03|0.03|N|O|1996-06-20|1996-04-29|1996-06-21|TAKE BACK RETURN|FOB|lyly regular, bold requ|
+4101|115|2|1|22|22332.42|0.05|0.02|R|F|1994-02-02|1994-02-19|1994-02-12|COLLECT COD|AIR|ly express instructions. careful|
+4102|10|3|1|17|15470.17|0.02|0.02|N|O|1996-06-03|1996-05-06|1996-07-02|COLLECT COD|AIR|ly silent theodolites sleep unusual exc|
+4102|69|8|2|5|4845.30|0.08|0.02|N|O|1996-05-11|1996-05-11|1996-05-16|COLLECT COD|AIR| the furiously even|
+4102|67|4|3|39|37715.34|0.08|0.01|N|O|1996-04-14|1996-05-18|1996-04-20|DELIVER IN PERSON|AIR|ffix blithely slyly special |
+4102|140|6|4|39|40565.46|0.02|0.00|N|O|1996-06-15|1996-06-06|1996-06-30|DELIVER IN PERSON|SHIP|y among the furiously special|
+4102|1|6|5|32|28832.00|0.08|0.01|N|O|1996-05-14|1996-04-29|1996-05-29|NONE|RAIL| the even requests; regular pinto|
+4102|137|8|6|7|7259.91|0.02|0.01|N|O|1996-06-19|1996-05-21|1996-07-15|NONE|REG AIR|bove the carefully pending the|
+4103|75|4|1|40|39002.80|0.05|0.03|R|F|1992-09-19|1992-08-14|1992-09-21|COLLECT COD|RAIL|usly across the slyly busy accounts! fin|
+4128|196|8|1|5|5480.95|0.04|0.04|N|O|1995-10-18|1995-11-28|1995-10-28|TAKE BACK RETURN|FOB|ake permanently |
+4129|56|8|1|32|30593.60|0.03|0.04|A|F|1993-09-16|1993-08-25|1993-09-25|TAKE BACK RETURN|MAIL|ckages haggl|
+4129|27|6|2|39|36153.78|0.06|0.07|R|F|1993-10-21|1993-08-04|1993-10-29|COLLECT COD|MAIL|y regular foxes. slyly ironic deposits |
+4130|178|6|1|44|47439.48|0.07|0.04|N|O|1996-05-14|1996-04-15|1996-05-15|COLLECT COD|TRUCK|eaves haggle qui|
+4130|63|10|2|2|1926.12|0.05|0.06|N|O|1996-05-19|1996-04-24|1996-06-17|TAKE BACK RETURN|RAIL|uriously regular instructions around th|
+4131|50|7|1|6|5700.30|0.05|0.01|N|O|1998-04-27|1998-04-18|1998-04-29|TAKE BACK RETURN|MAIL|ns cajole slyly. even, iro|
+4131|178|8|2|32|34501.44|0.08|0.01|N|O|1998-03-02|1998-03-21|1998-03-07|TAKE BACK RETURN|TRUCK| furiously regular asymptotes nod sly|
+4131|26|9|3|25|23150.50|0.02|0.07|N|O|1998-02-24|1998-03-01|1998-02-27|TAKE BACK RETURN|FOB|uickly exp|
+4131|36|7|4|8|7488.24|0.04|0.01|N|O|1998-03-03|1998-03-15|1998-03-26|COLLECT COD|FOB| after the furiously ironic d|
+4131|125|6|5|30|30753.60|0.01|0.01|N|O|1998-04-01|1998-04-13|1998-04-08|TAKE BACK RETURN|FOB|he fluffily express depen|
+4131|102|7|6|47|47098.70|0.02|0.00|N|O|1998-03-09|1998-04-05|1998-03-13|TAKE BACK RETURN|RAIL|ges. ironic pinto be|
+4132|138|4|1|28|29067.64|0.07|0.03|N|O|1995-08-16|1995-08-01|1995-08-29|TAKE BACK RETURN|SHIP|pths wake against the stealthily special pi|
+4132|15|5|2|23|21045.23|0.07|0.07|N|O|1995-06-27|1995-07-27|1995-07-13|TAKE BACK RETURN|FOB|d deposits. fluffily even requests haggle b|
+4132|87|8|3|18|17767.44|0.09|0.04|A|F|1995-06-01|1995-08-01|1995-06-02|TAKE BACK RETURN|RAIL|y final de|
+4133|24|5|1|35|32340.70|0.02|0.00|A|F|1992-11-25|1992-09-15|1992-12-25|NONE|AIR|g above the quickly bold packages. ev|
+4134|121|4|1|34|34718.08|0.02|0.05|R|F|1995-04-29|1995-03-13|1995-05-11|DELIVER IN PERSON|FOB|e furiously regular sheaves sleep|
+4134|96|10|2|34|33867.06|0.01|0.03|A|F|1995-05-06|1995-03-28|1995-05-13|DELIVER IN PERSON|SHIP|ual asymptotes wake carefully alo|
+4134|171|9|3|12|12854.04|0.05|0.04|A|F|1995-03-19|1995-03-27|1995-04-14|COLLECT COD|TRUCK|kly above the quickly regular |
+4134|100|4|4|45|45004.50|0.08|0.02|A|F|1995-04-11|1995-03-27|1995-04-17|TAKE BACK RETURN|MAIL|ironic pin|
+4135|2|3|1|23|20746.00|0.06|0.01|N|O|1997-04-09|1997-05-12|1997-04-16|TAKE BACK RETURN|FOB|posits cajole furiously carefully|
+4135|120|1|2|32|32643.84|0.07|0.00|N|O|1997-03-14|1997-04-23|1997-04-12|TAKE BACK RETURN|TRUCK| ideas. requests use. furiously|
+4135|160|5|3|33|34985.28|0.05|0.05|N|O|1997-05-01|1997-05-23|1997-05-23|DELIVER IN PERSON|AIR|he fluffil|
+4135|195|6|4|13|14237.47|0.04|0.07|N|O|1997-03-16|1997-05-19|1997-04-03|COLLECT COD|RAIL|efully special account|
+4160|113|10|1|25|25327.75|0.10|0.04|N|O|1996-09-22|1996-10-17|1996-09-24|NONE|SHIP|ar accounts sleep blithe|
+4160|122|7|2|12|12265.44|0.00|0.03|N|O|1996-11-22|1996-09-25|1996-12-10|DELIVER IN PERSON|REG AIR|y bold package|
+4160|63|4|3|48|46226.88|0.04|0.04|N|O|1996-09-19|1996-11-02|1996-09-24|COLLECT COD|FOB| unusual dolphins |
+4161|122|7|1|12|12265.44|0.08|0.02|R|F|1993-08-25|1993-10-04|1993-09-22|COLLECT COD|RAIL|onic dolphins. in|
+4161|28|3|2|47|43616.94|0.05|0.00|A|F|1993-12-20|1993-10-29|1994-01-19|TAKE BACK RETURN|RAIL|r requests about the final, even foxes hag|
+4161|138|4|3|42|43601.46|0.03|0.04|R|F|1993-11-12|1993-10-04|1993-11-27|COLLECT COD|MAIL|thely across the even attainments. express|
+4161|10|5|4|45|40950.45|0.02|0.06|A|F|1993-10-22|1993-10-17|1993-10-30|COLLECT COD|REG AIR|about the ironic packages cajole blithe|
+4161|29|10|5|46|42734.92|0.05|0.01|A|F|1993-11-09|1993-11-17|1993-11-17|TAKE BACK RETURN|TRUCK|he stealthily ironic foxes. ideas haggl|
+4161|148|9|6|19|19914.66|0.07|0.00|R|F|1993-08-22|1993-11-11|1993-09-01|TAKE BACK RETURN|REG AIR|beans breach s|
+4162|74|3|1|45|43833.15|0.10|0.07|A|F|1992-03-21|1992-05-02|1992-03-29|DELIVER IN PERSON|AIR|elets. slyly regular i|
+4162|90|1|2|29|28712.61|0.00|0.05|R|F|1992-02-25|1992-04-25|1992-03-17|NONE|REG AIR|nding pinto beans haggle blithe|
+4163|33|4|1|13|12129.39|0.08|0.03|A|F|1993-02-17|1993-03-13|1993-03-15|DELIVER IN PERSON|REG AIR|phins wake. pending requests inte|
+4164|120|7|1|9|9181.08|0.07|0.02|N|O|1998-08-25|1998-08-13|1998-09-19|DELIVER IN PERSON|SHIP|re fluffily slyly bold requests. |
+4165|41|2|1|12|11292.48|0.00|0.01|N|O|1997-09-20|1997-10-20|1997-10-12|TAKE BACK RETURN|REG AIR|nwind slow theodolites. carefully pending |
+4166|141|10|1|8|8329.12|0.00|0.08|A|F|1993-06-05|1993-04-10|1993-07-05|COLLECT COD|MAIL|uickly. blithely pending de|
+4166|93|5|2|8|7944.72|0.06|0.04|A|F|1993-06-07|1993-04-17|1993-06-16|DELIVER IN PERSON|REG AIR|es along the furiously regular acc|
+4166|7|10|3|17|15419.00|0.02|0.06|R|F|1993-06-29|1993-05-15|1993-07-24|DELIVER IN PERSON|SHIP|ackages. re|
+4166|86|7|4|36|35498.88|0.06|0.05|R|F|1993-03-01|1993-05-25|1993-03-05|COLLECT COD|MAIL|unts. furiously express accounts w|
+4166|77|6|5|5|4885.35|0.08|0.01|A|F|1993-06-19|1993-04-24|1993-06-27|NONE|REG AIR|hely unusual packages are above the f|
+4166|102|5|6|6|6012.60|0.04|0.08|R|F|1993-04-30|1993-04-17|1993-05-08|DELIVER IN PERSON|MAIL|ily ironic deposits print furiously. iron|
+4166|24|5|7|26|24024.52|0.09|0.01|R|F|1993-03-17|1993-05-09|1993-03-25|NONE|MAIL|lar dependencies. s|
+4167|61|8|1|47|45169.82|0.04|0.02|N|O|1998-08-02|1998-08-24|1998-08-28|DELIVER IN PERSON|REG AIR| carefully final asymptotes. slyly bo|
+4167|87|8|2|17|16780.36|0.06|0.07|N|O|1998-09-18|1998-09-06|1998-10-07|COLLECT COD|REG AIR|ly around the even instr|
+4167|73|3|3|1|973.07|0.03|0.06|N|O|1998-10-11|1998-08-14|1998-10-13|COLLECT COD|TRUCK|xpress platelets. blithely |
+4192|11|1|1|36|32796.36|0.06|0.08|N|O|1998-04-25|1998-05-26|1998-05-03|COLLECT COD|TRUCK|eodolites sleep|
+4192|121|6|2|15|15316.80|0.04|0.08|N|O|1998-06-26|1998-05-26|1998-07-16|COLLECT COD|AIR|e slyly special grouches. express pinto b|
+4192|135|6|3|7|7245.91|0.06|0.03|N|O|1998-05-19|1998-07-08|1998-05-31|COLLECT COD|FOB|y; excuses use. ironic, close instru|
+4192|24|3|4|32|29568.64|0.09|0.04|N|O|1998-06-23|1998-06-25|1998-07-17|NONE|FOB|ounts are fluffily slyly bold req|
+4192|48|7|5|48|45505.92|0.08|0.01|N|O|1998-08-17|1998-07-11|1998-09-03|NONE|AIR|ests. quickly bol|
+4192|150|7|6|44|46206.60|0.10|0.02|N|O|1998-08-06|1998-07-09|1998-08-20|NONE|FOB|structions mai|
+4192|170|5|7|27|28894.59|0.02|0.00|N|O|1998-07-03|1998-06-26|1998-07-13|TAKE BACK RETURN|AIR| carefully even escapades. care|
+4193|131|7|1|37|38151.81|0.09|0.06|A|F|1994-04-25|1994-02-24|1994-05-08|NONE|AIR|er the quickly regular dependencies wake|
+4193|117|7|2|3|3051.33|0.09|0.05|R|F|1994-04-29|1994-03-20|1994-05-29|TAKE BACK RETURN|REG AIR|osits above the depo|
+4193|179|10|3|10|10791.70|0.06|0.03|A|F|1994-02-10|1994-03-22|1994-03-09|COLLECT COD|RAIL|uffily spe|
+4193|51|9|4|29|27580.45|0.09|0.05|A|F|1994-02-11|1994-03-11|1994-03-13|TAKE BACK RETURN|RAIL|ly. final packages use blit|
+4193|20|7|5|50|46001.00|0.01|0.01|R|F|1994-04-28|1994-03-23|1994-05-09|NONE|FOB| beans. regular accounts cajole. de|
+4193|66|1|6|21|20287.26|0.02|0.04|R|F|1994-04-26|1994-03-22|1994-05-23|DELIVER IN PERSON|TRUCK|accounts cajole b|
+4194|197|1|1|43|47179.17|0.08|0.06|A|F|1994-11-06|1994-12-09|1994-11-16|NONE|TRUCK|olites are after the exp|
+4194|47|10|2|18|17046.72|0.07|0.07|A|F|1995-02-14|1994-12-04|1995-03-11|TAKE BACK RETURN|TRUCK|ld packages. quickly eve|
+4195|6|9|1|14|12684.00|0.09|0.04|R|F|1993-09-06|1993-07-21|1993-09-18|DELIVER IN PERSON|REG AIR|ironic packages. carefully express|
+4195|66|1|2|22|21253.32|0.10|0.08|R|F|1993-07-01|1993-07-23|1993-07-28|COLLECT COD|RAIL|lly express pinto bea|
+4195|194|8|3|19|20789.61|0.01|0.06|R|F|1993-09-06|1993-08-13|1993-09-15|TAKE BACK RETURN|REG AIR|telets sleep even requests. final, even i|
+4196|156|4|1|30|31684.50|0.02|0.06|N|O|1998-08-09|1998-06-30|1998-09-05|COLLECT COD|SHIP|egular foxes us|
+4196|9|6|2|31|28179.00|0.09|0.08|N|O|1998-06-12|1998-07-28|1998-07-11|NONE|MAIL|ut the blithely ironic inst|
+4196|178|9|3|46|49595.82|0.05|0.00|N|O|1998-09-05|1998-06-28|1998-09-10|TAKE BACK RETURN|MAIL|according to t|
+4196|114|8|4|42|42592.62|0.04|0.06|N|O|1998-08-13|1998-07-18|1998-09-07|TAKE BACK RETURN|AIR| instructions. courts cajole slyly ev|
+4196|72|2|5|3|2916.21|0.01|0.03|N|O|1998-05-17|1998-07-21|1998-05-18|DELIVER IN PERSON|TRUCK| accounts. fu|
+4196|87|8|6|43|42444.44|0.01|0.06|N|O|1998-08-12|1998-07-12|1998-08-22|DELIVER IN PERSON|FOB|es. slyly even |
+4196|4|1|7|3|2712.00|0.00|0.06|N|O|1998-08-05|1998-07-28|1998-08-15|DELIVER IN PERSON|REG AIR|y regular packages haggle furiously alongs|
+4197|129|8|1|50|51456.00|0.06|0.03|N|O|1996-11-15|1996-11-01|1996-11-20|NONE|FOB|. carefully bold asymptotes nag blithe|
+4197|70|9|2|39|37832.73|0.02|0.08|N|O|1996-10-07|1996-10-11|1996-10-18|DELIVER IN PERSON|RAIL|ronic requests. quickly bold packages in|
+4197|32|8|3|28|26096.84|0.06|0.02|N|O|1996-10-05|1996-10-24|1996-10-22|TAKE BACK RETURN|AIR|regular pin|
+4197|96|7|4|23|22910.07|0.00|0.03|N|O|1996-09-10|1996-10-10|1996-09-25|NONE|AIR|l instructions print slyly past the reg|
+4197|121|6|5|37|37781.44|0.03|0.04|N|O|1996-10-20|1996-10-10|1996-11-10|COLLECT COD|TRUCK|carefully enticing decoys boo|
+4197|31|7|6|48|44689.44|0.08|0.00|N|O|1996-10-07|1996-10-25|1996-10-23|COLLECT COD|REG AIR| final instructions. blithe, spe|
+4198|146|9|1|48|50214.72|0.09|0.05|N|O|1997-09-03|1997-07-18|1997-09-11|NONE|REG AIR|cajole carefully final, ironic ide|
+4198|143|6|2|46|47984.44|0.09|0.01|N|O|1997-08-17|1997-09-08|1997-09-11|COLLECT COD|TRUCK|posits among th|
+4198|145|4|3|13|13586.82|0.03|0.04|N|O|1997-07-18|1997-07-24|1997-08-10|NONE|REG AIR| furious excuses. bli|
+4199|70|5|1|16|15521.12|0.10|0.00|A|F|1992-06-11|1992-04-10|1992-07-10|COLLECT COD|TRUCK|ncies. furiously special accounts|
+4199|9|10|2|18|16362.00|0.00|0.01|A|F|1992-06-01|1992-03-30|1992-06-28|DELIVER IN PERSON|RAIL|pending, regular accounts. carefully|
+4224|199|10|1|27|29678.13|0.05|0.03|N|O|1997-09-05|1997-08-19|1997-09-30|NONE|SHIP|ly special deposits sleep qui|
+4224|37|3|2|20|18740.60|0.07|0.05|N|O|1997-11-09|1997-08-23|1997-11-14|NONE|FOB|unts promise across the requests. blith|
+4224|24|7|3|4|3696.08|0.08|0.05|N|O|1997-09-07|1997-09-05|1997-09-25|TAKE BACK RETURN|FOB| even dinos. carefull|
+4224|160|2|4|50|53008.00|0.10|0.06|N|O|1997-07-30|1997-09-10|1997-08-19|COLLECT COD|RAIL|side of the carefully silent dep|
+4224|85|6|5|48|47283.84|0.00|0.04|N|O|1997-10-03|1997-08-31|1997-10-10|NONE|RAIL| final, regular asymptotes use alway|
+4225|49|8|1|25|23726.00|0.08|0.04|N|O|1997-07-10|1997-08-08|1997-07-31|TAKE BACK RETURN|TRUCK|se fluffily. busily ironic requests are;|
+4225|96|8|2|23|22910.07|0.02|0.04|N|O|1997-09-18|1997-08-31|1997-10-11|TAKE BACK RETURN|RAIL|. quickly b|
+4225|98|10|3|28|27946.52|0.08|0.03|N|O|1997-07-11|1997-09-01|1997-08-03|DELIVER IN PERSON|FOB|ts are requests. even, bold depos|
+4226|188|9|1|27|29380.86|0.06|0.08|A|F|1993-05-03|1993-04-12|1993-05-16|COLLECT COD|AIR|sly alongside of the slyly ironic pac|
+4227|158|6|1|19|20104.85|0.01|0.08|A|F|1995-05-05|1995-05-03|1995-05-22|COLLECT COD|REG AIR|ns sleep along the blithely even theodolit|
+4227|33|4|2|8|7464.24|0.09|0.00|N|F|1995-06-11|1995-04-30|1995-06-28|COLLECT COD|REG AIR| packages since the bold, u|
+4227|75|6|3|11|10725.77|0.10|0.04|A|F|1995-03-30|1995-05-02|1995-04-26|DELIVER IN PERSON|SHIP|l requests-- bold requests cajole dogg|
+4227|200|4|4|2|2200.40|0.02|0.05|R|F|1995-04-24|1995-05-09|1995-05-21|DELIVER IN PERSON|AIR|ep. specia|
+4227|147|6|5|49|51309.86|0.05|0.06|R|F|1995-05-19|1995-04-12|1995-06-12|TAKE BACK RETURN|REG AIR|ts sleep blithely carefully unusual ideas.|
+4228|141|10|1|20|20822.80|0.00|0.06|N|O|1997-04-24|1997-05-29|1997-05-17|NONE|RAIL|f the slyly fluffy pinto beans are|
+4229|96|9|1|44|43827.96|0.02|0.05|N|O|1998-05-29|1998-05-12|1998-06-16|DELIVER IN PERSON|AIR|s. carefully e|
+4229|5|8|2|34|30770.00|0.07|0.05|N|O|1998-05-26|1998-04-13|1998-06-08|DELIVER IN PERSON|MAIL|thely final accounts use even packa|
+4230|46|5|1|38|35949.52|0.10|0.03|A|F|1992-04-28|1992-04-21|1992-05-28|TAKE BACK RETURN|FOB|ly regular packages. regular ideas boost|
+4230|199|3|2|43|47265.17|0.02|0.08|R|F|1992-03-14|1992-05-13|1992-03-28|NONE|FOB|ses lose blithely slyly final e|
+4230|196|9|3|10|10961.90|0.06|0.02|A|F|1992-06-11|1992-04-11|1992-07-02|TAKE BACK RETURN|MAIL|ar packages are |
+4230|75|6|4|28|27301.96|0.01|0.03|R|F|1992-05-12|1992-05-10|1992-06-01|TAKE BACK RETURN|MAIL|nt instruct|
+4230|125|10|5|50|51256.00|0.00|0.01|A|F|1992-03-29|1992-05-19|1992-04-20|TAKE BACK RETURN|SHIP|ts. final instructions in|
+4230|35|6|6|30|28050.90|0.05|0.07|A|F|1992-03-11|1992-04-29|1992-03-30|NONE|AIR|s. final excuses across the|
+4230|152|3|7|18|18938.70|0.10|0.04|R|F|1992-06-23|1992-05-10|1992-07-04|COLLECT COD|SHIP| the final acco|
+4231|142|3|1|47|48980.58|0.09|0.03|N|O|1997-11-27|1998-01-26|1997-12-17|NONE|REG AIR|hely along the silent at|
+4231|166|3|2|4|4264.64|0.06|0.02|N|O|1997-11-28|1998-01-26|1997-12-12|TAKE BACK RETURN|MAIL|lithely even packages. |
+4231|121|2|3|31|31654.72|0.07|0.08|N|O|1998-02-14|1997-12-27|1998-03-01|DELIVER IN PERSON|FOB|ublate. theodoli|
+4231|40|1|4|35|32901.40|0.10|0.00|N|O|1998-02-21|1998-01-24|1998-03-18|DELIVER IN PERSON|FOB|le quickly regular, unus|
+4256|151|9|1|22|23125.30|0.05|0.05|R|F|1992-07-30|1992-05-14|1992-08-14|NONE|TRUCK|, final platelets are slyly final pint|
+4257|65|10|1|3|2895.18|0.10|0.03|N|O|1995-06-18|1995-05-01|1995-07-12|DELIVER IN PERSON|MAIL|thin the theodolites use after the bl|
+4257|35|6|2|5|4675.15|0.01|0.04|R|F|1995-04-29|1995-06-05|1995-05-13|TAKE BACK RETURN|TRUCK|n deposits. furiously e|
+4257|128|9|3|33|33927.96|0.03|0.04|A|F|1995-05-23|1995-05-03|1995-05-31|COLLECT COD|AIR|uffily regular accounts ar|
+4258|166|7|1|36|38381.76|0.02|0.06|N|O|1997-02-23|1997-01-25|1997-02-27|TAKE BACK RETURN|SHIP|ns use alongs|
+4258|162|1|2|19|20181.04|0.03|0.02|N|O|1997-01-14|1996-12-12|1997-01-20|TAKE BACK RETURN|AIR|ly busily ironic foxes. f|
+4258|31|7|3|46|42827.38|0.04|0.07|N|O|1997-01-02|1996-12-26|1997-01-12|DELIVER IN PERSON|AIR| furiously pend|
+4258|35|6|4|22|20570.66|0.04|0.04|N|O|1996-12-12|1996-12-06|1996-12-20|TAKE BACK RETURN|AIR|e regular, even asym|
+4258|163|10|5|9|9568.44|0.04|0.03|N|O|1996-12-04|1996-12-08|1996-12-20|DELIVER IN PERSON|TRUCK|counts wake permanently after the bravely|
+4259|43|6|1|14|13202.56|0.05|0.03|N|O|1998-01-09|1997-11-21|1998-01-29|TAKE BACK RETURN|RAIL| furiously pending excuses. ideas hagg|
+4260|24|7|1|21|19404.42|0.08|0.04|R|F|1992-08-06|1992-06-18|1992-08-22|NONE|AIR|al, pending accounts must|
+4261|110|1|1|12|12121.32|0.05|0.01|A|F|1992-11-01|1993-01-01|1992-11-12|NONE|FOB|into beans |
+4261|82|3|2|4|3928.32|0.02|0.07|R|F|1992-12-11|1992-12-18|1992-12-24|DELIVER IN PERSON|FOB|ackages unwind furiously fluff|
+4261|175|5|3|3|3225.51|0.07|0.02|R|F|1992-11-10|1992-12-14|1992-11-17|COLLECT COD|RAIL|ly even deposits eat blithely alo|
+4261|174|3|4|36|38670.12|0.04|0.06|R|F|1992-12-02|1992-12-18|1992-12-25|NONE|REG AIR| slyly pendi|
+4261|24|7|5|28|25872.56|0.07|0.06|A|F|1992-10-08|1992-12-23|1992-10-11|TAKE BACK RETURN|MAIL|packages. fluffily i|
+4262|76|7|1|30|29282.10|0.01|0.03|N|O|1996-08-11|1996-10-11|1996-09-09|TAKE BACK RETURN|RAIL|tes after the carefully|
+4262|96|7|2|5|4980.45|0.02|0.05|N|O|1996-09-27|1996-09-05|1996-10-25|COLLECT COD|SHIP|blithely final asymptotes integrate|
+4262|162|1|3|5|5310.80|0.08|0.00|N|O|1996-10-02|1996-10-16|1996-10-05|NONE|REG AIR|ironic accounts are unusu|
+4262|74|2|4|45|43833.15|0.02|0.01|N|O|1996-11-09|1996-09-09|1996-11-12|DELIVER IN PERSON|SHIP|ackages boost. pending, even instruction|
+4262|100|3|5|28|28002.80|0.06|0.02|N|O|1996-10-22|1996-09-06|1996-11-13|DELIVER IN PERSON|FOB|ironic, regular depend|
+4262|17|7|6|26|23842.26|0.03|0.02|N|O|1996-08-29|1996-09-25|1996-08-31|NONE|RAIL|s boost slyly along the bold, iro|
+4262|160|5|7|41|43466.56|0.03|0.01|N|O|1996-08-28|1996-09-14|1996-09-20|COLLECT COD|RAIL|cuses unwind ac|
+4263|18|9|1|9|8262.09|0.08|0.07|N|O|1998-04-04|1998-04-29|1998-05-04|COLLECT COD|AIR|structions cajole quic|
+4263|196|10|2|28|30693.32|0.05|0.03|N|O|1998-06-24|1998-06-08|1998-07-14|NONE|MAIL|ideas for the carefully re|
+4263|11|1|3|38|34618.38|0.01|0.01|N|O|1998-07-10|1998-05-08|1998-07-17|NONE|TRUCK|rding to the dep|
+4263|19|3|4|20|18380.20|0.02|0.07|N|O|1998-04-09|1998-04-30|1998-05-04|NONE|RAIL|uietly regular deposits. sly deposits w|
+4263|198|2|5|14|15374.66|0.09|0.06|N|O|1998-05-06|1998-04-17|1998-05-11|DELIVER IN PERSON|TRUCK|d accounts. daringly regular accounts hagg|
+4263|113|10|6|47|47616.17|0.08|0.06|N|O|1998-06-28|1998-05-09|1998-07-02|DELIVER IN PERSON|TRUCK|y. theodolites wake idly ironic do|
+4263|29|4|7|6|5574.12|0.04|0.04|N|O|1998-05-01|1998-06-02|1998-05-14|TAKE BACK RETURN|REG AIR|g the final, regular instructions: |
+4288|74|5|1|32|31170.24|0.10|0.07|R|F|1993-03-19|1993-01-26|1993-04-18|TAKE BACK RETURN|AIR|e blithely even instructions. speci|
+4288|105|6|2|39|39198.90|0.05|0.02|R|F|1993-03-25|1993-02-06|1993-03-28|DELIVER IN PERSON|AIR|uffy theodolites run|
+4288|125|8|3|7|7175.84|0.03|0.01|A|F|1993-01-15|1993-02-05|1993-01-26|NONE|TRUCK|ngside of the special platelet|
+4289|196|7|1|19|20827.61|0.06|0.06|R|F|1993-12-31|1993-11-06|1994-01-23|DELIVER IN PERSON|TRUCK|e carefully regular ideas. sl|
+4290|137|3|1|23|23853.99|0.06|0.04|R|F|1995-04-04|1995-02-16|1995-04-07|TAKE BACK RETURN|REG AIR|uests cajole carefully.|
+4290|99|2|2|3|2997.27|0.09|0.03|A|F|1995-03-25|1995-03-07|1995-04-11|NONE|RAIL|lar platelets cajole|
+4291|192|6|1|3|3276.57|0.08|0.08|A|F|1994-03-17|1994-02-21|1994-03-27|COLLECT COD|SHIP|tes sleep slyly above the quickly sl|
+4291|125|8|2|43|44080.16|0.01|0.06|A|F|1994-02-01|1994-02-27|1994-02-06|DELIVER IN PERSON|REG AIR|s. quietly regular |
+4291|8|1|3|25|22700.00|0.09|0.08|R|F|1994-02-14|1994-02-08|1994-03-15|COLLECT COD|AIR|uctions. furiously regular ins|
+4292|44|3|1|22|20768.88|0.08|0.03|R|F|1992-02-14|1992-02-16|1992-03-01|NONE|FOB|refully expres|
+4292|40|6|2|1|940.04|0.03|0.01|A|F|1992-02-07|1992-03-16|1992-02-10|DELIVER IN PERSON|FOB| the furiously ev|
+4292|120|10|3|35|35704.20|0.03|0.06|A|F|1992-03-23|1992-04-04|1992-04-02|COLLECT COD|TRUCK|dugouts use. furiously bold packag|
+4292|163|10|4|40|42526.40|0.05|0.04|A|F|1992-04-27|1992-03-07|1992-05-04|COLLECT COD|REG AIR|ounts according to the furiously |
+4292|131|7|5|6|6186.78|0.07|0.08|R|F|1992-03-03|1992-02-24|1992-03-25|COLLECT COD|FOB|bove the silently regula|
+4292|4|1|6|47|42488.00|0.05|0.00|R|F|1992-05-02|1992-03-21|1992-05-27|TAKE BACK RETURN|FOB|y packages; even ideas boost|
+4293|1|6|1|34|30634.00|0.03|0.08|N|O|1996-11-05|1996-10-12|1996-12-04|NONE|FOB|ions sleep blithely on|
+4293|77|5|2|50|48853.50|0.01|0.05|N|O|1996-11-27|1996-10-30|1996-12-22|COLLECT COD|MAIL| special deposits. furiousl|
+4293|199|1|3|47|51661.93|0.08|0.02|N|O|1996-09-07|1996-10-24|1996-09-15|NONE|RAIL|ithely pending deposits af|
+4293|88|9|4|25|24702.00|0.04|0.04|N|O|1996-09-11|1996-11-14|1996-09-22|DELIVER IN PERSON|FOB|inal asympt|
+4293|181|2|5|1|1081.18|0.06|0.05|N|O|1996-11-15|1996-10-09|1996-11-26|COLLECT COD|AIR|eposits should boost along the |
+4293|79|7|6|45|44058.15|0.10|0.04|N|O|1996-11-04|1996-11-06|1996-11-23|NONE|MAIL|lar ideas use carefully|
+4294|105|8|1|19|19096.90|0.03|0.04|A|F|1992-10-16|1992-11-13|1992-10-26|DELIVER IN PERSON|AIR|nt dependencies. furiously regular ideas d|
+4294|27|2|2|16|14832.32|0.01|0.02|R|F|1992-08-17|1992-09-24|1992-09-04|TAKE BACK RETURN|REG AIR|lithely pint|
+4294|198|1|3|30|32945.70|0.01|0.00|A|F|1992-09-12|1992-11-06|1992-09-25|NONE|MAIL|olites. bold foxes affix ironic theodolite|
+4294|105|2|4|34|34173.40|0.02|0.01|R|F|1992-09-09|1992-11-06|1992-10-04|TAKE BACK RETURN|REG AIR|pendencies!|
+4294|119|3|5|37|37707.07|0.05|0.01|R|F|1992-09-07|1992-10-13|1992-09-08|NONE|REG AIR|cial packages nag f|
+4294|87|8|6|42|41457.36|0.02|0.03|A|F|1992-09-30|1992-11-13|1992-10-15|DELIVER IN PERSON|FOB| carefully; furiously ex|
+4294|175|3|7|47|50532.99|0.02|0.08|R|F|1992-11-09|1992-11-03|1992-12-05|TAKE BACK RETURN|SHIP|es. blithely r|
+4295|29|2|1|49|45521.98|0.09|0.01|N|O|1996-05-25|1996-03-17|1996-06-19|TAKE BACK RETURN|REG AIR|refully silent requests. f|
+4295|71|9|2|4|3884.28|0.09|0.07|N|O|1996-06-05|1996-04-26|1996-06-13|DELIVER IN PERSON|TRUCK|arefully according to the pending ac|
+4295|193|4|3|3|3279.57|0.04|0.00|N|O|1996-06-04|1996-04-24|1996-06-24|DELIVER IN PERSON|AIR|telets cajole bravely|
+4295|80|9|4|30|29402.40|0.07|0.06|N|O|1996-03-22|1996-04-23|1996-04-20|NONE|SHIP|yly ironic frets. pending foxes after |
+4320|46|5|1|28|26489.12|0.02|0.06|N|O|1997-01-28|1997-02-07|1997-02-07|COLLECT COD|FOB|nts. even, ironic excuses hagg|
+4320|140|6|2|6|6240.84|0.08|0.08|N|O|1997-01-11|1997-01-26|1997-01-22|DELIVER IN PERSON|SHIP|against the carefully careful asym|
+4320|188|9|3|33|35909.94|0.09|0.02|N|O|1996-12-11|1997-02-27|1997-01-08|TAKE BACK RETURN|SHIP|ess asymptotes so|
+4321|147|6|1|33|34555.62|0.09|0.02|A|F|1994-09-01|1994-08-17|1994-09-05|DELIVER IN PERSON|TRUCK|yly special excuses. fluffily |
+4321|54|2|2|45|42932.25|0.00|0.08|R|F|1994-11-13|1994-09-15|1994-11-18|DELIVER IN PERSON|SHIP| haggle ironically bold theodolites. quick|
+4321|186|7|3|23|24982.14|0.01|0.05|A|F|1994-11-03|1994-10-08|1994-11-06|DELIVER IN PERSON|SHIP|ly even orbits slee|
+4321|91|2|4|4|3964.36|0.02|0.00|R|F|1994-09-10|1994-10-06|1994-09-11|NONE|FOB|ironic deposi|
+4321|172|2|5|10|10721.70|0.04|0.03|A|F|1994-09-07|1994-08-23|1994-09-17|TAKE BACK RETURN|SHIP|wake carefully alongside of |
+4322|69|4|1|39|37793.34|0.04|0.02|N|O|1998-04-27|1998-06-03|1998-05-04|TAKE BACK RETURN|MAIL|its integrate fluffily |
+4322|140|1|2|9|9361.26|0.05|0.08|N|O|1998-05-18|1998-04-27|1998-05-28|COLLECT COD|AIR|ual instructio|
+4322|8|9|3|12|10896.00|0.09|0.05|N|O|1998-03-29|1998-06-05|1998-04-16|DELIVER IN PERSON|TRUCK|e blithely against the slyly unusu|
+4322|46|7|4|17|16082.68|0.09|0.08|N|O|1998-05-31|1998-05-31|1998-06-10|TAKE BACK RETURN|FOB|ructions boost |
+4322|102|7|5|10|10021.00|0.00|0.05|N|O|1998-05-31|1998-04-27|1998-06-25|TAKE BACK RETURN|REG AIR| regular ideas engage carefully quick|
+4322|60|8|6|39|37442.34|0.09|0.08|N|O|1998-03-16|1998-05-21|1998-04-11|COLLECT COD|AIR|ccounts. dogged pin|
+4322|14|4|7|34|31076.34|0.05|0.00|N|O|1998-05-27|1998-04-12|1998-06-16|NONE|REG AIR|ounts haggle fluffily ideas. pend|
+4323|1|2|1|33|29733.00|0.09|0.02|A|F|1994-05-04|1994-03-06|1994-05-23|COLLECT COD|TRUCK|the slyly bold deposits slee|
+4324|51|2|1|44|41846.20|0.05|0.04|N|O|1995-10-15|1995-09-07|1995-11-07|DELIVER IN PERSON|AIR|ainst the u|
+4324|48|7|2|12|11376.48|0.04|0.02|N|O|1995-10-05|1995-09-07|1995-10-18|NONE|REG AIR|c packages. furiously express sauternes|
+4324|82|3|3|14|13749.12|0.07|0.06|N|O|1995-11-12|1995-08-26|1995-11-21|COLLECT COD|AIR| packages nag express excuses. qui|
+4324|50|7|4|14|13300.70|0.02|0.04|N|O|1995-09-20|1995-10-08|1995-10-06|COLLECT COD|RAIL| express ideas. blithely blit|
+4324|84|5|5|22|21649.76|0.07|0.03|N|O|1995-09-13|1995-10-04|1995-09-23|DELIVER IN PERSON|SHIP|ke express, special ideas.|
+4324|43|2|6|31|29234.24|0.08|0.04|N|O|1995-10-23|1995-09-14|1995-11-09|COLLECT COD|RAIL|efully flu|
+4324|154|6|7|46|48490.90|0.00|0.03|N|O|1995-11-03|1995-09-28|1995-11-22|NONE|SHIP|ular, final theodo|
+4325|160|2|1|18|19082.88|0.01|0.07|N|O|1996-10-07|1996-09-28|1996-10-31|DELIVER IN PERSON|RAIL|. blithely|
+4326|163|4|1|11|11694.76|0.01|0.01|N|O|1997-02-02|1996-12-10|1997-02-20|DELIVER IN PERSON|TRUCK|press reque|
+4326|167|6|2|27|28813.32|0.06|0.01|N|O|1996-11-29|1997-01-20|1996-12-23|COLLECT COD|AIR|inal packages. final asymptotes about t|
+4327|95|8|1|18|17911.62|0.08|0.00|N|F|1995-06-16|1995-04-20|1995-07-12|COLLECT COD|RAIL|y final excuses. ironic, special requests a|
+4327|106|9|2|40|40244.00|0.07|0.01|N|F|1995-05-26|1995-04-17|1995-06-18|NONE|AIR|quests. packages are after th|
+4327|145|2|3|11|11496.54|0.10|0.07|R|F|1995-04-24|1995-05-27|1995-05-24|TAKE BACK RETURN|FOB| ironic dolphins|
+4327|21|10|4|8|7368.16|0.04|0.08|N|F|1995-05-26|1995-05-28|1995-06-19|DELIVER IN PERSON|AIR|eodolites cajole; unusual Tiresias|
+4327|190|1|5|39|42517.41|0.01|0.00|N|O|1995-06-23|1995-04-18|1995-07-13|TAKE BACK RETURN|FOB|kages against the blit|
+4327|152|4|6|10|10521.50|0.00|0.06|A|F|1995-04-28|1995-06-11|1995-05-07|TAKE BACK RETURN|TRUCK|arefully sile|
+4352|106|9|1|18|18109.80|0.00|0.03|N|O|1998-02-27|1998-02-02|1998-03-01|DELIVER IN PERSON|RAIL|ding to th|
+4353|94|8|1|22|21869.98|0.05|0.05|N|O|1998-01-19|1998-01-23|1998-02-10|COLLECT COD|FOB|ent packages. accounts are slyly. |
+4354|15|9|1|30|27450.30|0.08|0.07|R|F|1995-01-27|1994-11-24|1995-02-25|TAKE BACK RETURN|REG AIR|around the ir|
+4354|153|8|2|23|24222.45|0.01|0.08|R|F|1994-11-20|1994-12-23|1994-11-27|TAKE BACK RETURN|AIR|kly along the ironic, ent|
+4354|51|6|3|2|1902.10|0.10|0.04|A|F|1995-01-09|1994-12-15|1995-01-24|TAKE BACK RETURN|REG AIR|s nag quickly |
+4354|86|7|4|36|35498.88|0.05|0.05|A|F|1994-11-20|1994-12-06|1994-12-06|DELIVER IN PERSON|AIR| wake slyly eve|
+4354|65|10|5|37|35707.22|0.06|0.02|R|F|1995-01-13|1994-12-29|1995-01-31|DELIVER IN PERSON|FOB|deas use blithely! special foxes print af|
+4354|108|3|6|36|36291.60|0.03|0.04|R|F|1994-12-03|1994-12-05|1995-01-02|TAKE BACK RETURN|TRUCK|efully special packages use fluffily|
+4354|139|5|7|18|18704.34|0.03|0.04|A|F|1994-12-07|1994-12-11|1994-12-11|TAKE BACK RETURN|SHIP|ross the furiously |
+4355|195|7|1|32|35046.08|0.10|0.02|N|O|1996-12-29|1997-02-08|1997-01-24|DELIVER IN PERSON|REG AIR|y silent deposits. b|
+4355|17|1|2|4|3668.04|0.05|0.02|N|O|1997-02-25|1997-01-29|1997-03-17|TAKE BACK RETURN|TRUCK|slyly blithely regular packag|
+4355|1|2|3|13|11713.00|0.07|0.05|N|O|1997-01-21|1996-12-22|1997-02-14|COLLECT COD|TRUCK| ought to mold. blithely pending ideas |
+4355|194|6|4|14|15318.66|0.04|0.02|N|O|1997-03-08|1997-01-22|1997-03-26|NONE|RAIL|he furiously ironic accounts. quickly iro|
+4355|31|7|5|50|46551.50|0.10|0.00|N|O|1996-11-25|1997-01-01|1996-12-06|DELIVER IN PERSON|REG AIR| regular accounts boost along the |
+4355|122|7|6|35|35774.20|0.00|0.08|N|O|1997-01-28|1997-01-28|1997-02-20|NONE|FOB|ess accounts affix ironic|
+4355|101|4|7|47|47051.70|0.09|0.02|N|O|1996-12-28|1996-12-29|1997-01-09|NONE|RAIL|e. realms integrate |
+4356|194|5|1|35|38296.65|0.00|0.04|R|F|1994-05-30|1994-06-14|1994-06-08|COLLECT COD|MAIL|arefully ironic |
+4357|84|5|1|50|49204.00|0.04|0.07|N|O|1997-11-25|1997-12-03|1997-12-17|DELIVER IN PERSON|RAIL|s. final, e|
+4357|108|9|2|17|17137.70|0.10|0.07|N|O|1998-02-01|1997-12-08|1998-02-09|DELIVER IN PERSON|MAIL|e carefully furiou|
+4358|126|5|1|47|48227.64|0.04|0.00|N|O|1997-10-15|1997-10-14|1997-11-04|DELIVER IN PERSON|SHIP|refully busy dep|
+4359|174|3|1|41|44040.97|0.03|0.07|A|F|1993-04-06|1993-05-06|1993-04-14|COLLECT COD|RAIL|s affix sly|
+4359|153|8|2|8|8425.20|0.03|0.08|R|F|1993-06-27|1993-05-16|1993-07-04|DELIVER IN PERSON|MAIL|packages affix. fluffily regular f|
+4359|193|6|3|32|34982.08|0.10|0.03|R|F|1993-06-18|1993-04-04|1993-07-18|COLLECT COD|MAIL|olites nag quietly caref|
+4359|78|8|4|1|978.07|0.05|0.03|R|F|1993-04-27|1993-05-09|1993-05-08|NONE|MAIL| fluffily ironic, bold pac|
+4359|33|4|5|22|20526.66|0.04|0.01|A|F|1993-03-28|1993-06-01|1993-04-13|NONE|REG AIR|accounts wake ironic deposits. ironic|
+4384|136|7|1|5|5180.65|0.09|0.01|A|F|1992-08-22|1992-08-24|1992-09-20|DELIVER IN PERSON|MAIL|instructions sleep. blithely express pa|
+4384|89|10|2|38|37585.04|0.07|0.06|A|F|1992-10-18|1992-09-24|1992-11-04|NONE|FOB|ly final requests. regu|
+4384|89|10|3|11|10879.88|0.05|0.04|R|F|1992-08-31|1992-10-04|1992-09-28|TAKE BACK RETURN|FOB|deposits promise carefully even, regular e|
+4385|111|8|1|38|38422.18|0.00|0.02|N|O|1996-11-22|1996-10-30|1996-12-21|DELIVER IN PERSON|TRUCK|inal frays. final, bold exc|
+4386|130|3|1|10|10301.30|0.05|0.07|N|O|1998-06-03|1998-04-16|1998-06-28|TAKE BACK RETURN|MAIL|gainst the quickly expre|
+4386|118|2|2|28|28507.08|0.03|0.06|N|O|1998-03-19|1998-05-01|1998-03-27|NONE|FOB|. quick packages play slyly |
+4386|140|1|3|4|4160.56|0.07|0.05|N|O|1998-04-07|1998-03-25|1998-04-19|COLLECT COD|FOB|ns wake carefully carefully iron|
+4386|121|2|4|21|21443.52|0.09|0.00|N|O|1998-05-05|1998-03-19|1998-05-13|NONE|RAIL|e pending, sp|
+4386|130|3|5|39|40175.07|0.09|0.06|N|O|1998-03-05|1998-03-15|1998-03-16|NONE|RAIL|structions cajole quickly express|
+4386|90|1|6|18|17821.62|0.02|0.05|N|O|1998-04-12|1998-04-09|1998-05-12|TAKE BACK RETURN|SHIP| deposits use according to the pending, |
+4386|20|4|7|16|14720.32|0.07|0.02|N|O|1998-05-05|1998-03-17|1998-06-03|COLLECT COD|AIR|e furiously final pint|
+4387|122|5|1|3|3066.36|0.02|0.01|N|O|1996-01-17|1996-01-14|1996-01-28|COLLECT COD|AIR| boost slyly ironic instructions. furiou|
+4387|177|5|2|48|51704.16|0.06|0.05|N|O|1995-10-29|1995-12-11|1995-11-01|NONE|REG AIR|sleep slyly. blithely sl|
+4387|2|5|3|15|13530.00|0.00|0.03|N|O|1996-01-11|1996-01-14|1996-01-30|TAKE BACK RETURN|REG AIR|s hinder quietly across the pla|
+4387|47|8|4|9|8523.36|0.00|0.03|N|O|1996-01-04|1995-12-26|1996-01-12|DELIVER IN PERSON|REG AIR|c ideas. slyly regular packages sol|
+4387|82|3|5|3|2946.24|0.05|0.08|N|O|1995-11-17|1995-12-28|1995-11-25|COLLECT COD|SHIP| pinto beans |
+4387|6|3|6|40|36240.00|0.02|0.04|N|O|1995-11-29|1995-12-10|1995-12-20|NONE|REG AIR|deas according to the blithely regular fox|
+4388|65|10|1|30|28951.80|0.02|0.07|N|O|1996-06-07|1996-05-07|1996-06-22|DELIVER IN PERSON|FOB|s cajole fluffil|
+4388|84|5|2|28|27554.24|0.05|0.04|N|O|1996-05-08|1996-06-20|1996-05-12|TAKE BACK RETURN|RAIL|ove the ide|
+4388|52|4|3|13|12376.65|0.07|0.05|N|O|1996-06-28|1996-05-23|1996-07-04|DELIVER IN PERSON|REG AIR|ly even, expre|
+4389|157|2|1|20|21143.00|0.08|0.00|A|F|1994-06-06|1994-06-17|1994-06-17|DELIVER IN PERSON|SHIP|ng the carefully express d|
+4389|153|5|2|13|13690.95|0.00|0.00|A|F|1994-08-18|1994-06-06|1994-08-20|NONE|RAIL|nal, regula|
+4389|79|9|3|39|38183.73|0.04|0.07|A|F|1994-06-08|1994-06-04|1994-06-10|TAKE BACK RETURN|TRUCK| unusual, final excuses cajole carefully |
+4389|160|2|4|5|5300.80|0.09|0.00|A|F|1994-09-03|1994-06-23|1994-09-16|NONE|FOB| ironic request|
+4389|11|5|5|22|20042.22|0.08|0.00|R|F|1994-07-05|1994-06-12|1994-07-12|NONE|TRUCK|lly silent de|
+4389|2|3|6|22|19844.00|0.01|0.04|R|F|1994-06-07|1994-06-29|1994-06-19|COLLECT COD|TRUCK|at the final excuses hinder carefully a|
+4389|185|6|7|4|4340.72|0.09|0.08|R|F|1994-06-14|1994-06-30|1994-07-06|NONE|REG AIR| blithely even d|
+4390|152|10|1|35|36825.25|0.07|0.04|R|F|1995-05-30|1995-07-02|1995-06-15|DELIVER IN PERSON|TRUCK|ongside of the slyly regular ideas|
+4390|196|8|2|28|30693.32|0.03|0.00|N|O|1995-09-07|1995-06-22|1995-10-05|COLLECT COD|SHIP|ld braids haggle atop the for|
+4390|101|8|3|42|42046.20|0.05|0.08|A|F|1995-06-12|1995-07-16|1995-06-17|NONE|AIR|arefully even accoun|
+4390|98|2|4|32|31938.88|0.07|0.08|N|O|1995-09-15|1995-08-12|1995-10-05|TAKE BACK RETURN|TRUCK|ctions across|
+4391|161|10|1|1|1061.16|0.09|0.00|R|F|1992-06-18|1992-04-27|1992-06-20|COLLECT COD|TRUCK|ong the silent deposits|
+4391|187|8|2|45|48923.10|0.07|0.04|R|F|1992-04-01|1992-05-01|1992-04-13|TAKE BACK RETURN|TRUCK|ep quickly after |
+4416|94|7|1|37|36781.33|0.08|0.03|A|F|1992-10-23|1992-08-23|1992-11-16|COLLECT COD|RAIL|fluffily ironic |
+4416|89|10|2|3|2967.24|0.06|0.03|R|F|1992-10-22|1992-08-06|1992-11-13|DELIVER IN PERSON|SHIP| requests sleep along the |
+4416|9|6|3|45|40905.00|0.09|0.03|A|F|1992-10-16|1992-09-09|1992-10-28|COLLECT COD|AIR|the final pinto beans. special frets |
+4417|75|5|1|28|27301.96|0.08|0.02|N|O|1998-09-04|1998-10-04|1998-09-19|TAKE BACK RETURN|REG AIR|ies across the furious|
+4417|181|2|2|1|1081.18|0.06|0.08|N|O|1998-10-23|1998-08-22|1998-10-24|NONE|REG AIR|press deposits promise stealthily amo|
+4417|98|2|3|35|34933.15|0.06|0.04|N|O|1998-08-08|1998-09-23|1998-09-02|DELIVER IN PERSON|FOB|slyly regular, silent courts. even packag|
+4418|35|1|1|32|29920.96|0.02|0.06|A|F|1993-05-28|1993-06-02|1993-05-30|TAKE BACK RETURN|RAIL|ly. bold pinto b|
+4418|22|5|2|14|12908.28|0.03|0.04|A|F|1993-05-20|1993-06-18|1993-06-05|TAKE BACK RETURN|SHIP| blithely regular requests. blith|
+4418|79|7|3|3|2937.21|0.00|0.02|R|F|1993-04-08|1993-06-04|1993-05-02|NONE|SHIP|luffily across the unusual ideas. reque|
+4419|108|9|1|45|45364.50|0.01|0.05|N|O|1996-07-20|1996-09-07|1996-08-18|DELIVER IN PERSON|TRUCK|s doze sometimes fluffily regular a|
+4419|32|8|2|42|39145.26|0.00|0.03|N|O|1996-09-18|1996-07-25|1996-09-21|COLLECT COD|RAIL|sts. furious|
+4419|132|3|3|6|6192.78|0.02|0.08|N|O|1996-06-25|1996-09-04|1996-07-20|DELIVER IN PERSON|AIR|ts wake slyly final dugou|
+4420|8|5|1|7|6356.00|0.07|0.03|R|F|1994-08-30|1994-09-03|1994-09-25|NONE|FOB| regular instructions sleep around|
+4421|98|2|1|37|36929.33|0.09|0.08|N|O|1997-07-22|1997-06-27|1997-07-25|DELIVER IN PERSON|SHIP|l accounts. ironic request|
+4421|56|1|2|46|43978.30|0.04|0.04|N|O|1997-04-21|1997-05-13|1997-05-15|DELIVER IN PERSON|FOB|reful packages. bold, |
+4421|167|6|3|46|49089.36|0.00|0.06|N|O|1997-05-25|1997-05-21|1997-06-23|COLLECT COD|TRUCK|g dependenci|
+4421|191|4|4|32|34918.08|0.06|0.04|N|O|1997-07-09|1997-06-03|1997-07-25|NONE|SHIP|ar ideas eat among the furiousl|
+4421|190|1|5|32|34886.08|0.06|0.04|N|O|1997-07-28|1997-06-14|1997-08-13|NONE|REG AIR|uickly final pinto beans impress. bold |
+4421|47|6|6|44|41669.76|0.09|0.06|N|O|1997-06-17|1997-06-20|1997-06-29|NONE|TRUCK|le carefully. bl|
+4421|116|3|7|18|18289.98|0.01|0.00|N|O|1997-06-07|1997-05-13|1997-06-10|DELIVER IN PERSON|FOB|. regular, s|
+4422|135|1|1|5|5175.65|0.09|0.07|N|O|1995-07-17|1995-08-13|1995-07-25|NONE|SHIP|e furiously about t|
+4422|48|5|2|41|38869.64|0.08|0.05|N|F|1995-06-12|1995-07-09|1995-06-20|COLLECT COD|TRUCK| theodolites shal|
+4422|103|10|3|39|39120.90|0.00|0.05|N|O|1995-09-02|1995-06-24|1995-09-14|NONE|TRUCK|en hockey players engage|
+4422|153|4|4|4|4212.60|0.02|0.05|N|O|1995-09-18|1995-08-12|1995-10-18|COLLECT COD|FOB|cies along the bo|
+4422|80|9|5|20|19601.60|0.07|0.05|N|O|1995-08-17|1995-07-16|1995-09-13|DELIVER IN PERSON|RAIL|ructions wake slyly al|
+4423|150|9|1|3|3150.45|0.03|0.00|A|F|1995-03-22|1995-04-06|1995-04-19|NONE|TRUCK| final theodolites nag after the bli|
+4423|60|5|2|2|1920.12|0.07|0.04|A|F|1995-03-04|1995-04-04|1995-03-08|TAKE BACK RETURN|REG AIR|old sheaves sleep|
+4448|52|7|1|24|22849.20|0.10|0.07|N|O|1998-09-09|1998-07-06|1998-09-27|DELIVER IN PERSON|SHIP|nal packages along the ironic instructi|
+4448|189|10|2|13|14159.34|0.00|0.01|N|O|1998-07-26|1998-07-03|1998-08-14|COLLECT COD|MAIL|fluffily express accounts integrate furiou|
+4448|41|4|3|35|32936.40|0.10|0.06|N|O|1998-09-18|1998-07-27|1998-10-08|NONE|REG AIR|aggle carefully alongside of the q|
+4448|141|2|4|3|3123.42|0.01|0.01|N|O|1998-07-20|1998-07-10|1998-08-07|DELIVER IN PERSON|TRUCK|ronic theod|
+4448|91|2|5|41|40634.69|0.00|0.08|N|O|1998-07-30|1998-08-09|1998-08-03|NONE|AIR|pon the permanently even excuses nag |
+4448|172|3|6|12|12866.04|0.06|0.03|N|O|1998-08-21|1998-06-30|1998-09-09|COLLECT COD|RAIL|sits about the ironic, bu|
+4449|32|3|1|42|39145.26|0.10|0.07|N|O|1998-03-22|1998-05-09|1998-04-03|NONE|FOB| packages. blithely final |
+4449|141|8|2|10|10411.40|0.02|0.03|N|O|1998-05-09|1998-05-04|1998-05-15|NONE|SHIP|ccounts alongside of the platelets integr|
+4450|174|5|1|44|47263.48|0.10|0.00|N|O|1997-10-12|1997-10-13|1997-10-29|DELIVER IN PERSON|RAIL| the slyly eve|
+4450|15|6|2|9|8235.09|0.03|0.03|N|O|1997-08-13|1997-08-16|1997-08-15|NONE|FOB|gular requests cajole carefully. regular c|
+4450|96|8|3|45|44824.05|0.08|0.01|N|O|1997-09-01|1997-10-06|1997-09-19|NONE|TRUCK|express ideas are furiously regular|
+4450|62|9|4|13|12506.78|0.00|0.00|N|O|1997-08-26|1997-09-18|1997-09-20|COLLECT COD|MAIL| brave foxes. slyly unusual|
+4450|56|7|5|6|5736.30|0.09|0.01|N|O|1997-09-02|1997-09-30|1997-09-09|NONE|FOB|eposits. foxes cajole unusual fox|
+4451|164|5|1|40|42566.40|0.03|0.03|A|F|1994-11-18|1994-12-25|1994-11-26|DELIVER IN PERSON|RAIL|y. slyly special deposits are sly|
+4451|63|4|2|34|32744.04|0.10|0.02|A|F|1994-11-30|1994-12-04|1994-12-13|COLLECT COD|SHIP| regular ideas.|
+4451|159|10|3|19|20123.85|0.05|0.06|R|F|1994-10-09|1994-11-26|1994-10-23|COLLECT COD|FOB|ly after the fluffi|
+4452|114|8|1|21|21296.31|0.07|0.03|R|F|1994-10-06|1994-08-23|1994-10-15|COLLECT COD|TRUCK|multipliers x-ray carefully in place of |
+4452|1|8|2|47|42347.00|0.01|0.06|A|F|1994-10-08|1994-08-09|1994-10-09|TAKE BACK RETURN|TRUCK|ts. slyly regular cour|
+4453|147|10|1|41|42932.74|0.00|0.08|N|O|1997-07-17|1997-05-15|1997-07-31|NONE|REG AIR|anent theodolites are slyly except t|
+4453|133|4|2|16|16530.08|0.03|0.00|N|O|1997-07-22|1997-05-05|1997-08-03|COLLECT COD|FOB|ar excuses nag quickly even accounts. b|
+4453|62|7|3|48|46178.88|0.02|0.07|N|O|1997-05-29|1997-06-24|1997-06-03|NONE|SHIP|eep. fluffily express accounts at the furi|
+4453|102|5|4|26|26054.60|0.06|0.07|N|O|1997-05-07|1997-06-07|1997-05-22|NONE|TRUCK|express packages are|
+4454|151|9|1|20|21023.00|0.10|0.08|R|F|1994-05-06|1994-03-17|1994-05-20|COLLECT COD|SHIP|lar theodolites. even instructio|
+4454|152|10|2|22|23147.30|0.06|0.02|A|F|1994-02-06|1994-04-11|1994-03-06|DELIVER IN PERSON|RAIL|ully. carefully final accounts accordi|
+4454|192|3|3|45|49148.55|0.07|0.04|A|F|1994-03-29|1994-03-26|1994-04-04|TAKE BACK RETURN|RAIL|ests promise. packages print fur|
+4454|2|3|4|1|902.00|0.09|0.05|A|F|1994-02-05|1994-04-19|1994-02-12|COLLECT COD|RAIL|equests run.|
+4454|52|4|5|48|45698.40|0.00|0.07|R|F|1994-04-23|1994-04-03|1994-04-26|COLLECT COD|FOB|to beans wake across th|
+4454|160|8|6|20|21203.20|0.10|0.03|A|F|1994-04-08|1994-03-06|1994-04-26|DELIVER IN PERSON|TRUCK|quickly regular requests. furiously|
+4455|70|5|1|20|19401.40|0.01|0.05|A|F|1994-01-31|1993-11-21|1994-03-02|DELIVER IN PERSON|MAIL| express packages. packages boost quickly|
+4455|153|4|2|47|49498.05|0.09|0.01|R|F|1994-01-01|1993-12-25|1994-01-05|COLLECT COD|FOB| requests. even, even accou|
+4455|123|2|3|34|34786.08|0.00|0.06|A|F|1993-10-24|1993-11-27|1993-11-04|TAKE BACK RETURN|AIR| slyly ironic requests. quickly even d|
+4480|108|5|1|30|30243.00|0.08|0.03|R|F|1994-07-29|1994-06-22|1994-08-01|NONE|FOB|ven braids us|
+4481|24|9|1|50|46201.00|0.02|0.06|N|O|1996-07-22|1996-05-13|1996-08-14|DELIVER IN PERSON|RAIL|ar packages. regula|
+4481|190|1|2|27|29435.13|0.02|0.03|N|O|1996-04-06|1996-05-17|1996-04-12|TAKE BACK RETURN|AIR|ackages haggle even, |
+4482|71|2|1|32|31074.24|0.06|0.03|A|F|1995-05-16|1995-07-22|1995-06-07|NONE|RAIL| quickly pendin|
+4482|96|9|2|32|31874.88|0.01|0.06|N|O|1995-08-16|1995-06-26|1995-09-10|DELIVER IN PERSON|AIR|eans wake according |
+4483|6|7|1|32|28992.00|0.07|0.07|R|F|1992-04-05|1992-05-25|1992-04-08|DELIVER IN PERSON|MAIL|ests haggle. slyl|
+4483|62|1|2|50|48103.00|0.01|0.06|A|F|1992-06-19|1992-05-12|1992-07-08|DELIVER IN PERSON|TRUCK|ag blithely even|
+4483|9|4|3|50|45450.00|0.00|0.04|R|F|1992-06-10|1992-04-18|1992-06-17|DELIVER IN PERSON|MAIL|ackages. furiously ironi|
+4484|95|9|1|4|3980.36|0.06|0.03|N|O|1997-04-09|1997-02-11|1997-04-12|TAKE BACK RETURN|TRUCK|packages de|
+4484|137|8|2|39|40448.07|0.05|0.02|N|O|1997-04-01|1997-01-26|1997-04-21|NONE|RAIL|onic accounts wake blithel|
+4484|190|1|3|38|41427.22|0.06|0.07|N|O|1997-03-07|1997-01-31|1997-04-01|COLLECT COD|REG AIR|. even requests un|
+4484|122|5|4|41|41906.92|0.06|0.03|N|O|1997-01-25|1997-02-15|1997-01-29|TAKE BACK RETURN|REG AIR|ress accounts. ironic deposits unwind fur|
+4484|3|4|5|42|37926.00|0.03|0.07|N|O|1997-03-25|1997-02-21|1997-04-05|DELIVER IN PERSON|REG AIR|ding, pending requests wake. fluffily |
+4484|36|7|6|29|27144.87|0.09|0.06|N|O|1996-12-27|1997-03-10|1997-01-13|NONE|FOB| wake blithely ironic|
+4484|103|8|7|50|50155.00|0.07|0.01|N|O|1997-03-17|1997-03-16|1997-03-21|COLLECT COD|FOB|the ironic, final theodo|
+4485|191|5|1|1|1091.19|0.03|0.05|R|F|1994-12-04|1995-02-07|1994-12-09|NONE|AIR|play according to the ironic, ironic|
+4485|141|10|2|46|47892.44|0.04|0.06|R|F|1995-03-09|1994-12-14|1995-03-23|DELIVER IN PERSON|AIR|. ironic foxes haggle. regular war|
+4485|175|6|3|43|46232.31|0.01|0.05|R|F|1995-01-17|1995-02-11|1995-02-07|DELIVER IN PERSON|TRUCK|al accounts according to the slyly r|
+4485|144|5|4|43|44898.02|0.08|0.06|R|F|1995-01-28|1995-01-26|1995-02-07|DELIVER IN PERSON|AIR|. blithely|
+4485|6|7|5|47|42582.00|0.08|0.04|R|F|1995-03-11|1995-01-11|1995-03-21|TAKE BACK RETURN|RAIL|luffily pending acc|
+4486|135|1|1|46|47615.98|0.08|0.00|N|O|1998-05-02|1998-04-05|1998-05-08|COLLECT COD|MAIL|ackages. specia|
+4486|49|2|2|19|18031.76|0.10|0.01|N|O|1998-06-07|1998-05-28|1998-07-02|NONE|MAIL|pending foxes after|
+4486|96|7|3|47|46816.23|0.02|0.07|N|O|1998-04-09|1998-05-24|1998-05-07|DELIVER IN PERSON|MAIL|ts around the quiet packages ar|
+4486|91|4|4|28|27750.52|0.07|0.02|N|O|1998-04-21|1998-04-19|1998-04-26|TAKE BACK RETURN|AIR|to the furious, regular foxes play abov|
+4487|138|4|1|37|38410.81|0.03|0.07|R|F|1993-02-28|1993-04-18|1993-03-17|TAKE BACK RETURN|MAIL|bove the fu|
+4487|113|10|2|49|49642.39|0.10|0.00|R|F|1993-06-13|1993-05-08|1993-07-10|COLLECT COD|FOB|sual packages should ha|
+4487|190|1|3|1|1090.19|0.02|0.07|A|F|1993-05-11|1993-05-23|1993-05-17|TAKE BACK RETURN|FOB|ithely final asym|
+4487|93|4|4|25|24827.25|0.07|0.03|A|F|1993-03-09|1993-04-27|1993-03-30|COLLECT COD|RAIL|g the final instructions. slyly c|
+4512|162|1|1|30|31864.80|0.07|0.07|N|O|1996-01-28|1995-12-22|1996-02-22|TAKE BACK RETURN|TRUCK|ly unusual package|
+4512|41|4|2|24|22584.96|0.04|0.06|N|O|1995-12-16|1996-01-16|1995-12-25|NONE|SHIP|ly regular pinto beans. carefully bold depo|
+4512|145|8|3|21|21947.94|0.00|0.00|N|O|1995-10-31|1995-12-30|1995-11-15|NONE|REG AIR|lly unusual pinto b|
+4512|141|2|4|32|33316.48|0.10|0.01|N|O|1995-11-25|1995-12-28|1995-12-06|NONE|FOB|counts are against the quickly regular |
+4512|133|4|5|43|44424.59|0.06|0.00|N|O|1995-12-20|1995-11-28|1996-01-14|NONE|AIR|are carefully. theodolites wake|
+4513|170|1|1|29|31034.93|0.03|0.01|N|O|1996-05-18|1996-05-23|1996-06-08|NONE|REG AIR|cajole. regular packages boost. s|
+4513|70|9|2|39|37832.73|0.01|0.04|N|O|1996-06-25|1996-05-14|1996-07-24|NONE|MAIL|slyly furiously unusual deposits. blit|
+4513|138|4|3|34|35296.42|0.00|0.03|N|O|1996-03-27|1996-06-12|1996-04-06|DELIVER IN PERSON|SHIP|sits. quickly even instructions |
+4513|192|6|4|13|14198.47|0.08|0.08|N|O|1996-04-12|1996-05-19|1996-04-25|DELIVER IN PERSON|AIR|l, final excuses detect furi|
+4514|164|9|1|27|28732.32|0.06|0.06|R|F|1994-07-01|1994-07-13|1994-07-26|COLLECT COD|AIR| even, silent foxes be|
+4514|46|3|2|15|14190.60|0.10|0.04|R|F|1994-08-24|1994-07-11|1994-09-14|DELIVER IN PERSON|RAIL|! unusual, special deposits afte|
+4514|78|8|3|10|9780.70|0.09|0.05|A|F|1994-06-19|1994-06-25|1994-07-01|COLLECT COD|SHIP|ake furiously. carefully regular requests|
+4514|81|2|4|9|8829.72|0.10|0.03|A|F|1994-08-04|1994-07-01|1994-09-01|DELIVER IN PERSON|REG AIR|wly. quick|
+4514|149|8|5|12|12589.68|0.02|0.03|R|F|1994-08-20|1994-06-09|1994-09-15|TAKE BACK RETURN|FOB| carefully ironic foxes nag caref|
+4514|189|10|6|38|41388.84|0.03|0.05|A|F|1994-07-28|1994-07-06|1994-08-25|NONE|AIR|ending excuses. sl|
+4514|177|8|7|27|29083.59|0.04|0.06|A|F|1994-06-24|1994-07-14|1994-06-30|TAKE BACK RETURN|TRUCK|. slyly sile|
+4515|39|10|1|15|14085.45|0.06|0.01|R|F|1992-05-26|1992-05-25|1992-06-03|NONE|SHIP|posits wake|
+4515|103|10|2|50|50155.00|0.06|0.03|A|F|1992-03-28|1992-05-16|1992-04-20|NONE|AIR|ding instructions again|
+4515|154|6|3|27|28462.05|0.09|0.01|A|F|1992-06-06|1992-06-08|1992-06-07|DELIVER IN PERSON|REG AIR| against the even re|
+4515|54|5|4|32|30529.60|0.06|0.03|R|F|1992-04-07|1992-05-11|1992-04-09|COLLECT COD|MAIL|carefully express depo|
+4515|45|8|5|22|20790.88|0.09|0.07|A|F|1992-07-16|1992-05-07|1992-07-23|NONE|SHIP|le quickly above the even, bold ideas.|
+4515|180|8|6|23|24844.14|0.04|0.00|R|F|1992-05-23|1992-06-15|1992-06-20|TAKE BACK RETURN|FOB|ns. bold r|
+4516|170|9|1|34|36385.78|0.05|0.04|A|F|1994-05-16|1994-06-23|1994-06-12|NONE|SHIP|even pinto beans wake qui|
+4517|43|4|1|50|47152.00|0.01|0.02|N|O|1998-06-08|1998-04-18|1998-06-20|DELIVER IN PERSON|MAIL|refully pending acco|
+4518|144|7|1|9|9397.26|0.09|0.04|N|O|1997-06-26|1997-07-07|1997-07-10|NONE|RAIL| pending deposits. slyly re|
+4518|45|6|2|19|17955.76|0.10|0.05|N|O|1997-08-09|1997-06-06|1997-08-27|COLLECT COD|RAIL|ter the slyly bo|
+4519|55|3|1|30|28651.50|0.09|0.07|R|F|1993-04-11|1993-06-05|1993-04-22|DELIVER IN PERSON|REG AIR|totes. slyly bold somas after the |
+4519|191|3|2|37|40374.03|0.06|0.08|R|F|1993-07-22|1993-06-16|1993-08-19|COLLECT COD|AIR|ly slyly furious depth|
+4544|131|7|1|40|41245.20|0.07|0.01|N|O|1997-08-15|1997-10-16|1997-08-20|DELIVER IN PERSON|RAIL| detect slyly. evenly pending instru|
+4544|172|2|2|19|20371.23|0.08|0.01|N|O|1997-08-14|1997-09-08|1997-08-25|NONE|SHIP|regular ideas are furiously about|
+4544|71|9|3|20|19421.40|0.02|0.07|N|O|1997-10-12|1997-10-11|1997-10-13|COLLECT COD|REG AIR| waters about the|
+4544|51|6|4|39|37090.95|0.07|0.05|N|O|1997-08-20|1997-09-07|1997-08-27|COLLECT COD|REG AIR|ular packages. s|
+4544|133|4|5|31|32027.03|0.09|0.03|N|O|1997-08-09|1997-09-29|1997-08-17|COLLECT COD|TRUCK|dolites detect quickly reg|
+4544|27|8|6|8|7416.16|0.10|0.03|N|O|1997-10-13|1997-10-06|1997-10-25|COLLECT COD|AIR|olites. fi|
+4545|173|1|1|38|40780.46|0.06|0.06|R|F|1993-01-27|1993-03-01|1993-02-04|NONE|TRUCK|nts serve according to th|
+4545|63|4|2|27|26002.62|0.01|0.06|R|F|1993-02-07|1993-02-18|1993-02-18|NONE|FOB|ously bold asymptotes! blithely pen|
+4545|87|8|3|9|8883.72|0.10|0.06|R|F|1993-03-20|1993-02-23|1993-04-11|TAKE BACK RETURN|AIR|xpress accounts|
+4545|64|9|4|2|1928.12|0.10|0.00|R|F|1993-04-16|1993-04-17|1993-05-03|NONE|REG AIR|ages use. slyly even i|
+4545|117|1|5|27|27461.97|0.08|0.05|A|F|1993-03-18|1993-02-22|1993-03-23|NONE|RAIL|ccounts haggle carefully. deposits |
+4545|109|2|6|8|8072.80|0.03|0.02|A|F|1993-05-01|1993-03-12|1993-05-15|NONE|FOB| boost slyly. slyly|
+4545|9|2|7|36|32724.00|0.10|0.04|R|F|1993-01-28|1993-03-30|1993-02-04|DELIVER IN PERSON|SHIP|sublate slyly. furiously ironic accounts b|
+4546|133|4|1|10|10331.30|0.09|0.02|N|O|1995-09-23|1995-10-10|1995-10-23|COLLECT COD|TRUCK|osits alongside of the|
+4546|171|10|2|15|16067.55|0.04|0.07|N|O|1995-07-31|1995-10-17|1995-08-06|NONE|REG AIR|ught to cajole furiously. qu|
+4546|77|8|3|4|3908.28|0.06|0.08|N|O|1995-08-14|1995-10-07|1995-08-16|COLLECT COD|MAIL|kly pending dependencies along the furio|
+4546|149|6|4|10|10491.40|0.08|0.02|N|O|1995-09-02|1995-09-16|1995-09-10|DELIVER IN PERSON|FOB|above the enticingly ironic dependencies|
+4547|188|9|1|15|16322.70|0.10|0.04|A|F|1993-12-08|1993-11-15|1993-12-22|NONE|REG AIR|ets haggle. regular dinos affix fu|
+4547|116|10|2|7|7112.77|0.10|0.02|A|F|1993-09-04|1993-09-29|1993-09-20|COLLECT COD|RAIL|slyly express a|
+4547|45|2|3|15|14175.60|0.00|0.00|R|F|1993-11-18|1993-10-06|1993-12-13|NONE|TRUCK|e carefully across the unus|
+4547|148|7|4|15|15722.10|0.05|0.08|R|F|1993-11-29|1993-10-12|1993-12-29|COLLECT COD|REG AIR|ironic gifts integrate |
+4548|14|8|1|21|19194.21|0.10|0.05|N|O|1996-07-11|1996-09-04|1996-07-30|COLLECT COD|REG AIR|pecial theodoli|
+4548|47|10|2|17|16099.68|0.00|0.08|N|O|1996-07-23|1996-09-21|1996-07-26|DELIVER IN PERSON|REG AIR|y ironic requests above the fluffily d|
+4548|123|2|3|47|48086.64|0.05|0.04|N|O|1996-07-24|1996-09-12|1996-08-08|NONE|MAIL|ts. excuses use slyly spec|
+4548|177|6|4|22|23697.74|0.07|0.01|N|O|1996-07-06|1996-08-23|1996-07-15|DELIVER IN PERSON|RAIL|s. furiously ironic theodolites c|
+4548|45|4|5|36|34021.44|0.04|0.06|N|O|1996-08-19|1996-09-12|1996-09-08|COLLECT COD|FOB|tions integrat|
+4549|159|1|1|44|46602.60|0.08|0.00|N|O|1998-03-13|1998-04-15|1998-03-27|TAKE BACK RETURN|TRUCK|ding to the regular, silent requests|
+4549|89|10|2|1|989.08|0.05|0.08|N|O|1998-05-04|1998-04-11|1998-05-14|TAKE BACK RETURN|AIR| requests wake. furiously even |
+4550|150|7|1|9|9451.35|0.05|0.06|R|F|1995-04-19|1995-02-07|1995-04-24|COLLECT COD|SHIP|l dependencies boost slyly after th|
+4550|66|5|2|19|18355.14|0.06|0.04|A|F|1995-01-01|1995-02-13|1995-01-20|NONE|AIR|quests. express |
+4551|11|1|1|6|5466.06|0.08|0.08|N|O|1996-05-18|1996-04-23|1996-06-13|DELIVER IN PERSON|TRUCK|fily silent fo|
+4551|179|8|2|26|28058.42|0.02|0.04|N|O|1996-04-14|1996-04-26|1996-04-17|TAKE BACK RETURN|RAIL|le. carefully dogged accounts use furiousl|
+4551|22|1|3|22|20284.44|0.08|0.01|N|O|1996-05-12|1996-03-17|1996-05-29|TAKE BACK RETURN|REG AIR|ly ironic reques|
+4551|198|10|4|27|29651.13|0.00|0.01|N|O|1996-04-28|1996-03-22|1996-05-22|TAKE BACK RETURN|RAIL|y along the slyly even |
+4576|90|1|1|5|4950.45|0.09|0.03|N|O|1996-08-23|1996-11-08|1996-09-20|TAKE BACK RETURN|AIR|ly express, special asymptote|
+4576|58|9|2|43|41196.15|0.08|0.06|N|O|1996-10-24|1996-09-23|1996-11-10|NONE|SHIP|ly final deposits. never|
+4576|42|1|3|14|13188.56|0.09|0.01|N|O|1996-09-12|1996-09-30|1996-09-24|COLLECT COD|MAIL|detect slyly.|
+4577|185|6|1|43|46662.74|0.01|0.03|N|O|1998-06-16|1998-07-09|1998-06-17|TAKE BACK RETURN|AIR|packages. |
+4577|177|6|2|43|46318.31|0.05|0.03|N|O|1998-08-24|1998-06-02|1998-09-14|TAKE BACK RETURN|RAIL|ly accounts. carefully |
+4577|69|6|3|12|11628.72|0.07|0.05|N|O|1998-07-29|1998-06-17|1998-08-04|DELIVER IN PERSON|TRUCK|equests alongsi|
+4578|74|2|1|10|9740.70|0.09|0.06|R|F|1993-01-01|1992-11-19|1993-01-28|TAKE BACK RETURN|REG AIR|uests. blithely unus|
+4578|169|10|2|42|44904.72|0.06|0.00|R|F|1993-01-05|1992-11-06|1993-01-13|DELIVER IN PERSON|FOB|s are caref|
+4578|179|8|3|15|16187.55|0.01|0.01|R|F|1992-10-23|1992-11-22|1992-11-09|DELIVER IN PERSON|REG AIR|gular theodo|
+4578|139|10|4|7|7273.91|0.09|0.08|A|F|1992-12-07|1992-11-27|1993-01-05|TAKE BACK RETURN|SHIP|odolites. carefully unusual ideas accor|
+4578|163|2|5|20|21263.20|0.04|0.02|A|F|1993-01-11|1992-11-09|1993-01-23|TAKE BACK RETURN|RAIL|iously pending theodolites--|
+4579|175|4|1|14|15052.38|0.02|0.02|N|O|1996-02-01|1996-01-08|1996-02-08|TAKE BACK RETURN|MAIL|nding theodolites. fluffil|
+4579|42|3|2|28|26377.12|0.02|0.05|N|O|1996-01-22|1996-02-13|1996-02-03|DELIVER IN PERSON|RAIL|slyly across the |
+4579|178|9|3|34|36657.78|0.05|0.02|N|O|1996-02-26|1996-02-22|1996-03-16|COLLECT COD|MAIL|hely. carefully blithe dependen|
+4579|120|1|4|8|8160.96|0.05|0.06|N|O|1995-12-16|1996-01-15|1995-12-18|TAKE BACK RETURN|AIR|posits. carefully perman|
+4580|92|5|1|22|21825.98|0.01|0.05|A|F|1994-01-16|1994-01-26|1994-02-05|COLLECT COD|AIR|nticingly final packag|
+4580|32|3|2|10|9320.30|0.05|0.04|R|F|1993-12-20|1993-12-30|1994-01-17|COLLECT COD|RAIL|gular, pending deposits. fina|
+4580|1|8|3|41|36941.00|0.00|0.07|R|F|1993-12-13|1994-01-31|1994-01-06|NONE|SHIP|requests. quickly silent asymptotes sle|
+4580|178|8|4|5|5390.85|0.07|0.00|A|F|1994-01-28|1993-12-17|1994-02-22|NONE|TRUCK|o beans. f|
+4580|189|10|5|39|42478.02|0.03|0.02|R|F|1993-12-28|1993-12-26|1994-01-23|NONE|RAIL|. fluffily final dolphins use furiously al|
+4581|165|4|1|37|39410.92|0.01|0.04|A|F|1992-10-17|1992-11-05|1992-11-04|DELIVER IN PERSON|MAIL|e the blithely bold pearls ha|
+4581|50|3|2|7|6650.35|0.01|0.02|A|F|1992-10-09|1992-10-20|1992-10-21|TAKE BACK RETURN|MAIL|express accounts d|
+4581|21|10|3|46|42366.92|0.04|0.04|A|F|1992-09-09|1992-11-27|1992-09-26|NONE|REG AIR|nag toward the carefully final accounts. |
+4582|192|5|1|17|18567.23|0.09|0.08|N|O|1996-08-17|1996-08-26|1996-08-20|COLLECT COD|REG AIR|ng packages. depo|
+4583|141|2|1|17|17699.38|0.01|0.05|A|F|1994-11-08|1994-11-03|1994-11-29|COLLECT COD|MAIL|romise. reques|
+4583|187|8|2|43|46748.74|0.04|0.04|A|F|1994-10-30|1994-12-17|1994-11-16|COLLECT COD|RAIL|fully after the speci|
+4583|196|10|3|28|30693.32|0.00|0.07|A|F|1994-10-29|1994-11-21|1994-11-28|NONE|SHIP|to beans haggle sly|
+4583|173|4|4|27|28975.59|0.08|0.03|R|F|1995-01-11|1994-12-24|1995-02-10|DELIVER IN PERSON|TRUCK| detect silent requests. furiously speci|
+4583|184|5|5|36|39030.48|0.09|0.06|A|F|1995-01-06|1994-11-25|1995-01-29|DELIVER IN PERSON|RAIL|ar requests haggle after the furiously |
+4583|122|7|6|14|14309.68|0.09|0.01|R|F|1994-11-17|1994-11-08|1994-11-21|DELIVER IN PERSON|AIR|detect. doggedly regular pi|
+4583|87|8|7|32|31586.56|0.04|0.00|A|F|1995-01-13|1994-10-29|1995-02-08|TAKE BACK RETURN|RAIL|across the pinto beans-- quickly|
+4608|173|1|1|30|32195.10|0.08|0.05|R|F|1994-10-08|1994-07-18|1994-10-25|DELIVER IN PERSON|SHIP|s cajole. slyly |
+4608|47|8|2|50|47352.00|0.06|0.01|A|F|1994-07-25|1994-09-01|1994-08-10|NONE|FOB| theodolites|
+4608|79|9|3|50|48953.50|0.03|0.01|A|F|1994-08-04|1994-09-10|1994-08-13|COLLECT COD|TRUCK| wake closely. even decoys haggle above|
+4608|31|2|4|36|33517.08|0.05|0.06|R|F|1994-10-04|1994-08-02|1994-10-21|COLLECT COD|FOB|ages wake quickly slyly iron|
+4609|47|6|1|28|26517.12|0.10|0.05|N|O|1997-02-02|1997-02-17|1997-03-02|DELIVER IN PERSON|REG AIR|ously. quickly final requests cajole fl|
+4609|185|6|2|3|3255.54|0.09|0.03|N|O|1996-12-28|1997-02-06|1997-01-20|NONE|FOB|nstructions. furious instructions |
+4609|23|4|3|46|42458.92|0.05|0.05|N|O|1997-02-11|1997-01-16|1997-03-07|NONE|FOB|r foxes. fluffily ironic ideas ha|
+4610|87|8|1|21|20728.68|0.07|0.07|R|F|1993-08-10|1993-08-05|1993-08-27|NONE|REG AIR|ly special theodolites. even,|
+4610|175|5|2|14|15052.38|0.00|0.07|R|F|1993-07-28|1993-07-25|1993-07-31|TAKE BACK RETURN|SHIP| ironic frays. dependencies detect blithel|
+4610|159|1|3|44|46602.60|0.05|0.03|A|F|1993-08-05|1993-07-20|1993-08-19|COLLECT COD|TRUCK| final theodolites |
+4610|75|3|4|26|25351.82|0.06|0.03|R|F|1993-07-01|1993-07-19|1993-07-19|NONE|MAIL| to the fluffily ironic requests h|
+4610|147|8|5|29|30367.06|0.08|0.04|R|F|1993-08-09|1993-07-27|1993-08-16|DELIVER IN PERSON|AIR| foxes. special, express package|
+4611|52|7|1|47|44746.35|0.09|0.06|A|F|1993-03-05|1993-03-01|1993-03-17|COLLECT COD|TRUCK|iously. furiously regular|
+4611|35|6|2|31|28985.93|0.04|0.02|A|F|1993-01-28|1993-02-14|1993-01-29|TAKE BACK RETURN|AIR| final pinto beans. permanent, sp|
+4611|82|3|3|50|49104.00|0.08|0.01|R|F|1993-01-22|1993-03-30|1993-02-16|TAKE BACK RETURN|AIR|l platelets. |
+4611|71|9|4|48|46611.36|0.02|0.08|R|F|1993-02-28|1993-02-12|1993-03-01|COLLECT COD|AIR|ular accounts |
+4612|6|9|1|20|18120.00|0.02|0.03|R|F|1993-09-24|1993-12-18|1993-10-22|NONE|AIR|beans sleep blithely iro|
+4612|50|7|2|17|16150.85|0.10|0.06|A|F|1994-01-09|1993-11-08|1994-02-06|TAKE BACK RETURN|REG AIR|equests haggle carefully silent excus|
+4612|137|8|3|40|41485.20|0.08|0.01|R|F|1993-10-08|1993-11-23|1993-10-24|DELIVER IN PERSON|RAIL|special platelets.|
+4612|185|6|4|10|10851.80|0.10|0.06|A|F|1993-11-11|1993-11-19|1993-11-13|TAKE BACK RETURN|SHIP|unusual theodol|
+4613|38|9|1|17|15946.51|0.09|0.07|N|O|1998-06-07|1998-05-11|1998-06-29|DELIVER IN PERSON|SHIP|liers cajole a|
+4613|108|1|2|25|25202.50|0.05|0.04|N|O|1998-05-22|1998-04-11|1998-05-27|TAKE BACK RETURN|SHIP|y pending platelets x-ray ironically! pend|
+4613|174|3|3|15|16112.55|0.10|0.02|N|O|1998-05-31|1998-04-16|1998-06-25|DELIVER IN PERSON|MAIL|against the quickly r|
+4613|8|1|4|36|32688.00|0.04|0.01|N|O|1998-04-22|1998-05-05|1998-05-04|DELIVER IN PERSON|AIR|gainst the furiously ironic|
+4613|111|8|5|35|35388.85|0.04|0.06|N|O|1998-06-04|1998-04-17|1998-06-20|COLLECT COD|MAIL|e blithely against the even, bold pi|
+4613|196|8|6|47|51520.93|0.04|0.04|N|O|1998-07-03|1998-05-26|1998-07-09|NONE|FOB|uriously special requests wak|
+4613|119|3|7|39|39745.29|0.09|0.05|N|O|1998-06-12|1998-06-01|1998-07-06|DELIVER IN PERSON|REG AIR|ously express|
+4614|7|2|1|19|17233.00|0.09|0.08|N|O|1996-05-17|1996-06-21|1996-06-08|TAKE BACK RETURN|AIR|ix. carefully regular |
+4614|65|6|2|3|2895.18|0.08|0.01|N|O|1996-07-22|1996-07-21|1996-08-07|NONE|MAIL|ions engage final, ironic |
+4614|8|1|3|36|32688.00|0.10|0.04|N|O|1996-07-05|1996-06-26|1996-07-07|NONE|REG AIR|onic foxes affix furi|
+4614|126|9|4|6|6156.72|0.09|0.01|N|O|1996-06-11|1996-05-30|1996-07-03|COLLECT COD|REG AIR|ake quickly quickly regular epitap|
+4614|73|3|5|24|23353.68|0.07|0.06|N|O|1996-07-01|1996-06-24|1996-07-08|COLLECT COD|REG AIR|regular, even|
+4614|34|5|6|32|29888.96|0.10|0.05|N|O|1996-08-21|1996-05-28|1996-09-16|NONE|REG AIR|ickly furio|
+4614|128|1|7|41|42152.92|0.01|0.07|N|O|1996-07-31|1996-07-12|1996-08-16|COLLECT COD|REG AIR|ackages haggle carefully about the even, b|
+4615|92|4|1|10|9920.90|0.02|0.08|A|F|1993-11-20|1993-10-05|1993-12-08|DELIVER IN PERSON|AIR|sits. slyly express deposits are|
+4640|88|9|1|5|4940.40|0.03|0.08|N|O|1996-02-05|1996-02-14|1996-02-15|TAKE BACK RETURN|RAIL| warthogs against the regular|
+4640|88|9|2|9|8892.72|0.03|0.05|N|O|1996-02-12|1996-02-14|1996-02-29|DELIVER IN PERSON|AIR| accounts. unu|
+4640|27|10|3|18|16686.36|0.02|0.07|N|O|1996-02-28|1996-03-06|1996-03-28|DELIVER IN PERSON|RAIL|boost furiously accord|
+4640|23|2|4|36|33228.72|0.06|0.08|N|O|1996-01-03|1996-03-09|1996-01-11|DELIVER IN PERSON|RAIL|iously furious accounts boost. carefully|
+4640|156|1|5|15|15842.25|0.03|0.02|N|O|1996-03-19|1996-02-09|1996-04-11|TAKE BACK RETURN|FOB|y regular instructions doze furiously. reg|
+4641|190|1|1|45|49058.55|0.07|0.03|R|F|1993-05-11|1993-04-19|1993-05-21|DELIVER IN PERSON|MAIL| about the close |
+4641|95|7|2|39|38808.51|0.06|0.00|R|F|1993-02-10|1993-03-06|1993-02-15|TAKE BACK RETURN|REG AIR| the bold reque|
+4641|36|7|3|15|14040.45|0.01|0.08|R|F|1993-01-25|1993-04-09|1993-02-05|TAKE BACK RETURN|AIR|s. carefully even exc|
+4642|194|7|1|11|12036.09|0.04|0.07|A|F|1995-05-23|1995-04-26|1995-06-04|COLLECT COD|TRUCK|lithely express asympt|
+4642|180|10|2|34|36726.12|0.04|0.07|R|F|1995-04-01|1995-05-11|1995-04-23|COLLECT COD|SHIP|theodolites detect among the ironically sp|
+4642|21|2|3|10|9210.20|0.04|0.02|R|F|1995-04-16|1995-04-28|1995-04-24|COLLECT COD|RAIL|urts. even deposits nag beneath |
+4642|94|7|4|18|17893.62|0.00|0.04|N|F|1995-06-16|1995-04-16|1995-06-21|NONE|TRUCK|ily pending accounts hag|
+4642|179|10|5|41|44245.97|0.10|0.00|R|F|1995-04-08|1995-04-13|1995-05-01|DELIVER IN PERSON|MAIL|s are blithely. requests wake above the fur|
+4643|185|6|1|50|54259.00|0.08|0.05|N|O|1995-09-11|1995-08-13|1995-09-30|DELIVER IN PERSON|SHIP|. ironic deposits cajo|
+4644|177|7|1|4|4308.68|0.06|0.03|N|O|1998-05-06|1998-03-19|1998-05-28|NONE|MAIL|gular requests? pendi|
+4644|97|8|2|16|15953.44|0.03|0.04|N|O|1998-03-13|1998-02-21|1998-04-03|COLLECT COD|SHIP|lar excuses across the |
+4644|115|9|3|10|10151.10|0.02|0.02|N|O|1998-02-21|1998-02-28|1998-03-19|COLLECT COD|REG AIR|osits according to the|
+4644|154|2|4|45|47436.75|0.10|0.07|N|O|1998-02-02|1998-04-08|1998-02-15|COLLECT COD|SHIP| carefully a|
+4644|87|8|5|10|9870.80|0.08|0.08|N|O|1998-03-12|1998-03-11|1998-03-19|TAKE BACK RETURN|REG AIR| the slow, final fo|
+4645|50|7|1|45|42752.25|0.09|0.05|A|F|1994-12-27|1994-11-02|1994-12-31|DELIVER IN PERSON|AIR|ular ideas. slyly|
+4645|66|7|2|32|30913.92|0.10|0.08|A|F|1994-11-17|1994-10-30|1994-11-18|COLLECT COD|REG AIR| final accounts alongside|
+4645|54|5|3|25|23851.25|0.03|0.00|R|F|1994-10-25|1994-12-11|1994-11-14|NONE|REG AIR|braids. ironic dependencies main|
+4645|37|8|4|42|39355.26|0.10|0.02|R|F|1994-12-02|1994-12-18|1994-12-16|COLLECT COD|TRUCK|regular pinto beans amon|
+4645|161|10|5|35|37140.60|0.03|0.07|A|F|1994-12-08|1994-11-25|1994-12-09|TAKE BACK RETURN|FOB|sias believe bl|
+4645|42|9|6|27|25435.08|0.09|0.08|R|F|1994-11-26|1994-10-25|1994-12-04|NONE|SHIP|ously express pinto beans. ironic depos|
+4645|31|2|7|42|39103.26|0.10|0.06|A|F|1994-12-31|1994-10-22|1995-01-28|DELIVER IN PERSON|AIR|e slyly regular pinto beans. thin|
+4646|191|3|1|24|26188.56|0.02|0.05|N|O|1996-09-18|1996-08-09|1996-09-21|TAKE BACK RETURN|RAIL|ic platelets lose carefully. blithely unu|
+4646|178|6|2|26|28032.42|0.07|0.00|N|O|1996-10-02|1996-08-25|1996-10-27|DELIVER IN PERSON|MAIL|ix according to the slyly spe|
+4646|34|10|3|18|16812.54|0.01|0.00|N|O|1996-06-30|1996-08-10|1996-07-12|TAKE BACK RETURN|TRUCK|beans sleep car|
+4646|40|1|4|38|35721.52|0.08|0.01|N|O|1996-09-01|1996-08-23|1996-09-27|COLLECT COD|SHIP|al platelets cajole. slyly final dol|
+4646|26|1|5|22|20372.44|0.01|0.08|N|O|1996-07-14|1996-08-06|1996-07-29|DELIVER IN PERSON|MAIL|cies are blithely after the slyly reg|
+4647|93|6|1|16|15889.44|0.09|0.07|R|F|1994-09-07|1994-07-15|1994-10-06|COLLECT COD|RAIL|o beans about the fluffily special the|
+4647|129|2|2|34|34990.08|0.01|0.02|R|F|1994-05-20|1994-06-20|1994-05-29|COLLECT COD|TRUCK|ly sly accounts|
+4647|147|8|3|27|28272.78|0.03|0.08|R|F|1994-05-20|1994-06-26|1994-05-30|NONE|FOB|ully even ti|
+4647|139|10|4|2|2078.26|0.04|0.07|R|F|1994-07-03|1994-07-22|1994-07-22|TAKE BACK RETURN|RAIL|dolites wake furiously special pinto be|
+4647|187|8|5|2|2174.36|0.07|0.06|A|F|1994-05-27|1994-08-05|1994-06-10|TAKE BACK RETURN|FOB| pinto beans believe furiously slyly silent|
+4647|29|4|6|28|26012.56|0.02|0.03|A|F|1994-08-25|1994-08-06|1994-09-18|DELIVER IN PERSON|FOB| are above the fluffily fin|
+4672|59|7|1|22|21099.10|0.01|0.07|N|O|1995-12-03|1995-12-08|1995-12-17|COLLECT COD|AIR|l instructions. blithely ironic packages |
+4672|61|10|2|41|39403.46|0.00|0.00|N|O|1995-12-01|1995-12-15|1995-12-12|COLLECT COD|RAIL| slyly quie|
+4672|163|10|3|24|25515.84|0.04|0.03|N|O|1995-11-11|1995-12-28|1995-12-04|NONE|REG AIR|y fluffily stealt|
+4672|57|2|4|13|12441.65|0.10|0.03|N|O|1996-02-02|1995-12-13|1996-03-02|DELIVER IN PERSON|RAIL|ar requests? pending accounts against|
+4672|55|10|5|45|42977.25|0.08|0.07|N|O|1996-02-07|1996-01-16|1996-02-14|DELIVER IN PERSON|MAIL| platelets use amon|
+4672|141|8|6|20|20822.80|0.02|0.07|N|O|1995-12-08|1996-01-25|1995-12-19|COLLECT COD|REG AIR|s boost at the ca|
+4672|72|10|7|38|36938.66|0.01|0.01|N|O|1995-11-28|1995-12-08|1995-12-13|COLLECT COD|SHIP|ests. idle, regular ex|
+4673|17|8|1|8|7336.08|0.08|0.01|N|O|1996-10-12|1996-10-05|1996-11-04|TAKE BACK RETURN|FOB|lithely final re|
+4673|101|2|2|44|44048.40|0.06|0.01|N|O|1996-12-11|1996-10-31|1997-01-08|DELIVER IN PERSON|RAIL| gifts cajole dari|
+4673|123|2|3|9|9208.08|0.04|0.07|N|O|1996-10-15|1996-09-30|1996-10-30|DELIVER IN PERSON|MAIL|ages nag across |
+4674|150|7|1|50|52507.50|0.07|0.08|A|F|1994-05-13|1994-06-15|1994-06-05|COLLECT COD|RAIL|haggle about the blithel|
+4674|189|10|2|35|38121.30|0.02|0.05|A|F|1994-08-02|1994-06-04|1994-08-21|COLLECT COD|FOB|le quickly after the express sent|
+4674|111|5|3|3|3033.33|0.01|0.05|A|F|1994-07-19|1994-05-28|1994-07-23|TAKE BACK RETURN|RAIL| regular requests na|
+4674|13|7|4|21|19173.21|0.02|0.08|R|F|1994-05-08|1994-07-02|1994-06-04|COLLECT COD|RAIL|ent accounts sublate deposits. instruc|
+4675|171|2|1|6|6427.02|0.00|0.05|R|F|1994-01-22|1994-01-06|1994-02-12|TAKE BACK RETURN|TRUCK| unusual ideas thrash bl|
+4675|144|7|2|12|12529.68|0.00|0.04|A|F|1993-12-22|1994-01-12|1993-12-23|TAKE BACK RETURN|AIR|posits affix carefully|
+4675|181|2|3|5|5405.90|0.05|0.05|A|F|1994-01-16|1994-01-05|1994-01-18|DELIVER IN PERSON|RAIL|lent pinto beans|
+4675|34|10|4|26|24284.78|0.03|0.01|A|F|1993-12-16|1993-12-29|1993-12-23|DELIVER IN PERSON|SHIP|nts. express requests are quickly |
+4675|81|2|5|18|17659.44|0.01|0.08|R|F|1994-02-23|1994-01-18|1994-03-05|TAKE BACK RETURN|FOB|cajole unusual dep|
+4675|119|10|6|1|1019.11|0.10|0.06|R|F|1994-03-18|1994-02-14|1994-04-17|NONE|SHIP|unts. caref|
+4676|165|2|1|47|50062.52|0.03|0.06|N|O|1995-12-20|1995-10-04|1996-01-09|NONE|AIR|lithely about the carefully special requ|
+4676|6|1|2|33|29898.00|0.08|0.05|N|O|1995-12-29|1995-10-01|1996-01-18|TAKE BACK RETURN|FOB|yly express |
+4676|146|3|3|4|4184.56|0.10|0.06|N|O|1995-12-12|1995-10-22|1995-12-13|TAKE BACK RETURN|TRUCK|detect above the ironic platelets. fluffily|
+4676|111|2|4|50|50555.50|0.07|0.01|N|O|1995-09-20|1995-11-20|1995-10-18|TAKE BACK RETURN|AIR|r deposits boost boldly quickly quick asymp|
+4676|122|7|5|29|29641.48|0.01|0.02|N|O|1995-12-29|1995-11-12|1996-01-06|TAKE BACK RETURN|RAIL|ly regular theodolites sleep.|
+4676|46|7|6|8|7568.32|0.08|0.08|N|O|1995-12-05|1995-10-18|1996-01-02|COLLECT COD|AIR|cuses boost above|
+4676|64|1|7|13|12532.78|0.05|0.07|N|O|1995-11-18|1995-11-07|1995-12-10|TAKE BACK RETURN|TRUCK| at the slyly bold attainments. silently e|
+4677|128|3|1|25|25703.00|0.04|0.04|N|O|1998-04-11|1998-05-11|1998-04-18|TAKE BACK RETURN|REG AIR|unts doubt furiousl|
+4678|58|6|1|35|33531.75|0.04|0.08|N|O|1998-11-27|1998-10-02|1998-12-17|TAKE BACK RETURN|AIR|he accounts. fluffily bold sheaves b|
+4678|117|1|2|18|18307.98|0.03|0.06|N|O|1998-10-30|1998-09-22|1998-11-25|TAKE BACK RETURN|SHIP|usly ironic |
+4678|96|9|3|13|12949.17|0.10|0.07|N|O|1998-11-03|1998-10-17|1998-11-06|TAKE BACK RETURN|SHIP|its. carefully final fr|
+4678|22|1|4|23|21206.46|0.06|0.05|N|O|1998-09-03|1998-09-20|1998-09-04|DELIVER IN PERSON|SHIP|ily sly deposi|
+4678|178|9|5|40|43126.80|0.03|0.07|N|O|1998-11-11|1998-10-27|1998-11-24|TAKE BACK RETURN|AIR|. final, unusual requests sleep thinl|
+4679|190|1|1|7|7631.33|0.10|0.05|R|F|1993-05-11|1993-04-11|1993-05-16|NONE|TRUCK|kages. bold, regular packa|
+4704|78|6|1|14|13692.98|0.04|0.04|N|O|1996-10-27|1996-11-02|1996-11-07|DELIVER IN PERSON|TRUCK| above the slyly final requests. quickly |
+4704|28|3|2|7|6496.14|0.03|0.04|N|O|1996-12-04|1996-10-30|1996-12-23|DELIVER IN PERSON|SHIP|ers wake car|
+4704|64|5|3|44|42418.64|0.02|0.05|N|O|1996-09-02|1996-10-07|1996-09-17|DELIVER IN PERSON|REG AIR|out the care|
+4705|111|8|1|22|22244.42|0.04|0.04|R|F|1992-07-05|1992-05-11|1992-07-29|DELIVER IN PERSON|SHIP| fluffily pending accounts ca|
+4705|31|7|2|14|13034.42|0.00|0.08|R|F|1992-07-14|1992-05-23|1992-07-25|DELIVER IN PERSON|TRUCK|ain carefully amon|
+4705|56|1|3|16|15296.80|0.07|0.08|R|F|1992-07-02|1992-06-06|1992-07-06|DELIVER IN PERSON|RAIL|special ideas nag sl|
+4705|130|3|4|31|31934.03|0.03|0.03|R|F|1992-04-03|1992-05-30|1992-04-05|COLLECT COD|TRUCK|furiously final accou|
+4705|163|10|5|28|29768.48|0.10|0.01|A|F|1992-06-03|1992-06-07|1992-06-22|DELIVER IN PERSON|MAIL|tes wake according to the unusual plate|
+4705|184|5|6|23|24936.14|0.06|0.03|R|F|1992-06-22|1992-06-11|1992-07-18|DELIVER IN PERSON|MAIL| above the furiously ev|
+4705|89|10|7|40|39563.20|0.08|0.06|A|F|1992-04-19|1992-04-28|1992-05-07|COLLECT COD|TRUCK|blithely. sly|
+4706|182|3|1|37|40040.66|0.02|0.06|A|F|1993-02-20|1993-03-05|1993-03-03|DELIVER IN PERSON|TRUCK|kly final deposits c|
+4706|122|3|2|23|23508.76|0.03|0.01|A|F|1993-04-01|1993-03-13|1993-05-01|COLLECT COD|FOB|deas across t|
+4706|68|5|3|6|5808.36|0.01|0.04|R|F|1993-01-20|1993-03-18|1993-01-26|NONE|MAIL|efully eve|
+4706|116|10|4|5|5080.55|0.06|0.06|R|F|1993-02-14|1993-01-31|1993-02-26|NONE|REG AIR|ptotes haggle ca|
+4706|50|7|5|27|25651.35|0.06|0.08|A|F|1993-04-04|1993-03-11|1993-04-09|COLLECT COD|REG AIR|into beans. finally special instruct|
+4707|34|5|1|7|6538.21|0.02|0.05|R|F|1995-05-14|1995-04-06|1995-06-06|COLLECT COD|SHIP|ecial sheaves boost blithely accor|
+4707|136|7|2|49|50770.37|0.00|0.07|N|F|1995-06-17|1995-05-16|1995-06-25|COLLECT COD|FOB| alongside of the slyly ironic instructio|
+4708|191|4|1|18|19641.42|0.02|0.04|A|F|1994-11-11|1994-11-15|1994-11-26|NONE|REG AIR|special, eve|
+4708|75|3|2|5|4875.35|0.05|0.05|A|F|1994-10-15|1994-12-02|1994-11-12|COLLECT COD|MAIL|ely. carefully sp|
+4708|77|7|3|32|31266.24|0.04|0.07|A|F|1994-11-12|1994-11-14|1994-11-23|TAKE BACK RETURN|MAIL|the accounts. e|
+4709|25|6|1|25|23125.50|0.03|0.05|N|O|1996-02-21|1996-02-11|1996-03-17|DELIVER IN PERSON|AIR|deposits grow. fluffily unusual accounts |
+4709|177|5|2|25|26929.25|0.05|0.03|N|O|1996-01-22|1996-03-03|1996-02-21|DELIVER IN PERSON|REG AIR|inst the ironic, regul|
+4710|183|4|1|40|43327.20|0.10|0.08|A|F|1995-03-09|1995-02-25|1995-03-29|TAKE BACK RETURN|AIR|cross the blithely bold packages. silen|
+4710|128|3|2|47|48321.64|0.04|0.01|R|F|1995-02-22|1995-01-12|1995-02-28|NONE|RAIL|blithely express packages. even, ironic re|
+4711|133|4|1|7|7231.91|0.03|0.01|N|O|1998-05-12|1998-06-24|1998-05-24|COLLECT COD|MAIL|ly. bold accounts use fluff|
+4711|145|6|2|15|15677.10|0.08|0.07|N|O|1998-06-09|1998-07-30|1998-06-18|COLLECT COD|SHIP| beans wake. deposits could bo|
+4711|150|1|3|22|23103.30|0.02|0.03|N|O|1998-06-21|1998-06-18|1998-07-19|TAKE BACK RETURN|REG AIR|along the quickly careful packages. bli|
+4711|65|10|4|8|7720.48|0.07|0.00|N|O|1998-06-17|1998-06-13|1998-06-27|TAKE BACK RETURN|SHIP|g to the carefully ironic deposits. specia|
+4711|49|2|5|15|14235.60|0.05|0.01|N|O|1998-09-03|1998-07-15|1998-09-13|TAKE BACK RETURN|SHIP|ld requests: furiously final inst|
+4711|116|7|6|45|45724.95|0.05|0.06|N|O|1998-05-19|1998-07-14|1998-05-21|COLLECT COD|SHIP| ironic theodolites |
+4711|46|5|7|18|17028.72|0.05|0.04|N|O|1998-07-03|1998-07-31|1998-07-23|DELIVER IN PERSON|RAIL| blithely. bold asymptote|
+4736|196|10|1|26|28500.94|0.03|0.03|N|O|1996-02-02|1996-01-18|1996-02-09|DELIVER IN PERSON|AIR|efully speci|
+4736|4|1|2|43|38872.00|0.06|0.07|N|O|1996-02-05|1995-12-21|1996-02-06|COLLECT COD|MAIL|quests. carefully |
+4737|191|5|1|37|40374.03|0.03|0.04|R|F|1993-05-17|1993-04-10|1993-05-30|DELIVER IN PERSON|TRUCK|s. fluffily regular |
+4737|69|8|2|22|21319.32|0.04|0.04|A|F|1993-03-29|1993-05-22|1993-04-16|TAKE BACK RETURN|RAIL| hang fluffily around t|
+4738|187|8|1|9|9784.62|0.04|0.04|A|F|1992-06-01|1992-06-26|1992-06-02|COLLECT COD|TRUCK|posits serve slyly. unusual pint|
+4738|173|3|2|16|17170.72|0.07|0.08|A|F|1992-06-17|1992-06-20|1992-06-21|NONE|MAIL|nic deposits are slyly! carefu|
+4738|100|2|3|50|50005.00|0.04|0.02|A|F|1992-06-18|1992-07-04|1992-07-07|TAKE BACK RETURN|TRUCK|the blithely ironic braids sleep slyly|
+4738|29|4|4|22|20438.44|0.02|0.08|A|F|1992-05-25|1992-05-19|1992-06-12|COLLECT COD|SHIP|ld, even packages. furio|
+4738|187|8|5|13|14133.34|0.04|0.05|R|F|1992-05-30|1992-06-11|1992-06-26|COLLECT COD|AIR| wake. unusual platelets for the|
+4738|159|1|6|10|10591.50|0.10|0.01|R|F|1992-07-10|1992-06-16|1992-07-25|TAKE BACK RETURN|SHIP|hins above the|
+4738|83|4|7|28|27526.24|0.05|0.07|A|F|1992-06-09|1992-07-05|1992-06-25|NONE|AIR|e furiously ironic excuses. care|
+4739|168|9|1|8|8545.28|0.07|0.07|R|F|1993-06-22|1993-05-10|1993-07-11|TAKE BACK RETURN|SHIP|cording to the |
+4739|185|6|2|31|33640.58|0.09|0.06|R|F|1993-06-20|1993-05-18|1993-06-26|COLLECT COD|SHIP|blithely special pin|
+4739|100|4|3|30|30003.00|0.09|0.00|A|F|1993-05-29|1993-04-12|1993-06-18|NONE|TRUCK|ly even packages use across th|
+4740|3|4|1|22|19866.00|0.06|0.01|N|O|1996-10-04|1996-08-17|1996-10-05|TAKE BACK RETURN|RAIL|final dependencies nag |
+4740|153|5|2|24|25275.60|0.08|0.02|N|O|1996-09-10|1996-09-27|1996-10-07|TAKE BACK RETURN|TRUCK|hely regular deposits|
+4741|73|2|1|24|23353.68|0.00|0.01|A|F|1992-09-16|1992-09-19|1992-09-20|DELIVER IN PERSON|RAIL|deas boost furiously slyly regular id|
+4741|113|4|2|16|16209.76|0.01|0.07|R|F|1992-08-25|1992-08-10|1992-08-29|TAKE BACK RETURN|FOB|final foxes haggle r|
+4741|156|8|3|24|25347.60|0.05|0.08|A|F|1992-11-04|1992-08-14|1992-11-06|TAKE BACK RETURN|MAIL|even requests.|
+4741|51|3|4|39|37090.95|0.09|0.06|R|F|1992-10-28|1992-10-03|1992-11-11|COLLECT COD|SHIP|t, regular requests|
+4741|179|10|5|40|43166.80|0.09|0.03|R|F|1992-09-20|1992-09-23|1992-10-09|TAKE BACK RETURN|REG AIR| fluffily slow deposits. fluffily regu|
+4741|157|5|6|34|35943.10|0.02|0.07|R|F|1992-08-25|1992-08-18|1992-09-20|DELIVER IN PERSON|RAIL|sly special packages after the furiously|
+4742|156|4|1|32|33796.80|0.10|0.08|R|F|1995-04-04|1995-06-12|1995-04-19|COLLECT COD|RAIL|eposits boost blithely. carefully regular a|
+4742|155|7|2|29|30599.35|0.02|0.03|N|F|1995-06-15|1995-05-05|1995-06-24|COLLECT COD|REG AIR|integrate closely among t|
+4742|72|10|3|15|14581.05|0.06|0.04|N|O|1995-07-20|1995-05-26|1995-08-11|NONE|SHIP|terns are sl|
+4742|188|9|4|31|33733.58|0.05|0.08|N|F|1995-06-13|1995-05-08|1995-06-24|COLLECT COD|REG AIR|ke slyly among the furiousl|
+4742|100|1|5|45|45004.50|0.05|0.00|R|F|1995-05-12|1995-05-14|1995-06-07|TAKE BACK RETURN|RAIL|ke carefully. do|
+4743|60|5|1|19|18241.14|0.04|0.07|A|F|1993-06-23|1993-05-03|1993-07-20|COLLECT COD|AIR|hely even accounts|
+4743|159|4|2|3|3177.45|0.01|0.03|R|F|1993-04-14|1993-06-08|1993-05-09|NONE|TRUCK|al requests. express idea|
+4743|73|2|3|21|20434.47|0.08|0.03|A|F|1993-07-02|1993-06-15|1993-07-26|DELIVER IN PERSON|RAIL|ake blithely against the packages. reg|
+4743|34|5|4|27|25218.81|0.08|0.05|R|F|1993-07-26|1993-05-27|1993-08-24|DELIVER IN PERSON|AIR|aids use. express deposits|
+4768|36|7|1|5|4680.15|0.00|0.03|R|F|1993-12-27|1994-02-09|1994-01-11|NONE|MAIL|egular accounts. bravely final fra|
+4769|35|1|1|16|14960.48|0.08|0.05|N|O|1995-07-16|1995-07-05|1995-07-22|TAKE BACK RETURN|FOB| deposits. slyly even asymptote|
+4769|63|8|2|34|32744.04|0.06|0.07|N|O|1995-07-26|1995-05-18|1995-08-03|COLLECT COD|REG AIR|ven instructions. ca|
+4769|47|10|3|36|34093.44|0.10|0.03|N|O|1995-07-22|1995-06-16|1995-08-11|NONE|RAIL|. slyly even deposit|
+4769|69|10|4|45|43607.70|0.08|0.06|R|F|1995-06-01|1995-07-13|1995-06-04|TAKE BACK RETURN|RAIL|accounts are. even accounts sleep|
+4769|112|6|5|15|15181.65|0.07|0.08|N|F|1995-06-12|1995-07-07|1995-07-04|NONE|SHIP|egular platelets can cajole across the |
+4770|32|8|1|41|38213.23|0.00|0.08|N|O|1995-09-04|1995-08-08|1995-09-10|COLLECT COD|FOB|ithely even packages sleep caref|
+4770|157|5|2|30|31714.50|0.09|0.07|N|O|1995-08-25|1995-08-27|1995-09-07|COLLECT COD|SHIP|ffily carefully ironic ideas. ironic d|
+4771|49|10|1|9|8541.36|0.01|0.00|R|F|1993-02-28|1993-02-19|1993-03-25|NONE|FOB|riously after the packages. fina|
+4771|16|7|2|21|19236.21|0.09|0.01|R|F|1993-01-19|1993-02-10|1993-02-01|NONE|FOB|fluffily pendi|
+4771|12|3|3|5|4560.05|0.06|0.08|R|F|1993-01-07|1993-01-19|1993-01-26|NONE|RAIL|ar, quiet accounts nag furiously express id|
+4771|9|4|4|21|19089.00|0.05|0.04|A|F|1992-12-20|1993-01-22|1992-12-26|TAKE BACK RETURN|SHIP| carefully re|
+4772|87|8|1|1|987.08|0.10|0.00|R|F|1994-11-13|1994-10-25|1994-11-15|DELIVER IN PERSON|AIR|ans. slyly even acc|
+4772|146|9|2|16|16738.24|0.07|0.06|R|F|1994-10-27|1994-12-07|1994-10-29|TAKE BACK RETURN|MAIL|egular accounts wake s|
+4772|95|6|3|31|30847.79|0.02|0.04|A|F|1994-10-02|1994-10-21|1994-10-13|TAKE BACK RETURN|FOB|ests are thinly. furiously unusua|
+4772|71|10|4|15|14566.05|0.02|0.07|R|F|1994-09-19|1994-10-22|1994-09-26|COLLECT COD|TRUCK| requests. express, regular th|
+4773|144|5|1|23|24015.22|0.00|0.08|N|O|1996-01-01|1996-03-19|1996-01-04|NONE|FOB|ly express grouches wak|
+4773|197|9|2|36|39498.84|0.09|0.04|N|O|1996-04-08|1996-03-03|1996-05-01|COLLECT COD|REG AIR| dependencies. quickly|
+4773|167|8|3|49|52290.84|0.05|0.02|N|O|1996-01-26|1996-02-29|1996-01-27|TAKE BACK RETURN|FOB|y final reque|
+4773|20|10|4|49|45080.98|0.09|0.04|N|O|1996-01-12|1996-02-17|1996-02-05|TAKE BACK RETURN|TRUCK|ly pending theodolites cajole caref|
+4773|150|3|5|20|21003.00|0.02|0.07|N|O|1995-12-28|1996-02-17|1996-01-15|COLLECT COD|TRUCK| blithely final deposits nag after t|
+4773|190|1|6|11|11992.09|0.10|0.06|N|O|1996-01-02|1996-01-29|1996-01-24|DELIVER IN PERSON|REG AIR|en accounts. slyly b|
+4773|158|3|7|6|6348.90|0.07|0.01|N|O|1996-03-09|1996-03-18|1996-03-27|NONE|AIR|latelets haggle s|
+4774|84|5|1|45|44283.60|0.10|0.00|R|F|1993-07-07|1993-06-08|1993-07-31|COLLECT COD|TRUCK| haggle busily afte|
+4774|39|5|2|4|3756.12|0.02|0.03|A|F|1993-08-03|1993-05-30|1993-08-19|COLLECT COD|FOB|xes according to the foxes wake above the f|
+4774|173|4|3|47|50438.99|0.10|0.08|R|F|1993-06-13|1993-07-04|1993-07-09|TAKE BACK RETURN|FOB|regular dolphins above the furi|
+4774|130|3|4|30|30903.90|0.05|0.08|A|F|1993-08-18|1993-06-08|1993-08-21|DELIVER IN PERSON|REG AIR|tions against the blithely final theodolit|
+4775|74|4|1|1|974.07|0.10|0.02|N|O|1995-09-06|1995-09-28|1995-09-29|DELIVER IN PERSON|MAIL|furiously ironic theodolite|
+4775|153|1|2|37|38966.55|0.02|0.01|N|O|1995-09-06|1995-09-28|1995-09-28|COLLECT COD|TRUCK|ts. pinto beans use according to th|
+4775|153|5|3|34|35807.10|0.09|0.06|N|O|1995-09-14|1995-10-15|1995-09-21|DELIVER IN PERSON|MAIL|onic epitaphs. f|
+4775|119|9|4|39|39745.29|0.07|0.04|N|O|1995-08-30|1995-10-12|1995-09-20|NONE|AIR|eep never with the slyly regular acc|
+4800|97|10|1|11|10967.99|0.03|0.03|R|F|1992-01-27|1992-03-16|1992-02-19|TAKE BACK RETURN|RAIL|ic dependenc|
+4800|26|5|2|1|926.02|0.06|0.06|A|F|1992-02-23|1992-03-16|1992-03-20|TAKE BACK RETURN|MAIL|nal accounts are blithely deposits. bol|
+4800|11|8|3|21|19131.21|0.09|0.05|A|F|1992-02-14|1992-03-15|1992-02-26|NONE|SHIP|ithely according to |
+4800|176|7|4|38|40894.46|0.10|0.08|R|F|1992-02-01|1992-02-28|1992-02-21|NONE|TRUCK|s sleep fluffily. furiou|
+4800|53|4|5|24|22873.20|0.08|0.04|R|F|1992-01-14|1992-02-23|1992-01-25|NONE|TRUCK|ully carefully r|
+4801|184|5|1|37|40114.66|0.10|0.02|N|O|1996-03-09|1996-02-29|1996-03-25|TAKE BACK RETURN|FOB|uests hinder blithely against the instr|
+4801|26|1|2|34|31484.68|0.03|0.02|N|O|1996-02-05|1996-04-16|1996-02-23|NONE|SHIP|y final requests |
+4801|110|1|3|4|4040.44|0.04|0.04|N|O|1996-03-23|1996-04-04|1996-03-25|COLLECT COD|RAIL|pitaphs. regular, reg|
+4801|92|3|4|39|38691.51|0.07|0.01|N|O|1996-03-19|1996-03-21|1996-04-17|TAKE BACK RETURN|REG AIR|warhorses wake never for the care|
+4802|40|1|1|6|5640.24|0.00|0.06|N|O|1997-04-16|1997-03-25|1997-04-21|TAKE BACK RETURN|SHIP|unusual accounts wake blithely. b|
+4803|132|3|1|2|2064.26|0.08|0.03|N|O|1996-04-16|1996-03-20|1996-05-15|NONE|REG AIR|gular reque|
+4803|176|4|2|47|50579.99|0.10|0.00|N|O|1996-03-14|1996-03-30|1996-03-15|DELIVER IN PERSON|FOB|ly final excuses. slyly express requ|
+4803|196|8|3|42|46039.98|0.04|0.08|N|O|1996-04-27|1996-05-05|1996-05-17|NONE|TRUCK| accounts affix quickly ar|
+4803|22|1|4|24|22128.48|0.10|0.04|N|O|1996-02-24|1996-04-02|1996-02-28|NONE|MAIL|t blithely slyly special decoys. |
+4803|189|10|5|21|22872.78|0.03|0.06|N|O|1996-05-25|1996-03-15|1996-06-09|COLLECT COD|FOB| silent packages use. b|
+4803|194|5|6|19|20789.61|0.07|0.00|N|O|1996-04-20|1996-03-25|1996-04-27|TAKE BACK RETURN|RAIL|sts. enticing, even|
+4804|128|1|1|44|45237.28|0.06|0.08|A|F|1992-05-02|1992-03-24|1992-05-28|TAKE BACK RETURN|AIR|aggle quickly among the slyly fi|
+4804|35|6|2|41|38336.23|0.10|0.02|R|F|1992-04-06|1992-04-12|1992-05-03|COLLECT COD|MAIL|. deposits haggle express tithes?|
+4804|65|2|3|33|31846.98|0.09|0.05|A|F|1992-03-02|1992-04-14|1992-03-13|DELIVER IN PERSON|AIR|, thin excuses. |
+4805|150|1|1|7|7351.05|0.09|0.03|A|F|1992-05-01|1992-07-09|1992-05-09|NONE|FOB| requests. regular deposit|
+4805|189|10|2|45|49013.10|0.02|0.03|R|F|1992-06-16|1992-06-08|1992-07-03|NONE|TRUCK|the furiously sly t|
+4805|154|6|3|44|46382.60|0.01|0.02|R|F|1992-05-14|1992-06-23|1992-05-25|DELIVER IN PERSON|SHIP|eposits sleep furiously qui|
+4805|65|2|4|13|12545.78|0.04|0.04|R|F|1992-07-16|1992-06-07|1992-08-10|COLLECT COD|AIR|its serve about the accounts. slyly regu|
+4805|9|10|5|42|38178.00|0.03|0.03|R|F|1992-08-17|1992-07-03|1992-09-14|NONE|REG AIR|the regular, fina|
+4805|136|7|6|18|18650.34|0.06|0.04|A|F|1992-06-07|1992-07-10|1992-06-12|COLLECT COD|TRUCK|o use pending, unusu|
+4806|16|7|1|26|23816.26|0.10|0.05|R|F|1993-05-28|1993-06-07|1993-05-29|DELIVER IN PERSON|SHIP| bold pearls sublate blithely. quickly pe|
+4806|72|10|2|6|5832.42|0.01|0.06|A|F|1993-05-17|1993-07-19|1993-05-29|TAKE BACK RETURN|SHIP|even theodolites. packages sl|
+4806|29|4|3|8|7432.16|0.09|0.00|A|F|1993-05-08|1993-07-16|1993-05-28|NONE|TRUCK|requests boost blithely. qui|
+4807|122|1|1|9|9199.08|0.04|0.08|N|O|1997-04-23|1997-03-01|1997-05-15|TAKE BACK RETURN|TRUCK|may are blithely. carefully even pinto b|
+4807|10|1|2|41|37310.41|0.07|0.08|N|O|1997-05-02|1997-03-31|1997-05-15|TAKE BACK RETURN|AIR| fluffily re|
+4807|145|6|3|34|35534.76|0.06|0.02|N|O|1997-01-31|1997-03-13|1997-02-01|NONE|SHIP|ecial ideas. deposits according to the fin|
+4807|190|1|4|32|34886.08|0.05|0.00|N|O|1997-04-04|1997-03-21|1997-04-16|NONE|RAIL|efully even dolphins slee|
+4807|159|1|5|2|2118.30|0.02|0.05|N|O|1997-05-09|1997-04-03|1997-06-05|TAKE BACK RETURN|RAIL|deas wake bli|
+4807|160|1|6|22|23323.52|0.09|0.06|N|O|1997-03-13|1997-02-23|1997-04-01|NONE|FOB|es use final excuses. furiously final|
+4832|15|6|1|23|21045.23|0.03|0.01|N|O|1997-12-05|1998-01-05|1997-12-10|NONE|RAIL|y express depo|
+4832|152|4|2|10|10521.50|0.00|0.06|N|O|1998-01-08|1998-02-01|1998-01-11|DELIVER IN PERSON|MAIL|ly. blithely bold pinto beans should have|
+4832|149|6|3|4|4196.56|0.04|0.01|N|O|1998-01-16|1998-02-12|1998-02-08|TAKE BACK RETURN|AIR|ages. slyly express deposits cajole car|
+4832|64|5|4|6|5784.36|0.02|0.01|N|O|1997-12-08|1998-02-03|1997-12-10|COLLECT COD|TRUCK|ages cajole after the bold requests. furi|
+4832|138|4|5|43|44639.59|0.10|0.08|N|O|1997-12-31|1998-02-20|1998-01-26|COLLECT COD|RAIL|oze according to the accou|
+4833|107|10|1|31|31220.10|0.08|0.04|N|O|1996-06-24|1996-07-15|1996-07-02|NONE|SHIP|ven instructions cajole against the caref|
+4833|117|7|2|11|11188.21|0.03|0.01|N|O|1996-08-24|1996-07-26|1996-09-19|NONE|REG AIR|s nag above the busily sile|
+4833|18|9|3|26|23868.26|0.08|0.04|N|O|1996-05-13|1996-07-12|1996-05-31|NONE|SHIP|s packages. even gif|
+4833|36|7|4|19|17784.57|0.07|0.07|N|O|1996-08-21|1996-07-09|1996-09-10|TAKE BACK RETURN|AIR|y quick theodolit|
+4833|35|1|5|4|3740.12|0.10|0.02|N|O|1996-08-16|1996-06-29|1996-08-22|NONE|AIR|y pending packages sleep blithely regular r|
+4834|183|4|1|27|29245.86|0.06|0.02|N|O|1997-01-09|1996-10-27|1997-01-27|DELIVER IN PERSON|RAIL|es nag blithe|
+4834|71|1|2|26|25247.82|0.01|0.00|N|O|1996-10-04|1996-10-21|1996-10-10|DELIVER IN PERSON|TRUCK|ages dazzle carefully. slyly daring foxes|
+4834|23|2|3|34|31382.68|0.03|0.01|N|O|1996-12-09|1996-11-26|1996-12-10|NONE|MAIL|ounts haggle bo|
+4834|143|10|4|38|39639.32|0.03|0.06|N|O|1997-01-10|1996-12-06|1997-01-22|COLLECT COD|FOB|alongside of the carefully even plate|
+4835|179|10|1|18|19425.06|0.00|0.03|R|F|1995-02-17|1994-12-14|1995-03-17|DELIVER IN PERSON|MAIL|eat furiously against the slyly |
+4835|91|3|2|3|2973.27|0.09|0.06|R|F|1995-01-24|1995-01-12|1995-02-16|COLLECT COD|AIR|etimes final pac|
+4835|86|7|3|27|26624.16|0.05|0.00|A|F|1994-12-10|1994-12-13|1995-01-02|DELIVER IN PERSON|REG AIR| accounts after the car|
+4835|102|7|4|23|23048.30|0.08|0.07|A|F|1995-02-05|1995-01-04|1995-02-28|NONE|SHIP|e carefully regular foxes. deposits are sly|
+4836|162|1|1|22|23367.52|0.01|0.03|N|O|1997-03-03|1997-02-23|1997-03-04|NONE|SHIP|al pinto beans. care|
+4836|48|5|2|16|15168.64|0.07|0.08|N|O|1997-01-14|1997-03-05|1997-01-30|COLLECT COD|MAIL|gular packages against the express reque|
+4836|76|4|3|14|13664.98|0.03|0.08|N|O|1997-02-21|1997-02-06|1997-03-08|COLLECT COD|MAIL|lites. unusual, bold dolphins ar|
+4836|106|1|4|15|15091.50|0.10|0.00|N|O|1997-03-08|1997-03-14|1997-03-30|TAKE BACK RETURN|TRUCK|eep slyly. even requests cajole|
+4836|51|6|5|12|11412.60|0.01|0.04|N|O|1997-02-02|1997-02-10|1997-02-03|COLLECT COD|TRUCK|sly ironic accoun|
+4837|42|1|1|16|15072.64|0.09|0.04|N|O|1998-08-12|1998-06-06|1998-08-26|COLLECT COD|FOB|ing requests are blithely regular instructi|
+4837|193|5|2|16|17491.04|0.01|0.02|N|O|1998-08-19|1998-06-18|1998-08-26|NONE|RAIL|counts cajole slyly furiou|
+4837|68|5|3|42|40658.52|0.10|0.00|N|O|1998-06-19|1998-07-06|1998-06-23|COLLECT COD|MAIL|o the furiously final theodolites boost|
+4838|122|3|1|35|35774.20|0.01|0.00|R|F|1992-10-30|1992-10-23|1992-11-21|TAKE BACK RETURN|RAIL|ly blithely unusual foxes. even package|
+4838|148|5|2|2|2096.28|0.03|0.08|R|F|1992-08-11|1992-09-16|1992-08-26|COLLECT COD|MAIL|hely final notornis are furiously blithe|
+4838|52|3|3|26|24753.30|0.06|0.04|R|F|1992-09-03|1992-10-25|1992-09-11|TAKE BACK RETURN|FOB|ular requests boost about the packages. r|
+4839|60|2|1|5|4800.30|0.10|0.07|A|F|1994-09-07|1994-07-15|1994-10-05|DELIVER IN PERSON|FOB|ses integrate. regular deposits are about |
+4839|10|1|2|25|22750.25|0.02|0.02|R|F|1994-05-20|1994-07-08|1994-05-30|NONE|REG AIR|regular packages ab|
+4839|60|1|3|18|17281.08|0.06|0.01|R|F|1994-05-18|1994-06-13|1994-06-09|TAKE BACK RETURN|FOB|blithely ironic theodolites use along|
+4839|100|1|4|19|19001.90|0.07|0.08|R|F|1994-05-20|1994-07-14|1994-05-30|NONE|REG AIR| deposits sublate furiously ir|
+4839|71|10|5|9|8739.63|0.05|0.01|R|F|1994-06-17|1994-06-18|1994-07-10|NONE|SHIP|ounts haggle carefully above|
+4864|150|9|1|28|29404.20|0.06|0.08|A|F|1993-02-06|1992-12-15|1993-02-10|COLLECT COD|REG AIR|thely around the bli|
+4864|38|4|2|38|35645.14|0.10|0.02|R|F|1992-12-20|1993-01-07|1993-01-06|TAKE BACK RETURN|SHIP|ording to the ironic, ir|
+4864|133|4|3|45|46490.85|0.02|0.01|A|F|1992-11-17|1993-01-02|1992-11-26|COLLECT COD|SHIP|round the furiously careful pa|
+4864|31|2|4|46|42827.38|0.07|0.03|A|F|1993-02-24|1993-01-02|1993-03-17|TAKE BACK RETURN|RAIL|sts use carefully across the carefull|
+4865|162|7|1|16|16994.56|0.07|0.05|N|O|1997-10-02|1997-08-20|1997-10-04|COLLECT COD|TRUCK|osits haggle. fur|
+4865|137|8|2|4|4148.52|0.07|0.01|N|O|1997-07-24|1997-07-25|1997-08-07|TAKE BACK RETURN|FOB|sts. blithely special instruction|
+4865|68|3|3|44|42594.64|0.10|0.08|N|O|1997-07-25|1997-08-20|1997-08-22|COLLECT COD|FOB|even deposits sleep against the quickly r|
+4865|50|3|4|21|19951.05|0.04|0.02|N|O|1997-07-17|1997-08-10|1997-07-21|NONE|RAIL|eposits detect sly|
+4865|54|9|5|33|31483.65|0.00|0.05|N|O|1997-07-17|1997-08-16|1997-07-30|TAKE BACK RETURN|FOB|y pending notornis ab|
+4865|65|2|6|47|45357.82|0.00|0.05|N|O|1997-08-26|1997-08-07|1997-08-31|NONE|RAIL|y unusual packages. packages|
+4866|11|8|1|9|8199.09|0.01|0.05|N|O|1997-08-30|1997-09-18|1997-09-24|TAKE BACK RETURN|MAIL|ven dependencies x-ray. quic|
+4866|102|3|2|1|1002.10|0.06|0.00|N|O|1997-10-15|1997-10-01|1997-11-14|TAKE BACK RETURN|AIR|latelets nag. q|
+4866|131|7|3|17|17529.21|0.07|0.00|N|O|1997-11-26|1997-10-11|1997-12-12|COLLECT COD|TRUCK|ess packages doubt. even somas wake f|
+4867|82|3|1|7|6874.56|0.09|0.03|A|F|1992-07-17|1992-08-17|1992-07-22|COLLECT COD|FOB|e carefully even packages. slyly ironic i|
+4867|160|8|2|3|3180.48|0.04|0.08|R|F|1992-07-04|1992-07-15|1992-07-21|NONE|AIR|yly silent deposits|
+4868|73|3|1|47|45734.29|0.03|0.03|N|O|1997-04-29|1997-04-27|1997-05-11|DELIVER IN PERSON|SHIP|gle unusual, fluffy packages. foxes cajol|
+4868|180|1|2|8|8641.44|0.10|0.08|N|O|1997-03-26|1997-05-09|1997-04-16|NONE|RAIL|ly special th|
+4868|191|2|3|49|53468.31|0.09|0.03|N|O|1997-04-23|1997-05-07|1997-04-26|NONE|SHIP|ys engage. th|
+4868|80|1|4|34|33322.72|0.04|0.02|N|O|1997-05-19|1997-04-27|1997-06-15|NONE|RAIL|en instructions about th|
+4868|122|3|5|22|22486.64|0.07|0.06|N|O|1997-04-26|1997-05-16|1997-05-01|DELIVER IN PERSON|FOB|osits. final foxes boost regular,|
+4869|41|8|1|31|29172.24|0.10|0.01|A|F|1995-01-17|1994-11-30|1995-02-02|NONE|SHIP|ins. always unusual ideas across the ir|
+4869|58|3|2|24|22993.20|0.09|0.06|A|F|1994-11-17|1994-11-07|1994-11-27|COLLECT COD|MAIL|olites cajole after the ideas. special t|
+4869|157|8|3|25|26428.75|0.00|0.05|R|F|1994-11-25|1994-11-14|1994-12-19|DELIVER IN PERSON|AIR|e according t|
+4869|103|8|4|24|24074.40|0.10|0.07|R|F|1994-11-23|1994-11-18|1994-12-11|DELIVER IN PERSON|MAIL|se deposits above the sly, q|
+4869|173|2|5|42|45073.14|0.07|0.04|R|F|1994-10-16|1994-12-10|1994-11-07|TAKE BACK RETURN|REG AIR| slyly even instructions. |
+4869|122|5|6|30|30663.60|0.00|0.05|A|F|1995-01-09|1994-11-20|1995-02-02|COLLECT COD|RAIL|gedly even requests. s|
+4870|48|5|1|49|46453.96|0.05|0.05|R|F|1994-11-14|1994-10-24|1994-12-12|TAKE BACK RETURN|SHIP| regular packages |
+4870|127|10|2|6|6162.72|0.06|0.08|A|F|1994-09-09|1994-10-16|1994-09-21|DELIVER IN PERSON|TRUCK|ress requests. bold, silent pinto bea|
+4870|31|2|3|5|4655.15|0.05|0.00|R|F|1994-10-11|1994-10-07|1994-10-24|NONE|AIR|s haggle furiously. slyly ironic dinos|
+4870|6|9|4|4|3624.00|0.03|0.08|A|F|1994-10-23|1994-09-16|1994-11-04|COLLECT COD|RAIL|its wake quickly. slyly quick|
+4870|71|1|5|36|34958.52|0.09|0.06|A|F|1994-09-06|1994-09-17|1994-10-01|COLLECT COD|REG AIR| instructions. carefully pending pac|
+4871|177|5|1|14|15080.38|0.07|0.03|N|O|1995-09-30|1995-07-29|1995-10-18|TAKE BACK RETURN|REG AIR|inst the never ironic |
+4871|161|6|2|17|18039.72|0.07|0.03|N|O|1995-09-09|1995-09-01|1995-10-02|DELIVER IN PERSON|AIR|es. carefully ev|
+4871|63|4|3|3|2889.18|0.03|0.06|N|O|1995-10-03|1995-08-10|1995-10-06|DELIVER IN PERSON|TRUCK|y special packages wak|
+4871|149|8|4|35|36719.90|0.08|0.07|N|O|1995-08-11|1995-07-18|1995-08-29|DELIVER IN PERSON|TRUCK|ackages sle|
+4871|152|3|5|10|10521.50|0.09|0.02|N|O|1995-09-12|1995-09-02|1995-10-05|TAKE BACK RETURN|AIR|s integrate after the a|
+4871|136|2|6|36|37300.68|0.02|0.08|N|O|1995-09-18|1995-08-29|1995-10-05|TAKE BACK RETURN|AIR|ely according|
+4871|140|6|7|10|10401.40|0.10|0.02|N|O|1995-07-13|1995-08-19|1995-07-29|NONE|REG AIR|p ironic theodolites. slyly even platel|
+4896|41|2|1|19|17879.76|0.09|0.05|A|F|1992-12-13|1992-11-13|1993-01-09|NONE|AIR|nusual requ|
+4896|140|1|2|44|45766.16|0.04|0.03|A|F|1992-11-24|1992-11-15|1992-12-18|COLLECT COD|MAIL|e after the slowly f|
+4896|58|10|3|6|5748.30|0.04|0.04|A|F|1992-10-30|1992-11-12|1992-11-28|DELIVER IN PERSON|TRUCK|usly regular deposits|
+4896|23|4|4|5|4615.10|0.08|0.02|R|F|1992-12-02|1992-11-11|1992-12-19|COLLECT COD|SHIP|eposits hang carefully. sly|
+4896|86|7|5|21|20707.68|0.07|0.08|R|F|1992-11-18|1992-11-18|1992-11-29|DELIVER IN PERSON|TRUCK|ly express deposits. carefully pending depo|
+4897|55|6|1|26|24831.30|0.01|0.01|R|F|1992-12-22|1992-10-25|1992-12-27|DELIVER IN PERSON|TRUCK|. carefully ironic dep|
+4897|143|6|2|34|35466.76|0.02|0.00|R|F|1992-12-31|1992-11-11|1993-01-30|COLLECT COD|AIR|ts. special dependencies use fluffily |
+4897|55|7|3|42|40112.10|0.09|0.03|A|F|1992-09-23|1992-10-28|1992-10-02|DELIVER IN PERSON|FOB|sts. blithely regular deposits will have|
+4897|104|5|4|19|19077.90|0.03|0.00|A|F|1992-11-08|1992-12-14|1992-12-03|DELIVER IN PERSON|FOB|! ironic, pending dependencies doze furiou|
+4898|72|1|1|44|42771.08|0.07|0.02|A|F|1994-09-13|1994-08-18|1994-09-16|NONE|FOB|y regular grouches about|
+4899|34|10|1|14|13076.42|0.06|0.00|R|F|1993-11-10|1994-01-10|1993-11-20|NONE|REG AIR| foxes eat|
+4900|116|3|1|40|40644.40|0.10|0.03|A|F|1992-09-02|1992-09-25|1992-09-21|COLLECT COD|TRUCK|heodolites. request|
+4900|77|8|2|33|32243.31|0.06|0.06|R|F|1992-08-18|1992-09-20|1992-08-19|COLLECT COD|MAIL|nto beans nag slyly reg|
+4900|103|8|3|48|48148.80|0.02|0.00|R|F|1992-09-18|1992-08-14|1992-09-28|TAKE BACK RETURN|MAIL|uickly ironic ideas kindle s|
+4900|32|3|4|20|18640.60|0.05|0.00|R|F|1992-09-22|1992-09-23|1992-09-27|TAKE BACK RETURN|MAIL|yers. accounts affix somet|
+4900|105|8|5|40|40204.00|0.03|0.02|R|F|1992-07-14|1992-09-05|1992-07-20|NONE|REG AIR|luffily final dol|
+4900|103|6|6|46|46142.60|0.06|0.08|R|F|1992-07-11|1992-09-19|1992-07-16|TAKE BACK RETURN|SHIP|ly final acco|
+4901|141|10|1|37|38522.18|0.00|0.04|N|O|1998-01-26|1998-02-20|1998-01-31|DELIVER IN PERSON|TRUCK| furiously ev|
+4901|165|4|2|12|12781.92|0.00|0.04|N|O|1998-01-12|1998-02-06|1998-02-03|COLLECT COD|REG AIR|y unusual deposits prom|
+4901|120|4|3|16|16321.92|0.05|0.08|N|O|1998-04-19|1998-03-18|1998-04-21|NONE|AIR|deposits. blithely fin|
+4901|36|7|4|41|38377.23|0.03|0.00|N|O|1998-03-18|1998-02-18|1998-04-14|TAKE BACK RETURN|AIR|efully bold packages affix carefully eve|
+4901|116|7|5|40|40644.40|0.06|0.02|N|O|1998-01-08|1998-01-30|1998-01-15|DELIVER IN PERSON|MAIL|ect across the furiou|
+4902|196|10|1|22|24116.18|0.00|0.04|N|O|1998-10-17|1998-08-10|1998-10-21|COLLECT COD|RAIL|r the furiously final fox|
+4902|83|4|2|1|983.08|0.09|0.04|N|O|1998-10-12|1998-08-20|1998-11-08|NONE|RAIL|daring foxes? even, bold requests wake f|
+4903|121|2|1|1|1021.12|0.06|0.03|R|F|1992-04-23|1992-06-13|1992-05-03|NONE|SHIP|nusual requests|
+4903|165|6|2|6|6390.96|0.09|0.07|R|F|1992-04-01|1992-05-16|1992-04-11|DELIVER IN PERSON|SHIP|azzle quickly along the blithely final pla|
+4903|120|10|3|27|27543.24|0.07|0.06|A|F|1992-06-29|1992-06-09|1992-07-08|COLLECT COD|RAIL|pinto beans are; |
+4928|100|1|1|4|4000.40|0.04|0.02|R|F|1993-10-25|1993-12-24|1993-11-16|TAKE BACK RETURN|REG AIR|bout the slyly final accounts. carefull|
+4928|93|4|2|20|19861.80|0.03|0.08|A|F|1994-01-19|1993-11-29|1994-02-13|DELIVER IN PERSON|SHIP|quiet theodolites ca|
+4928|149|8|3|34|35670.76|0.06|0.05|A|F|1993-10-12|1993-12-31|1993-10-14|DELIVER IN PERSON|AIR|, regular depos|
+4929|14|1|1|20|18280.20|0.00|0.04|N|O|1996-03-12|1996-05-23|1996-03-20|COLLECT COD|REG AIR| final pinto beans detect. final,|
+4929|79|7|2|40|39162.80|0.08|0.03|N|O|1996-05-30|1996-04-13|1996-06-22|TAKE BACK RETURN|AIR|unts against |
+4929|77|7|3|32|31266.24|0.08|0.02|N|O|1996-04-28|1996-05-23|1996-04-30|COLLECT COD|TRUCK|usly at the blithely pending pl|
+4929|109|4|4|26|26236.60|0.00|0.05|N|O|1996-06-10|1996-05-29|1996-06-26|DELIVER IN PERSON|RAIL| slyly. fl|
+4929|67|8|5|24|23209.44|0.09|0.05|N|O|1996-04-15|1996-04-30|1996-05-09|NONE|MAIL| accounts boost|
+4930|187|8|1|35|38051.30|0.03|0.01|A|F|1994-07-09|1994-07-30|1994-07-15|NONE|RAIL|lose slyly regular dependencies. fur|
+4930|115|5|2|20|20302.20|0.02|0.04|A|F|1994-08-21|1994-06-17|1994-08-24|COLLECT COD|FOB|he carefully|
+4930|168|7|3|28|29908.48|0.00|0.08|R|F|1994-08-27|1994-06-27|1994-09-18|COLLECT COD|TRUCK|e ironic, unusual courts. regula|
+4930|166|7|4|42|44778.72|0.00|0.00|A|F|1994-06-18|1994-06-22|1994-07-10|COLLECT COD|AIR|ions haggle. furiously regular ideas use |
+4930|190|1|5|38|41427.22|0.02|0.03|A|F|1994-06-06|1994-06-18|1994-07-03|TAKE BACK RETURN|AIR|bold requests sleep never|
+4931|194|7|1|1|1094.19|0.08|0.06|A|F|1995-01-24|1994-12-19|1995-02-07|DELIVER IN PERSON|SHIP| furiously |
+4931|151|3|2|8|8409.20|0.06|0.02|R|F|1994-12-15|1995-01-14|1995-01-06|NONE|SHIP|ts boost. packages wake sly|
+4931|144|5|3|20|20882.80|0.09|0.00|A|F|1995-01-25|1994-12-21|1995-02-06|DELIVER IN PERSON|MAIL|the furious|
+4931|200|4|4|50|55010.00|0.04|0.01|A|F|1994-12-15|1994-12-18|1994-12-23|COLLECT COD|REG AIR|s haggle al|
+4931|150|7|5|25|26253.75|0.05|0.05|R|F|1994-12-19|1995-01-05|1994-12-21|COLLECT COD|FOB|aggle bravely according to the quic|
+4931|103|6|6|8|8024.80|0.02|0.03|A|F|1995-02-16|1994-12-30|1995-03-15|DELIVER IN PERSON|SHIP|dependencies are slyly|
+4932|51|3|1|13|12363.65|0.04|0.03|A|F|1993-09-13|1993-10-16|1993-09-20|DELIVER IN PERSON|SHIP|slyly according to the furiously fin|
+4932|103|10|2|15|15046.50|0.01|0.02|R|F|1993-11-15|1993-10-25|1993-11-29|NONE|REG AIR|yly. unusu|
+4932|87|8|3|5|4935.40|0.06|0.06|A|F|1993-10-01|1993-09-13|1993-10-04|NONE|MAIL| haggle furiously. slyly ironic packages sl|
+4932|98|1|4|11|10978.99|0.09|0.06|A|F|1993-09-21|1993-09-30|1993-09-23|COLLECT COD|SHIP|as. special depende|
+4933|32|8|1|48|44737.44|0.08|0.00|N|O|1995-10-10|1995-10-03|1995-11-04|COLLECT COD|SHIP|ideas. sly|
+4933|82|3|2|2|1964.16|0.09|0.00|N|O|1995-10-01|1995-09-29|1995-10-19|DELIVER IN PERSON|MAIL|ctions nag final instructions. accou|
+4934|97|10|1|48|47860.32|0.00|0.01|N|O|1997-05-20|1997-04-22|1997-06-02|TAKE BACK RETURN|SHIP| ideas cajol|
+4934|110|1|2|41|41414.51|0.06|0.06|N|O|1997-06-04|1997-04-11|1997-06-25|TAKE BACK RETURN|FOB|wake final, ironic f|
+4934|140|1|3|8|8321.12|0.03|0.06|N|O|1997-05-20|1997-04-30|1997-05-27|TAKE BACK RETURN|MAIL|arefully express pains cajo|
+4934|148|5|4|9|9433.26|0.06|0.08|N|O|1997-06-10|1997-04-09|1997-06-12|TAKE BACK RETURN|REG AIR| haggle alongside of the|
+4934|138|9|5|29|30105.77|0.09|0.03|N|O|1997-04-10|1997-05-05|1997-05-04|DELIVER IN PERSON|AIR|aggle furiously among the busily final re|
+4934|52|3|6|42|39986.10|0.00|0.07|N|O|1997-03-19|1997-05-05|1997-03-25|NONE|MAIL|ven, ironic ideas|
+4934|11|5|7|2|1822.02|0.10|0.06|N|O|1997-06-05|1997-03-26|1997-06-09|COLLECT COD|MAIL|ongside of the brave, regula|
+4935|161|2|1|13|13795.08|0.09|0.01|A|F|1993-06-20|1993-08-13|1993-06-27|COLLECT COD|REG AIR|ly requests. final deposits might |
+4935|40|6|2|37|34781.48|0.01|0.05|R|F|1993-08-30|1993-07-23|1993-09-07|TAKE BACK RETURN|RAIL|y even dependencies nag a|
+4935|11|8|3|24|21864.24|0.06|0.04|A|F|1993-05-29|1993-08-17|1993-06-22|NONE|RAIL|ly quickly s|
+4935|45|6|4|49|46306.96|0.06|0.01|A|F|1993-09-16|1993-08-21|1993-10-12|COLLECT COD|TRUCK|ffily after the furiou|
+4935|10|1|5|14|12740.14|0.08|0.08|A|F|1993-05-30|1993-07-25|1993-05-31|COLLECT COD|FOB|slowly. blith|
+4935|188|9|6|36|39174.48|0.10|0.00|R|F|1993-07-11|1993-07-04|1993-08-01|DELIVER IN PERSON|RAIL|requests across the quick|
+4960|18|5|1|36|33048.36|0.01|0.05|R|F|1995-03-06|1995-05-04|1995-04-05|TAKE BACK RETURN|RAIL|c, unusual accou|
+4960|45|8|2|6|5670.24|0.03|0.08|R|F|1995-03-21|1995-05-13|1995-04-14|TAKE BACK RETURN|SHIP|ual package|
+4960|149|8|3|9|9442.26|0.01|0.03|A|F|1995-03-20|1995-05-05|1995-04-17|COLLECT COD|RAIL|e blithely carefully fina|
+4960|120|7|4|14|14281.68|0.00|0.06|A|F|1995-04-03|1995-04-17|1995-04-07|NONE|RAIL|accounts. warhorses are. grouches |
+4960|98|1|5|8|7984.72|0.07|0.04|R|F|1995-03-14|1995-04-18|1995-04-09|NONE|FOB|as. busily regular packages nag. |
+4960|146|7|6|37|38707.18|0.10|0.04|R|F|1995-05-23|1995-04-12|1995-06-01|DELIVER IN PERSON|MAIL|ending theodolites w|
+4960|170|1|7|42|44947.14|0.08|0.07|A|F|1995-04-19|1995-04-11|1995-05-08|NONE|SHIP|s requests cajole. |
+4961|44|7|1|38|35873.52|0.10|0.07|N|O|1998-07-09|1998-06-03|1998-07-11|TAKE BACK RETURN|FOB|e on the blithely bold accounts. unu|
+4961|60|5|2|1|960.06|0.08|0.08|N|O|1998-07-08|1998-05-25|1998-07-12|DELIVER IN PERSON|MAIL|s affix carefully silent dependen|
+4961|162|3|3|41|43548.56|0.02|0.02|N|O|1998-07-15|1998-06-15|1998-08-05|TAKE BACK RETURN|REG AIR|ily against the n|
+4961|100|3|4|10|10001.00|0.02|0.04|N|O|1998-04-15|1998-07-03|1998-04-18|DELIVER IN PERSON|MAIL|quests. regular, ironic ideas at the ironi|
+4962|19|6|1|46|42274.46|0.01|0.07|R|F|1993-08-23|1993-09-04|1993-08-27|COLLECT COD|REG AIR| pinto beans grow about the sl|
+4963|168|5|1|38|40590.08|0.08|0.02|N|O|1996-12-25|1996-12-12|1997-01-02|COLLECT COD|AIR|tegrate daringly accou|
+4963|76|4|2|16|15617.12|0.00|0.03|N|O|1996-11-20|1997-01-13|1996-12-06|COLLECT COD|MAIL| carefully slyly u|
+4964|133|9|1|29|29960.77|0.04|0.01|N|O|1997-10-18|1997-08-30|1997-11-01|NONE|AIR|k accounts nag carefully-- ironic, fin|
+4964|148|5|2|46|48214.44|0.06|0.06|N|O|1997-10-05|1997-09-12|1997-10-11|NONE|TRUCK|althy deposits|
+4964|143|4|3|18|18776.52|0.00|0.06|N|O|1997-10-13|1997-09-01|1997-11-10|DELIVER IN PERSON|AIR| platelets. furio|
+4964|180|10|4|12|12962.16|0.08|0.01|N|O|1997-09-03|1997-10-25|1997-09-15|NONE|TRUCK|ully silent instructions ca|
+4964|41|10|5|42|39523.68|0.06|0.04|N|O|1997-09-04|1997-08-28|1997-10-02|TAKE BACK RETURN|AIR| hinder. idly even|
+4964|193|7|6|22|24050.18|0.04|0.08|N|O|1997-09-11|1997-10-06|1997-09-29|NONE|AIR|equests doubt quickly. caref|
+4964|173|4|7|28|30048.76|0.00|0.05|N|O|1997-08-30|1997-09-15|1997-09-18|COLLECT COD|RAIL|among the carefully regula|
+4965|131|2|1|28|28871.64|0.05|0.03|A|F|1994-01-02|1993-11-20|1994-01-04|TAKE BACK RETURN|REG AIR| deposits. requests sublate quickly |
+4965|13|10|2|25|22825.25|0.10|0.02|R|F|1994-02-05|1993-12-15|1994-02-24|TAKE BACK RETURN|MAIL|wake at the carefully speci|
+4965|101|8|3|27|27029.70|0.05|0.06|R|F|1993-11-06|1993-12-24|1993-11-30|TAKE BACK RETURN|SHIP|efully final foxes|
+4965|138|9|4|33|34258.29|0.04|0.04|A|F|1993-12-31|1993-11-29|1994-01-27|DELIVER IN PERSON|REG AIR|iously slyly|
+4966|76|6|1|10|9760.70|0.06|0.03|N|O|1996-09-23|1996-11-02|1996-10-07|TAKE BACK RETURN|SHIP| requests. carefully pending requests|
+4966|194|6|2|6|6565.14|0.02|0.01|N|O|1996-12-09|1996-11-29|1996-12-30|NONE|AIR|d deposits are sly excuses. slyly iro|
+4966|165|6|3|7|7456.12|0.00|0.01|N|O|1996-12-08|1996-10-09|1997-01-06|COLLECT COD|MAIL|ckly ironic tithe|
+4966|16|6|4|26|23816.26|0.08|0.03|N|O|1996-11-14|1996-11-29|1996-12-05|COLLECT COD|REG AIR|nt pearls haggle carefully slyly even |
+4966|144|1|5|12|12529.68|0.02|0.07|N|O|1996-12-07|1996-11-23|1996-12-20|DELIVER IN PERSON|RAIL|eodolites. ironic requests across the exp|
+4967|71|1|1|50|48553.50|0.07|0.01|N|O|1997-05-27|1997-05-13|1997-06-12|NONE|REG AIR|kages. final, unusual accounts c|
+4967|53|5|2|43|40981.15|0.00|0.07|N|O|1997-05-28|1997-04-10|1997-06-09|NONE|TRUCK|ons. slyly ironic requests|
+4967|50|1|3|15|14250.75|0.08|0.02|N|O|1997-04-16|1997-04-12|1997-05-08|TAKE BACK RETURN|MAIL|y. blithel|
+4967|123|2|4|1|1023.12|0.10|0.07|N|O|1997-06-04|1997-03-29|1997-06-23|NONE|FOB|osits. unusual frets thrash furiously|
+4992|184|5|1|42|45535.56|0.07|0.01|R|F|1992-07-19|1992-06-16|1992-08-17|TAKE BACK RETURN|RAIL|foxes about the quickly final platele|
+4992|147|4|2|47|49215.58|0.10|0.08|A|F|1992-09-04|1992-08-05|1992-09-21|COLLECT COD|MAIL|atterns use fluffily.|
+4992|144|7|3|17|17750.38|0.03|0.03|A|F|1992-07-05|1992-07-19|1992-07-30|TAKE BACK RETURN|FOB|s along the perma|
+4992|70|7|4|25|24251.75|0.04|0.06|R|F|1992-08-06|1992-07-11|1992-08-20|NONE|SHIP|ly about the never ironic requests. pe|
+4992|139|5|5|23|23899.99|0.01|0.08|R|F|1992-06-28|1992-07-15|1992-07-12|DELIVER IN PERSON|MAIL|uickly regul|
+4992|163|8|6|44|46779.04|0.05|0.02|A|F|1992-06-01|1992-07-22|1992-06-03|NONE|RAIL|rmanent, sly packages print slyly. regula|
+4993|38|4|1|34|31893.02|0.05|0.00|R|F|1994-09-21|1994-10-31|1994-09-24|TAKE BACK RETURN|REG AIR|ular, pending packages at the even packa|
+4993|129|4|2|39|40135.68|0.03|0.08|R|F|1994-09-10|1994-09-04|1994-09-26|COLLECT COD|SHIP|pending, regular requests solve caref|
+4993|166|1|3|42|44778.72|0.06|0.00|A|F|1994-08-27|1994-09-24|1994-09-05|NONE|MAIL| final packages at the q|
+4993|158|6|4|31|32802.65|0.10|0.06|A|F|1994-10-02|1994-10-29|1994-10-15|NONE|AIR|nwind thinly platelets. a|
+4994|156|8|1|36|38021.40|0.00|0.06|N|O|1996-09-29|1996-07-30|1996-10-03|TAKE BACK RETURN|TRUCK|ess ideas. blithely silent brai|
+4994|80|9|2|47|46063.76|0.04|0.05|N|O|1996-09-20|1996-08-04|1996-10-15|COLLECT COD|TRUCK|sts. blithely close ideas sleep quic|
+4994|183|4|3|29|31412.22|0.08|0.01|N|O|1996-08-26|1996-09-27|1996-09-25|DELIVER IN PERSON|RAIL|ptotes boost carefully|
+4994|39|10|4|40|37561.20|0.01|0.06|N|O|1996-08-25|1996-08-16|1996-09-07|TAKE BACK RETURN|REG AIR|eposits. regula|
+4994|42|9|5|24|22608.96|0.01|0.07|N|O|1996-08-19|1996-09-24|1996-08-25|TAKE BACK RETURN|FOB|s. slyly ironic deposits cajole f|
+4994|73|4|6|6|5838.42|0.01|0.02|N|O|1996-09-05|1996-08-04|1996-09-30|TAKE BACK RETURN|FOB|grate carefully around th|
+4994|130|1|7|31|31934.03|0.07|0.04|N|O|1996-10-14|1996-09-23|1996-11-08|TAKE BACK RETURN|RAIL|lar decoys cajole fluffil|
+4995|65|4|1|16|15440.96|0.02|0.05|N|O|1996-02-27|1996-04-03|1996-02-29|DELIVER IN PERSON|MAIL|egular, bold packages. accou|
+4995|81|2|2|43|42186.44|0.00|0.06|N|O|1996-02-24|1996-02-20|1996-03-07|NONE|AIR|ts. blithely silent ideas after t|
+4995|156|7|3|22|23235.30|0.03|0.06|N|O|1996-03-17|1996-03-12|1996-04-01|DELIVER IN PERSON|MAIL|s wake furious, express dependencies.|
+4995|40|1|4|9|8460.36|0.07|0.07|N|O|1996-03-07|1996-03-17|1996-03-11|DELIVER IN PERSON|FOB| ironic packages cajole across t|
+4995|148|7|5|48|50310.72|0.08|0.07|N|O|1996-03-22|1996-04-01|1996-04-07|NONE|SHIP|t blithely. requests affix blithely. |
+4995|110|5|6|48|48485.28|0.09|0.07|N|O|1996-04-14|1996-04-04|1996-05-07|DELIVER IN PERSON|RAIL|nstructions. carefully final depos|
+4996|56|1|1|35|33461.75|0.07|0.01|A|F|1992-10-30|1992-10-27|1992-11-05|TAKE BACK RETURN|SHIP|s. unusual, regular dolphins integrate care|
+4996|156|7|2|39|41189.85|0.02|0.07|A|F|1992-09-19|1992-10-19|1992-10-06|COLLECT COD|FOB|equests are carefully final|
+4996|128|7|3|12|12337.44|0.04|0.06|R|F|1993-01-09|1992-11-22|1993-02-04|DELIVER IN PERSON|SHIP|usly bold requests sleep dogge|
+4996|144|3|4|13|13573.82|0.00|0.00|A|F|1992-09-17|1992-12-02|1992-10-07|DELIVER IN PERSON|TRUCK|o beans use about the furious|
+4997|79|7|1|44|43079.08|0.02|0.05|N|O|1998-06-09|1998-06-12|1998-07-07|NONE|RAIL|r escapades ca|
+4997|17|7|2|5|4585.05|0.02|0.04|N|O|1998-05-16|1998-06-05|1998-06-07|COLLECT COD|REG AIR|cuses are furiously unusual asymptotes|
+4997|58|9|3|24|22993.20|0.04|0.06|N|O|1998-04-20|1998-04-23|1998-05-16|NONE|AIR|xpress, bo|
+4997|40|6|4|5|4700.20|0.10|0.03|N|O|1998-06-12|1998-04-24|1998-06-13|DELIVER IN PERSON|TRUCK|aggle slyly alongside of the slyly i|
+4997|22|7|5|46|42412.92|0.00|0.04|N|O|1998-04-28|1998-06-04|1998-05-08|TAKE BACK RETURN|SHIP|ecial courts are carefully|
+4997|29|2|6|2|1858.04|0.07|0.01|N|O|1998-07-09|1998-06-10|1998-07-21|TAKE BACK RETURN|REG AIR|counts. slyl|
+4998|154|2|1|12|12649.80|0.04|0.03|A|F|1992-02-20|1992-03-06|1992-03-01|TAKE BACK RETURN|RAIL| sleep slyly furiously final accounts. ins|
+4998|183|4|2|15|16247.70|0.06|0.00|R|F|1992-04-24|1992-03-21|1992-05-02|NONE|REG AIR|heodolites sleep quickly.|
+4998|59|10|3|27|25894.35|0.06|0.02|R|F|1992-03-17|1992-02-26|1992-04-05|DELIVER IN PERSON|MAIL|the blithely ironic |
+4998|63|10|4|47|45263.82|0.10|0.04|A|F|1992-02-07|1992-03-07|1992-02-19|DELIVER IN PERSON|TRUCK|mong the careful|
+4998|145|4|5|24|25083.36|0.01|0.04|R|F|1992-01-25|1992-03-16|1992-01-27|COLLECT COD|REG AIR| unwind about|
+4998|99|1|6|8|7992.72|0.03|0.07|A|F|1992-05-01|1992-03-03|1992-05-24|TAKE BACK RETURN|AIR|ions nag quickly according to the theodolit|
+4999|153|8|1|30|31594.50|0.00|0.02|A|F|1993-08-20|1993-08-15|1993-08-30|NONE|AIR|ades cajole carefully unusual ide|
+4999|10|1|2|44|40040.44|0.03|0.01|A|F|1993-08-01|1993-08-04|1993-08-17|COLLECT COD|REG AIR|ependencies. slowly regu|
+4999|86|7|3|30|29582.40|0.09|0.01|R|F|1993-07-21|1993-08-11|1993-08-20|DELIVER IN PERSON|RAIL|s cajole among the blithel|
+5024|166|3|1|17|18124.72|0.10|0.02|N|O|1996-11-24|1997-01-10|1996-12-04|NONE|AIR| to the expre|
+5024|58|6|2|41|39280.05|0.06|0.01|N|O|1996-11-09|1996-12-03|1996-12-01|COLLECT COD|REG AIR|osits hinder carefully |
+5024|112|6|3|18|18217.98|0.04|0.03|N|O|1996-12-02|1997-01-16|1996-12-05|NONE|MAIL|zle carefully sauternes. quickly|
+5024|123|8|4|42|42971.04|0.03|0.06|N|O|1996-12-02|1996-12-08|1996-12-04|DELIVER IN PERSON|RAIL|tegrate. busily spec|
+5025|30|9|1|11|10230.33|0.00|0.04|N|O|1997-02-21|1997-04-16|1997-03-14|COLLECT COD|SHIP|the carefully final esc|
+5025|78|7|2|10|9780.70|0.07|0.04|N|O|1997-06-04|1997-04-29|1997-06-28|COLLECT COD|RAIL|lly silent deposits boost busily again|
+5026|96|8|1|13|12949.17|0.02|0.04|N|O|1997-12-23|1997-11-02|1998-01-03|TAKE BACK RETURN|SHIP|endencies sleep carefully alongs|
+5027|98|2|1|6|5988.54|0.04|0.05|N|O|1997-09-28|1997-11-24|1997-10-25|NONE|FOB|ar, ironic deposi|
+5027|62|3|2|39|37520.34|0.06|0.01|N|O|1997-09-09|1997-11-13|1997-09-21|TAKE BACK RETURN|FOB|ess requests! quickly regular pac|
+5027|126|5|3|32|32835.84|0.00|0.01|N|O|1997-11-13|1997-10-29|1997-11-18|TAKE BACK RETURN|RAIL|cording to|
+5027|26|7|4|37|34262.74|0.02|0.00|N|O|1997-10-05|1997-10-30|1997-10-26|NONE|REG AIR|ost slyly fluffily|
+5027|143|4|5|3|3129.42|0.03|0.06|N|O|1997-09-30|1997-11-26|1997-10-05|DELIVER IN PERSON|AIR|t the even mu|
+5027|87|8|6|25|24677.00|0.06|0.00|N|O|1997-09-16|1997-11-25|1997-10-08|TAKE BACK RETURN|RAIL|ic ideas. requests sleep fluffily am|
+5027|81|2|7|50|49054.00|0.07|0.02|N|O|1997-09-18|1997-11-07|1997-10-05|DELIVER IN PERSON|MAIL| beans dazzle according to the fluffi|
+5028|14|1|1|15|13710.15|0.07|0.07|R|F|1992-07-17|1992-07-16|1992-08-05|COLLECT COD|REG AIR|es are quickly final pains. furiously pend|
+5028|199|10|2|15|16487.85|0.03|0.07|R|F|1992-08-02|1992-07-09|1992-08-30|NONE|REG AIR|gular, bold pinto bea|
+5029|154|5|1|17|17920.55|0.02|0.01|A|F|1993-03-12|1992-12-18|1993-04-02|DELIVER IN PERSON|FOB|! packages boost blithely. furious|
+5029|97|9|2|2|1994.18|0.00|0.04|A|F|1992-11-25|1993-01-04|1992-12-20|DELIVER IN PERSON|MAIL|packages. furiously ironi|
+5030|102|3|1|22|22046.20|0.04|0.06|N|O|1998-09-01|1998-08-15|1998-09-30|TAKE BACK RETURN|TRUCK|. quickly regular foxes believe|
+5030|80|9|2|50|49004.00|0.05|0.06|N|O|1998-08-22|1998-07-25|1998-09-18|TAKE BACK RETURN|FOB|ss excuses serve bli|
+5031|50|1|1|15|14250.75|0.02|0.05|R|F|1995-04-01|1995-02-24|1995-04-12|DELIVER IN PERSON|AIR|yly pending theodolites.|
+5031|161|6|2|40|42446.40|0.10|0.04|A|F|1994-12-04|1995-01-27|1995-01-01|NONE|TRUCK|ns hang blithely across th|
+5031|154|6|3|4|4216.60|0.01|0.07|R|F|1994-12-26|1995-02-24|1995-01-11|NONE|RAIL|after the even frays: ironic, unusual th|
+5031|181|2|4|31|33516.58|0.10|0.08|R|F|1995-01-15|1995-01-08|1995-02-09|COLLECT COD|MAIL|ts across the even requests doze furiously|
+5056|48|7|1|7|6636.28|0.09|0.01|N|O|1997-04-28|1997-04-07|1997-05-15|DELIVER IN PERSON|TRUCK|rouches after the pending instruc|
+5056|197|1|2|19|20846.61|0.04|0.00|N|O|1997-03-24|1997-05-05|1997-04-23|DELIVER IN PERSON|AIR|c theodolites. ironic a|
+5056|90|1|3|23|22772.07|0.02|0.05|N|O|1997-05-12|1997-04-28|1997-05-25|NONE|SHIP|ickly regular requests cajole. depos|
+5056|87|8|4|14|13819.12|0.08|0.00|N|O|1997-06-09|1997-04-13|1997-07-06|COLLECT COD|SHIP|sts haggle carefully along the slyl|
+5057|37|3|1|38|35607.14|0.02|0.03|N|O|1997-10-24|1997-09-07|1997-10-30|TAKE BACK RETURN|MAIL|packages. stealthily bold wa|
+5057|8|1|2|45|40860.00|0.08|0.07|N|O|1997-09-20|1997-10-02|1997-10-20|NONE|FOB| asymptotes wake slyl|
+5058|193|5|1|16|17491.04|0.09|0.07|N|O|1998-07-12|1998-06-09|1998-07-15|DELIVER IN PERSON|SHIP| the special foxes |
+5059|70|5|1|5|4850.35|0.03|0.08|R|F|1993-12-23|1994-01-12|1993-12-24|TAKE BACK RETURN|FOB|ts affix slyly accordi|
+5059|123|2|2|19|19439.28|0.06|0.04|R|F|1994-03-02|1993-12-26|1994-03-14|TAKE BACK RETURN|MAIL| special ideas poach blithely qu|
+5059|77|7|3|45|43968.15|0.02|0.00|A|F|1994-01-28|1994-01-08|1994-02-18|DELIVER IN PERSON|MAIL|enly. requests doze. express, close pa|
+5060|25|8|1|27|24975.54|0.10|0.07|R|F|1992-07-23|1992-09-05|1992-08-07|COLLECT COD|SHIP|s. ironic |
+5060|32|8|2|28|26096.84|0.04|0.04|R|F|1992-09-25|1992-08-11|1992-10-09|NONE|REG AIR|c requests|
+5060|161|2|3|15|15917.40|0.06|0.01|A|F|1992-08-28|1992-08-20|1992-09-01|DELIVER IN PERSON|AIR|ular deposits sl|
+5061|165|2|1|18|19172.88|0.03|0.00|A|F|1993-10-20|1993-10-05|1993-10-28|TAKE BACK RETURN|SHIP|atelets among the ca|
+5061|198|1|2|8|8785.52|0.01|0.02|R|F|1993-09-07|1993-10-31|1993-10-04|DELIVER IN PERSON|REG AIR|regular foxes. ir|
+5061|24|5|3|26|24024.52|0.02|0.05|A|F|1993-11-07|1993-09-13|1993-11-13|NONE|REG AIR| cajole slyly. carefully spe|
+5062|101|4|1|9|9009.90|0.08|0.00|R|F|1993-01-02|1992-12-01|1993-01-20|TAKE BACK RETURN|MAIL| silent theodolites wake. c|
+5062|75|6|2|4|3900.28|0.02|0.02|R|F|1993-02-06|1992-12-14|1993-03-03|DELIVER IN PERSON|AIR|ke furiously express theodolites. |
+5062|159|10|3|50|52957.50|0.09|0.07|A|F|1992-12-25|1992-12-13|1992-12-29|TAKE BACK RETURN|MAIL| the regular, unusual pains. specia|
+5062|161|10|4|18|19100.88|0.03|0.07|R|F|1992-11-04|1992-12-25|1992-11-05|NONE|SHIP|furiously pending requests are ruthles|
+5062|194|8|5|25|27354.75|0.08|0.02|R|F|1992-12-15|1992-11-17|1993-01-01|NONE|TRUCK|uthless excuses ag|
+5063|129|10|1|31|31902.72|0.08|0.01|N|O|1997-06-02|1997-06-20|1997-06-27|NONE|RAIL|kages. ironic, ironic courts wake. carefu|
+5063|174|2|2|43|46189.31|0.04|0.08|N|O|1997-09-14|1997-07-05|1997-10-05|TAKE BACK RETURN|TRUCK|latelets might nod blithely regular requ|
+5063|167|4|3|2|2134.32|0.02|0.03|N|O|1997-06-17|1997-07-27|1997-06-24|COLLECT COD|SHIP|kly regular i|
+5063|135|6|4|18|18632.34|0.08|0.05|N|O|1997-06-02|1997-06-18|1997-06-06|TAKE BACK RETURN|RAIL|refully quiet reques|
+5063|161|8|5|1|1061.16|0.06|0.07|N|O|1997-09-03|1997-06-26|1997-10-03|NONE|FOB|ously special |
+5088|78|6|1|23|22495.61|0.06|0.06|R|F|1993-03-03|1993-03-07|1993-03-08|NONE|FOB|cording to the fluffily expr|
+5088|51|3|2|41|38993.05|0.09|0.00|R|F|1993-01-22|1993-03-07|1993-02-09|TAKE BACK RETURN|TRUCK|ing requests. |
+5088|86|7|3|36|35498.88|0.10|0.05|A|F|1993-04-16|1993-04-03|1993-05-14|NONE|TRUCK|the furiously final deposits. furiously re|
+5088|109|6|4|10|10091.00|0.04|0.05|R|F|1993-04-07|1993-02-06|1993-04-26|NONE|FOB|beans. special requests af|
+5089|158|6|1|4|4232.60|0.05|0.06|R|F|1992-09-18|1992-09-28|1992-10-13|DELIVER IN PERSON|TRUCK|nts sleep blithely |
+5089|162|3|2|20|21243.20|0.00|0.07|R|F|1992-10-10|1992-10-07|1992-11-06|COLLECT COD|RAIL| ironic accounts|
+5089|124|7|3|46|47109.52|0.03|0.04|A|F|1992-11-09|1992-10-13|1992-11-10|TAKE BACK RETURN|RAIL|above the express accounts. exc|
+5089|34|10|4|38|35493.14|0.05|0.03|R|F|1992-11-23|1992-09-11|1992-12-22|TAKE BACK RETURN|SHIP|regular instructions are|
+5090|22|3|1|22|20284.44|0.07|0.00|N|O|1997-05-10|1997-05-25|1997-05-24|TAKE BACK RETURN|TRUCK|ets integrate ironic, regul|
+5090|129|10|2|46|47339.52|0.05|0.00|N|O|1997-04-05|1997-04-14|1997-05-01|COLLECT COD|REG AIR|lose theodolites sleep blit|
+5090|2|9|3|22|19844.00|0.09|0.05|N|O|1997-07-03|1997-04-12|1997-07-26|NONE|REG AIR|ular requests su|
+5090|114|8|4|2|2028.22|0.03|0.06|N|O|1997-04-07|1997-04-23|1997-05-01|TAKE BACK RETURN|AIR|tes. slowly iro|
+5090|48|9|5|21|19908.84|0.10|0.02|N|O|1997-03-29|1997-04-24|1997-04-25|TAKE BACK RETURN|FOB|ly express accounts. slyly even r|
+5090|80|9|6|30|29402.40|0.02|0.03|N|O|1997-05-04|1997-04-14|1997-05-30|COLLECT COD|MAIL|osits nag slyly. fluffily ex|
+5091|78|6|1|50|48903.50|0.05|0.03|N|O|1998-07-21|1998-06-22|1998-07-26|COLLECT COD|REG AIR|al dependencies. r|
+5092|164|1|1|30|31924.80|0.06|0.00|N|O|1995-12-27|1995-12-08|1996-01-09|DELIVER IN PERSON|MAIL|ss, ironic deposits. furiously stea|
+5092|45|4|2|34|32131.36|0.04|0.02|N|O|1995-12-09|1995-12-26|1995-12-21|TAKE BACK RETURN|AIR|ckages nag |
+5092|140|6|3|13|13521.82|0.06|0.01|N|O|1995-11-21|1996-01-05|1995-12-19|TAKE BACK RETURN|SHIP|es detect sly|
+5092|180|1|4|14|15122.52|0.04|0.00|N|O|1996-02-20|1995-11-30|1996-03-20|DELIVER IN PERSON|REG AIR| deposits cajole furiously against the sly|
+5092|186|7|5|42|45619.56|0.01|0.02|N|O|1995-11-06|1996-01-01|1995-12-06|DELIVER IN PERSON|AIR|s use along t|
+5092|178|6|6|11|11859.87|0.03|0.03|N|O|1995-12-02|1995-12-27|1995-12-11|COLLECT COD|MAIL|ly against the slyly silen|
+5092|159|10|7|50|52957.50|0.10|0.03|N|O|1995-11-30|1996-01-14|1995-12-19|NONE|REG AIR|r platelets maintain car|
+5093|168|9|1|40|42726.40|0.05|0.01|R|F|1993-09-16|1993-11-04|1993-10-05|TAKE BACK RETURN|REG AIR|ing pinto beans. quickly bold dependenci|
+5093|74|2|2|15|14611.05|0.01|0.04|A|F|1993-12-02|1993-11-18|1994-01-01|DELIVER IN PERSON|FOB|ly among the unusual foxe|
+5093|151|9|3|31|32585.65|0.00|0.02|R|F|1993-09-22|1993-11-14|1993-09-26|TAKE BACK RETURN|REG AIR| against the|
+5093|156|1|4|37|39077.55|0.04|0.00|A|F|1993-10-26|1993-12-02|1993-10-27|NONE|TRUCK|courts. qui|
+5093|115|2|5|30|30453.30|0.06|0.05|A|F|1993-11-22|1993-11-27|1993-12-14|DELIVER IN PERSON|TRUCK|ithely ironic sheaves use fluff|
+5093|121|6|6|31|31654.72|0.01|0.08|A|F|1993-12-17|1993-11-14|1994-01-02|NONE|SHIP|he final foxes. fluffily ironic |
+5094|143|10|1|19|19819.66|0.03|0.03|R|F|1993-03-31|1993-06-12|1993-04-04|NONE|AIR|ronic foxes. furi|
+5094|108|5|2|23|23186.30|0.05|0.07|R|F|1993-06-13|1993-05-19|1993-07-06|NONE|MAIL|st furiously above the fluffily care|
+5094|92|6|3|11|10912.99|0.04|0.08|A|F|1993-06-25|1993-06-24|1993-07-18|TAKE BACK RETURN|MAIL|s cajole quickly against the furiously ex|
+5094|79|10|4|21|20560.47|0.09|0.08|R|F|1993-07-26|1993-05-03|1993-08-16|NONE|MAIL| blithely furiously final re|
+5095|65|10|1|46|44392.76|0.07|0.01|A|F|1992-06-26|1992-06-25|1992-07-05|COLLECT COD|RAIL|egular instruction|
+5095|106|3|2|2|2012.20|0.07|0.08|A|F|1992-07-09|1992-05-25|1992-07-21|DELIVER IN PERSON|REG AIR|detect car|
+5095|123|8|3|28|28647.36|0.01|0.04|A|F|1992-06-20|1992-06-27|1992-06-22|DELIVER IN PERSON|AIR| into the final courts. ca|
+5095|178|7|4|42|45283.14|0.08|0.08|R|F|1992-05-23|1992-06-01|1992-06-18|COLLECT COD|TRUCK|ccounts. packages could have t|
+5095|166|7|5|9|9595.44|0.10|0.07|R|F|1992-08-14|1992-06-23|1992-08-16|TAKE BACK RETURN|REG AIR|bold theodolites wake about the expr|
+5095|97|8|6|15|14956.35|0.01|0.06|A|F|1992-07-11|1992-07-12|1992-08-09|COLLECT COD|AIR| to the packages wake sly|
+5095|169|10|7|40|42766.40|0.05|0.02|A|F|1992-07-11|1992-06-07|1992-07-26|DELIVER IN PERSON|MAIL|carefully unusual plat|
+5120|133|4|1|28|28927.64|0.06|0.03|N|O|1996-07-20|1996-08-31|1996-08-06|NONE|RAIL| across the silent requests. caref|
+5121|184|5|1|23|24936.14|0.06|0.01|A|F|1992-05-18|1992-06-20|1992-06-02|TAKE BACK RETURN|REG AIR|even courts are blithely ironically |
+5121|111|1|2|45|45499.95|0.08|0.04|A|F|1992-08-13|1992-07-27|1992-09-12|NONE|TRUCK|pecial accounts cajole ca|
+5121|97|10|3|27|26921.43|0.08|0.07|R|F|1992-06-17|1992-06-11|1992-06-19|NONE|MAIL|ly silent theodolit|
+5121|68|7|4|10|9680.60|0.04|0.05|R|F|1992-06-08|1992-07-10|1992-07-02|TAKE BACK RETURN|FOB|e quickly according |
+5121|89|10|5|46|45497.68|0.03|0.02|R|F|1992-05-27|1992-07-19|1992-05-28|TAKE BACK RETURN|FOB|use express foxes. slyly |
+5121|1|8|6|2|1802.00|0.04|0.07|R|F|1992-08-10|1992-06-28|1992-08-11|NONE|FOB| final, regular account|
+5122|183|4|1|28|30329.04|0.03|0.00|N|O|1996-04-20|1996-03-29|1996-04-29|DELIVER IN PERSON|FOB|g the busily ironic accounts boos|
+5122|82|3|2|43|42229.44|0.09|0.03|N|O|1996-05-31|1996-04-12|1996-06-13|NONE|MAIL|ut the carefully special foxes. idle,|
+5122|45|6|3|12|11340.48|0.07|0.03|N|O|1996-04-02|1996-04-27|1996-04-10|DELIVER IN PERSON|AIR|lar instructions |
+5123|26|7|1|13|12038.26|0.08|0.07|N|O|1998-05-17|1998-03-23|1998-06-02|COLLECT COD|MAIL|regular pearls|
+5124|55|7|1|43|41067.15|0.00|0.02|N|O|1997-07-10|1997-05-13|1997-07-31|COLLECT COD|AIR|onic package|
+5124|6|3|2|41|37146.00|0.02|0.06|N|O|1997-07-05|1997-06-29|1997-07-23|DELIVER IN PERSON|RAIL|wake across the|
+5124|125|6|3|44|45105.28|0.03|0.03|N|O|1997-07-13|1997-06-26|1997-08-01|DELIVER IN PERSON|RAIL|equests. carefully unusual d|
+5124|70|9|4|36|34922.52|0.10|0.07|N|O|1997-04-20|1997-07-03|1997-05-04|TAKE BACK RETURN|AIR|r deposits ab|
+5125|6|9|1|38|34428.00|0.09|0.05|N|O|1998-03-20|1998-04-14|1998-03-22|COLLECT COD|MAIL|ily even deposits w|
+5125|160|1|2|5|5300.80|0.08|0.06|N|O|1998-04-07|1998-04-14|1998-04-29|COLLECT COD|RAIL| thinly even pack|
+5126|24|3|1|33|30492.66|0.02|0.02|R|F|1993-02-04|1992-12-23|1993-02-14|NONE|RAIL|ipliers promise furiously whithout the |
+5126|101|6|2|43|43047.30|0.09|0.04|R|F|1993-01-07|1992-12-19|1993-01-16|COLLECT COD|MAIL|e silently. ironic, unusual accounts|
+5126|78|8|3|23|22495.61|0.08|0.01|R|F|1993-01-02|1993-01-02|1993-01-05|COLLECT COD|TRUCK|egular, blithe packages.|
+5127|19|3|1|33|30327.33|0.08|0.04|N|O|1997-03-25|1997-03-02|1997-04-04|NONE|SHIP| bold deposits use carefully a|
+5127|32|8|2|20|18640.60|0.01|0.03|N|O|1997-05-11|1997-02-26|1997-05-12|TAKE BACK RETURN|SHIP|dolites about the final platelets w|
+5152|105|2|1|9|9045.90|0.04|0.03|N|O|1997-04-11|1997-02-11|1997-04-18|COLLECT COD|AIR| cajole furiously alongside of the bo|
+5152|134|10|2|50|51706.50|0.04|0.04|N|O|1997-03-10|1997-02-04|1997-03-15|COLLECT COD|FOB| the final deposits. slyly ironic warth|
+5153|35|1|1|42|39271.26|0.03|0.01|N|O|1995-10-03|1995-11-09|1995-10-11|COLLECT COD|RAIL|re thinly. ironic|
+5153|53|5|2|14|13342.70|0.05|0.05|N|O|1995-11-29|1995-10-21|1995-12-08|TAKE BACK RETURN|TRUCK| slyly daring pinto beans lose blithely fi|
+5153|68|7|3|30|29041.80|0.09|0.01|N|O|1995-11-10|1995-11-14|1995-11-16|DELIVER IN PERSON|AIR|beans sleep bl|
+5153|173|2|4|32|34341.44|0.10|0.08|N|O|1995-12-05|1995-09-25|1996-01-03|DELIVER IN PERSON|MAIL|egular deposits. ironi|
+5153|112|2|5|36|36435.96|0.01|0.03|N|O|1995-12-15|1995-11-08|1995-12-30|COLLECT COD|TRUCK| ironic instru|
+5153|136|2|6|42|43517.46|0.00|0.03|N|O|1995-10-19|1995-11-23|1995-11-06|TAKE BACK RETURN|RAIL|ickly even deposi|
+5154|190|1|1|11|11992.09|0.02|0.05|N|O|1997-08-06|1997-06-30|1997-09-04|NONE|RAIL|luffily bold foxes. final|
+5154|144|5|2|15|15662.10|0.07|0.08|N|O|1997-06-23|1997-07-11|1997-07-11|NONE|AIR|even packages. packages use|
+5155|48|9|1|1|948.04|0.00|0.00|A|F|1994-07-03|1994-08-11|1994-07-29|COLLECT COD|FOB|oze slyly after the silent, regular idea|
+5155|188|9|2|5|5440.90|0.08|0.02|A|F|1994-06-30|1994-08-13|1994-07-15|TAKE BACK RETURN|AIR|ole blithely slyly ironic |
+5155|106|3|3|28|28170.80|0.05|0.02|R|F|1994-07-01|1994-07-19|1994-07-18|COLLECT COD|REG AIR|s cajole. accounts wake. thinly quiet pla|
+5155|79|7|4|39|38183.73|0.09|0.06|A|F|1994-08-25|1994-09-01|1994-09-18|COLLECT COD|TRUCK|l dolphins nag caref|
+5156|117|4|1|21|21359.31|0.06|0.03|N|O|1997-01-01|1997-01-30|1997-01-11|TAKE BACK RETURN|TRUCK|ts detect against the furiously reg|
+5156|148|1|2|36|37733.04|0.04|0.07|N|O|1997-02-12|1996-12-10|1997-03-13|TAKE BACK RETURN|REG AIR| slyly even orbi|
+5157|55|7|1|35|33426.75|0.06|0.08|N|O|1997-07-28|1997-09-30|1997-08-15|TAKE BACK RETURN|REG AIR|to the furiously sil|
+5157|138|9|2|18|18686.34|0.10|0.04|N|O|1997-09-06|1997-10-03|1997-09-19|COLLECT COD|MAIL|y bold deposits nag blithely. final reque|
+5157|167|8|3|15|16007.40|0.09|0.00|N|O|1997-07-27|1997-08-30|1997-08-08|DELIVER IN PERSON|RAIL|cajole. spec|
+5157|59|7|4|25|23976.25|0.00|0.03|N|O|1997-08-24|1997-09-23|1997-08-28|COLLECT COD|REG AIR| packages detect. even requests against th|
+5157|149|8|5|40|41965.60|0.09|0.06|N|O|1997-08-11|1997-08-28|1997-09-01|TAKE BACK RETURN|FOB|ial packages according to |
+5157|150|9|6|26|27303.90|0.10|0.01|N|O|1997-07-28|1997-08-22|1997-08-22|NONE|FOB|nto beans cajole car|
+5157|49|8|7|12|11388.48|0.10|0.08|N|O|1997-10-19|1997-08-07|1997-10-26|NONE|FOB|es. busily |
+5158|45|4|1|43|40636.72|0.10|0.04|N|O|1997-04-10|1997-03-06|1997-04-15|DELIVER IN PERSON|AIR|nusual platelets. slyly even foxes cajole |
+5158|85|6|2|18|17731.44|0.04|0.04|N|O|1997-04-30|1997-03-28|1997-05-12|COLLECT COD|REG AIR|hely regular pa|
+5158|142|9|3|41|42727.74|0.05|0.05|N|O|1997-02-25|1997-03-19|1997-03-03|COLLECT COD|AIR|deposits. quickly special |
+5158|131|7|4|49|50525.37|0.05|0.01|N|O|1997-04-10|1997-03-21|1997-04-30|NONE|REG AIR|r requests sleep q|
+5158|119|9|5|20|20382.20|0.01|0.04|N|O|1997-02-03|1997-02-20|1997-02-08|TAKE BACK RETURN|AIR|latelets use accordin|
+5158|88|9|6|39|38535.12|0.08|0.04|N|O|1997-05-15|1997-04-04|1997-06-02|DELIVER IN PERSON|FOB|lithely fina|
+5158|91|5|7|38|37661.42|0.10|0.05|N|O|1997-05-09|1997-03-03|1997-06-04|NONE|SHIP|uffily regular ac|
+5159|124|7|1|39|39940.68|0.06|0.07|N|O|1996-12-17|1996-12-08|1997-01-10|COLLECT COD|MAIL|re furiously after the pending dolphin|
+5159|17|1|2|46|42182.46|0.01|0.01|N|O|1996-12-15|1996-12-07|1996-12-30|DELIVER IN PERSON|SHIP|s kindle slyly carefully regular|
+5159|152|4|3|22|23147.30|0.01|0.02|N|O|1996-11-06|1996-11-04|1996-11-15|TAKE BACK RETURN|SHIP|he furiously sile|
+5159|52|3|4|5|4760.25|0.10|0.00|N|O|1996-11-25|1996-12-19|1996-12-25|TAKE BACK RETURN|FOB|nal deposits. pending, ironic ideas grow|
+5159|198|10|5|36|39534.84|0.06|0.01|N|O|1997-01-24|1996-11-07|1997-02-08|NONE|REG AIR|packages wake.|
+5184|153|8|1|33|34753.95|0.07|0.04|N|O|1998-08-17|1998-10-16|1998-08-24|TAKE BACK RETURN|AIR|posits. carefully express asympto|
+5184|16|6|2|47|43052.47|0.05|0.01|N|O|1998-11-02|1998-08-19|1998-11-07|COLLECT COD|TRUCK|se. carefully express pinto beans x|
+5184|88|9|3|39|38535.12|0.03|0.06|N|O|1998-10-27|1998-10-17|1998-11-19|DELIVER IN PERSON|FOB|es above the care|
+5184|176|7|4|26|27980.42|0.05|0.08|N|O|1998-11-11|1998-08-26|1998-12-01|TAKE BACK RETURN|TRUCK| packages are|
+5184|124|9|5|19|19458.28|0.06|0.03|N|O|1998-11-15|1998-10-12|1998-11-21|COLLECT COD|REG AIR|refully express platelets sleep carefull|
+5184|80|9|6|49|48023.92|0.02|0.00|N|O|1998-09-18|1998-08-28|1998-10-14|COLLECT COD|FOB|thlessly closely even reque|
+5185|197|1|1|37|40596.03|0.00|0.04|N|O|1997-08-08|1997-09-08|1997-08-14|COLLECT COD|SHIP|gainst the courts dazzle care|
+5185|25|8|2|32|29600.64|0.06|0.00|N|O|1997-08-17|1997-09-30|1997-08-24|TAKE BACK RETURN|REG AIR|ackages. slyly even requests|
+5185|196|9|3|41|44943.79|0.00|0.05|N|O|1997-10-15|1997-10-11|1997-11-02|COLLECT COD|REG AIR|ly blithe deposits. furi|
+5185|96|7|4|30|29882.70|0.09|0.04|N|O|1997-10-17|1997-09-16|1997-10-23|TAKE BACK RETURN|SHIP|ress packages are furiously|
+5185|128|9|5|8|8224.96|0.04|0.00|N|O|1997-08-30|1997-09-02|1997-09-22|COLLECT COD|REG AIR|sts around the slyly perma|
+5185|146|9|6|50|52307.00|0.03|0.04|N|O|1997-10-15|1997-10-19|1997-11-06|TAKE BACK RETURN|FOB|final platelets. ideas sleep careful|
+5186|55|10|1|38|36291.90|0.06|0.02|N|O|1996-11-23|1996-09-21|1996-12-11|DELIVER IN PERSON|MAIL|y ruthless foxes. fluffily |
+5186|91|2|2|31|30723.79|0.09|0.03|N|O|1996-10-19|1996-09-26|1996-10-25|TAKE BACK RETURN|REG AIR| accounts use furiously slyly spe|
+5186|89|10|3|26|25716.08|0.03|0.02|N|O|1996-08-08|1996-10-05|1996-08-21|DELIVER IN PERSON|FOB|capades. accounts sublate. pinto|
+5186|90|1|4|8|7920.72|0.10|0.05|N|O|1996-09-23|1996-09-29|1996-09-30|COLLECT COD|RAIL|y regular notornis k|
+5186|18|2|5|28|25704.28|0.09|0.03|N|O|1996-10-05|1996-10-27|1996-10-19|TAKE BACK RETURN|RAIL|al decoys. blit|
+5186|82|3|6|35|34372.80|0.00|0.05|N|O|1996-10-20|1996-10-12|1996-11-12|TAKE BACK RETURN|RAIL|sly silent pack|
+5186|198|10|7|44|48320.36|0.00|0.08|N|O|1996-09-23|1996-10-14|1996-10-01|NONE|TRUCK|old, final accounts cajole sl|
+5187|11|1|1|49|44639.49|0.04|0.06|N|O|1997-10-20|1997-10-12|1997-10-26|DELIVER IN PERSON|AIR|l, regular platelets instead of the foxes w|
+5187|83|4|2|1|983.08|0.10|0.08|N|O|1997-08-08|1997-08-24|1997-08-22|DELIVER IN PERSON|REG AIR|aggle never bold |
+5188|118|2|1|18|18325.98|0.04|0.03|N|O|1995-06-19|1995-05-19|1995-06-25|DELIVER IN PERSON|AIR|p according to the sometimes regu|
+5188|194|8|2|36|39390.84|0.04|0.02|A|F|1995-03-09|1995-05-16|1995-03-19|NONE|TRUCK|packages? blithely s|
+5188|148|1|3|9|9433.26|0.06|0.08|A|F|1995-05-09|1995-05-22|1995-05-19|TAKE BACK RETURN|REG AIR|r attainments are across the |
+5189|138|9|1|44|45677.72|0.02|0.06|A|F|1994-01-13|1994-02-07|1994-01-21|DELIVER IN PERSON|MAIL|y finally pendin|
+5189|16|3|2|38|34808.38|0.06|0.00|A|F|1994-03-26|1994-01-28|1994-04-20|DELIVER IN PERSON|REG AIR|ideas. idle, final deposits de|
+5189|110|5|3|4|4040.44|0.09|0.02|A|F|1993-12-21|1994-02-23|1994-01-09|DELIVER IN PERSON|REG AIR|. blithely exp|
+5189|94|7|4|49|48710.41|0.05|0.01|R|F|1994-01-22|1994-01-19|1994-02-04|TAKE BACK RETURN|SHIP| requests |
+5189|123|2|5|14|14323.68|0.02|0.03|A|F|1994-01-23|1994-01-05|1994-02-12|DELIVER IN PERSON|REG AIR|unusual packag|
+5189|17|8|6|41|37597.41|0.02|0.06|R|F|1993-12-12|1994-02-05|1994-01-09|DELIVER IN PERSON|RAIL|ial theodolites cajole slyly. slyly unus|
+5190|56|1|1|43|41110.15|0.09|0.06|A|F|1992-08-19|1992-06-10|1992-09-01|DELIVER IN PERSON|FOB|encies use fluffily unusual requests? hoc|
+5190|132|3|2|6|6192.78|0.10|0.08|A|F|1992-08-08|1992-07-14|1992-08-22|COLLECT COD|RAIL|furiously regular pinto beans. furiously i|
+5190|89|10|3|45|44508.60|0.04|0.03|A|F|1992-07-23|1992-06-16|1992-08-04|NONE|FOB|y carefully final ideas. f|
+5191|115|6|1|41|41619.51|0.00|0.08|A|F|1995-02-05|1995-02-27|1995-02-15|DELIVER IN PERSON|AIR|uests! ironic theodolites cajole care|
+5191|168|7|2|40|42726.40|0.02|0.01|A|F|1995-03-31|1995-02-21|1995-04-02|NONE|AIR|nes haggle sometimes. requests eng|
+5191|43|4|3|27|25462.08|0.07|0.05|A|F|1994-12-26|1995-01-24|1995-01-14|DELIVER IN PERSON|RAIL|tructions nag bravely within the re|
+5191|183|4|4|7|7582.26|0.01|0.04|A|F|1995-03-24|1995-01-30|1995-03-30|NONE|RAIL|eposits. express|
+5216|69|10|1|17|16474.02|0.04|0.06|N|O|1997-08-20|1997-11-07|1997-09-14|COLLECT COD|FOB|s according to the accounts bo|
+5217|80|1|1|50|49004.00|0.05|0.02|N|O|1995-12-26|1995-11-21|1996-01-24|DELIVER IN PERSON|MAIL|s. express, express accounts c|
+5217|16|7|2|23|21068.23|0.06|0.07|N|O|1996-01-18|1995-12-24|1996-02-10|COLLECT COD|RAIL|ven ideas. requests amo|
+5217|102|7|3|23|23048.30|0.03|0.02|N|O|1995-11-15|1995-12-17|1995-11-27|DELIVER IN PERSON|FOB|pending packages cajole ne|
+5217|81|2|4|47|46110.76|0.04|0.00|N|O|1995-11-24|1995-12-25|1995-11-25|COLLECT COD|AIR|ronic packages i|
+5218|83|4|1|43|42272.44|0.05|0.04|A|F|1992-08-04|1992-09-12|1992-08-17|DELIVER IN PERSON|SHIP|k theodolites. express, even id|
+5218|125|4|2|33|33828.96|0.06|0.01|R|F|1992-09-16|1992-09-30|1992-09-27|NONE|TRUCK|ronic instructi|
+5219|135|6|1|2|2070.26|0.08|0.00|N|O|1997-06-26|1997-04-29|1997-07-08|TAKE BACK RETURN|FOB| blithely according to the stea|
+5219|119|9|2|20|20382.20|0.05|0.00|N|O|1997-04-20|1997-05-26|1997-05-13|COLLECT COD|FOB|e along the ironic,|
+5220|83|4|1|27|26543.16|0.10|0.04|R|F|1992-09-21|1992-08-29|1992-10-16|DELIVER IN PERSON|RAIL|s cajole blithely furiously iron|
+5221|104|9|1|24|24098.40|0.07|0.03|N|O|1995-10-04|1995-08-11|1995-10-30|COLLECT COD|REG AIR|s pinto beans sleep. sly|
+5221|9|10|2|34|30906.00|0.01|0.05|N|O|1995-09-11|1995-07-17|1995-10-10|COLLECT COD|SHIP|eans. furio|
+5221|180|10|3|16|17282.88|0.04|0.01|N|O|1995-08-29|1995-09-06|1995-09-12|TAKE BACK RETURN|TRUCK|ending request|
+5222|151|3|1|1|1051.15|0.00|0.00|A|F|1994-08-19|1994-07-16|1994-09-08|TAKE BACK RETURN|FOB|idle requests. carefully pending pinto bean|
+5223|45|4|1|24|22680.96|0.00|0.00|A|F|1994-10-03|1994-09-20|1994-10-11|TAKE BACK RETURN|TRUCK|refully bold courts besides the regular,|
+5223|124|9|2|25|25603.00|0.09|0.02|R|F|1994-07-12|1994-08-13|1994-08-01|NONE|FOB|y express ideas impress|
+5223|6|3|3|19|17214.00|0.04|0.01|R|F|1994-10-28|1994-08-26|1994-10-31|COLLECT COD|REG AIR|ntly. furiously even excuses a|
+5223|130|9|4|40|41205.20|0.01|0.04|R|F|1994-10-01|1994-09-18|1994-10-28|COLLECT COD|SHIP|kly pending |
+5248|81|2|1|39|38262.12|0.05|0.03|N|O|1995-08-10|1995-07-04|1995-09-09|TAKE BACK RETURN|MAIL|yly even accounts. spe|
+5248|138|9|2|45|46715.85|0.00|0.06|A|F|1995-05-09|1995-07-12|1995-05-27|DELIVER IN PERSON|FOB|. bold, pending foxes h|
+5249|50|9|1|31|29451.55|0.07|0.03|A|F|1994-11-21|1994-11-19|1994-12-08|NONE|REG AIR|f the excuses. furiously fin|
+5249|31|7|2|44|40965.32|0.05|0.00|A|F|1994-12-28|1994-11-29|1994-12-29|TAKE BACK RETURN|MAIL|ole furiousl|
+5249|32|8|3|13|12116.39|0.09|0.00|R|F|1994-09-27|1994-10-20|1994-10-05|DELIVER IN PERSON|SHIP|ites. finally exp|
+5249|146|3|4|29|30338.06|0.00|0.05|A|F|1994-09-16|1994-11-03|1994-10-06|NONE|TRUCK| players. f|
+5249|158|6|5|12|12697.80|0.01|0.08|R|F|1994-12-28|1994-11-07|1995-01-15|COLLECT COD|MAIL|press depths could have to sleep carefu|
+5250|44|3|1|2|1888.08|0.08|0.04|N|O|1995-08-09|1995-10-10|1995-08-13|COLLECT COD|AIR|its. final pinto|
+5250|192|6|2|27|29489.13|0.10|0.05|N|O|1995-10-24|1995-09-03|1995-11-18|COLLECT COD|TRUCK|l forges are. furiously unusual pin|
+5251|139|10|1|36|37408.68|0.10|0.01|N|O|1995-07-16|1995-07-05|1995-07-28|DELIVER IN PERSON|FOB|slowly! bli|
+5252|141|10|1|13|13534.82|0.02|0.01|N|O|1996-03-02|1996-05-10|1996-03-11|NONE|FOB|boost fluffily across |
+5252|139|5|2|39|40526.07|0.06|0.05|N|O|1996-05-17|1996-04-23|1996-05-23|COLLECT COD|AIR|gular requests.|
+5252|195|9|3|9|9856.71|0.09|0.03|N|O|1996-05-30|1996-05-03|1996-06-26|TAKE BACK RETURN|RAIL|x. slyly special depos|
+5252|87|8|4|48|47379.84|0.01|0.06|N|O|1996-04-17|1996-03-19|1996-05-03|COLLECT COD|AIR|bold requests. furious|
+5252|68|5|5|24|23233.44|0.04|0.05|N|O|1996-05-11|1996-04-17|1996-05-12|COLLECT COD|REG AIR|posits after the fluffi|
+5252|3|10|6|41|37023.00|0.02|0.03|N|O|1996-03-16|1996-04-18|1996-03-17|NONE|TRUCK|ording to the blithely express somas sho|
+5253|31|2|1|35|32586.05|0.02|0.00|N|O|1995-07-23|1995-06-12|1995-08-03|DELIVER IN PERSON|AIR|ven deposits. careful|
+5253|150|7|2|38|39905.70|0.02|0.06|N|O|1995-08-03|1995-06-14|1995-08-27|DELIVER IN PERSON|REG AIR|onic dependencies are furiou|
+5253|14|5|3|9|8226.09|0.03|0.08|N|F|1995-06-08|1995-05-12|1995-06-23|DELIVER IN PERSON|REG AIR|lyly express deposits use furiou|
+5253|166|1|4|25|26654.00|0.04|0.03|A|F|1995-05-21|1995-06-13|1995-06-09|COLLECT COD|TRUCK|urts. even theodoli|
+5254|111|2|1|35|35388.85|0.01|0.07|A|F|1992-07-28|1992-09-05|1992-08-07|COLLECT COD|REG AIR|ntegrate carefully among the pending|
+5254|135|6|2|10|10351.30|0.05|0.04|A|F|1992-11-19|1992-10-20|1992-12-15|COLLECT COD|SHIP| accounts. silent deposit|
+5254|192|5|3|32|34950.08|0.00|0.08|A|F|1992-08-10|1992-09-21|1992-08-16|NONE|RAIL|ts impress closely furi|
+5254|163|2|4|45|47842.20|0.05|0.06|A|F|1992-11-11|1992-09-01|1992-12-07|COLLECT COD|REG AIR| wake. blithely silent excuse|
+5254|29|8|5|23|21367.46|0.02|0.06|A|F|1992-08-16|1992-09-05|1992-09-15|COLLECT COD|RAIL|lyly regular accounts. furiously pendin|
+5254|158|3|6|34|35977.10|0.09|0.02|R|F|1992-08-29|1992-10-16|1992-09-20|TAKE BACK RETURN|RAIL| furiously above the furiously |
+5254|20|7|7|9|8280.18|0.09|0.03|R|F|1992-07-29|1992-10-15|1992-08-20|TAKE BACK RETURN|REG AIR| wake blithely fluff|
+5255|131|7|1|2|2062.26|0.04|0.08|N|O|1996-09-27|1996-10-04|1996-10-04|DELIVER IN PERSON|RAIL|ajole blithely fluf|
+5255|172|10|2|30|32165.10|0.04|0.08|N|O|1996-09-20|1996-08-18|1996-10-09|TAKE BACK RETURN|AIR| to the silent requests cajole b|
+5255|130|3|3|41|42235.33|0.09|0.03|N|O|1996-08-21|1996-09-24|1996-09-05|COLLECT COD|FOB|tect blithely against t|
+5280|97|9|1|16|15953.44|0.02|0.03|N|O|1998-03-29|1998-01-28|1998-04-03|TAKE BACK RETURN|SHIP| foxes are furiously. theodoli|
+5280|176|5|2|46|49503.82|0.01|0.06|N|O|1998-01-04|1998-01-21|1998-02-03|TAKE BACK RETURN|FOB|efully carefully pen|
+5281|114|1|1|37|37522.07|0.05|0.02|N|O|1995-11-10|1996-01-31|1995-11-22|DELIVER IN PERSON|MAIL|ronic dependencies. fluffily final p|
+5281|105|2|2|38|38193.80|0.00|0.05|N|O|1996-02-17|1995-12-19|1996-02-29|NONE|RAIL|n asymptotes could wake about th|
+5281|127|2|3|23|23623.76|0.08|0.00|N|O|1995-12-30|1996-01-26|1996-01-23|COLLECT COD|REG AIR|. final theodolites cajole. ironic p|
+5281|87|8|4|48|47379.84|0.03|0.05|N|O|1996-01-31|1995-12-23|1996-02-08|TAKE BACK RETURN|REG AIR|ss the furiously |
+5281|43|10|5|33|31120.32|0.01|0.07|N|O|1996-03-01|1995-12-28|1996-03-05|COLLECT COD|RAIL|ly brave foxes. bold deposits above the |
+5282|118|2|1|36|36651.96|0.05|0.02|N|O|1998-05-20|1998-04-10|1998-06-14|DELIVER IN PERSON|TRUCK|re slyly accor|
+5282|52|10|2|32|30465.60|0.02|0.05|N|O|1998-03-01|1998-03-31|1998-03-03|NONE|FOB|onic deposits; furiou|
+5282|58|10|3|28|26825.40|0.03|0.06|N|O|1998-05-06|1998-04-24|1998-05-30|COLLECT COD|SHIP|fily final instruc|
+5283|5|2|1|20|18100.00|0.05|0.02|A|F|1994-09-16|1994-08-03|1994-10-15|TAKE BACK RETURN|TRUCK|al deposits? blithely even pinto beans|
+5283|186|7|2|1|1086.18|0.10|0.08|R|F|1994-06-20|1994-08-03|1994-07-01|COLLECT COD|FOB|deposits within the furio|
+5284|173|1|1|16|17170.72|0.04|0.02|N|O|1995-08-17|1995-08-23|1995-08-26|DELIVER IN PERSON|TRUCK|unts detect furiously even d|
+5284|44|7|2|24|22656.96|0.03|0.08|N|O|1995-10-21|1995-08-23|1995-10-27|COLLECT COD|AIR| haggle according |
+5285|193|5|1|31|33888.89|0.08|0.00|A|F|1994-04-17|1994-04-05|1994-05-09|NONE|RAIL|ubt. quickly blithe |
+5285|31|2|2|37|34448.11|0.09|0.02|R|F|1994-02-26|1994-02-18|1994-03-27|NONE|SHIP|uffily regu|
+5285|34|10|3|24|22416.72|0.02|0.04|A|F|1994-04-19|1994-04-03|1994-04-25|DELIVER IN PERSON|FOB|ess packages. quick, even deposits snooze b|
+5285|43|2|4|12|11316.48|0.05|0.06|A|F|1994-04-22|1994-04-07|1994-05-19|NONE|AIR| deposits-- quickly bold requests hag|
+5285|71|2|5|1|971.07|0.03|0.05|R|F|1994-03-14|1994-02-20|1994-04-10|COLLECT COD|TRUCK|e fluffily about the slyly special pa|
+5285|146|7|6|1|1046.14|0.06|0.01|R|F|1994-02-08|1994-04-02|1994-02-17|COLLECT COD|SHIP|ing deposits integra|
+5286|199|1|1|1|1099.19|0.01|0.07|N|O|1997-11-25|1997-11-07|1997-12-17|COLLECT COD|REG AIR|ly! furiously final pack|
+5286|97|1|2|7|6979.63|0.06|0.05|N|O|1997-10-23|1997-12-10|1997-11-20|TAKE BACK RETURN|RAIL|y express instructions sleep carefull|
+5286|16|10|3|3|2748.03|0.06|0.08|N|O|1997-12-04|1997-11-06|1997-12-09|COLLECT COD|MAIL|re fluffily|
+5286|40|6|4|6|5640.24|0.04|0.03|N|O|1997-10-15|1997-12-05|1997-11-12|COLLECT COD|RAIL|y special a|
+5286|186|7|5|38|41274.84|0.07|0.05|N|O|1997-11-29|1997-11-26|1997-12-15|TAKE BACK RETURN|SHIP|fluffily. special, ironic deposit|
+5286|138|9|6|24|24915.12|0.08|0.00|N|O|1997-09-27|1997-12-21|1997-09-30|COLLECT COD|TRUCK|s. express foxes of the|
+5287|39|10|1|32|30048.96|0.01|0.01|A|F|1994-01-29|1994-01-27|1994-02-08|NONE|RAIL|heodolites haggle caref|
+5312|61|6|1|27|25948.62|0.04|0.08|A|F|1995-04-20|1995-04-09|1995-04-25|COLLECT COD|SHIP|tructions cajol|
+5312|2|5|2|43|38786.00|0.05|0.08|A|F|1995-03-24|1995-05-07|1995-03-28|NONE|TRUCK|ly unusual|
+5313|17|1|1|34|31178.34|0.10|0.02|N|O|1997-08-07|1997-08-12|1997-08-24|DELIVER IN PERSON|FOB|ccording to the blithely final account|
+5313|13|10|2|17|15521.17|0.00|0.02|N|O|1997-09-02|1997-08-20|1997-09-07|NONE|SHIP|uests wake|
+5313|112|9|3|47|47569.17|0.06|0.08|N|O|1997-08-12|1997-08-18|1997-08-13|TAKE BACK RETURN|RAIL|pinto beans across the |
+5313|197|1|4|16|17555.04|0.08|0.00|N|O|1997-10-04|1997-08-02|1997-10-25|COLLECT COD|REG AIR|ckages wake carefully aga|
+5313|72|1|5|30|29162.10|0.06|0.08|N|O|1997-06-27|1997-07-18|1997-06-30|NONE|SHIP|nding packages use|
+5313|120|7|6|21|21422.52|0.05|0.05|N|O|1997-09-26|1997-09-02|1997-10-18|COLLECT COD|FOB|he blithely regular packages. quickly|
+5314|118|9|1|10|10181.10|0.07|0.05|N|O|1995-09-26|1995-07-24|1995-10-19|DELIVER IN PERSON|RAIL|latelets haggle final|
+5314|125|6|2|16|16401.92|0.00|0.04|N|O|1995-09-25|1995-07-08|1995-10-17|COLLECT COD|SHIP|hely unusual packages acc|
+5315|35|1|1|12|11220.36|0.08|0.06|R|F|1992-12-18|1993-01-16|1993-01-10|NONE|AIR|ccounts. furiously ironi|
+5315|179|10|2|39|42087.63|0.00|0.06|R|F|1992-11-09|1992-12-29|1992-12-07|NONE|SHIP|ly alongside of the ca|
+5316|108|1|1|29|29234.90|0.10|0.05|R|F|1994-03-28|1994-04-29|1994-04-09|DELIVER IN PERSON|REG AIR|ckly unusual foxes bo|
+5316|136|7|2|31|32120.03|0.00|0.08|A|F|1994-04-01|1994-04-21|1994-04-12|DELIVER IN PERSON|MAIL|s. deposits cajole around t|
+5317|82|3|1|29|28480.32|0.02|0.06|A|F|1994-11-28|1994-11-27|1994-12-16|COLLECT COD|FOB|oss the carefull|
+5317|171|2|2|18|19281.06|0.06|0.06|A|F|1995-01-02|1994-10-29|1995-01-16|NONE|RAIL|g to the blithely p|
+5317|120|4|3|37|37744.44|0.09|0.00|R|F|1994-09-15|1994-10-24|1994-09-23|NONE|TRUCK|totes nag theodolites. pend|
+5317|67|6|4|50|48353.00|0.09|0.01|A|F|1994-10-17|1994-10-25|1994-11-03|NONE|REG AIR|cajole furiously. accounts use quick|
+5317|95|8|5|19|18906.71|0.07|0.07|R|F|1994-12-15|1994-10-18|1994-12-27|NONE|MAIL|onic requests boost bli|
+5317|115|9|6|48|48725.28|0.01|0.03|A|F|1994-09-19|1994-11-25|1994-10-03|COLLECT COD|MAIL|ts about the packages cajole furio|
+5317|169|4|7|30|32074.80|0.07|0.07|A|F|1994-10-13|1994-10-31|1994-10-28|NONE|AIR|cross the attainments. slyly |
+5318|61|6|1|13|12493.78|0.10|0.04|R|F|1993-07-15|1993-06-25|1993-08-13|COLLECT COD|REG AIR|ly silent ideas. ideas haggle among the |
+5318|180|1|2|26|28084.68|0.00|0.04|R|F|1993-07-07|1993-05-23|1993-07-28|COLLECT COD|SHIP|al, express foxes. bold requests sleep alwa|
+5318|7|10|3|37|33559.00|0.07|0.05|A|F|1993-07-09|1993-06-22|1993-07-21|COLLECT COD|SHIP|ickly final deposi|
+5318|142|5|4|31|32306.34|0.01|0.04|R|F|1993-07-28|1993-05-06|1993-08-06|DELIVER IN PERSON|REG AIR|requests must sleep slyly quickly|
+5319|150|9|1|31|32554.65|0.04|0.07|N|O|1996-03-26|1996-03-07|1996-04-24|COLLECT COD|TRUCK|d carefully about the courts. fluffily spe|
+5319|44|3|2|39|36817.56|0.09|0.05|N|O|1996-05-17|1996-03-14|1996-06-11|NONE|TRUCK|unts. furiously silent|
+5344|19|3|1|6|5514.06|0.07|0.01|N|O|1998-08-04|1998-09-03|1998-08-11|TAKE BACK RETURN|REG AIR|ithely about the pending plate|
+5344|79|9|2|37|36225.59|0.03|0.07|N|O|1998-10-09|1998-07-26|1998-11-08|NONE|TRUCK|thely express packages|
+5344|67|8|3|26|25143.56|0.02|0.06|N|O|1998-08-27|1998-08-22|1998-09-24|NONE|AIR|furiously pending, silent multipliers.|
+5344|39|10|4|21|19719.63|0.03|0.01|N|O|1998-08-31|1998-09-06|1998-09-02|NONE|MAIL|xes. furiously even pinto beans sleep f|
+5345|83|4|1|3|2949.24|0.05|0.01|N|O|1997-12-10|1997-10-03|1998-01-05|COLLECT COD|SHIP|ites wake carefully unusual |
+5345|146|5|2|2|2092.28|0.10|0.02|N|O|1997-11-18|1997-10-12|1997-12-08|NONE|MAIL|ut the slyly specia|
+5345|192|5|3|46|50240.74|0.06|0.04|N|O|1997-10-06|1997-09-27|1997-10-18|COLLECT COD|REG AIR|slyly special deposits. fin|
+5345|114|4|4|37|37522.07|0.01|0.01|N|O|1997-11-01|1997-10-09|1997-11-26|DELIVER IN PERSON|AIR| along the ironically fina|
+5345|34|10|5|22|20548.66|0.02|0.02|N|O|1997-08-27|1997-11-22|1997-09-10|TAKE BACK RETURN|MAIL|leep slyly regular fox|
+5346|149|8|1|21|22031.94|0.07|0.08|R|F|1994-03-11|1994-03-07|1994-04-04|DELIVER IN PERSON|RAIL|integrate blithely a|
+5346|192|5|2|13|14198.47|0.04|0.04|A|F|1994-02-03|1994-02-05|1994-02-09|COLLECT COD|TRUCK|y. fluffily bold accounts grow. furio|
+5346|109|2|3|7|7063.70|0.08|0.05|A|F|1994-01-30|1994-03-26|1994-01-31|DELIVER IN PERSON|SHIP|equests use carefully care|
+5346|162|3|4|35|37175.60|0.06|0.02|A|F|1994-02-09|1994-03-01|1994-02-14|TAKE BACK RETURN|FOB|nic excuses cajole entic|
+5346|121|2|5|25|25528.00|0.05|0.06|R|F|1993-12-28|1994-03-19|1994-01-09|TAKE BACK RETURN|REG AIR|he ironic ideas are boldly slyly ironi|
+5346|33|9|6|6|5598.18|0.08|0.04|R|F|1994-03-01|1994-02-04|1994-03-09|NONE|REG AIR|escapades sleep furiously beside the |
+5346|80|9|7|41|40183.28|0.05|0.04|R|F|1994-01-10|1994-02-15|1994-01-26|TAKE BACK RETURN|REG AIR|fully close instructi|
+5347|83|4|1|48|47187.84|0.04|0.08|A|F|1995-02-25|1995-04-26|1995-03-26|NONE|SHIP|equests are slyly. blithely regu|
+5347|124|3|2|47|48133.64|0.02|0.01|N|F|1995-06-05|1995-03-29|1995-06-28|COLLECT COD|AIR|across the slyly bol|
+5347|23|2|3|34|31382.68|0.06|0.00|A|F|1995-05-18|1995-04-04|1995-06-02|DELIVER IN PERSON|SHIP| pending deposits. fluffily regular senti|
+5347|40|1|4|4|3760.16|0.06|0.03|A|F|1995-03-24|1995-04-03|1995-04-01|NONE|SHIP|ldly pending asymptotes ki|
+5347|131|2|5|21|21653.73|0.08|0.04|R|F|1995-04-01|1995-04-16|1995-04-23|NONE|SHIP|sly slyly final requests. careful|
+5347|56|1|6|6|5736.30|0.06|0.02|A|F|1995-04-11|1995-04-14|1995-05-02|NONE|TRUCK|lly unusual ideas. sl|
+5347|50|7|7|18|17100.90|0.01|0.01|N|F|1995-05-24|1995-05-07|1995-06-19|NONE|FOB|he ideas among the requests |
+5348|69|4|1|21|20350.26|0.10|0.04|N|O|1997-12-11|1997-12-24|1997-12-28|NONE|REG AIR| regular theodolites haggle car|
+5348|156|1|2|31|32740.65|0.07|0.02|N|O|1998-01-04|1997-12-09|1998-01-17|COLLECT COD|RAIL|are finally|
+5348|17|8|3|16|14672.16|0.06|0.08|N|O|1998-02-28|1997-12-25|1998-03-12|DELIVER IN PERSON|AIR|uriously thin pinto beans |
+5348|20|4|4|7|6440.14|0.04|0.00|N|O|1998-01-29|1997-12-20|1998-02-10|DELIVER IN PERSON|RAIL|even foxes. epitap|
+5348|2|5|5|37|33374.00|0.06|0.07|N|O|1997-12-01|1998-02-02|1997-12-07|NONE|FOB|y according to the carefully pending acco|
+5348|143|10|6|14|14603.96|0.06|0.05|N|O|1997-12-16|1998-01-12|1997-12-24|COLLECT COD|FOB|en pinto beans. somas cajo|
+5349|156|7|1|19|20066.85|0.06|0.01|N|O|1996-09-11|1996-11-18|1996-09-22|TAKE BACK RETURN|FOB|endencies use whithout the special |
+5349|168|3|2|14|14954.24|0.06|0.00|N|O|1996-11-07|1996-11-17|1996-11-20|TAKE BACK RETURN|TRUCK|fully regular |
+5349|4|5|3|6|5424.00|0.10|0.01|N|O|1996-12-30|1996-10-08|1997-01-01|DELIVER IN PERSON|MAIL|inal deposits affix carefully|
+5350|122|3|1|19|19420.28|0.02|0.06|R|F|1993-10-20|1993-11-15|1993-11-17|DELIVER IN PERSON|RAIL|romise slyly alongsi|
+5350|191|4|2|44|48012.36|0.04|0.06|R|F|1993-10-30|1993-11-23|1993-11-25|DELIVER IN PERSON|AIR|p above the ironic, pending dep|
+5350|54|9|3|12|11448.60|0.10|0.04|A|F|1994-01-30|1993-11-21|1994-02-15|COLLECT COD|REG AIR| cajole. even instructions haggle. blithe|
+5350|155|10|4|7|7386.05|0.08|0.00|R|F|1993-10-19|1993-12-28|1993-11-04|NONE|SHIP|alongside of th|
+5350|129|10|5|27|27786.24|0.07|0.04|A|F|1993-11-25|1993-12-27|1993-12-08|COLLECT COD|TRUCK|es. blithe theodolites haggl|
+5351|7|2|1|36|32652.00|0.06|0.05|N|O|1998-07-27|1998-07-06|1998-08-25|NONE|MAIL|ss the ironic, regular asymptotes cajole |
+5351|33|9|2|47|43852.41|0.04|0.01|N|O|1998-05-30|1998-08-08|1998-06-23|DELIVER IN PERSON|REG AIR|s. grouches cajole. sile|
+5351|106|3|3|2|2012.20|0.00|0.02|N|O|1998-05-12|1998-07-15|1998-05-24|NONE|TRUCK|g accounts wake furiously slyly even dolph|
+5376|61|6|1|42|40364.52|0.10|0.04|A|F|1994-09-20|1994-08-30|1994-09-29|TAKE BACK RETURN|REG AIR|y even asymptotes. courts are unusual pa|
+5376|91|4|2|44|43607.96|0.05|0.02|R|F|1994-08-30|1994-08-05|1994-09-07|COLLECT COD|AIR|ithe packages detect final theodolites. f|
+5376|65|6|3|18|17371.08|0.02|0.08|A|F|1994-10-29|1994-09-13|1994-11-01|COLLECT COD|MAIL| accounts boo|
+5377|79|8|1|40|39162.80|0.00|0.04|N|O|1997-05-21|1997-06-15|1997-05-26|DELIVER IN PERSON|AIR|lithely ironic theodolites are care|
+5377|30|3|2|17|15810.51|0.09|0.00|N|O|1997-07-05|1997-05-25|1997-07-22|COLLECT COD|RAIL|dencies. carefully regular re|
+5377|103|8|3|23|23071.30|0.07|0.08|N|O|1997-06-26|1997-07-13|1997-07-08|COLLECT COD|RAIL| silent wa|
+5377|104|7|4|12|12049.20|0.05|0.07|N|O|1997-05-08|1997-06-15|1997-05-15|DELIVER IN PERSON|MAIL| ironic, final|
+5377|173|3|5|27|28975.59|0.08|0.02|N|O|1997-07-11|1997-06-12|1997-08-08|TAKE BACK RETURN|MAIL|press theodolites. e|
+5378|155|3|1|39|41150.85|0.07|0.04|R|F|1992-11-25|1992-12-22|1992-12-02|COLLECT COD|AIR|ts are quickly around the|
+5378|62|9|2|46|44254.76|0.01|0.04|A|F|1993-02-17|1993-01-20|1993-02-26|COLLECT COD|REG AIR|into beans sleep. fu|
+5378|10|7|3|18|16380.18|0.02|0.03|R|F|1992-11-25|1992-12-21|1992-12-10|COLLECT COD|FOB|onic accounts was bold, |
+5379|199|1|1|40|43967.60|0.01|0.08|N|O|1995-10-01|1995-10-19|1995-10-30|COLLECT COD|MAIL|carefully final accounts haggle blithely. |
+5380|182|3|1|14|15150.52|0.10|0.01|N|O|1997-12-18|1997-12-03|1998-01-06|NONE|RAIL|final platelets.|
+5380|147|6|2|10|10471.40|0.09|0.05|N|O|1997-11-24|1998-01-10|1997-12-21|COLLECT COD|AIR|refully pending deposits. special, even t|
+5380|184|5|3|40|43367.20|0.02|0.08|N|O|1997-12-30|1997-11-27|1998-01-09|DELIVER IN PERSON|SHIP|ar asymptotes. blithely r|
+5380|66|3|4|6|5796.36|0.09|0.05|N|O|1997-11-15|1998-01-08|1997-12-11|COLLECT COD|MAIL|es. fluffily brave accounts across t|
+5380|107|8|5|48|48340.80|0.04|0.03|N|O|1997-12-01|1997-12-28|1997-12-05|DELIVER IN PERSON|FOB|encies haggle car|
+5381|188|9|1|37|40262.66|0.04|0.01|A|F|1993-04-08|1993-04-07|1993-04-12|DELIVER IN PERSON|SHIP|ly final deposits print carefully. unusua|
+5381|111|8|2|48|48533.28|0.04|0.03|R|F|1993-04-22|1993-04-17|1993-05-14|TAKE BACK RETURN|FOB|luffily spec|
+5381|192|3|3|13|14198.47|0.08|0.03|R|F|1993-05-09|1993-04-26|1993-05-25|NONE|FOB|s after the f|
+5381|168|3|4|17|18158.72|0.05|0.05|R|F|1993-05-25|1993-04-14|1993-06-17|NONE|MAIL|ckly final requests haggle qui|
+5381|63|8|5|49|47189.94|0.06|0.02|R|F|1993-05-08|1993-04-07|1993-06-03|NONE|FOB| accounts. regular, regula|
+5381|132|3|6|33|34060.29|0.10|0.00|A|F|1993-04-09|1993-04-03|1993-04-22|DELIVER IN PERSON|SHIP|ly special deposits |
+5381|44|3|7|31|29265.24|0.04|0.05|A|F|1993-04-10|1993-03-22|1993-04-13|TAKE BACK RETURN|MAIL|the carefully expre|
+5382|153|8|1|34|35807.10|0.03|0.03|R|F|1992-02-22|1992-02-18|1992-03-02|DELIVER IN PERSON|FOB|gular accounts. even accounts integrate|
+5382|55|3|2|13|12415.65|0.09|0.06|A|F|1992-01-16|1992-03-12|1992-02-06|NONE|MAIL|eodolites. final foxes |
+5382|149|10|3|3|3147.42|0.10|0.06|A|F|1992-03-22|1992-03-06|1992-04-19|TAKE BACK RETURN|AIR|efully unusua|
+5382|62|9|4|20|19241.20|0.08|0.02|A|F|1992-03-26|1992-02-17|1992-04-15|DELIVER IN PERSON|FOB|carefully regular accounts. slyly ev|
+5382|177|8|5|14|15080.38|0.02|0.02|A|F|1992-04-05|1992-04-05|1992-05-04|TAKE BACK RETURN|FOB| brave platelets. ev|
+5382|180|9|6|6|6481.08|0.02|0.01|A|F|1992-03-07|1992-04-02|1992-03-18|TAKE BACK RETURN|FOB|y final foxes by the sl|
+5382|105|2|7|48|48244.80|0.05|0.05|A|F|1992-02-14|1992-03-19|1992-02-25|DELIVER IN PERSON|REG AIR|nts integrate quickly ca|
+5383|96|7|1|12|11953.08|0.04|0.00|N|O|1995-07-02|1995-08-16|1995-08-01|TAKE BACK RETURN|AIR|y regular instructi|
+5408|102|7|1|2|2004.20|0.07|0.04|R|F|1992-08-21|1992-10-03|1992-08-28|DELIVER IN PERSON|MAIL|cross the dolphins h|
+5408|118|2|2|35|35633.85|0.04|0.05|R|F|1992-10-02|1992-10-17|1992-10-13|TAKE BACK RETURN|AIR|thely ironic requests alongside of the sl|
+5408|76|6|3|34|33186.38|0.10|0.02|A|F|1992-10-22|1992-08-25|1992-11-16|DELIVER IN PERSON|TRUCK|requests detect blithely a|
+5408|54|2|4|48|45794.40|0.04|0.05|R|F|1992-09-30|1992-08-27|1992-10-27|NONE|TRUCK|. furiously regular |
+5408|183|4|5|8|8665.44|0.03|0.07|A|F|1992-10-24|1992-09-06|1992-11-03|NONE|AIR|thely regular hocke|
+5409|194|8|1|27|29543.13|0.01|0.02|A|F|1992-02-14|1992-03-18|1992-02-23|DELIVER IN PERSON|AIR|eodolites |
+5409|104|5|2|38|38155.80|0.01|0.02|A|F|1992-03-17|1992-03-29|1992-04-13|NONE|REG AIR|onic, regular accounts! blithely even|
+5409|141|10|3|17|17699.38|0.07|0.00|A|F|1992-01-13|1992-04-05|1992-01-20|DELIVER IN PERSON|AIR|cross the sil|
+5409|1|8|4|9|8109.00|0.07|0.03|A|F|1992-02-15|1992-04-02|1992-02-28|DELIVER IN PERSON|AIR| unusual, unusual reques|
+5409|159|10|5|37|39188.55|0.06|0.04|R|F|1992-05-07|1992-02-10|1992-05-20|DELIVER IN PERSON|FOB|ously regular packages. packages|
+5409|64|3|6|14|13496.84|0.03|0.08|R|F|1992-02-14|1992-03-26|1992-02-29|DELIVER IN PERSON|AIR|osits cajole furiously|
+5410|117|8|1|48|48821.28|0.04|0.08|N|O|1998-09-27|1998-09-11|1998-10-01|TAKE BACK RETURN|AIR| about the slyly even courts. quickly regul|
+5410|105|8|2|41|41209.10|0.01|0.07|N|O|1998-08-25|1998-10-20|1998-09-01|DELIVER IN PERSON|REG AIR|sly. slyly ironic theodolites|
+5410|29|4|3|40|37160.80|0.07|0.08|N|O|1998-11-17|1998-10-02|1998-11-27|COLLECT COD|TRUCK|iously special accounts are along th|
+5410|50|7|4|8|7600.40|0.05|0.04|N|O|1998-09-12|1998-10-22|1998-09-22|DELIVER IN PERSON|TRUCK|ly. fluffily ironic platelets alon|
+5411|96|9|1|17|16933.53|0.05|0.01|N|O|1997-07-22|1997-07-14|1997-07-30|TAKE BACK RETURN|REG AIR| slyly slyly even deposits. carefully b|
+5411|113|7|2|10|10131.10|0.08|0.01|N|O|1997-07-19|1997-08-04|1997-07-26|TAKE BACK RETURN|MAIL|nding, special foxes unw|
+5411|56|7|3|5|4780.25|0.10|0.01|N|O|1997-09-12|1997-08-03|1997-09-23|DELIVER IN PERSON|FOB| bold, ironic theodo|
+5411|129|8|4|15|15436.80|0.08|0.05|N|O|1997-07-01|1997-07-15|1997-07-07|COLLECT COD|RAIL|attainments sleep slyly ironic|
+5411|4|5|5|19|17176.00|0.05|0.08|N|O|1997-05-25|1997-07-30|1997-06-19|COLLECT COD|RAIL|ial accounts according to the f|
+5412|54|9|1|2|1908.10|0.03|0.07|N|O|1998-04-14|1998-04-02|1998-04-19|TAKE BACK RETURN|REG AIR| sleep above the furiou|
+5412|66|1|2|48|46370.88|0.01|0.08|N|O|1998-02-22|1998-03-28|1998-03-18|TAKE BACK RETURN|TRUCK|s. slyly final packages cajole blithe|
+5412|74|2|3|31|30196.17|0.05|0.08|N|O|1998-03-23|1998-04-17|1998-04-10|NONE|SHIP|t the accounts detect slyly about the c|
+5412|97|10|4|26|25924.34|0.02|0.08|N|O|1998-01-22|1998-04-19|1998-02-17|NONE|AIR| the blithel|
+5413|126|7|1|48|49253.76|0.02|0.08|N|O|1998-01-25|1997-11-20|1998-02-22|COLLECT COD|SHIP| theodolites. furiously ironic instr|
+5413|142|9|2|37|38559.18|0.02|0.07|N|O|1997-12-08|1998-01-01|1997-12-13|COLLECT COD|TRUCK|usly bold instructions affix idly unusual, |
+5413|111|8|3|36|36399.96|0.02|0.07|N|O|1997-12-12|1997-11-28|1997-12-25|NONE|TRUCK|ular, regular ideas mold! final requests|
+5413|110|3|4|22|22222.42|0.02|0.08|N|O|1997-11-10|1997-11-24|1997-11-22|DELIVER IN PERSON|FOB|posits. quick|
+5413|189|10|5|5|5445.90|0.10|0.01|N|O|1997-11-28|1997-11-24|1997-12-05|NONE|RAIL|tes are al|
+5413|190|1|6|32|34886.08|0.02|0.03|N|O|1997-10-28|1998-01-03|1997-11-10|NONE|TRUCK|refully special package|
+5413|31|7|7|32|29792.96|0.06|0.07|N|O|1997-10-23|1997-12-09|1997-11-17|NONE|TRUCK|he quickly ironic ideas. slyly ironic ide|
+5414|68|9|1|40|38722.40|0.07|0.06|R|F|1993-04-07|1993-05-18|1993-04-23|COLLECT COD|AIR|ts are evenly across|
+5414|123|8|2|48|49109.76|0.06|0.07|R|F|1993-06-08|1993-05-14|1993-07-06|DELIVER IN PERSON|FOB| silent dolphins; fluffily regular tithe|
+5414|35|1|3|23|21505.69|0.10|0.00|A|F|1993-07-22|1993-05-26|1993-08-08|COLLECT COD|MAIL|e bold, express dolphins. spec|
+5414|133|4|4|15|15496.95|0.06|0.08|R|F|1993-05-18|1993-06-09|1993-05-27|DELIVER IN PERSON|REG AIR|e slyly about the carefully regula|
+5414|9|2|5|19|17271.00|0.01|0.05|R|F|1993-04-06|1993-05-12|1993-05-02|DELIVER IN PERSON|RAIL|ffily silent theodolites na|
+5414|98|1|6|28|27946.52|0.10|0.05|A|F|1993-03-27|1993-06-04|1993-04-07|TAKE BACK RETURN|SHIP|ts sleep sl|
+5415|102|5|1|44|44092.40|0.00|0.06|A|F|1992-08-19|1992-10-26|1992-09-17|TAKE BACK RETURN|TRUCK| requests. unusual theodolites sleep agains|
+5415|31|7|2|16|14896.48|0.08|0.00|A|F|1992-09-29|1992-09-12|1992-10-10|NONE|AIR|pinto beans haggle furiously|
+5415|102|7|3|6|6012.60|0.10|0.03|A|F|1992-10-28|1992-09-09|1992-11-20|COLLECT COD|RAIL|ges around the fur|
+5415|16|7|4|43|39388.43|0.01|0.02|R|F|1992-11-17|1992-09-14|1992-12-14|DELIVER IN PERSON|SHIP|yly blithely stealthy deposits. carefu|
+5415|161|6|5|11|11672.76|0.00|0.01|R|F|1992-11-22|1992-10-19|1992-12-10|DELIVER IN PERSON|SHIP|gle among t|
+5415|144|1|6|46|48030.44|0.03|0.03|R|F|1992-08-25|1992-09-10|1992-09-22|DELIVER IN PERSON|REG AIR|ve the fluffily |
+5415|153|4|7|11|11584.65|0.08|0.06|A|F|1992-08-21|1992-09-04|1992-08-23|NONE|TRUCK|unts maintain carefully unusual|
+5440|115|2|1|3|3045.33|0.02|0.08|N|O|1997-02-18|1997-02-28|1997-03-15|NONE|SHIP|y. accounts haggle along the blit|
+5441|164|1|1|3|3192.48|0.00|0.02|R|F|1994-08-12|1994-10-14|1994-09-01|TAKE BACK RETURN|REG AIR|are. unusual, |
+5441|131|2|2|49|50525.37|0.02|0.03|A|F|1994-09-23|1994-09-22|1994-10-22|NONE|FOB|ording to the furio|
+5441|144|3|3|33|34456.62|0.09|0.02|R|F|1994-10-09|1994-10-06|1994-10-30|DELIVER IN PERSON|TRUCK|ges. final instruction|
+5441|67|4|4|47|45451.82|0.07|0.08|R|F|1994-11-19|1994-10-16|1994-12-16|TAKE BACK RETURN|FOB|ounts wake slyly about the express instr|
+5442|42|5|1|16|15072.64|0.00|0.00|N|O|1998-04-12|1998-03-03|1998-05-04|TAKE BACK RETURN|RAIL|r packages. accounts haggle dependencies. f|
+5442|88|9|2|45|44463.60|0.08|0.01|N|O|1998-03-30|1998-02-24|1998-04-18|TAKE BACK RETURN|AIR|old slyly after |
+5442|61|8|3|12|11532.72|0.01|0.08|N|O|1998-04-15|1998-03-18|1998-05-05|DELIVER IN PERSON|TRUCK|fully final|
+5442|158|9|4|21|22221.15|0.07|0.06|N|O|1998-03-13|1998-02-19|1998-04-06|COLLECT COD|MAIL|ffily furiously ironic theodolites. furio|
+5442|16|7|5|25|22900.25|0.04|0.00|N|O|1998-03-29|1998-02-13|1998-04-13|TAKE BACK RETURN|REG AIR|ake furiously. slyly express th|
+5442|144|3|6|26|27147.64|0.08|0.07|N|O|1998-03-21|1998-03-21|1998-03-25|TAKE BACK RETURN|AIR|have to sleep furiously bold ideas. blith|
+5443|178|9|1|14|15094.38|0.02|0.00|N|O|1996-10-27|1996-11-11|1996-11-21|DELIVER IN PERSON|RAIL|s after the regular, regular deposits hag|
+5443|72|3|2|39|37910.73|0.03|0.07|N|O|1996-11-01|1996-11-30|1996-11-19|NONE|RAIL|gage carefully across the furiously|
+5443|160|5|3|25|26504.00|0.05|0.00|N|O|1996-12-07|1997-01-08|1997-01-05|NONE|FOB|use carefully above the pinto bea|
+5443|191|4|4|6|6547.14|0.05|0.02|N|O|1996-11-17|1996-12-03|1996-11-30|TAKE BACK RETURN|AIR|p fluffily foxe|
+5443|83|4|5|40|39323.20|0.03|0.03|N|O|1997-01-28|1996-12-10|1997-02-13|NONE|FOB|n courts. special re|
+5444|186|7|1|21|22809.78|0.01|0.07|A|F|1995-04-11|1995-04-25|1995-04-21|DELIVER IN PERSON|RAIL|ar packages haggle above th|
+5444|43|6|2|40|37721.60|0.05|0.08|N|O|1995-07-09|1995-04-25|1995-07-19|COLLECT COD|TRUCK|ously bold ideas. instructions wake slyl|
+5444|150|9|3|40|42006.00|0.08|0.01|A|F|1995-04-06|1995-05-08|1995-05-06|DELIVER IN PERSON|AIR| even packages.|
+5444|59|4|4|33|31648.65|0.05|0.04|N|O|1995-06-24|1995-04-24|1995-07-13|DELIVER IN PERSON|SHIP|ut the courts cajole blithely excuses|
+5444|171|9|5|21|22494.57|0.04|0.00|R|F|1995-05-05|1995-05-25|1995-05-29|TAKE BACK RETURN|REG AIR|aves serve sly|
+5444|20|7|6|21|19320.42|0.07|0.01|A|F|1995-03-30|1995-05-01|1995-03-31|COLLECT COD|AIR|furiously even theodolites.|
+5445|90|1|1|33|32672.97|0.08|0.07|A|F|1993-10-21|1993-10-14|1993-10-28|DELIVER IN PERSON|REG AIR|ests. final instructions|
+5445|131|2|2|12|12373.56|0.09|0.08|R|F|1993-11-02|1993-09-05|1993-11-26|COLLECT COD|FOB| slyly pending pinto beans was slyly al|
+5445|103|8|3|46|46142.60|0.04|0.07|A|F|1993-10-06|1993-09-15|1993-10-28|DELIVER IN PERSON|RAIL|old depend|
+5445|149|10|4|10|10491.40|0.08|0.06|A|F|1993-09-16|1993-10-05|1993-10-01|NONE|TRUCK|ncies abou|
+5445|13|10|5|14|12782.14|0.00|0.02|R|F|1993-11-19|1993-10-18|1993-12-07|NONE|RAIL| requests. bravely i|
+5446|190|1|1|27|29435.13|0.05|0.07|R|F|1994-07-21|1994-08-25|1994-08-17|TAKE BACK RETURN|RAIL|ously across the quic|
+5447|99|1|1|31|30971.79|0.09|0.03|N|O|1996-07-14|1996-05-07|1996-07-17|COLLECT COD|SHIP| foxes sleep. blithely unusual accounts det|
+5472|59|10|1|27|25894.35|0.09|0.06|A|F|1993-08-04|1993-07-07|1993-09-03|COLLECT COD|TRUCK|fily pending attainments. unus|
+5472|68|3|2|28|27105.68|0.00|0.03|A|F|1993-07-28|1993-05-28|1993-08-11|TAKE BACK RETURN|FOB|ffily pendin|
+5472|178|7|3|45|48517.65|0.06|0.02|R|F|1993-06-05|1993-05-14|1993-06-10|NONE|TRUCK| idle packages. furi|
+5472|184|5|4|37|40114.66|0.07|0.05|R|F|1993-06-15|1993-07-03|1993-07-09|DELIVER IN PERSON|RAIL|egrate carefully dependencies. |
+5472|75|6|5|40|39002.80|0.02|0.05|A|F|1993-04-13|1993-07-04|1993-05-04|NONE|REG AIR|e requests detect furiously. ruthlessly un|
+5472|167|2|6|39|41619.24|0.02|0.03|R|F|1993-04-18|1993-07-10|1993-05-12|TAKE BACK RETURN|MAIL|uriously carefully |
+5472|15|5|7|1|915.01|0.03|0.02|A|F|1993-04-14|1993-06-28|1993-04-16|NONE|RAIL|s use furiou|
+5473|48|5|1|9|8532.36|0.03|0.07|R|F|1992-06-03|1992-05-30|1992-06-09|TAKE BACK RETURN|AIR| excuses sleep blithely! regular dep|
+5473|70|1|2|27|26191.89|0.01|0.03|A|F|1992-04-06|1992-04-26|1992-04-29|TAKE BACK RETURN|MAIL|the deposits. warthogs wake fur|
+5473|15|5|3|33|30195.33|0.09|0.00|R|F|1992-05-18|1992-06-10|1992-06-13|TAKE BACK RETURN|MAIL|efully above the even, |
+5474|184|5|1|38|41198.84|0.01|0.08|A|F|1992-07-15|1992-07-16|1992-07-20|NONE|REG AIR| slyly beneath |
+5474|94|8|2|10|9940.90|0.06|0.00|R|F|1992-08-08|1992-08-10|1992-08-24|TAKE BACK RETURN|TRUCK|pinto bean|
+5474|48|1|3|31|29389.24|0.00|0.08|R|F|1992-08-02|1992-07-12|1992-08-04|NONE|TRUCK|the furiously express ideas. speci|
+5474|90|1|4|46|45544.14|0.03|0.04|A|F|1992-06-07|1992-07-11|1992-06-22|NONE|TRUCK|nstructions. furio|
+5475|183|4|1|10|10831.80|0.09|0.08|N|O|1996-07-19|1996-08-22|1996-07-23|COLLECT COD|AIR|ding to the deposits wake fina|
+5476|48|1|1|13|12324.52|0.01|0.04|N|O|1997-12-27|1997-12-08|1997-12-29|COLLECT COD|TRUCK|iously special ac|
+5476|20|4|2|17|15640.34|0.10|0.01|N|O|1998-02-02|1998-01-28|1998-02-14|COLLECT COD|FOB|ng dependencies until the f|
+5477|80|8|1|20|19601.60|0.03|0.01|N|O|1998-03-21|1998-02-09|1998-04-07|TAKE BACK RETURN|SHIP|platelets about the ironic|
+5477|77|7|2|21|20518.47|0.03|0.00|N|O|1998-01-28|1998-02-15|1998-02-24|TAKE BACK RETURN|SHIP|blate slyly. silent|
+5477|134|5|3|31|32058.03|0.04|0.01|N|O|1998-01-11|1998-01-30|1998-02-04|DELIVER IN PERSON|MAIL| special Tiresias cajole furiously. pending|
+5477|193|6|4|16|17491.04|0.00|0.01|N|O|1998-03-07|1998-03-12|1998-04-06|COLLECT COD|RAIL|regular, s|
+5477|96|9|5|23|22910.07|0.00|0.06|N|O|1998-01-04|1998-02-23|1998-01-24|NONE|REG AIR|telets wake blithely ab|
+5477|121|6|6|19|19401.28|0.10|0.03|N|O|1998-02-03|1998-01-30|1998-03-04|TAKE BACK RETURN|MAIL|ost carefully packages.|
+5478|8|9|1|39|35412.00|0.09|0.06|N|O|1996-08-19|1996-06-25|1996-09-08|DELIVER IN PERSON|SHIP|s. furiously |
+5478|2|5|2|47|42394.00|0.10|0.01|N|O|1996-08-15|1996-07-12|1996-08-31|NONE|RAIL| instructions; slyly even accounts hagg|
+5478|119|3|3|25|25477.75|0.09|0.07|N|O|1996-06-08|1996-07-12|1996-07-07|NONE|TRUCK|unusual, pending requests haggle accoun|
+5479|138|4|1|50|51906.50|0.02|0.02|A|F|1993-12-24|1994-02-14|1994-01-18|DELIVER IN PERSON|MAIL|ironic gifts. even dependencies sno|
+5479|104|5|2|19|19077.90|0.05|0.03|A|F|1994-01-22|1994-03-07|1994-02-11|DELIVER IN PERSON|SHIP|arefully bo|
+5504|68|5|1|4|3872.24|0.10|0.07|A|F|1993-04-30|1993-03-01|1993-05-22|DELIVER IN PERSON|AIR|into beans boost. |
+5504|177|8|2|7|7540.19|0.03|0.05|R|F|1993-04-25|1993-03-15|1993-05-06|NONE|TRUCK|packages detect furiously express reques|
+5504|160|2|3|29|30744.64|0.05|0.03|A|F|1993-01-28|1993-02-13|1993-02-27|NONE|SHIP|ajole carefully. care|
+5505|25|8|1|43|39775.86|0.07|0.01|N|O|1997-12-30|1997-11-28|1998-01-09|TAKE BACK RETURN|TRUCK|y alongside of the special requests.|
+5505|182|3|2|33|35711.94|0.05|0.08|N|O|1998-01-11|1997-11-11|1998-01-30|TAKE BACK RETURN|AIR|ithely unusual excuses integrat|
+5505|155|10|3|10|10551.50|0.06|0.01|N|O|1997-10-28|1997-11-27|1997-10-29|DELIVER IN PERSON|AIR| furiously special asym|
+5505|40|1|4|18|16920.72|0.04|0.04|N|O|1997-10-25|1997-12-12|1997-10-30|TAKE BACK RETURN|RAIL| to the quickly express pac|
+5505|162|9|5|46|48859.36|0.05|0.00|N|O|1998-01-06|1997-11-04|1998-02-04|TAKE BACK RETURN|SHIP|usly ironic dependencies haggle across |
+5506|140|1|1|2|2080.28|0.00|0.03|R|F|1994-02-04|1994-01-13|1994-02-17|COLLECT COD|MAIL|onic theodolites are fluffil|
+5506|160|1|2|6|6360.96|0.07|0.06|R|F|1994-02-21|1994-01-30|1994-02-27|DELIVER IN PERSON|MAIL|hely according to the furiously unusua|
+5507|10|5|1|23|20930.23|0.05|0.04|N|O|1998-09-04|1998-07-04|1998-09-18|TAKE BACK RETURN|AIR|ously slow packages poach whithout the|
+5507|138|9|2|48|49830.24|0.03|0.01|N|O|1998-08-03|1998-08-10|1998-08-24|DELIVER IN PERSON|AIR|yly idle deposits. final, final fox|
+5507|45|2|3|4|3780.16|0.04|0.06|N|O|1998-06-06|1998-07-02|1998-06-27|TAKE BACK RETURN|RAIL|into beans are|
+5507|67|6|4|22|21275.32|0.07|0.01|N|O|1998-07-08|1998-08-10|1998-07-22|DELIVER IN PERSON|TRUCK|gular ideas. carefully unu|
+5507|132|3|5|48|49542.24|0.06|0.01|N|O|1998-07-21|1998-07-15|1998-07-31|DELIVER IN PERSON|SHIP|uriously regular acc|
+5508|117|7|1|4|4068.44|0.10|0.04|N|O|1996-09-01|1996-08-02|1996-09-17|COLLECT COD|AIR|fluffily about the even |
+5509|197|10|1|3|3291.57|0.03|0.02|A|F|1994-06-14|1994-05-11|1994-06-17|NONE|SHIP| quickly fin|
+5509|99|3|2|17|16984.53|0.03|0.07|R|F|1994-07-01|1994-06-30|1994-07-31|COLLECT COD|AIR|ccounts wake ar|
+5509|93|7|3|30|29792.70|0.04|0.04|A|F|1994-07-23|1994-06-01|1994-08-08|NONE|AIR|counts haggle pinto beans. furiously |
+5509|100|3|4|45|45004.50|0.00|0.07|A|F|1994-07-24|1994-05-28|1994-08-20|COLLECT COD|AIR|counts sleep. f|
+5509|156|8|5|35|36965.25|0.04|0.03|A|F|1994-04-17|1994-06-29|1994-04-24|COLLECT COD|RAIL|c accounts. ca|
+5510|16|6|1|8|7328.08|0.01|0.01|A|F|1993-03-16|1993-03-29|1993-03-24|DELIVER IN PERSON|FOB|n packages boost sly|
+5510|20|10|2|46|42320.92|0.02|0.07|A|F|1993-03-12|1993-02-09|1993-03-19|NONE|TRUCK|silent packages cajole doggedly regular |
+5510|162|3|3|47|49921.52|0.03|0.01|A|F|1993-01-20|1993-03-25|1993-02-15|DELIVER IN PERSON|SHIP|riously even requests. slyly bold accou|
+5510|24|7|4|29|26796.58|0.09|0.08|A|F|1993-02-28|1993-03-28|1993-03-12|COLLECT COD|AIR|lithely fluffily ironic req|
+5511|165|4|1|16|17042.56|0.10|0.05|A|F|1995-02-02|1995-01-06|1995-02-19|TAKE BACK RETURN|RAIL|thely bold theodolites |
+5511|165|10|2|31|33019.96|0.09|0.01|A|F|1995-02-23|1995-01-21|1995-03-02|COLLECT COD|REG AIR|gular excuses. fluffily even pinto beans c|
+5511|128|3|3|49|50377.88|0.05|0.05|R|F|1994-12-21|1995-01-27|1994-12-26|NONE|REG AIR|bout the requests. theodolites |
+5511|122|7|4|4|4088.48|0.08|0.02|R|F|1994-12-28|1995-01-16|1995-01-24|TAKE BACK RETURN|RAIL|lphins. carefully blithe de|
+5511|9|2|5|23|20907.00|0.10|0.07|A|F|1995-03-11|1995-01-21|1995-03-27|TAKE BACK RETURN|TRUCK|ing dugouts |
+5511|188|9|6|5|5440.90|0.08|0.05|R|F|1994-12-29|1995-01-16|1995-01-24|DELIVER IN PERSON|MAIL|al theodolites. blithely final de|
+5511|143|2|7|23|23992.22|0.02|0.07|R|F|1995-02-03|1995-01-05|1995-02-18|COLLECT COD|REG AIR|ully deposits. warthogs hagg|
+5536|90|1|1|14|13861.26|0.08|0.06|N|O|1998-05-18|1998-05-08|1998-06-05|COLLECT COD|MAIL|instructions sleep |
+5536|62|1|2|20|19241.20|0.08|0.04|N|O|1998-05-08|1998-05-10|1998-05-31|DELIVER IN PERSON|REG AIR|equests mo|
+5536|197|9|3|35|38401.65|0.07|0.02|N|O|1998-05-19|1998-06-08|1998-06-05|NONE|MAIL|c, final theo|
+5536|9|10|4|30|27270.00|0.05|0.07|N|O|1998-04-15|1998-05-23|1998-05-03|NONE|FOB|arefully regular theodolites according|
+5536|141|2|5|11|11452.54|0.02|0.08|N|O|1998-03-18|1998-05-12|1998-03-28|TAKE BACK RETURN|FOB| snooze furio|
+5537|45|8|1|10|9450.40|0.05|0.08|N|O|1997-01-13|1996-12-25|1997-01-28|TAKE BACK RETURN|AIR| sleep carefully slyly bold depos|
+5537|150|9|2|15|15752.25|0.07|0.04|N|O|1997-01-13|1996-12-25|1997-01-27|COLLECT COD|AIR|eposits. permanently pending packag|
+5537|151|6|3|39|40994.85|0.03|0.00|N|O|1996-12-17|1996-11-08|1997-01-15|COLLECT COD|REG AIR| slyly bold packages are. qu|
+5537|97|1|4|38|37889.42|0.01|0.00|N|O|1996-11-06|1996-11-23|1996-11-12|TAKE BACK RETURN|MAIL|s above the carefully ironic deposits |
+5538|154|9|1|42|44274.30|0.05|0.00|A|F|1994-04-08|1994-03-17|1994-05-05|DELIVER IN PERSON|REG AIR|vely ironic accounts. furiously unusual acc|
+5538|121|2|2|4|4084.48|0.02|0.03|R|F|1994-03-21|1994-02-17|1994-04-11|TAKE BACK RETURN|REG AIR|ithely along the c|
+5538|19|3|3|38|34922.38|0.03|0.06|R|F|1994-03-17|1994-02-11|1994-04-10|TAKE BACK RETURN|FOB|ular pinto beans. silent ideas above |
+5538|78|6|4|9|8802.63|0.00|0.01|R|F|1993-12-26|1994-01-31|1994-01-03|TAKE BACK RETURN|REG AIR|encies across the blithely fina|
+5539|65|10|1|42|40532.52|0.10|0.08|A|F|1994-09-29|1994-09-17|1994-10-20|DELIVER IN PERSON|RAIL|ons across the carefully si|
+5540|181|2|1|42|45409.56|0.02|0.08|N|O|1996-11-12|1996-12-18|1996-12-05|TAKE BACK RETURN|RAIL|ss dolphins haggle |
+5540|102|3|2|2|2004.20|0.06|0.02|N|O|1996-12-12|1997-01-09|1996-12-25|DELIVER IN PERSON|MAIL|nic asymptotes could hav|
+5540|64|3|3|19|18317.14|0.01|0.03|N|O|1997-02-06|1996-11-18|1997-02-20|DELIVER IN PERSON|SHIP| slyly slyl|
+5540|72|10|4|24|23329.68|0.10|0.05|N|O|1997-01-09|1996-12-02|1997-01-23|COLLECT COD|FOB|deposits! ironic depths may engage-- b|
+5541|96|8|1|39|38847.51|0.08|0.05|N|O|1997-11-17|1997-12-27|1997-12-11|TAKE BACK RETURN|RAIL|ding theodolites haggle against the slyly |
+5542|189|10|1|6|6535.08|0.03|0.01|N|O|1996-06-14|1996-05-28|1996-07-11|DELIVER IN PERSON|TRUCK| foxes doubt. theodolites ca|
+5543|143|10|1|14|14603.96|0.02|0.03|R|F|1993-10-09|1993-12-09|1993-10-21|NONE|SHIP|ecial reque|
+5543|162|7|2|22|23367.52|0.04|0.00|A|F|1993-11-06|1993-11-02|1993-12-02|DELIVER IN PERSON|SHIP|instructions. deposits use quickly. ir|
+5543|67|6|3|3|2901.18|0.08|0.05|R|F|1993-11-18|1993-11-05|1993-12-17|NONE|FOB|ress, even |
+5543|147|10|4|8|8377.12|0.05|0.01|R|F|1993-10-28|1993-11-18|1993-11-07|NONE|SHIP|totes? iron|
+5543|80|1|5|32|31362.56|0.03|0.03|R|F|1993-10-04|1993-11-14|1993-11-03|DELIVER IN PERSON|AIR|ully around the |
+5543|184|5|6|1|1084.18|0.03|0.07|A|F|1993-10-29|1993-11-11|1993-11-23|TAKE BACK RETURN|FOB|uriously. slyly|
+5543|129|8|7|39|40135.68|0.06|0.00|R|F|1993-10-07|1993-11-15|1993-10-28|TAKE BACK RETURN|MAIL|l excuses are furiously. slyly unusual requ|
+5568|166|5|1|50|53308.00|0.05|0.05|N|O|1995-07-14|1995-09-04|1995-08-03|COLLECT COD|TRUCK|furious ide|
+5568|44|5|2|18|16992.72|0.01|0.08|N|O|1995-08-19|1995-08-18|1995-08-24|DELIVER IN PERSON|SHIP|structions haggle. carefully regular |
+5568|89|10|3|35|34617.80|0.08|0.07|N|O|1995-09-17|1995-09-04|1995-10-14|NONE|SHIP|lyly. blit|
+5569|29|4|1|25|23225.50|0.10|0.03|R|F|1993-06-29|1993-07-18|1993-07-05|TAKE BACK RETURN|TRUCK| deposits cajole above|
+5569|58|10|2|26|24909.30|0.09|0.06|A|F|1993-08-21|1993-07-22|1993-09-09|DELIVER IN PERSON|MAIL|pitaphs. ironic req|
+5569|55|3|3|48|45842.40|0.02|0.03|R|F|1993-06-16|1993-06-15|1993-07-09|COLLECT COD|SHIP|the fluffily|
+5569|147|10|4|19|19895.66|0.10|0.08|R|F|1993-07-30|1993-06-21|1993-08-13|TAKE BACK RETURN|FOB| detect ca|
+5569|59|1|5|15|14385.75|0.02|0.06|A|F|1993-06-29|1993-07-06|1993-07-05|DELIVER IN PERSON|MAIL|lithely bold requests boost fur|
+5570|161|6|1|37|39262.92|0.08|0.02|N|O|1996-08-29|1996-10-23|1996-09-11|NONE|RAIL|y ironic pin|
+5570|39|10|2|15|14085.45|0.09|0.02|N|O|1996-10-04|1996-10-05|1996-10-28|TAKE BACK RETURN|REG AIR|beans nag slyly special, regular pack|
+5570|60|1|3|29|27841.74|0.02|0.05|N|O|1996-10-12|1996-10-20|1996-11-08|TAKE BACK RETURN|SHIP|he silent, enticing requests.|
+5571|154|2|1|32|33732.80|0.05|0.01|R|F|1992-12-25|1993-03-01|1993-01-23|NONE|FOB| the blithely even packages nag q|
+5571|94|8|2|31|30816.79|0.09|0.07|R|F|1993-01-05|1993-01-18|1993-02-04|DELIVER IN PERSON|SHIP|uffily even accounts. quickly re|
+5571|92|6|3|18|17857.62|0.10|0.05|R|F|1993-03-11|1993-02-28|1993-04-03|COLLECT COD|REG AIR|uests haggle furiously pending d|
+5572|22|1|1|24|22128.48|0.08|0.08|R|F|1994-10-30|1994-10-02|1994-11-27|TAKE BACK RETURN|MAIL|ests cajole. evenly ironic exc|
+5572|172|10|2|27|28948.59|0.03|0.04|A|F|1994-08-29|1994-09-10|1994-08-30|TAKE BACK RETURN|SHIP| accounts. carefully final accoun|
+5572|87|8|3|19|18754.52|0.10|0.00|A|F|1994-08-12|1994-10-07|1994-09-01|DELIVER IN PERSON|RAIL|es. final, final requests wake blithely ag|
+5572|135|1|4|46|47615.98|0.02|0.01|R|F|1994-09-08|1994-10-14|1994-10-01|NONE|REG AIR|ully regular platelet|
+5572|24|3|5|34|31416.68|0.10|0.08|R|F|1994-10-22|1994-08-16|1994-11-08|NONE|TRUCK|asymptotes integrate. s|
+5572|101|2|6|14|14015.40|0.04|0.05|A|F|1994-11-02|1994-09-20|1994-11-03|COLLECT COD|RAIL|he fluffily express packages. fluffily fina|
+5572|26|1|7|24|22224.48|0.01|0.05|R|F|1994-09-26|1994-09-04|1994-10-22|DELIVER IN PERSON|FOB| beans. foxes sleep fluffily across th|
+5573|21|6|1|32|29472.64|0.05|0.07|N|O|1996-09-30|1996-10-25|1996-10-15|DELIVER IN PERSON|RAIL|egular depths haggl|
+5573|50|3|2|2|1900.10|0.01|0.07|N|O|1996-08-26|1996-09-29|1996-09-04|COLLECT COD|TRUCK| even foxes. specia|
+5573|11|8|3|46|41906.46|0.06|0.01|N|O|1996-11-04|1996-10-02|1996-11-15|DELIVER IN PERSON|MAIL|s haggle qu|
+5573|169|4|4|43|45973.88|0.10|0.03|N|O|1996-10-22|1996-11-03|1996-11-02|TAKE BACK RETURN|FOB| furiously pending packages against |
+5573|138|9|5|43|44639.59|0.05|0.04|N|O|1996-09-09|1996-09-24|1996-09-28|COLLECT COD|AIR| bold package|
+5574|185|6|1|46|49918.28|0.02|0.07|A|F|1992-06-20|1992-04-19|1992-07-11|NONE|FOB|arefully express requests wake furiousl|
+5574|33|4|2|21|19593.63|0.05|0.08|A|F|1992-03-22|1992-04-26|1992-04-16|TAKE BACK RETURN|TRUCK|fully final dugouts. express foxes nag |
+5574|119|6|3|27|27515.97|0.10|0.06|R|F|1992-05-08|1992-05-19|1992-06-05|TAKE BACK RETURN|REG AIR|ecial realms. furiously entici|
+5574|94|6|4|14|13917.26|0.09|0.01|R|F|1992-05-20|1992-04-09|1992-05-23|COLLECT COD|REG AIR| use slyly carefully special requests? slyl|
+5574|85|6|5|19|18716.52|0.05|0.03|A|F|1992-05-28|1992-04-24|1992-06-11|TAKE BACK RETURN|REG AIR|old deposits int|
+5575|58|10|1|7|6706.35|0.01|0.07|N|O|1995-10-01|1995-09-30|1995-10-06|NONE|FOB|s. slyly pending theodolites prin|
+5575|31|7|2|23|21413.69|0.04|0.02|N|O|1995-10-26|1995-10-09|1995-11-13|TAKE BACK RETURN|AIR|enticingly final requests. ironically|
+5575|63|8|3|16|15408.96|0.00|0.07|N|O|1995-08-17|1995-10-14|1995-08-30|NONE|RAIL|jole boldly beyond the final as|
+5575|110|1|4|7|7070.77|0.01|0.04|N|O|1995-10-15|1995-09-14|1995-10-18|DELIVER IN PERSON|RAIL|special requests. final, final |
+5600|187|8|1|34|36964.12|0.02|0.00|N|O|1997-03-22|1997-04-05|1997-04-09|TAKE BACK RETURN|MAIL|ly above the stealthy ideas. permane|
+5600|8|5|2|19|17252.00|0.00|0.01|N|O|1997-04-10|1997-03-24|1997-04-16|TAKE BACK RETURN|TRUCK|dencies. carefully p|
+5601|38|4|1|29|27202.87|0.09|0.04|A|F|1992-04-06|1992-02-24|1992-04-29|DELIVER IN PERSON|TRUCK| ironic ideas. final|
+5601|164|1|2|45|47887.20|0.10|0.07|A|F|1992-03-25|1992-04-03|1992-04-04|TAKE BACK RETURN|MAIL|ts-- blithely final accounts cajole. carefu|
+5601|73|4|3|38|36976.66|0.07|0.00|A|F|1992-01-08|1992-03-01|1992-01-09|TAKE BACK RETURN|REG AIR|ter the evenly final deposit|
+5601|148|5|4|12|12577.68|0.03|0.01|A|F|1992-02-27|1992-03-16|1992-03-27|COLLECT COD|TRUCK|ep carefully a|
+5602|176|4|1|9|9685.53|0.08|0.03|N|O|1997-10-14|1997-09-14|1997-11-11|COLLECT COD|FOB|lar foxes; quickly ironic ac|
+5602|62|7|2|31|29823.86|0.04|0.08|N|O|1997-09-04|1997-10-24|1997-09-07|NONE|TRUCK|rate fluffily regular platelets. blithel|
+5602|68|5|3|30|29041.80|0.04|0.00|N|O|1997-09-20|1997-10-25|1997-10-12|DELIVER IN PERSON|FOB|e slyly even packages. careful|
+5603|98|2|1|50|49904.50|0.03|0.02|A|F|1992-10-06|1992-08-20|1992-10-08|COLLECT COD|SHIP|final theodolites accor|
+5603|116|6|2|49|49789.39|0.06|0.05|A|F|1992-06-24|1992-07-28|1992-07-01|DELIVER IN PERSON|FOB|fully silent requests. carefully fin|
+5603|32|8|3|49|45669.47|0.00|0.02|R|F|1992-10-07|1992-07-21|1992-10-10|DELIVER IN PERSON|TRUCK|nic, pending dependencies print|
+5604|136|7|1|44|45589.72|0.05|0.01|N|O|1998-08-06|1998-07-08|1998-09-04|NONE|RAIL|efully ironi|
+5604|136|2|2|49|50770.37|0.10|0.00|N|O|1998-05-02|1998-07-07|1998-05-20|NONE|FOB|ove the regula|
+5604|78|8|3|10|9780.70|0.07|0.05|N|O|1998-08-03|1998-06-23|1998-08-04|COLLECT COD|SHIP|ly final realms wake blit|
+5605|87|8|1|50|49354.00|0.08|0.05|N|O|1996-08-26|1996-10-15|1996-09-04|TAKE BACK RETURN|RAIL|instructions sleep carefully ironic req|
+5605|151|2|2|7|7358.05|0.06|0.01|N|O|1996-12-13|1996-10-13|1996-12-15|TAKE BACK RETURN|FOB|lowly special courts nag among the furi|
+5605|173|2|3|3|3219.51|0.01|0.02|N|O|1996-09-01|1996-10-02|1996-09-20|TAKE BACK RETURN|AIR|posits. accounts boost. t|
+5605|55|3|4|45|42977.25|0.00|0.01|N|O|1996-09-05|1996-10-04|1996-09-13|COLLECT COD|FOB|ly unusual instructions. carefully ironic p|
+5605|70|7|5|39|37832.73|0.00|0.08|N|O|1996-12-13|1996-11-03|1996-12-24|DELIVER IN PERSON|REG AIR|cial deposits. theodolites w|
+5605|166|7|6|29|30918.64|0.08|0.08|N|O|1996-09-19|1996-10-22|1996-10-06|DELIVER IN PERSON|SHIP| quickly. quickly pending sen|
+5606|174|5|1|47|50485.99|0.10|0.04|N|O|1996-12-23|1997-01-31|1997-01-20|DELIVER IN PERSON|REG AIR|carefully final foxes. pending, final|
+5606|92|3|2|34|33731.06|0.09|0.06|N|O|1997-02-23|1997-02-08|1997-03-09|TAKE BACK RETURN|REG AIR|uses. slyly final |
+5606|127|8|3|46|47247.52|0.04|0.00|N|O|1997-03-11|1997-01-13|1997-03-23|DELIVER IN PERSON|REG AIR|ter the ironic accounts. even, ironic depos|
+5606|82|3|4|30|29462.40|0.08|0.04|N|O|1997-02-06|1997-01-26|1997-02-16|DELIVER IN PERSON|REG AIR| nag always. blithely express packages |
+5606|7|2|5|25|22675.00|0.06|0.00|N|O|1996-12-25|1997-01-12|1997-01-11|TAKE BACK RETURN|AIR|breach about the furiously bold |
+5606|154|5|6|3|3162.45|0.04|0.06|N|O|1997-01-11|1997-01-04|1997-02-08|COLLECT COD|AIR| sauternes. asympto|
+5606|74|5|7|46|44807.22|0.07|0.01|N|O|1997-02-01|1997-01-31|1997-02-15|DELIVER IN PERSON|TRUCK|ow requests wake around the regular accoun|
+5607|132|8|1|23|23738.99|0.02|0.06|R|F|1992-04-17|1992-02-12|1992-04-30|DELIVER IN PERSON|MAIL|the special, final patterns |
+5632|10|3|1|48|43680.48|0.06|0.06|N|O|1996-05-08|1996-03-24|1996-06-04|TAKE BACK RETURN|FOB|unts. decoys u|
+5632|106|7|2|21|21128.10|0.02|0.08|N|O|1996-03-22|1996-03-10|1996-04-10|NONE|AIR|refully regular pinto beans. ironic reques|
+5632|67|2|3|24|23209.44|0.04|0.06|N|O|1996-03-23|1996-04-02|1996-03-30|TAKE BACK RETURN|MAIL|beans detect. quickly final i|
+5633|160|2|1|28|29684.48|0.02|0.00|N|O|1998-08-14|1998-07-24|1998-08-26|TAKE BACK RETURN|SHIP|as boost quickly. unusual pinto |
+5633|102|3|2|10|10021.00|0.09|0.04|N|O|1998-07-15|1998-08-03|1998-08-03|COLLECT COD|AIR|its cajole fluffily fluffily special pinto|
+5633|46|7|3|27|25543.08|0.03|0.02|N|O|1998-09-28|1998-07-28|1998-10-12|DELIVER IN PERSON|AIR|ructions. even ideas haggle carefully r|
+5633|164|5|4|50|53208.00|0.02|0.05|N|O|1998-07-23|1998-07-09|1998-08-21|DELIVER IN PERSON|TRUCK|ts. slyly regular |
+5633|100|2|5|48|48004.80|0.01|0.05|N|O|1998-06-24|1998-07-22|1998-07-18|DELIVER IN PERSON|TRUCK|even courts haggle slyly at the requ|
+5633|107|2|6|1|1007.10|0.02|0.03|N|O|1998-09-29|1998-08-28|1998-10-19|NONE|RAIL|thely notornis: |
+5633|11|5|7|39|35529.39|0.02|0.08|N|O|1998-07-12|1998-07-03|1998-07-13|COLLECT COD|TRUCK|ding ideas cajole furiously after|
+5634|185|6|1|26|28214.68|0.10|0.08|N|O|1996-10-29|1996-09-15|1996-11-24|COLLECT COD|REG AIR|ptotes mold qu|
+5634|175|3|2|22|23653.74|0.02|0.05|N|O|1996-09-01|1996-08-31|1996-09-05|DELIVER IN PERSON|MAIL|silently unusual foxes above the blithely|
+5634|109|6|3|16|16145.60|0.08|0.02|N|O|1996-11-15|1996-09-14|1996-12-04|NONE|AIR|ess ideas are carefully pending, even re|
+5634|182|3|4|29|31383.22|0.00|0.01|N|O|1996-08-10|1996-10-29|1996-08-11|TAKE BACK RETURN|MAIL|ely final ideas. deposits sleep. reg|
+5634|1|2|5|1|901.00|0.04|0.02|N|O|1996-10-02|1996-10-21|1996-10-27|COLLECT COD|MAIL|ctions haggle carefully. carefully clo|
+5635|83|4|1|43|42272.44|0.03|0.00|R|F|1992-10-12|1992-09-29|1992-11-01|TAKE BACK RETURN|TRUCK|cross the d|
+5635|72|3|2|5|4860.35|0.05|0.08|R|F|1992-10-02|1992-11-05|1992-10-26|TAKE BACK RETURN|REG AIR|yly along the ironic, fi|
+5635|72|1|3|12|11664.84|0.09|0.02|A|F|1992-10-18|1992-09-24|1992-11-17|NONE|REG AIR|ke slyly against the carefully final req|
+5635|8|5|4|40|36320.00|0.03|0.01|A|F|1992-09-25|1992-11-05|1992-10-11|NONE|FOB|pending foxes. regular packages|
+5635|169|10|5|38|40628.08|0.05|0.06|A|F|1992-10-09|1992-09-25|1992-10-18|NONE|MAIL|ckly pendin|
+5635|162|9|6|23|24429.68|0.05|0.04|A|F|1992-08-24|1992-11-10|1992-09-21|NONE|AIR|ily pending packages. bold,|
+5635|137|3|7|32|33188.16|0.03|0.08|R|F|1992-11-24|1992-09-20|1992-12-17|TAKE BACK RETURN|TRUCK|slyly even|
+5636|70|9|1|18|17461.26|0.05|0.03|R|F|1995-05-14|1995-05-17|1995-06-12|DELIVER IN PERSON|REG AIR|slyly express requests. furiously pen|
+5636|70|5|2|26|25221.82|0.03|0.06|A|F|1995-03-05|1995-05-16|1995-03-23|TAKE BACK RETURN|AIR| furiously final pinto beans o|
+5636|90|1|3|21|20791.89|0.03|0.03|A|F|1995-03-13|1995-05-11|1995-03-24|COLLECT COD|AIR| are furiously unusual |
+5636|109|6|4|15|15136.50|0.03|0.04|R|F|1995-04-21|1995-04-30|1995-05-05|DELIVER IN PERSON|REG AIR|efully special|
+5636|47|4|5|13|12311.52|0.10|0.03|A|F|1995-05-11|1995-04-27|1995-05-26|COLLECT COD|AIR|en, fluffy accounts amon|
+5636|12|3|6|33|30096.33|0.06|0.04|A|F|1995-03-09|1995-04-05|1995-03-23|DELIVER IN PERSON|MAIL|ding to the |
+5636|134|10|7|24|24819.12|0.10|0.05|R|F|1995-04-12|1995-03-27|1995-04-16|DELIVER IN PERSON|RAIL|counts sleep furiously b|
+5637|47|4|1|14|13258.56|0.03|0.05|N|O|1996-07-20|1996-07-26|1996-08-14|COLLECT COD|MAIL|y bold deposits wak|
+5637|172|3|2|35|37525.95|0.09|0.08|N|O|1996-08-01|1996-08-04|1996-08-20|NONE|AIR|s sleep blithely alongside of the ironic|
+5637|96|10|3|22|21913.98|0.01|0.07|N|O|1996-08-28|1996-07-30|1996-09-17|COLLECT COD|REG AIR|nding requests are ca|
+5637|66|1|4|16|15456.96|0.03|0.03|N|O|1996-09-08|1996-08-31|1996-09-29|TAKE BACK RETURN|TRUCK|d packages. express requests|
+5637|196|7|5|10|10961.90|0.01|0.00|N|O|1996-08-25|1996-08-11|1996-09-23|TAKE BACK RETURN|MAIL|ickly ironic gifts. blithely even cour|
+5637|129|4|6|27|27786.24|0.01|0.05|N|O|1996-06-27|1996-08-09|1996-07-27|DELIVER IN PERSON|REG AIR|oss the carefully express warhorses|
+5638|138|9|1|45|46715.85|0.09|0.07|A|F|1994-05-17|1994-03-09|1994-06-15|NONE|TRUCK|ar foxes. fluffily pending accounts |
+5638|168|3|2|12|12817.92|0.02|0.05|A|F|1994-02-05|1994-04-01|1994-02-25|COLLECT COD|TRUCK|n, even requests. furiously ironic not|
+5638|162|9|3|21|22305.36|0.08|0.00|A|F|1994-03-13|1994-03-27|1994-03-17|DELIVER IN PERSON|TRUCK|press courts use f|
+5639|47|10|1|11|10417.44|0.09|0.02|R|F|1994-09-18|1994-07-10|1994-10-12|TAKE BACK RETURN|SHIP|g the unusual pinto beans caj|
+5664|122|1|1|25|25553.00|0.00|0.06|N|O|1998-10-29|1998-09-23|1998-11-25|COLLECT COD|FOB|eposits: furiously ironic grouch|
+5664|173|2|2|9|9658.53|0.07|0.05|N|O|1998-07-31|1998-08-26|1998-08-12|COLLECT COD|RAIL| ironic deposits haggle furiously. re|
+5664|53|4|3|31|29544.55|0.01|0.03|N|O|1998-11-10|1998-09-12|1998-12-07|TAKE BACK RETURN|FOB|ainst the never silent request|
+5664|138|9|4|33|34258.29|0.08|0.03|N|O|1998-08-29|1998-09-17|1998-09-25|DELIVER IN PERSON|RAIL|d the final |
+5664|112|2|5|44|44532.84|0.01|0.06|N|O|1998-09-24|1998-09-26|1998-10-23|NONE|TRUCK|ang thinly bold pa|
+5664|68|5|6|34|32914.04|0.09|0.01|N|O|1998-09-10|1998-10-05|1998-09-15|COLLECT COD|RAIL|st. fluffily pending foxes na|
+5664|182|3|7|9|9739.62|0.01|0.05|N|O|1998-11-04|1998-10-15|1998-11-20|TAKE BACK RETURN|REG AIR|yly. express ideas agai|
+5665|101|2|1|32|32035.20|0.00|0.02|A|F|1993-08-11|1993-08-01|1993-09-07|NONE|AIR|f the slyly even requests! regular request|
+5665|5|8|2|14|12670.00|0.02|0.00|R|F|1993-06-29|1993-09-16|1993-07-16|DELIVER IN PERSON|AIR|- special pinto beans sleep quickly blithel|
+5665|158|9|3|41|43384.15|0.09|0.02|A|F|1993-08-23|1993-09-22|1993-09-11|COLLECT COD|REG AIR| idle ideas across |
+5665|46|9|4|47|44463.88|0.01|0.01|A|F|1993-10-06|1993-09-19|1993-11-01|NONE|RAIL|s mold fluffily. final deposits along the|
+5666|122|5|1|7|7154.84|0.09|0.08|R|F|1994-05-10|1994-04-06|1994-05-21|NONE|FOB| ideas. regular packag|
+5666|36|7|2|14|13104.42|0.08|0.01|A|F|1994-02-27|1994-04-11|1994-03-06|DELIVER IN PERSON|TRUCK|lar deposits nag against the slyly final d|
+5666|193|6|3|39|42634.41|0.00|0.01|A|F|1994-05-13|1994-04-02|1994-06-12|DELIVER IN PERSON|TRUCK|the even, final foxes. quickly iron|
+5666|131|2|4|24|24747.12|0.07|0.01|R|F|1994-02-14|1994-03-09|1994-03-06|DELIVER IN PERSON|FOB|on the carefully pending asympto|
+5666|109|10|5|36|36327.60|0.07|0.07|R|F|1994-03-15|1994-03-16|1994-03-18|COLLECT COD|TRUCK|accounts. furiousl|
+5667|145|4|1|37|38670.18|0.09|0.06|N|O|1995-09-24|1995-09-17|1995-10-03|NONE|REG AIR|s cajole blit|
+5668|4|9|1|15|13560.00|0.03|0.04|A|F|1995-04-06|1995-05-12|1995-04-17|COLLECT COD|FOB| the express, pending requests. bo|
+5669|191|2|1|7|7638.33|0.06|0.06|N|O|1996-06-19|1996-07-07|1996-07-11|COLLECT COD|SHIP|yly regular requests lose blithely. careful|
+5669|156|8|2|2|2112.30|0.06|0.07|N|O|1996-08-04|1996-06-15|1996-08-20|NONE|SHIP| blithely excuses. slyly|
+5669|158|9|3|40|42326.00|0.00|0.02|N|O|1996-08-30|1996-06-15|1996-09-07|TAKE BACK RETURN|FOB|ar accounts alongside of the final, p|
+5669|90|1|4|31|30692.79|0.04|0.05|N|O|1996-08-05|1996-06-10|1996-08-29|COLLECT COD|AIR|to beans against the regular depo|
+5669|140|6|5|30|31204.20|0.07|0.01|N|O|1996-07-14|1996-07-28|1996-08-10|TAKE BACK RETURN|TRUCK|l accounts. care|
+5670|90|1|1|27|26732.43|0.10|0.06|R|F|1993-05-09|1993-05-30|1993-06-06|TAKE BACK RETURN|REG AIR| ideas promise bli|
+5670|186|7|2|43|46705.74|0.06|0.00|A|F|1993-07-09|1993-06-03|1993-07-14|DELIVER IN PERSON|FOB|ests in place of the carefully sly depos|
+5670|7|8|3|24|21768.00|0.09|0.04|A|F|1993-07-17|1993-07-01|1993-08-03|NONE|AIR|press, express requests haggle|
+5670|142|9|4|11|11463.54|0.06|0.06|R|F|1993-07-11|1993-06-26|1993-07-24|DELIVER IN PERSON|MAIL|etect furiously among the even pin|
+5671|120|7|1|25|25503.00|0.00|0.08|N|O|1998-04-17|1998-03-28|1998-05-06|DELIVER IN PERSON|AIR|cording to the quickly final requests-- |
+5671|129|8|2|46|47339.52|0.05|0.08|N|O|1998-03-28|1998-04-22|1998-04-19|TAKE BACK RETURN|MAIL|lar pinto beans detect care|
+5671|172|10|3|13|13938.21|0.10|0.06|N|O|1998-03-02|1998-04-03|1998-03-08|TAKE BACK RETURN|TRUCK|bold theodolites about|
+5671|111|1|4|42|42466.62|0.00|0.07|N|O|1998-02-17|1998-04-24|1998-03-17|TAKE BACK RETURN|SHIP|carefully slyly special deposit|
+5671|129|4|5|13|13378.56|0.09|0.00|N|O|1998-04-24|1998-03-26|1998-04-27|NONE|REG AIR|ers according to the ironic, unusual excu|
+5671|114|1|6|30|30423.30|0.09|0.07|N|O|1998-06-06|1998-04-15|1998-07-01|DELIVER IN PERSON|TRUCK|fily ironi|
+5696|137|3|1|28|29039.64|0.03|0.06|N|O|1995-07-03|1995-06-14|1995-07-27|COLLECT COD|REG AIR| the fluffily brave pearls |
+5696|59|1|2|46|44116.30|0.01|0.00|N|O|1995-08-10|1995-07-08|1995-08-25|COLLECT COD|AIR|ter the instruct|
+5696|167|2|3|42|44820.72|0.04|0.01|N|F|1995-06-06|1995-06-11|1995-06-19|TAKE BACK RETURN|SHIP|te furious|
+5696|98|10|4|20|19961.80|0.08|0.00|N|O|1995-06-25|1995-07-18|1995-07-16|NONE|TRUCK|silent, pending ideas sleep fluffil|
+5696|124|9|5|19|19458.28|0.07|0.05|N|O|1995-08-31|1995-06-13|1995-09-10|COLLECT COD|SHIP|unusual requests sleep furiously ru|
+5696|132|8|6|37|38188.81|0.04|0.05|N|O|1995-07-21|1995-06-23|1995-08-19|NONE|RAIL| carefully expres|
+5696|102|9|7|6|6012.60|0.07|0.05|N|O|1995-08-03|1995-07-15|1995-09-01|DELIVER IN PERSON|REG AIR|n patterns lose slyly fina|
+5697|55|7|1|24|22921.20|0.10|0.07|R|F|1992-10-27|1992-11-28|1992-11-20|NONE|RAIL|uffily iro|
+5697|16|10|2|43|39388.43|0.06|0.02|R|F|1992-12-08|1992-12-03|1992-12-17|TAKE BACK RETURN|FOB|blithely reg|
+5697|56|8|3|42|40154.10|0.03|0.01|A|F|1992-12-19|1992-12-08|1993-01-03|COLLECT COD|TRUCK|inal theodolites cajole after the bli|
+5698|11|8|1|30|27330.30|0.01|0.05|A|F|1994-05-26|1994-08-16|1994-06-19|COLLECT COD|AIR|its. quickly regular foxes aro|
+5698|163|4|2|25|26579.00|0.08|0.07|R|F|1994-08-06|1994-06-21|1994-08-25|NONE|SHIP| asymptotes sleep slyly above the|
+5698|155|3|3|45|47481.75|0.03|0.01|A|F|1994-06-23|1994-08-13|1994-07-02|NONE|FOB|ng excuses. slyly express asymptotes|
+5698|58|6|4|15|14370.75|0.07|0.08|R|F|1994-06-29|1994-07-03|1994-07-02|COLLECT COD|REG AIR|ly ironic frets haggle carefully |
+5698|140|1|5|37|38485.18|0.06|0.06|A|F|1994-06-30|1994-06-23|1994-07-22|TAKE BACK RETURN|SHIP|ts. even, ironic |
+5698|188|9|6|1|1088.18|0.06|0.04|R|F|1994-05-31|1994-07-10|1994-06-03|DELIVER IN PERSON|MAIL|nts. slyly quiet pinto beans nag carefu|
+5699|2|7|1|24|21648.00|0.01|0.07|A|F|1992-10-21|1992-09-04|1992-11-04|COLLECT COD|AIR|kages. fin|
+5699|55|10|2|26|24831.30|0.06|0.06|R|F|1992-08-11|1992-09-21|1992-08-14|COLLECT COD|MAIL|y final deposits wake fluffily u|
+5699|18|2|3|48|44064.48|0.10|0.05|R|F|1992-11-23|1992-10-20|1992-11-29|DELIVER IN PERSON|TRUCK|s. carefully regul|
+5699|55|3|4|46|43932.30|0.08|0.02|A|F|1992-11-28|1992-09-23|1992-12-27|TAKE BACK RETURN|FOB|o the slyly|
+5699|28|7|5|21|19488.42|0.02|0.02|A|F|1992-10-13|1992-09-30|1992-10-19|NONE|MAIL|lyly final pla|
+5699|191|5|6|30|32735.70|0.08|0.05|R|F|1992-11-13|1992-10-01|1992-12-11|DELIVER IN PERSON|AIR| the carefully final |
+5699|129|8|7|45|46310.40|0.09|0.06|A|F|1992-09-23|1992-10-22|1992-10-04|DELIVER IN PERSON|SHIP|rmanent packages sleep across the f|
+5700|168|5|1|24|25635.84|0.09|0.00|N|O|1997-12-26|1998-01-28|1998-01-18|DELIVER IN PERSON|REG AIR|ix carefully |
+5700|123|8|2|30|30693.60|0.00|0.06|N|O|1998-04-19|1998-03-13|1998-04-27|COLLECT COD|MAIL|ly blithely final instructions. fl|
+5700|126|5|3|23|23600.76|0.03|0.05|N|O|1998-01-30|1998-01-31|1998-01-31|NONE|REG AIR| wake quickly carefully fluffy hockey|
+5701|54|2|1|17|16218.85|0.02|0.05|N|O|1997-03-27|1997-04-08|1997-04-21|DELIVER IN PERSON|RAIL|tes. quickly final a|
+5702|77|7|1|44|42991.08|0.06|0.02|R|F|1994-01-04|1993-11-25|1994-01-22|NONE|RAIL|lites. carefully final requests doze b|
+5702|86|7|2|37|36484.96|0.10|0.05|R|F|1993-12-14|1993-10-21|1994-01-08|NONE|FOB|ix slyly. regular instructions slee|
+5702|131|7|3|44|45369.72|0.00|0.02|R|F|1993-11-28|1993-12-02|1993-12-22|NONE|TRUCK|ake according to th|
+5702|63|8|4|31|29854.86|0.00|0.04|A|F|1994-01-04|1993-10-22|1994-01-26|DELIVER IN PERSON|TRUCK|pinto beans. blithely |
+5703|88|9|1|2|1976.16|0.09|0.01|R|F|1993-05-29|1993-07-26|1993-06-05|TAKE BACK RETURN|REG AIR|nts against the blithely sile|
+5728|44|1|1|47|44369.88|0.10|0.05|A|F|1994-12-13|1995-01-25|1994-12-25|TAKE BACK RETURN|MAIL|nd the bravely final deposits. final ideas|
+5728|159|1|2|40|42366.00|0.05|0.08|A|F|1995-03-28|1995-01-17|1995-04-14|TAKE BACK RETURN|SHIP|final deposits. theodolite|
+5729|143|4|1|5|5215.70|0.07|0.00|R|F|1994-11-27|1994-11-11|1994-12-23|TAKE BACK RETURN|MAIL|s. even sheaves nag courts. |
+5729|107|10|2|39|39276.90|0.10|0.00|A|F|1995-01-22|1994-11-21|1995-02-13|TAKE BACK RETURN|MAIL|. special pl|
+5729|12|3|3|50|45600.50|0.00|0.05|R|F|1994-12-09|1994-12-31|1994-12-24|TAKE BACK RETURN|AIR|ly special sentiments. car|
+5730|151|2|1|2|2102.30|0.08|0.00|N|O|1998-02-24|1998-03-15|1998-03-11|COLLECT COD|SHIP|ely ironic foxes. carefu|
+5730|200|1|2|9|9901.80|0.10|0.01|N|O|1998-03-05|1998-02-02|1998-03-28|DELIVER IN PERSON|MAIL|s lose blithely. specia|
+5731|192|6|1|13|14198.47|0.02|0.04|N|O|1997-07-30|1997-06-23|1997-08-13|COLLECT COD|RAIL|ngside of the quickly regular depos|
+5731|105|6|2|11|11056.10|0.00|0.08|N|O|1997-06-06|1997-07-08|1997-06-25|NONE|MAIL| furiously final accounts wake. d|
+5731|111|2|3|6|6066.66|0.01|0.04|N|O|1997-07-02|1997-07-01|1997-07-08|COLLECT COD|SHIP|sits integrate slyly close platelets. quick|
+5731|14|1|4|6|5484.06|0.03|0.06|N|O|1997-09-07|1997-06-20|1997-09-20|TAKE BACK RETURN|RAIL|rs. quickly regular theo|
+5731|195|6|5|19|20808.61|0.08|0.02|N|O|1997-06-29|1997-06-27|1997-07-15|NONE|REG AIR|ly unusual ideas above the |
+5732|139|5|1|26|27017.38|0.02|0.07|N|O|1997-08-18|1997-10-25|1997-09-12|TAKE BACK RETURN|TRUCK|totes cajole according to the theodolites.|
+5733|33|4|1|39|36388.17|0.01|0.07|A|F|1993-03-22|1993-05-24|1993-04-04|DELIVER IN PERSON|FOB|side of the|
+5734|183|4|1|29|31412.22|0.05|0.01|N|O|1997-12-01|1997-12-08|1997-12-23|NONE|RAIL|structions cajole final, express |
+5734|150|3|2|6|6300.90|0.07|0.00|N|O|1997-10-27|1997-12-19|1997-11-02|COLLECT COD|RAIL|s. regular platelets cajole furiously. regu|
+5734|67|8|3|10|9670.60|0.01|0.03|N|O|1997-12-28|1997-12-24|1998-01-24|DELIVER IN PERSON|TRUCK|equests; accounts above|
+5735|60|1|1|41|39362.46|0.01|0.01|R|F|1994-12-23|1995-02-10|1995-01-22|COLLECT COD|MAIL|lthily ruthless i|
+5760|1|8|1|6|5406.00|0.09|0.03|R|F|1994-07-30|1994-07-31|1994-08-16|COLLECT COD|REG AIR|ng the acco|
+5760|6|1|2|24|21744.00|0.04|0.05|A|F|1994-07-15|1994-07-04|1994-08-08|NONE|MAIL|s. bravely ironic accounts among|
+5760|148|5|3|8|8385.12|0.07|0.04|A|F|1994-09-06|1994-08-03|1994-10-06|NONE|AIR|l accounts among the carefully even de|
+5760|123|4|4|19|19439.28|0.10|0.01|R|F|1994-08-02|1994-08-02|1994-08-15|COLLECT COD|SHIP|sits nag. even, regular ideas cajole b|
+5760|166|1|5|6|6396.96|0.03|0.07|R|F|1994-06-09|1994-07-06|1994-06-16|DELIVER IN PERSON|MAIL| shall have to cajole along the |
+5761|47|6|1|41|38828.64|0.08|0.00|N|O|1998-07-31|1998-08-09|1998-08-08|TAKE BACK RETURN|TRUCK|pecial deposits. qu|
+5761|108|9|2|36|36291.60|0.00|0.07|N|O|1998-09-07|1998-09-21|1998-09-11|TAKE BACK RETURN|TRUCK| pinto beans thrash alongside of the pendi|
+5761|198|2|3|49|53811.31|0.04|0.08|N|O|1998-07-14|1998-08-20|1998-07-25|NONE|SHIP|ly bold accounts wake above the|
+5762|175|6|1|6|6451.02|0.05|0.02|N|O|1997-04-07|1997-03-25|1997-05-02|NONE|AIR|ironic dependencies doze carefu|
+5762|102|9|2|27|27056.70|0.02|0.08|N|O|1997-02-21|1997-05-08|1997-03-23|NONE|REG AIR|across the bold ideas. carefully sp|
+5762|89|10|3|40|39563.20|0.00|0.08|N|O|1997-04-30|1997-05-09|1997-05-08|COLLECT COD|SHIP|al instructions. furiousl|
+5762|133|4|4|47|48557.11|0.05|0.06|N|O|1997-03-02|1997-03-23|1997-03-19|NONE|RAIL|equests sleep after the furiously ironic pa|
+5762|25|6|5|28|25900.56|0.02|0.06|N|O|1997-02-22|1997-03-25|1997-02-24|TAKE BACK RETURN|SHIP|ic foxes among the blithely qui|
+5762|12|6|6|12|10944.12|0.00|0.06|N|O|1997-04-18|1997-04-27|1997-05-11|DELIVER IN PERSON|REG AIR|ages are abo|
+5763|131|2|1|32|32996.16|0.02|0.06|N|O|1998-07-16|1998-09-13|1998-08-02|DELIVER IN PERSON|FOB|ding instruct|
+5763|136|2|2|23|23830.99|0.09|0.04|N|O|1998-07-25|1998-09-21|1998-08-15|DELIVER IN PERSON|SHIP|re after the blithel|
+5763|13|3|3|25|22825.25|0.01|0.02|N|O|1998-10-04|1998-08-16|1998-10-09|DELIVER IN PERSON|REG AIR|inal theodolites. even re|
+5763|121|6|4|47|47992.64|0.09|0.00|N|O|1998-08-22|1998-09-22|1998-09-04|NONE|REG AIR|gle slyly. slyly final re|
+5763|123|4|5|8|8184.96|0.06|0.05|N|O|1998-09-23|1998-09-15|1998-09-27|DELIVER IN PERSON|TRUCK|foxes wake slyly. car|
+5763|190|1|6|9|9811.71|0.08|0.02|N|O|1998-09-24|1998-09-01|1998-10-02|NONE|AIR| deposits. instru|
+5764|101|2|1|28|28030.80|0.04|0.04|A|F|1993-12-07|1993-12-20|1993-12-26|TAKE BACK RETURN|RAIL|sleep furi|
+5764|200|3|2|20|22004.00|0.10|0.05|A|F|1993-10-17|1993-12-24|1993-10-18|TAKE BACK RETURN|FOB|ng to the fluffily qu|
+5764|188|9|3|4|4352.72|0.03|0.05|A|F|1993-10-25|1993-12-23|1993-11-06|DELIVER IN PERSON|AIR|ily regular courts haggle|
+5765|162|7|1|31|32926.96|0.00|0.06|A|F|1995-01-11|1995-02-13|1995-01-23|TAKE BACK RETURN|AIR|r foxes. ev|
+5765|124|9|2|29|29699.48|0.07|0.08|A|F|1994-12-29|1995-02-01|1995-01-26|NONE|RAIL|nic requests. deposits wake quickly among |
+5765|139|10|3|31|32213.03|0.05|0.01|R|F|1995-03-01|1995-01-23|1995-03-31|TAKE BACK RETURN|REG AIR|the furiou|
+5765|152|4|4|46|48398.90|0.07|0.07|R|F|1995-03-13|1995-02-12|1995-03-20|DELIVER IN PERSON|MAIL|ccounts sleep about th|
+5765|174|3|5|48|51560.16|0.09|0.02|A|F|1995-03-30|1995-01-14|1995-04-09|DELIVER IN PERSON|SHIP|theodolites integrate furiously|
+5765|83|4|6|41|40306.28|0.04|0.00|A|F|1994-12-31|1995-02-11|1995-01-17|TAKE BACK RETURN|SHIP| furiously. slyly sile|
+5765|42|5|7|21|19782.84|0.05|0.04|R|F|1995-04-05|1995-02-12|1995-05-05|COLLECT COD|TRUCK|ole furiously. quick, special dependencies |
+5766|188|9|1|1|1088.18|0.10|0.01|R|F|1994-01-16|1993-11-16|1994-01-23|NONE|MAIL|blithely regular the|
+5766|149|8|2|39|40916.46|0.02|0.07|A|F|1993-10-24|1993-12-07|1993-11-08|DELIVER IN PERSON|SHIP| furiously unusual courts. slyly final pear|
+5766|118|8|3|4|4072.44|0.08|0.08|R|F|1993-11-10|1993-10-30|1993-12-01|COLLECT COD|TRUCK|ly even requests. furiou|
+5767|167|8|1|11|11738.76|0.08|0.01|A|F|1992-06-02|1992-05-30|1992-06-08|NONE|TRUCK|instructions. carefully final accou|
+5767|69|8|2|15|14535.90|0.07|0.05|R|F|1992-06-05|1992-07-28|1992-06-08|DELIVER IN PERSON|MAIL|warthogs. carefully unusual g|
+5767|191|3|3|42|45829.98|0.06|0.01|R|F|1992-07-31|1992-06-09|1992-08-09|COLLECT COD|TRUCK| blithe deposi|
+5767|153|4|4|34|35807.10|0.06|0.01|R|F|1992-06-02|1992-06-23|1992-06-17|NONE|FOB|sits among the|
+5767|46|7|5|36|34057.44|0.03|0.00|A|F|1992-07-17|1992-06-10|1992-07-19|COLLECT COD|AIR|ake carefully. packages |
+5792|178|8|1|34|36657.78|0.08|0.07|R|F|1993-05-23|1993-06-25|1993-06-12|NONE|RAIL|requests are against t|
+5792|157|5|2|47|49686.05|0.10|0.00|A|F|1993-06-08|1993-05-10|1993-06-26|COLLECT COD|AIR|regular, ironic excuses n|
+5792|183|4|3|32|34661.76|0.05|0.08|R|F|1993-06-26|1993-05-23|1993-07-07|COLLECT COD|RAIL|s are slyly against the ev|
+5792|14|8|4|14|12796.14|0.09|0.02|A|F|1993-07-28|1993-06-17|1993-08-27|DELIVER IN PERSON|RAIL|olites print carefully|
+5792|102|9|5|31|31065.10|0.02|0.01|A|F|1993-06-17|1993-05-05|1993-07-01|COLLECT COD|TRUCK|s? furiously even instructions |
+5793|53|5|1|20|19061.00|0.05|0.03|N|O|1997-10-05|1997-09-04|1997-10-30|COLLECT COD|AIR|e carefully ex|
+5793|170|5|2|41|43876.97|0.06|0.06|N|O|1997-08-04|1997-10-10|1997-08-12|DELIVER IN PERSON|TRUCK|snooze quick|
+5793|43|4|3|8|7544.32|0.07|0.03|N|O|1997-08-16|1997-09-08|1997-08-28|COLLECT COD|AIR|al foxes l|
+5793|148|7|4|48|50310.72|0.02|0.02|N|O|1997-09-27|1997-08-23|1997-10-27|DELIVER IN PERSON|REG AIR|quickly enticing excuses use slyly abov|
+5794|158|9|1|42|44442.30|0.06|0.05|R|F|1993-06-29|1993-05-30|1993-07-28|COLLECT COD|REG AIR|he careful|
+5794|115|2|2|14|14211.54|0.09|0.02|R|F|1993-04-19|1993-07-02|1993-05-18|COLLECT COD|SHIP|uriously carefully ironic reque|
+5794|7|8|3|15|13605.00|0.09|0.06|R|F|1993-06-25|1993-06-27|1993-07-09|NONE|MAIL|blithely regular ideas. final foxes haggle |
+5794|137|3|4|47|48745.11|0.00|0.08|A|F|1993-07-16|1993-06-21|1993-08-05|TAKE BACK RETURN|REG AIR|quests. blithely final excu|
+5795|193|6|1|34|37168.46|0.09|0.05|A|F|1992-08-21|1992-07-30|1992-08-27|COLLECT COD|REG AIR|al instructions must affix along the ironic|
+5796|58|3|1|27|25867.35|0.10|0.00|N|O|1996-04-06|1996-02-29|1996-04-20|DELIVER IN PERSON|RAIL|s wake quickly aro|
+5797|61|6|1|17|16338.02|0.09|0.03|N|O|1997-12-13|1998-01-12|1997-12-23|NONE|REG AIR|the ironic, even theodoli|
+5798|127|8|1|2|2054.24|0.09|0.00|N|O|1998-05-25|1998-06-22|1998-06-09|COLLECT COD|FOB|e furiously across |
+5798|124|9|2|14|14337.68|0.06|0.05|N|O|1998-04-01|1998-06-14|1998-04-27|NONE|RAIL|he special, bold packages. carefully iron|
+5798|134|5|3|22|22750.86|0.02|0.01|N|O|1998-06-24|1998-06-06|1998-07-20|COLLECT COD|TRUCK|sits poach carefully|
+5798|146|3|4|40|41845.60|0.08|0.06|N|O|1998-07-09|1998-06-24|1998-07-16|NONE|TRUCK| integrate carefu|
+5798|149|8|5|7|7343.98|0.06|0.07|N|O|1998-06-06|1998-05-10|1998-06-07|NONE|SHIP|ts against the blithely final p|
+5798|38|4|6|9|8442.27|0.06|0.02|N|O|1998-05-05|1998-05-25|1998-05-09|DELIVER IN PERSON|REG AIR|e blithely|
+5798|115|9|7|32|32483.52|0.08|0.01|N|O|1998-04-27|1998-05-03|1998-05-08|TAKE BACK RETURN|REG AIR|ubt blithely above the |
+5799|95|6|1|41|40798.69|0.04|0.02|N|O|1995-11-13|1995-10-31|1995-11-16|COLLECT COD|TRUCK|al accounts sleep ruthlessl|
+5799|100|3|2|30|30003.00|0.03|0.08|N|O|1995-09-12|1995-09-13|1995-09-19|NONE|RAIL| furiously s|
+5824|77|7|1|40|39082.80|0.06|0.06|N|O|1997-01-14|1997-01-17|1997-02-02|NONE|REG AIR|he final packag|
+5824|182|3|2|42|45451.56|0.09|0.00|N|O|1997-02-01|1997-02-20|1997-02-07|COLLECT COD|SHIP|ts sleep. carefully regular accounts h|
+5824|73|1|3|16|15569.12|0.03|0.02|N|O|1997-02-13|1997-01-07|1997-02-17|TAKE BACK RETURN|TRUCK|sly express Ti|
+5824|92|5|4|32|31746.88|0.03|0.02|N|O|1997-02-16|1997-01-24|1997-02-20|DELIVER IN PERSON|RAIL|ven requests. |
+5824|107|8|5|44|44312.40|0.08|0.03|N|O|1997-01-24|1997-01-31|1997-02-11|COLLECT COD|TRUCK|fily fluffily bold|
+5825|159|7|1|23|24360.45|0.10|0.05|R|F|1995-05-10|1995-04-28|1995-05-13|DELIVER IN PERSON|TRUCK| special pinto beans. dependencies haggl|
+5826|144|1|1|4|4176.56|0.03|0.06|N|O|1998-07-31|1998-09-10|1998-08-27|NONE|AIR| packages across the fluffily spec|
+5826|64|5|2|18|17353.08|0.04|0.01|N|O|1998-07-17|1998-09-03|1998-07-22|NONE|SHIP|atelets use above t|
+5827|187|8|1|30|32615.40|0.03|0.05|N|O|1998-11-11|1998-09-27|1998-11-30|DELIVER IN PERSON|RAIL|ounts may c|
+5827|103|6|2|23|23071.30|0.09|0.05|N|O|1998-11-16|1998-09-14|1998-11-17|COLLECT COD|RAIL|ans. furiously special instruct|
+5827|164|1|3|3|3192.48|0.03|0.06|N|O|1998-10-17|1998-09-29|1998-10-28|DELIVER IN PERSON|MAIL|uses eat along the furiously|
+5827|200|1|4|26|28605.20|0.06|0.00|N|O|1998-07-29|1998-09-24|1998-07-30|DELIVER IN PERSON|SHIP|arefully special packages wake thin|
+5827|112|9|5|38|38460.18|0.03|0.06|N|O|1998-10-18|1998-08-27|1998-10-23|TAKE BACK RETURN|TRUCK|ly ruthless accounts|
+5827|17|4|6|14|12838.14|0.05|0.01|N|O|1998-08-31|1998-09-06|1998-09-13|TAKE BACK RETURN|RAIL|rges. fluffily pending |
+5828|2|9|1|28|25256.00|0.10|0.03|A|F|1994-05-15|1994-05-20|1994-06-08|DELIVER IN PERSON|MAIL| special ideas haggle slyly ac|
+5828|158|3|2|37|39151.55|0.01|0.00|R|F|1994-06-07|1994-05-30|1994-06-17|NONE|RAIL|e carefully spec|
+5829|40|1|1|4|3760.16|0.01|0.02|N|O|1997-03-01|1997-02-17|1997-03-22|NONE|TRUCK|ithely; accounts cajole ideas. regular foxe|
+5829|107|10|2|40|40284.00|0.04|0.01|N|O|1997-04-21|1997-02-12|1997-05-04|COLLECT COD|TRUCK| the carefully ironic accounts. a|
+5829|129|8|3|6|6174.72|0.05|0.06|N|O|1997-01-22|1997-03-12|1997-02-02|TAKE BACK RETURN|AIR|sts. slyly special fo|
+5829|90|1|4|42|41583.78|0.02|0.07|N|O|1997-03-26|1997-04-01|1997-03-30|COLLECT COD|REG AIR|pearls. slyly bold deposits solve final|
+5829|191|5|5|49|53468.31|0.05|0.01|N|O|1997-01-31|1997-03-13|1997-02-18|NONE|MAIL| ironic excuses use fluf|
+5829|18|5|6|17|15606.17|0.09|0.02|N|O|1997-04-10|1997-03-29|1997-04-22|COLLECT COD|AIR|after the furiously ironic ideas no|
+5829|78|9|7|27|26407.89|0.08|0.04|N|O|1997-02-25|1997-03-31|1997-03-03|DELIVER IN PERSON|AIR|ns about the excuses are c|
+5830|160|2|1|29|30744.64|0.10|0.02|R|F|1993-06-19|1993-05-10|1993-07-13|DELIVER IN PERSON|REG AIR|y bold excuses|
+5831|191|2|1|2|2182.38|0.10|0.01|N|O|1997-02-09|1997-01-20|1997-03-07|TAKE BACK RETURN|TRUCK|quickly silent req|
+5831|74|3|2|33|32144.31|0.04|0.03|N|O|1996-11-20|1997-01-18|1996-12-18|TAKE BACK RETURN|MAIL| instructions wake. slyly sil|
+5831|82|3|3|6|5892.48|0.05|0.07|N|O|1997-01-29|1997-01-14|1997-02-09|NONE|MAIL|ly ironic accounts nag pendin|
+5831|13|10|4|46|41998.46|0.06|0.02|N|O|1997-02-24|1997-01-18|1997-03-02|COLLECT COD|MAIL|ly final pa|
+5831|43|4|5|37|34892.48|0.05|0.01|N|O|1997-01-17|1997-02-08|1997-02-01|NONE|FOB|uriously even requests|
+5856|4|1|1|1|904.00|0.03|0.02|A|F|1994-12-29|1995-01-07|1995-01-10|TAKE BACK RETURN|MAIL|tly. special deposits wake blithely even|
+5856|35|6|2|35|32726.05|0.09|0.02|R|F|1994-11-24|1994-12-23|1994-11-30|COLLECT COD|AIR|excuses. finally ir|
+5856|153|4|3|39|41072.85|0.05|0.03|A|F|1995-01-18|1995-01-11|1995-01-19|DELIVER IN PERSON|TRUCK|uickly quickly fluffy in|
+5857|58|9|1|25|23951.25|0.03|0.02|N|O|1997-12-02|1997-12-17|1997-12-08|DELIVER IN PERSON|REG AIR|ding platelets. pending excu|
+5857|195|9|2|50|54759.50|0.06|0.07|N|O|1997-12-04|1997-12-16|1997-12-20|NONE|TRUCK|y regular d|
+5857|68|3|3|1|968.06|0.03|0.01|N|O|1998-02-01|1997-12-09|1998-02-20|TAKE BACK RETURN|SHIP|instructions detect final reques|
+5857|118|2|4|12|12217.32|0.03|0.08|N|O|1998-01-24|1997-12-27|1998-02-10|TAKE BACK RETURN|AIR|counts. express, final|
+5857|192|4|5|14|15290.66|0.07|0.07|N|O|1997-12-10|1998-01-06|1998-01-04|TAKE BACK RETURN|TRUCK|ffily pendin|
+5857|93|5|6|49|48661.41|0.00|0.04|N|O|1998-01-23|1997-12-12|1998-01-28|DELIVER IN PERSON|REG AIR|egular pinto beans|
+5858|121|4|1|20|20422.40|0.02|0.06|A|F|1992-07-23|1992-08-26|1992-07-24|COLLECT COD|SHIP|uffily unusual pinto beans sleep|
+5858|16|7|2|36|32976.36|0.00|0.05|A|F|1992-09-25|1992-08-16|1992-10-11|NONE|SHIP|osits wake quickly quickly sile|
+5858|148|5|3|7|7336.98|0.08|0.02|A|F|1992-10-07|1992-08-16|1992-10-15|TAKE BACK RETURN|REG AIR|. doggedly regular packages use pendin|
+5858|164|9|4|46|48951.36|0.07|0.06|R|F|1992-09-07|1992-10-06|1992-10-06|DELIVER IN PERSON|MAIL|posits withi|
+5858|161|8|5|18|19100.88|0.00|0.07|A|F|1992-11-05|1992-10-08|1992-12-03|NONE|TRUCK|al excuses. bold|
+5858|154|9|6|7|7379.05|0.04|0.00|A|F|1992-09-14|1992-10-01|1992-10-01|TAKE BACK RETURN|RAIL|dly pending ac|
+5858|11|5|7|50|45550.50|0.06|0.00|R|F|1992-07-20|1992-10-07|1992-07-25|NONE|TRUCK|r the ironic ex|
+5859|175|4|1|50|53758.50|0.07|0.01|N|O|1997-07-08|1997-06-20|1997-07-27|COLLECT COD|MAIL|ly regular deposits use. ironic|
+5859|9|6|2|17|15453.00|0.03|0.03|N|O|1997-05-15|1997-06-30|1997-05-26|DELIVER IN PERSON|AIR|ly ironic requests. quickly unusual pin|
+5859|46|3|3|33|31219.32|0.10|0.04|N|O|1997-07-08|1997-06-22|1997-07-18|TAKE BACK RETURN|TRUCK|eposits unwind furiously final pinto bea|
+5859|93|4|4|40|39723.60|0.09|0.02|N|O|1997-08-05|1997-06-17|1997-08-20|NONE|REG AIR|l dependenci|
+5859|153|8|5|35|36860.25|0.00|0.08|N|O|1997-05-28|1997-07-14|1997-06-15|COLLECT COD|TRUCK|egular acco|
+5859|44|5|6|9|8496.36|0.01|0.02|N|O|1997-06-15|1997-06-06|1997-06-20|NONE|RAIL|ges boost quickly. blithely r|
+5859|191|5|7|27|29462.13|0.05|0.08|N|O|1997-07-30|1997-07-08|1997-08-08|NONE|MAIL| across th|
+5860|51|3|1|10|9510.50|0.04|0.04|A|F|1992-03-11|1992-03-30|1992-03-31|NONE|MAIL|ual patterns try to eat carefully above|
+5861|191|5|1|32|34918.08|0.00|0.03|N|O|1997-05-27|1997-05-29|1997-05-28|TAKE BACK RETURN|MAIL|nt asymptotes. carefully express request|
+5861|86|7|2|6|5916.48|0.10|0.03|N|O|1997-07-28|1997-05-18|1997-08-24|TAKE BACK RETURN|TRUCK|olites. slyly|
+5862|113|7|1|4|4052.44|0.09|0.06|N|O|1997-06-04|1997-04-26|1997-06-19|NONE|TRUCK|yly silent deposit|
+5862|2|7|2|29|26158.00|0.03|0.05|N|O|1997-04-02|1997-04-16|1997-04-04|NONE|FOB|e fluffily. furiously|
+5863|161|10|1|45|47752.20|0.07|0.06|A|F|1993-12-19|1994-01-25|1994-01-05|NONE|REG AIR| deposits are ab|
+5863|160|8|2|21|22263.36|0.09|0.03|R|F|1994-01-13|1994-01-09|1994-01-28|DELIVER IN PERSON|FOB|atelets nag blithely furi|
+5888|62|7|1|46|44254.76|0.02|0.00|N|O|1996-11-18|1996-11-05|1996-12-08|TAKE BACK RETURN|FOB|yly final accounts hag|
+5888|112|3|2|24|24290.64|0.03|0.01|N|O|1996-11-07|1996-11-30|1996-11-20|COLLECT COD|SHIP|ing to the spe|
+5889|77|7|1|17|16610.19|0.09|0.02|N|O|1995-07-01|1995-08-12|1995-07-25|NONE|AIR|blithely pending packages. flu|
+5890|113|4|1|38|38498.18|0.01|0.08|A|F|1993-02-14|1992-12-09|1993-02-27|COLLECT COD|FOB| accounts. carefully final asymptotes|
+5891|85|6|1|22|21671.76|0.00|0.06|R|F|1993-01-01|1993-02-18|1993-01-14|DELIVER IN PERSON|TRUCK|iresias cajole deposits. special, ir|
+5891|186|7|2|9|9775.62|0.03|0.07|R|F|1993-01-20|1993-02-27|1993-02-10|COLLECT COD|REG AIR|cajole carefully |
+5891|30|9|3|10|9300.30|0.08|0.01|A|F|1993-04-14|1993-02-07|1993-04-15|DELIVER IN PERSON|RAIL|nding requests. b|
+5892|148|9|1|7|7336.98|0.02|0.03|N|O|1995-06-26|1995-07-18|1995-07-25|COLLECT COD|AIR|e furiously. quickly even deposits da|
+5892|150|9|2|37|38855.55|0.09|0.06|N|O|1995-08-12|1995-06-11|1995-09-05|NONE|REG AIR|maintain. bold, expre|
+5892|3|4|3|28|25284.00|0.03|0.06|N|O|1995-08-16|1995-07-06|1995-08-22|DELIVER IN PERSON|MAIL|ithely unusual accounts will have to integ|
+5892|75|6|4|23|22426.61|0.08|0.04|R|F|1995-05-18|1995-07-06|1995-05-29|COLLECT COD|MAIL| foxes nag slyly about the qui|
+5893|134|10|1|43|44467.59|0.05|0.02|R|F|1992-11-02|1992-09-27|1992-11-21|TAKE BACK RETURN|RAIL|s. regular courts above the carefully silen|
+5893|2|9|2|2|1804.00|0.10|0.04|R|F|1992-07-18|1992-09-10|1992-08-12|NONE|RAIL|ckages wake sly|
+5894|8|5|1|23|20884.00|0.04|0.08|A|F|1994-09-05|1994-10-27|1994-09-13|NONE|TRUCK| furiously even deposits haggle alw|
+5894|79|8|2|48|46995.36|0.04|0.08|A|F|1994-09-04|1994-11-03|1994-09-17|NONE|TRUCK| asymptotes among the blithely silent |
+5895|15|9|1|38|34770.38|0.05|0.08|N|O|1997-04-05|1997-03-06|1997-05-03|DELIVER IN PERSON|RAIL|ts are furiously. regular, final excuses |
+5895|122|3|2|47|48039.64|0.04|0.06|N|O|1997-04-27|1997-03-17|1997-05-07|DELIVER IN PERSON|AIR|r packages wake carefull|
+5895|84|5|3|49|48219.92|0.03|0.07|N|O|1997-03-15|1997-02-17|1997-04-04|NONE|TRUCK|permanent foxes. packages|
+5895|146|7|4|31|32430.34|0.03|0.01|N|O|1997-03-03|1997-03-30|1997-03-08|TAKE BACK RETURN|TRUCK| final deposits nod slyly careful|
+5895|200|1|5|20|22004.00|0.07|0.00|N|O|1997-04-30|1997-02-07|1997-05-08|DELIVER IN PERSON|AIR|gular deposits wake blithely carefully fin|
+5895|78|7|6|15|14671.05|0.08|0.08|N|O|1997-04-19|1997-03-09|1997-05-13|TAKE BACK RETURN|RAIL|silent package|
+5920|187|8|1|50|54359.00|0.06|0.00|A|F|1995-03-13|1995-01-03|1995-03-31|TAKE BACK RETURN|RAIL|across the carefully pending platelets|
+5920|58|9|2|24|22993.20|0.01|0.05|A|F|1994-12-28|1995-01-21|1994-12-31|DELIVER IN PERSON|FOB|fully regular dolphins. furiousl|
+5920|117|1|3|2|2034.22|0.08|0.07|A|F|1995-02-18|1995-01-13|1995-03-04|NONE|SHIP| evenly spe|
+5920|12|2|4|28|25536.28|0.06|0.02|R|F|1994-12-17|1995-02-13|1994-12-31|NONE|SHIP|le slyly slyly even deposits. f|
+5920|100|4|5|42|42004.20|0.09|0.08|A|F|1994-12-18|1995-01-07|1995-01-14|COLLECT COD|AIR|lar, ironic dependencies sno|
+5921|99|3|1|44|43959.96|0.07|0.01|R|F|1994-07-14|1994-06-30|1994-07-15|NONE|TRUCK|ain about the special|
+5921|146|9|2|25|26153.50|0.06|0.01|A|F|1994-05-19|1994-06-15|1994-06-17|COLLECT COD|TRUCK|nd the slyly regular deposits. quick|
+5921|68|5|3|17|16457.02|0.06|0.01|R|F|1994-05-20|1994-05-26|1994-05-23|NONE|FOB|final asymptotes. even packages boost |
+5921|28|7|4|26|24128.52|0.03|0.04|A|F|1994-05-03|1994-07-06|1994-05-06|NONE|AIR|hy dependenc|
+5921|143|10|5|41|42768.74|0.04|0.02|R|F|1994-04-13|1994-05-31|1994-04-26|DELIVER IN PERSON|AIR|nusual, regular theodol|
+5921|115|6|6|5|5075.55|0.02|0.00|R|F|1994-06-01|1994-05-07|1994-06-10|COLLECT COD|TRUCK|eas cajole across the final, fi|
+5922|196|10|1|9|9865.71|0.07|0.00|N|O|1996-12-04|1997-01-20|1996-12-08|DELIVER IN PERSON|RAIL|haggle slyly even packages. packages|
+5922|157|2|2|37|39114.55|0.01|0.04|N|O|1996-12-19|1996-12-16|1997-01-15|COLLECT COD|RAIL|s wake slyly. requests cajole furiously asy|
+5922|90|1|3|35|34653.15|0.08|0.00|N|O|1996-12-12|1997-01-21|1997-01-01|DELIVER IN PERSON|SHIP|accounts. regu|
+5922|66|7|4|13|12558.78|0.08|0.07|N|O|1997-03-08|1996-12-26|1997-04-03|DELIVER IN PERSON|FOB|sly special accounts wake ironically.|
+5922|57|5|5|39|37324.95|0.04|0.07|N|O|1997-03-04|1997-01-17|1997-03-25|TAKE BACK RETURN|SHIP|e of the instructions. quick|
+5922|179|9|6|10|10791.70|0.04|0.01|N|O|1997-02-23|1996-12-26|1997-03-04|NONE|REG AIR|sly regular deposits haggle quickly ins|
+5923|177|8|1|27|29083.59|0.08|0.03|N|O|1997-08-16|1997-06-27|1997-08-29|DELIVER IN PERSON|RAIL|arefully i|
+5923|119|3|2|42|42802.62|0.01|0.08|N|O|1997-09-16|1997-07-23|1997-09-27|COLLECT COD|REG AIR|y regular theodolites w|
+5923|108|5|3|2|2016.20|0.06|0.05|N|O|1997-06-19|1997-07-31|1997-06-28|TAKE BACK RETURN|TRUCK|express patterns. even deposits|
+5923|174|4|4|46|49411.82|0.05|0.04|N|O|1997-07-29|1997-07-23|1997-08-23|COLLECT COD|SHIP|nto beans cajole blithe|
+5923|59|4|5|35|33566.75|0.04|0.05|N|O|1997-07-21|1997-07-11|1997-08-01|DELIVER IN PERSON|AIR|sts affix unusual, final requests. request|
+5924|176|5|1|38|40894.46|0.06|0.05|N|O|1995-12-17|1995-12-11|1996-01-06|TAKE BACK RETURN|AIR|ions cajole carefully along the |
+5924|53|1|2|49|46699.45|0.04|0.00|N|O|1995-10-25|1995-12-11|1995-11-08|NONE|MAIL|inly final excuses. blithely regular requ|
+5924|17|8|3|24|22008.24|0.09|0.08|N|O|1996-01-12|1995-12-13|1996-01-25|COLLECT COD|REG AIR| use carefully. special, e|
+5925|87|8|1|42|41457.36|0.05|0.02|N|O|1996-03-05|1996-01-13|1996-03-10|COLLECT COD|SHIP|to the furiously|
+5925|125|4|2|31|31778.72|0.03|0.03|N|O|1996-01-02|1995-12-14|1996-01-07|TAKE BACK RETURN|FOB|e slyly. furiously regular deposi|
+5925|89|10|3|50|49454.00|0.03|0.04|N|O|1996-02-14|1996-01-10|1996-02-15|NONE|TRUCK|es. stealthily express pains print bli|
+5925|54|9|4|30|28621.50|0.02|0.07|N|O|1996-02-21|1996-02-11|1996-03-10|NONE|TRUCK| the packa|
+5925|160|1|5|41|43466.56|0.00|0.06|N|O|1996-02-03|1995-12-24|1996-02-20|NONE|SHIP| across the pending deposits nag caref|
+5925|50|9|6|48|45602.40|0.02|0.00|N|O|1996-02-03|1996-01-19|1996-03-04|DELIVER IN PERSON|REG AIR| haggle after the fo|
+5926|90|1|1|8|7920.72|0.02|0.00|R|F|1994-07-17|1994-07-20|1994-08-11|COLLECT COD|MAIL|gle furiously express foxes. bo|
+5926|50|9|2|27|25651.35|0.09|0.05|A|F|1994-07-05|1994-08-11|1994-08-02|DELIVER IN PERSON|MAIL|ironic requests|
+5926|127|8|3|46|47247.52|0.01|0.03|R|F|1994-09-05|1994-08-12|1994-09-11|COLLECT COD|RAIL|ts integrate. courts haggl|
+5926|190|1|4|23|25074.37|0.01|0.02|A|F|1994-07-23|1994-08-10|1994-07-27|DELIVER IN PERSON|FOB|ickly special packages among |
+5927|90|1|1|44|43563.96|0.04|0.05|N|O|1997-11-29|1997-11-21|1997-12-13|DELIVER IN PERSON|TRUCK|rding to the special, final decoy|
+5927|115|2|2|8|8120.88|0.04|0.05|N|O|1997-09-24|1997-11-15|1997-10-22|TAKE BACK RETURN|SHIP|ilent dependencies nod c|
+5927|167|6|3|32|34149.12|0.10|0.07|N|O|1997-12-26|1997-10-27|1997-12-31|COLLECT COD|AIR|telets. carefully bold accounts was|
+5952|200|2|1|49|53909.80|0.10|0.02|N|O|1997-06-30|1997-07-10|1997-07-02|COLLECT COD|AIR|e furiously regular|
+5952|191|5|2|11|12003.09|0.10|0.05|N|O|1997-05-13|1997-06-04|1997-05-27|DELIVER IN PERSON|FOB|y nag blithely aga|
+5952|71|2|3|43|41756.01|0.01|0.01|N|O|1997-06-29|1997-06-06|1997-07-15|COLLECT COD|MAIL|posits sleep furiously quickly final p|
+5952|158|3|4|23|24337.45|0.00|0.07|N|O|1997-05-13|1997-06-27|1997-05-20|NONE|TRUCK|e blithely packages. eve|
+5953|129|10|1|36|37048.32|0.03|0.00|R|F|1992-05-28|1992-06-24|1992-05-29|DELIVER IN PERSON|FOB| cajole furio|
+5953|13|7|2|34|31042.34|0.03|0.04|A|F|1992-05-04|1992-06-12|1992-06-02|NONE|RAIL|hockey players use furiously against th|
+5953|162|9|3|5|5310.80|0.07|0.06|A|F|1992-04-10|1992-04-27|1992-04-14|NONE|SHIP|s. blithely |
+5953|169|8|4|23|24590.68|0.09|0.02|R|F|1992-06-05|1992-06-03|1992-06-29|TAKE BACK RETURN|FOB|he silent ideas. silent foxes po|
+5954|147|6|1|8|8377.12|0.03|0.00|A|F|1993-03-27|1993-01-22|1993-04-04|TAKE BACK RETURN|AIR|unusual th|
+5954|81|2|2|40|39243.20|0.02|0.01|A|F|1992-12-30|1993-01-16|1993-01-09|COLLECT COD|RAIL|iously ironic deposits after|
+5954|94|8|3|20|19881.80|0.09|0.07|A|F|1992-12-25|1993-02-05|1992-12-31|COLLECT COD|REG AIR| accounts wake carefu|
+5954|145|4|4|20|20902.80|0.00|0.01|R|F|1993-02-27|1993-01-04|1993-03-08|NONE|TRUCK|ke furiously blithely special packa|
+5954|100|4|5|35|35003.50|0.04|0.06|A|F|1993-03-17|1993-02-06|1993-04-10|NONE|SHIP|tions maintain slyly. furious|
+5954|193|5|6|39|42634.41|0.04|0.08|A|F|1993-02-27|1993-02-25|1993-03-29|DELIVER IN PERSON|REG AIR| always regular dolphins. furiously p|
+5955|140|1|1|14|14561.96|0.08|0.08|N|O|1995-06-22|1995-05-23|1995-06-24|DELIVER IN PERSON|TRUCK| unusual, bold theodolit|
+5955|62|7|2|15|14430.90|0.08|0.07|R|F|1995-04-22|1995-05-28|1995-04-27|NONE|FOB|y final accounts above the regu|
+5955|112|9|3|40|40484.40|0.03|0.00|R|F|1995-04-01|1995-06-11|1995-04-27|NONE|FOB|oss the fluffily regular|
+5956|155|3|1|10|10551.50|0.04|0.05|N|O|1998-07-27|1998-07-04|1998-08-21|NONE|MAIL|ic packages am|
+5956|55|7|2|23|21966.15|0.08|0.03|N|O|1998-06-06|1998-07-10|1998-06-15|DELIVER IN PERSON|RAIL|ly slyly special |
+5956|175|5|3|47|50532.99|0.04|0.06|N|O|1998-09-06|1998-06-29|1998-09-18|TAKE BACK RETURN|MAIL|lyly express theodol|
+5956|20|10|4|40|36800.80|0.09|0.05|N|O|1998-06-11|1998-07-19|1998-06-21|NONE|MAIL|final theodolites sleep carefully ironic c|
+5957|15|9|1|37|33855.37|0.07|0.00|A|F|1994-04-18|1994-02-19|1994-05-11|NONE|AIR| ideas use ruthlessly.|
+5957|59|4|2|46|44116.30|0.04|0.08|A|F|1994-01-23|1994-01-30|1994-02-07|NONE|SHIP|platelets. furiously unusual requests |
+5957|2|7|3|17|15334.00|0.01|0.01|A|F|1994-01-24|1994-02-16|1994-02-08|TAKE BACK RETURN|SHIP|. final, pending packages|
+5957|132|3|4|29|29931.77|0.01|0.03|R|F|1994-02-24|1994-03-04|1994-03-08|COLLECT COD|REG AIR|sits. final, even asymptotes cajole quickly|
+5957|88|9|5|40|39523.20|0.04|0.04|R|F|1994-01-07|1994-02-05|1994-01-26|DELIVER IN PERSON|SHIP|ironic asymptotes sleep blithely again|
+5957|6|1|6|41|37146.00|0.10|0.07|R|F|1994-03-25|1994-02-20|1994-03-31|DELIVER IN PERSON|MAIL|es across the regular requests maint|
+5957|159|1|7|32|33892.80|0.10|0.07|A|F|1994-03-05|1994-02-20|1994-03-09|NONE|TRUCK| boost carefully across the |
+5958|149|8|1|33|34621.62|0.02|0.04|N|O|1995-09-24|1995-12-12|1995-10-05|COLLECT COD|MAIL|lar, regular accounts wake furi|
+5958|43|6|2|23|21689.92|0.03|0.04|N|O|1995-09-26|1995-10-19|1995-09-27|COLLECT COD|SHIP|regular requests. bold, bold deposits unwin|
+5958|153|8|3|42|44232.30|0.10|0.00|N|O|1995-12-12|1995-10-19|1996-01-09|NONE|AIR|n accounts. final, ironic packages |
+5958|39|10|4|18|16902.54|0.04|0.05|N|O|1995-12-02|1995-10-17|1995-12-22|COLLECT COD|FOB|regular requests haggle|
+5958|132|8|5|32|33028.16|0.06|0.00|N|O|1995-09-20|1995-12-10|1995-10-14|COLLECT COD|REG AIR|e carefully special theodolites. carefully |
+5959|135|1|1|49|50721.37|0.07|0.03|R|F|1992-07-16|1992-08-09|1992-08-14|DELIVER IN PERSON|SHIP|usual packages haggle slyly pi|
+5959|147|8|2|17|17801.38|0.09|0.07|R|F|1992-06-10|1992-07-06|1992-06-23|COLLECT COD|MAIL|ackages. blithely ex|
+5959|5|6|3|4|3620.00|0.04|0.03|R|F|1992-06-14|1992-07-05|1992-07-01|NONE|MAIL|gular requests ar|
+5959|196|7|4|13|14250.47|0.03|0.00|A|F|1992-07-29|1992-07-13|1992-08-20|COLLECT COD|SHIP|ar forges. deposits det|
+5959|40|6|5|37|34781.48|0.04|0.01|R|F|1992-06-05|1992-07-18|1992-06-29|NONE|TRUCK|endencies. brai|
+5959|119|3|6|35|35668.85|0.03|0.00|A|F|1992-05-27|1992-06-19|1992-06-23|NONE|TRUCK|ely silent deposits. |
+5959|43|10|7|47|44322.88|0.02|0.01|R|F|1992-08-28|1992-07-24|1992-09-09|TAKE BACK RETURN|RAIL|deposits. slyly special cou|
+5984|70|5|1|13|12610.91|0.06|0.07|R|F|1994-10-16|1994-09-06|1994-11-11|NONE|MAIL|lar platelets. f|
+5984|102|3|2|25|25052.50|0.05|0.08|R|F|1994-10-06|1994-07-21|1994-10-28|COLLECT COD|RAIL|gular accounts. even packages nag slyly|
+5984|1|4|3|8|7208.00|0.10|0.00|R|F|1994-09-17|1994-08-28|1994-09-25|COLLECT COD|RAIL|its. express,|
+5984|190|1|4|35|38156.65|0.00|0.01|A|F|1994-08-25|1994-08-05|1994-08-31|DELIVER IN PERSON|SHIP|le fluffily regula|
+5985|86|7|1|4|3944.32|0.02|0.02|A|F|1995-05-04|1995-04-01|1995-05-17|DELIVER IN PERSON|MAIL|ole along the quickly slow d|
+5986|79|7|1|26|25455.82|0.00|0.00|R|F|1992-08-10|1992-05-23|1992-08-24|TAKE BACK RETURN|SHIP|e fluffily ironic ideas. silent |
+5986|196|8|2|25|27404.75|0.03|0.06|A|F|1992-06-16|1992-07-17|1992-06-29|TAKE BACK RETURN|MAIL| instructions. slyly regular de|
+5986|30|5|3|1|930.03|0.07|0.06|A|F|1992-05-21|1992-06-21|1992-05-24|DELIVER IN PERSON|REG AIR|fix quickly quickly final deposits. fluffil|
+5986|90|1|4|31|30692.79|0.00|0.03|A|F|1992-08-21|1992-06-29|1992-09-14|NONE|AIR|structions! furiously pending instructi|
+5986|136|7|5|6|6216.78|0.05|0.02|A|F|1992-07-16|1992-06-10|1992-07-29|DELIVER IN PERSON|RAIL|al foxes within the slyly speci|
+5987|23|2|1|1|923.02|0.01|0.04|N|O|1996-09-13|1996-10-29|1996-09-21|DELIVER IN PERSON|REG AIR|refully final excuses haggle furiously ag|
+5987|176|5|2|20|21523.40|0.10|0.06|N|O|1996-11-28|1996-09-17|1996-12-05|TAKE BACK RETURN|RAIL|ing excuses nag quickly always bold|
+5987|92|3|3|43|42659.87|0.08|0.04|N|O|1996-10-30|1996-10-13|1996-11-12|NONE|AIR|theodolites wake above the furiously b|
+5987|97|1|4|37|36892.33|0.08|0.08|N|O|1996-10-15|1996-10-27|1996-11-09|NONE|MAIL|le furiously carefully special |
+5988|172|1|1|41|43958.97|0.08|0.03|R|F|1994-01-20|1994-02-06|1994-02-10|COLLECT COD|AIR|the pending, express reque|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/nation.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/nation.tbl
new file mode 100644
index 0000000..ed3fd5b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/nation.tbl
@@ -0,0 +1,25 @@
+0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|
+1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|
+2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |
+3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|
+4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|
+5|ETHIOPIA|0|ven packages wake quickly. regu|
+6|FRANCE|3|refully final requests. regular, ironi|
+7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|
+8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|
+9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|
+10|IRAN|4|efully alongside of the slyly final dependencies. |
+11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|
+12|JAPAN|2|ously. final, express gifts cajole a|
+13|JORDAN|4|ic deposits are blithely about the carefully regular pa|
+14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|
+15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|
+16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|
+17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|
+18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|
+19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|
+20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|
+21|VIETNAM|2|hely enticingly express accounts. even, final |
+22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|
+23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|
+24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders-part1.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders-part1.tbl
new file mode 100644
index 0000000..ed78053
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders-part1.tbl
@@ -0,0 +1,750 @@
+1|37|O|131251.81|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |
+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 sil|
+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 reques|
+802|137|F|156381.95|1995-01-05|1-URGENT|Clerk#000000516|0|posits. ironic, pending requests cajole. even theodol|
+803|16|O|27629.66|1997-04-29|5-LOW|Clerk#000000260|0|ic instructions. even deposits haggle furiously at the deposits-- regular de|
+804|50|F|94400.43|1993-03-12|3-MEDIUM|Clerk#000000931|0|s. blithely final foxes are about the packag|
+805|127|O|90042.41|1995-07-05|4-NOT SPECIFIED|Clerk#000000856|0|y according to the fluffily |
+806|131|O|26839.16|1996-06-20|2-HIGH|Clerk#000000240|0| the ironic packages wake carefully fina|
+807|145|F|222392.53|1993-11-24|3-MEDIUM|Clerk#000000012|0|refully special tithes. blithely regular accoun|
+832|29|F|68494.08|1992-04-19|5-LOW|Clerk#000000495|0|xes. bravely regular packages sleep up the furiously bold accou|
+833|56|F|49033.69|1994-02-13|3-MEDIUM|Clerk#000000437|0|ts haggle quickly across the slyl|
+834|43|F|46459.92|1994-05-23|3-MEDIUM|Clerk#000000805|0| sleep. quickly even foxes are boldly. slyly express requests use slyly|
+835|65|O|62430.67|1995-10-08|4-NOT SPECIFIED|Clerk#000000416|0|s about the carefully special foxes haggle quickly about the|
+836|70|O|72843.48|1996-11-25|4-NOT SPECIFIED|Clerk#000000729|0|ely bold excuses sleep regular ideas. furiously unusual ideas wake furiou|
+837|116|F|60918.41|1994-06-15|4-NOT SPECIFIED|Clerk#000000563|0|kages sleep slyly above the ironic, final orbits|
+838|17|O|82918.36|1998-01-29|5-LOW|Clerk#000000213|0| slyly around the slyly even|
+839|28|O|70182.63|1995-08-08|1-URGENT|Clerk#000000951|0|the carefully even platelets. furiously unusual fo|
+864|139|O|74710.74|1997-08-17|1-URGENT|Clerk#000000036|0|ly after the slyly regular deposits. express, regular asymptotes nag ca|
+865|4|F|70430.54|1993-05-04|3-MEDIUM|Clerk#000000337|0|. special packages wake after the carefully final accounts. express pinto be|
+866|40|F|4766.19|1992-11-28|3-MEDIUM|Clerk#000000718|0|ins after the even, even accounts nod blithel|
+867|26|F|7471.75|1993-11-16|3-MEDIUM|Clerk#000000877|0|pades nag quickly final, |
+868|104|F|127345.45|1992-06-09|4-NOT SPECIFIED|Clerk#000000782|0|onic theodolites print carefully. blithely dogge|
+869|136|O|58932.19|1997-01-12|2-HIGH|Clerk#000000245|0|ar sheaves are slowly. slyly even attainments boost theodolites. furiously|
+870|34|F|40492.37|1993-06-20|4-NOT SPECIFIED|Clerk#000000123|0|blithely ironic ideas nod. sly, r|
+871|16|O|172861.58|1995-11-15|5-LOW|Clerk#000000882|0|oss the ironic theodolites.|
+896|2|F|169847.63|1993-03-09|1-URGENT|Clerk#000000187|0|inal packages eat blithely according to the warhorses. furiously quiet de|
+897|49|P|57697.44|1995-03-20|1-URGENT|Clerk#000000316|0| wake quickly against |
+898|55|F|101020.75|1993-06-03|2-HIGH|Clerk#000000611|0|. unusual pinto beans haggle quickly across |
+899|109|O|125562.09|1998-04-08|5-LOW|Clerk#000000575|0|rts engage carefully final theodolites.|
+900|46|F|120073.51|1994-10-01|4-NOT SPECIFIED|Clerk#000000060|0| fluffily express deposits nag furiousl|
+901|13|O|81826.12|1998-07-21|4-NOT SPECIFIED|Clerk#000000929|0|lyly even foxes are furious, silent requests. requests about the quickly |
+902|10|F|37348.62|1994-07-27|4-NOT SPECIFIED|Clerk#000000811|0|yly final requests over the furiously regula|
+903|11|O|109351.87|1995-07-07|4-NOT SPECIFIED|Clerk#000000793|0|e slyly about the final pl|
+928|67|F|228136.49|1995-03-02|5-LOW|Clerk#000000450|0|ithely express pinto beans. |
+929|83|F|109301.02|1992-10-02|2-HIGH|Clerk#000000160|0|its. furiously even foxes affix carefully finally silent accounts. express req|
+930|131|F|199102.23|1994-12-17|1-URGENT|Clerk#000000004|0| accounts nag slyly. ironic, ironic accounts wake blithel|
+931|103|F|117909.23|1992-12-07|1-URGENT|Clerk#000000881|0|ss packages haggle furiously express, regular deposits. even, e|
+932|41|O|40234.50|1997-05-16|2-HIGH|Clerk#000000218|0|ly express instructions boost furiously reg|
+933|97|F|71349.30|1992-08-05|4-NOT SPECIFIED|Clerk#000000752|0|ial courts wake permanently against the furiously regular ideas. unusual |
+934|52|O|17213.59|1996-07-03|1-URGENT|Clerk#000000229|0|ts integrate carefully. sly, regular deposits af|
+935|50|O|97733.87|1997-09-24|5-LOW|Clerk#000000180|0|iously final deposits cajole. blithely even packages |
+960|35|F|63537.13|1994-09-21|3-MEDIUM|Clerk#000000120|0|regular accounts. requests|
+961|56|P|158893.16|1995-06-04|4-NOT SPECIFIED|Clerk#000000720|0|ons nag furiously among the quickl|
+962|37|F|98258.73|1994-05-06|5-LOW|Clerk#000000463|0|ments nag deposits. fluffily ironic a|
+963|26|F|53287.25|1994-05-26|3-MEDIUM|Clerk#000000497|0|uses haggle carefully. slyly even dependencies after the packages ha|
+964|76|O|131146.47|1995-05-20|3-MEDIUM|Clerk#000000657|0|print blithely ironic, careful theodolit|
+965|70|P|41758.44|1995-05-15|5-LOW|Clerk#000000218|0|iously special packages. slyly pending requests are carefully |
+966|14|O|120516.93|1998-04-30|2-HIGH|Clerk#000000239|0|special deposits. furious|
+967|110|F|179287.95|1992-06-21|3-MEDIUM|Clerk#000000167|0|excuses engage quickly bold dep|
+992|55|O|133665.12|1997-11-11|3-MEDIUM|Clerk#000000875|0|ts. regular pinto beans thrash carefully sl|
+993|80|O|198238.65|1995-09-10|3-MEDIUM|Clerk#000000894|0|quickly express accounts among the furiously bol|
+994|2|F|41433.48|1994-04-20|5-LOW|Clerk#000000497|0|ole. slyly bold excuses nag caref|
+995|116|P|135157.92|1995-05-31|3-MEDIUM|Clerk#000000439|0|deas. blithely final deposits play. express accounts wake blithely caref|
+996|71|O|47447.63|1997-12-29|1-URGENT|Clerk#000000497|0|arefully final packages into the slyly final requests affix blit|
+997|109|O|27561.82|1997-05-19|2-HIGH|Clerk#000000651|0|ly express depths. furiously final requests haggle furiously. carefu|
+998|32|F|65269.38|1994-11-26|4-NOT SPECIFIED|Clerk#000000956|0|ronic dolphins. ironic, bold ideas haggle furiously furious|
+999|61|F|145249.13|1993-09-05|5-LOW|Clerk#000000464|0|pitaphs sleep. regular accounts use. f|
+1024|4|O|176084.63|1997-12-23|5-LOW|Clerk#000000903|0| blithely. even, express theodolites cajole slyly across|
+1025|103|F|82034.03|1995-05-05|2-HIGH|Clerk#000000376|0|ross the slyly final pa|
+1026|73|O|36464.76|1997-06-04|5-LOW|Clerk#000000223|0|s wake blithely. special acco|
+1027|128|F|112770.89|1992-06-03|3-MEDIUM|Clerk#000000241|0|equests cajole. slyly final pinto bean|
+1028|70|F|153864.67|1994-01-01|2-HIGH|Clerk#000000131|0|ts are. final, silent deposits are among the fl|
+1029|130|F|47440.91|1994-06-21|2-HIGH|Clerk#000000700|0|quests sleep. slyly even foxes wake quickly final theodolites. clo|
+1030|134|F|16346.94|1994-06-15|5-LOW|Clerk#000000422|0|ully ironic accounts sleep carefully. requests are carefully alongside of the |
+1031|4|F|128024.71|1994-09-01|3-MEDIUM|Clerk#000000448|0|s; ironic theodolites along the carefully ex|
+1056|28|F|38446.39|1995-02-11|1-URGENT|Clerk#000000125|0|t, even deposits hang about the slyly special i|
+1057|76|F|108107.42|1992-02-20|1-URGENT|Clerk#000000124|0|cuses dazzle carefully careful, ironic pinto beans. carefully even theod|
+1058|53|F|89359.11|1993-04-26|3-MEDIUM|Clerk#000000373|0|kly pending courts haggle. blithely regular sheaves integrate carefully fi|
+1059|127|F|198360.22|1994-02-27|1-URGENT|Clerk#000000104|0|en accounts. carefully bold packages cajole daringly special depende|
+1060|140|F|121994.04|1993-02-21|3-MEDIUM|Clerk#000000989|0|l platelets sleep quickly slyly special requests. furiously |
+1061|103|O|166947.75|1998-05-15|5-LOW|Clerk#000000576|0|uests sleep at the packages. fur|
+1062|106|O|39805.04|1997-01-15|1-URGENT|Clerk#000000152|0|eposits use blithely |
+1063|37|F|41392.31|1994-04-02|2-HIGH|Clerk#000000024|0|deposits nag quickly regular deposits. quickl|
+1088|148|F|47120.41|1992-05-21|5-LOW|Clerk#000000347|0|counts are blithely. platelets print. carefully |
+1089|49|O|103192.74|1996-05-04|4-NOT SPECIFIED|Clerk#000000226|0|ns haggle ruthlessly. even requests are quickly abov|
+1090|19|O|32929.30|1997-11-15|2-HIGH|Clerk#000000300|0| furiously regular platelets haggle along the slyly unusual foxes! |
+1091|83|O|35795.22|1996-08-27|1-URGENT|Clerk#000000549|0| even pinto beans haggle quickly alongside of the eve|
+1092|124|P|85552.21|1995-03-04|3-MEDIUM|Clerk#000000006|0|re quickly along the blithe|
+1093|101|O|79189.58|1997-07-31|4-NOT SPECIFIED|Clerk#000000159|0| after the carefully ironic requests. carefully ironic packages wake fluffil|
+1094|145|O|9006.25|1997-12-24|3-MEDIUM|Clerk#000000570|0|beans affix furiously about the pending, even deposits. finally pendi|
+1095|145|O|178491.24|1995-08-22|3-MEDIUM|Clerk#000000709|0|sly bold requests cajole carefully according to|
+1120|140|O|107958.62|1997-11-07|3-MEDIUM|Clerk#000000319|0|lly special requests. slyly pending platelets are quickly pending requ|
+1121|29|O|241837.88|1997-01-13|3-MEDIUM|Clerk#000000541|0|r escapades. deposits above the fluffily bold requests hag|
+1122|121|O|179747.47|1997-01-10|1-URGENT|Clerk#000000083|0|uffily carefully final theodolites. furiously express packages affix|
+1123|73|O|93259.93|1996-08-03|3-MEDIUM|Clerk#000000929|0|uriously pending requests. slyly regular instruction|
+1124|80|O|141858.97|1998-07-30|5-LOW|Clerk#000000326|0|regular pinto beans along the fluffily silent packages|
+1125|25|F|80438.38|1994-10-27|2-HIGH|Clerk#000000510|0|ithely final requests. i|
+1126|145|O|59982.31|1998-01-28|4-NOT SPECIFIED|Clerk#000000928|0|d slyly regular ideas: special ideas believe slyly. slyly ironic sheaves w|
+1127|58|O|103320.91|1995-09-19|4-NOT SPECIFIED|Clerk#000000397|0|usly silent, regular pinto beans. blithely express requests boos|
+1152|49|F|51775.54|1994-08-14|4-NOT SPECIFIED|Clerk#000000496|0|equests. deposits ab|
+1153|121|O|220727.97|1996-04-18|5-LOW|Clerk#000000059|0| across the pending deposi|
+1154|37|F|192417.85|1992-02-15|1-URGENT|Clerk#000000268|0|old asymptotes are special requests. blithely even deposits sleep furiously|
+1155|149|O|126902.81|1997-10-06|2-HIGH|Clerk#000000164|0|c deposits haggle among the ironic, even requests. carefully ironic sheaves n|
+1156|133|O|217682.81|1996-10-19|1-URGENT|Clerk#000000200|0| blithely ironic dolphins. furiously pendi|
+1157|97|O|85394.06|1998-01-14|4-NOT SPECIFIED|Clerk#000000207|0|out the regular excuses boost carefully against the furio|
+1158|142|O|31075.51|1996-06-30|2-HIGH|Clerk#000000549|0|integrate slyly furiously ironic deposit|
+1159|70|F|55553.68|1992-09-18|3-MEDIUM|Clerk#000000992|0|ts may sleep. requests according to the|
+1184|89|O|39700.29|1997-10-26|5-LOW|Clerk#000000777|0|iously even packages haggle fluffily care|
+1185|74|F|47033.21|1992-08-24|5-LOW|Clerk#000000344|0| even escapades are. package|
+1186|59|O|82026.18|1996-08-15|4-NOT SPECIFIED|Clerk#000000798|0|ingly regular pinto beans: instructi|
+1187|134|F|85948.02|1992-11-20|3-MEDIUM|Clerk#000000047|0|s after the furiously final deposits boost slyly under the|
+1188|20|O|54655.07|1996-04-11|2-HIGH|Clerk#000000256|0|ully ironic deposits. slyl|
+1189|46|F|71017.99|1994-04-09|1-URGENT|Clerk#000000243|0|f the even accounts. courts print blithely ironic accounts. sile|
+1190|13|O|31043.39|1997-03-16|5-LOW|Clerk#000000575|0|ccounts above the foxes integrate carefully after the |
+1191|112|O|28623.04|1995-11-07|3-MEDIUM|Clerk#000000011|0|uests nag furiously. carefully even requests|
+1216|122|F|68056.57|1992-12-07|5-LOW|Clerk#000000918|0|nal foxes around the e|
+1217|7|F|40982.08|1992-04-26|4-NOT SPECIFIED|Clerk#000000538|0| foxes nag quickly. ironic excuses nod. blithely pending|
+1218|10|F|99834.47|1994-06-20|4-NOT SPECIFIED|Clerk#000000994|0|s cajole. special, silent deposits about the theo|
+1219|28|O|10163.56|1995-10-05|3-MEDIUM|Clerk#000000800|0|od carefully. slyly final dependencies across the even fray|
+1220|49|O|122157.14|1996-08-29|1-URGENT|Clerk#000000712|0|inal theodolites wake. fluffily ironic asymptotes cajol|
+1221|14|F|117397.16|1992-04-19|4-NOT SPECIFIED|Clerk#000000852|0| detect against the silent, even deposits. carefully ironic|
+1222|10|F|47623.94|1993-02-05|3-MEDIUM|Clerk#000000811|0|theodolites use quickly even accounts. carefully final asympto|
+1223|10|O|26714.67|1996-05-25|4-NOT SPECIFIED|Clerk#000000238|0|posits was blithely fr|
+1248|49|F|210713.88|1992-01-02|1-URGENT|Clerk#000000890|0|t the carefully regular dugouts. s|
+1249|149|F|45889.09|1994-01-05|1-URGENT|Clerk#000000095|0|al ideas sleep above the pending pin|
+1250|37|F|12907.62|1992-09-29|4-NOT SPECIFIED|Clerk#000000652|0|ts after the fluffily pending instructions use slyly about the s|
+1251|38|O|109536.55|1997-10-30|1-URGENT|Clerk#000000276|0|, brave sauternes. deposits boost fluffily.|
+1252|149|O|93403.05|1997-08-04|5-LOW|Clerk#000000348|0|ng the slyly regular excuses. special courts nag furiously blithely e|
+1253|115|F|92730.74|1993-01-26|1-URGENT|Clerk#000000775|0| requests sleep furiously even foxes. ruthless packag|
+1254|70|O|94649.25|1995-12-22|1-URGENT|Clerk#000000607|0| pinto beans. carefully regular request|
+1255|122|F|62518.31|1994-05-30|4-NOT SPECIFIED|Clerk#000000798|0|ct slyly regular accounts. quick|
+1280|97|F|91664.85|1993-01-11|5-LOW|Clerk#000000160|0|posits thrash quickly after the theodolites. furiously iro|
+1281|62|F|165454.51|1994-12-11|1-URGENT|Clerk#000000430|0|counts. carefully pending accounts eat |
+1282|116|F|61297.42|1992-02-29|4-NOT SPECIFIED|Clerk#000000168|0|he quickly special packages. furiously final re|
+1283|118|O|202623.92|1996-08-30|4-NOT SPECIFIED|Clerk#000000260|0| pinto beans boost slyly ac|
+1284|134|O|106122.38|1996-01-07|2-HIGH|Clerk#000000492|0|s. blithely silent deposits s|
+1285|11|F|139124.72|1992-06-01|1-URGENT|Clerk#000000423|0|cial deposits cajole after the ironic requests. p|
+1286|109|F|207291.83|1993-05-14|4-NOT SPECIFIED|Clerk#000000939|0| deposits use carefully from the excuses. slyly bold p|
+1287|19|F|131432.42|1994-07-05|2-HIGH|Clerk#000000288|0|ly ironic dolphins integrate furiously among the final packages. st|
+1312|112|F|58111.00|1994-05-19|3-MEDIUM|Clerk#000000538|0|n, express accounts across the ironic|
+1313|148|F|46598.65|1994-09-13|1-URGENT|Clerk#000000774|0|ld accounts. regular deposits cajole. ironically pending theodolites use car|
+1314|143|F|56207.66|1994-05-13|3-MEDIUM|Clerk#000000485|0|ickly blithe packages nod ideas. furiously bold braids boost around the car|
+1315|22|O|121935.23|1998-03-22|5-LOW|Clerk#000000840|0|final theodolites alongside of the carefu|
+1316|16|F|163746.47|1993-12-03|1-URGENT|Clerk#000000857|0|ully bold theodolites? pending, bold pin|
+1317|100|P|139714.71|1995-05-19|2-HIGH|Clerk#000000373|0|sts. furiously special deposits lose fur|
+1318|128|O|81663.65|1998-06-27|3-MEDIUM|Clerk#000000581|0|s hang bold requests. pending, re|
+1319|32|O|31103.83|1996-09-27|2-HIGH|Clerk#000000257|0|y across the ruthlessly ironic accounts. unusu|
+1344|17|F|43809.37|1992-04-16|5-LOW|Clerk#000000178|0|omise close, silent requests. pending theodolites boost pending |
+1345|95|F|111207.93|1992-10-28|5-LOW|Clerk#000000447|0| regular tithes. quickly fluffy de|
+1346|76|F|171975.62|1992-06-18|2-HIGH|Clerk#000000374|0|ges sleep quickly-- even pint|
+1347|41|O|173444.60|1997-06-20|5-LOW|Clerk#000000977|0|he furiously even foxes use carefully express req|
+1348|19|O|94135.77|1998-04-18|5-LOW|Clerk#000000206|0|tly. quickly even deposi|
+1349|64|O|46376.09|1997-10-26|1-URGENT|Clerk#000000543|0|yly! blithely special theodolites cajole. unusual, reg|
+1350|52|F|49305.98|1993-08-24|1-URGENT|Clerk#000000635|0|iously about the blithely special a|
+1351|106|O|24637.96|1998-04-20|1-URGENT|Clerk#000000012|0| cajole. regular, special re|
+1376|47|O|23984.88|1997-05-04|4-NOT SPECIFIED|Clerk#000000730|0|der furiously final, final frets. carefull|
+1377|20|O|108334.30|1998-04-24|4-NOT SPECIFIED|Clerk#000000625|0|lly across the blithely express accounts. ironic excuses promise carefully de|
+1378|20|O|118495.12|1996-03-09|4-NOT SPECIFIED|Clerk#000000705|0| furiously even tithes cajole slyly among the quick|
+1379|65|O|84627.76|1998-05-25|5-LOW|Clerk#000000861|0|y deposits are caref|
+1380|137|O|94969.41|1996-07-07|3-MEDIUM|Clerk#000000969|0|inal deposits wake slyly daringly even requests. bold, even foxe|
+1381|127|O|58212.22|1998-05-25|3-MEDIUM|Clerk#000000107|0|even requests breach after the bold, ironic instructions. slyly even|
+1382|133|F|173522.71|1993-08-17|5-LOW|Clerk#000000241|0|fully final packages sl|
+1383|121|F|34797.72|1993-04-27|2-HIGH|Clerk#000000785|0|ts. express requests sleep blithel|
+1408|55|O|183965.61|1997-12-26|4-NOT SPECIFIED|Clerk#000000942|0|t the quickly final asymptotes. unusual|
+1409|143|F|72440.52|1992-12-31|4-NOT SPECIFIED|Clerk#000000065|0|ructions. furiously unusual excuses are regular, unusual theodolites. fin|
+1410|113|O|114879.19|1997-04-12|5-LOW|Clerk#000000123|0|iously along the bravely regular dolphins. pinto beans cajole furiously sp|
+1411|95|F|164462.61|1994-12-21|2-HIGH|Clerk#000000566|0|s. furiously special excuses across the pending pinto beans haggle sp|
+1412|53|F|78676.54|1993-03-13|4-NOT SPECIFIED|Clerk#000000083|0|uffily daring theodolit|
+1413|91|O|75733.58|1997-06-14|3-MEDIUM|Clerk#000000342|0|, ironic instructions. carefully even packages dazzle|
+1414|77|O|38057.81|1995-08-16|1-URGENT|Clerk#000000883|0|ccounts. ironic foxes haggle car|
+1415|79|F|24654.79|1994-05-29|4-NOT SPECIFIED|Clerk#000000601|0|rays. blithely final ideas affix quickl|
+1440|98|O|50201.16|1995-08-10|5-LOW|Clerk#000000956|0| pending requests. closely s|
+1441|122|O|156477.94|1997-03-06|4-NOT SPECIFIED|Clerk#000000156|0|ter the excuses. ironic dependencies m|
+1442|112|F|7108.12|1994-07-05|4-NOT SPECIFIED|Clerk#000000935|0|nal pinto beans. slyly ironic ideas cajol|
+1443|44|O|44672.03|1996-12-16|5-LOW|Clerk#000000185|0|x blithely against the carefully final somas. even asymptotes are. quickly spe|
+1444|134|F|207907.60|1994-12-06|3-MEDIUM|Clerk#000000783|0|ove the bold accounts cajole fluffily about|
+1445|115|F|154653.32|1995-01-10|3-MEDIUM|Clerk#000000211|0|even packages wake fluffily |
+1446|41|O|27663.16|1998-02-16|5-LOW|Clerk#000000274|0|lly regular notornis above the requests sleep final accounts! |
+1447|91|F|108171.38|1992-10-15|2-HIGH|Clerk#000000880|0|inly against the blithely pending excuses. regular, pe|
+1472|149|O|65331.05|1996-10-06|5-LOW|Clerk#000000303|0|y special dolphins around the final dependencies wake quick|
+1473|94|O|80624.38|1997-03-17|3-MEDIUM|Clerk#000000960|0|furiously close accoun|
+1474|70|F|51697.18|1995-01-09|1-URGENT|Clerk#000000438|0|detect quickly above the carefully even |
+1475|5|O|185496.66|1997-11-12|2-HIGH|Clerk#000000972|0|cally final packages boost. blithely ironic packa|
+1476|145|O|18795.62|1996-06-27|2-HIGH|Clerk#000000673|0|ding accounts hinder alongside of the quickly pending requests. fluf|
+1477|76|O|231831.35|1997-08-24|5-LOW|Clerk#000000612|0|ly bold foxes. final ideas would cajo|
+1478|50|O|20791.50|1997-08-03|2-HIGH|Clerk#000000827|0|lessly. carefully express|
+1479|16|O|31471.04|1995-12-16|4-NOT SPECIFIED|Clerk#000000697|0|he furiously even foxes. thinly bold deposits|
+1504|2|F|89399.40|1992-08-28|3-MEDIUM|Clerk#000000381|0|, brave deposits. bold de|
+1505|37|F|55892.35|1992-08-21|2-HIGH|Clerk#000000544|0|s. slyly ironic packages cajole. carefully regular packages haggle |
+1506|148|F|195844.84|1992-09-21|3-MEDIUM|Clerk#000000620|0| dependencies. accounts affix blithely slowly unusual deposits. slyly regular |
+1507|121|F|96166.92|1993-10-14|3-MEDIUM|Clerk#000000305|0|stealthy, ironic de|
+1508|103|O|151282.65|1998-04-10|5-LOW|Clerk#000000117|0| after the furiously regular pinto beans hang slyly quickly ironi|
+1509|64|F|180455.98|1993-07-08|5-LOW|Clerk#000000770|0|the regular ideas. regul|
+1510|53|O|154590.05|1996-09-17|5-LOW|Clerk#000000128|0|ld carefully. furiously final asymptotes haggle furiously|
+1511|79|O|59651.38|1996-12-22|4-NOT SPECIFIED|Clerk#000000386|0|ts above the depend|
+1536|94|O|5184.26|1997-01-26|3-MEDIUM|Clerk#000000117|0|ges are! furiously final deposits cajole iron|
+1537|109|F|108317.51|1992-02-15|4-NOT SPECIFIED|Clerk#000000862|0|g to the even deposits. ironic, final packages |
+1538|29|O|179554.41|1995-06-18|4-NOT SPECIFIED|Clerk#000000258|0| instructions. regular theod|
+1539|112|F|39612.63|1995-03-10|5-LOW|Clerk#000000840|0|nstructions boost pa|
+1540|16|F|128014.15|1992-08-05|2-HIGH|Clerk#000000927|0|r ideas hinder blithe|
+1541|94|P|47286.32|1995-05-18|1-URGENT|Clerk#000000906|0|y. slyly ironic warhorses around the furiously regul|
+1542|143|F|132972.24|1993-09-15|3-MEDIUM|Clerk#000000435|0|t the furiously close deposits do was f|
+1543|52|O|139047.22|1997-02-20|1-URGENT|Clerk#000000398|0|unts. furiously pend|
+1568|17|O|76119.72|1997-01-30|4-NOT SPECIFIED|Clerk#000000554|0|d notornis. carefully |
+1569|104|O|87803.55|1998-04-02|5-LOW|Clerk#000000786|0|orbits. fluffily even decoys serve blithely. furiously furious realms nag acro|
+1570|124|O|35589.57|1998-03-16|1-URGENT|Clerk#000000745|0|pinto beans haggle furiousl|
+1571|103|F|151404.78|1992-12-05|2-HIGH|Clerk#000000565|0|ously furiously bold warthogs. slyly ironic instructions are quickly a|
+1572|11|O|47232.79|1996-02-24|2-HIGH|Clerk#000000994|0|fluffily ironic accounts haggle blithely final platelets! slyly regular foxes|
+1573|148|F|86918.57|1992-12-28|2-HIGH|Clerk#000000940|0|ess, ironic deposits use along the carefu|
+1574|134|O|179923.54|1996-12-12|3-MEDIUM|Clerk#000000809|0| ideas hinder after the carefully unusual |
+1575|145|O|197031.52|1995-09-13|3-MEDIUM|Clerk#000000497|0|. furiously regular dep|
+1600|94|F|130515.61|1993-03-03|3-MEDIUM|Clerk#000000627|0|tions cajole quietly above the regular, silent requests. slyly fin|
+1601|53|F|73962.95|1994-08-27|5-LOW|Clerk#000000469|0|ent deposits are ca|
+1602|1|F|4225.26|1993-08-05|5-LOW|Clerk#000000660|0|deposits. busily silent instructions haggle furiously. fin|
+1603|2|F|29305.47|1993-07-31|4-NOT SPECIFIED|Clerk#000000869|0|s. slyly silent deposits boo|
+1604|113|F|107139.29|1993-07-17|5-LOW|Clerk#000000512|0|lithely silent waters. blithely unusual packages alongside |
+1605|58|O|130687.64|1998-04-24|4-NOT SPECIFIED|Clerk#000000616|0|sleep furiously? ruthless, even pinto beans |
+1606|53|O|115877.40|1997-04-17|4-NOT SPECIFIED|Clerk#000000550|0|r requests. quickly even platelets breach before the ironically|
+1607|149|O|166335.03|1995-12-16|2-HIGH|Clerk#000000498|0| bold, pending foxes haggle. slyly silent |
+1632|67|O|183286.33|1997-01-08|3-MEDIUM|Clerk#000000351|0|onic requests are accounts. bold a|
+1633|16|O|52359.51|1995-10-14|2-HIGH|Clerk#000000666|0|y silent accounts sl|
+1634|70|O|145898.47|1996-09-10|1-URGENT|Clerk#000000360|0|arefully blithely ironic requests. slyly unusual instructions alongside|
+1635|4|O|70232.26|1997-02-13|3-MEDIUM|Clerk#000000958|0|s. slyly ironic requests affix slyly |
+1636|79|O|172021.87|1997-06-17|3-MEDIUM|Clerk#000000457|0|ding requests. slyly ironic courts wake quickl|
+1637|73|F|180912.15|1995-02-08|4-NOT SPECIFIED|Clerk#000000189|0| final accounts. blithely silent ideas cajole bravely. carefully express |
+1638|139|O|172436.30|1997-08-13|2-HIGH|Clerk#000000643|0|he fluffily regular asymp|
+1639|5|O|104166.56|1995-08-20|4-NOT SPECIFIED|Clerk#000000939|0|haggle furiously. final requests detect furious|
+1664|64|O|178060.22|1996-03-03|1-URGENT|Clerk#000000090|0|y quickly even asymptotes. furiously regular packages haggle quickly fin|
+1665|76|F|4819.91|1994-05-08|2-HIGH|Clerk#000000920|0|ly regular packages are fluffily even ideas. fluffily final|
+1666|95|O|128367.97|1995-10-18|1-URGENT|Clerk#000000849|0|ffily pending dependencies wake fluffily. pending, final accounts |
+1667|5|O|125030.37|1997-10-10|2-HIGH|Clerk#000000103|0|e accounts. slyly express accounts must are a|
+1668|142|O|137576.19|1997-07-12|4-NOT SPECIFIED|Clerk#000000148|0|eodolites. carefully dogged dolphins haggle q|
+1669|2|O|24362.39|1997-06-09|3-MEDIUM|Clerk#000000663|0|er ironic requests detect furiously blithely sp|
+1670|25|O|89999.72|1997-05-24|2-HIGH|Clerk#000000320|0|unusual dependencies. furiously special platelets main|
+1671|35|O|104391.11|1996-07-27|4-NOT SPECIFIED|Clerk#000000275|0|ly. slyly pending requests was above the |
+1696|4|O|102665.03|1998-01-08|4-NOT SPECIFIED|Clerk#000000041|0|bravely bold accounts above the quickly bold|
+1697|76|O|122621.31|1996-10-07|1-URGENT|Clerk#000000815|0|o x-ray blithely. pl|
+1698|40|O|141118.87|1997-04-23|2-HIGH|Clerk#000000432|0|slyly. carefully express deposit|
+1699|85|F|66408.29|1993-12-30|1-URGENT|Clerk#000000125|0|jole blithely. furiously un|
+1700|65|O|89143.36|1996-06-15|3-MEDIUM|Clerk#000000328|0|ely final dolphins wake sometimes above the quietly regular deposits. fur|
+1701|130|F|72835.95|1992-05-19|2-HIGH|Clerk#000000395|0|furiously. regular, close theodoli|
+1702|67|P|194119.31|1995-05-07|2-HIGH|Clerk#000000300|0|around the carefully final deposits cajole carefully according to the b|
+1703|134|F|121220.59|1993-01-28|3-MEDIUM|Clerk#000000463|0| pinto beans poach. bold courts boost. regular, express deposits at|
+1728|64|O|131604.34|1996-05-22|2-HIGH|Clerk#000000711|0|beans. slyly regular instructions sleep! slyly final packages|
+1729|133|F|12137.76|1992-05-19|2-HIGH|Clerk#000000158|0|pending foxes wake. accounts|
+1730|124|O|150886.49|1998-07-24|5-LOW|Clerk#000000794|0| fluffily pending deposits serve. furiously even requests wake furiou|
+1731|128|O|190490.78|1996-01-06|1-URGENT|Clerk#000000268|0|lithely regular, final instructions. ironic, express packages are above|
+1732|146|F|179854.51|1993-11-29|5-LOW|Clerk#000000903|0|inal requests integrate dolph|
+1733|148|O|165489.52|1996-05-12|2-HIGH|Clerk#000000789|0|e carefully according to the accounts. furiously pending instructions sleep|
+1734|7|F|44002.53|1994-06-11|2-HIGH|Clerk#000000722|0| final ideas haggle. blithely quick foxes sleep busily bold ideas. i|
+1735|22|F|98541.95|1992-12-27|1-URGENT|Clerk#000000458|0|ully idle requests wake qu|
+1760|115|O|82151.12|1996-05-17|5-LOW|Clerk#000000917|0| deposits. busily regular deposits wake blithely along the furiously even re|
+1761|106|F|211925.95|1993-12-24|2-HIGH|Clerk#000000817|0|efully slyly bold frets. packages boost b|
+1762|77|F|202227.17|1994-08-20|4-NOT SPECIFIED|Clerk#000000653|0|ly ironic packages. furi|
+1763|121|O|140685.01|1996-10-29|2-HIGH|Clerk#000000321|0|es. bold dependencies haggle furiously along |
+1764|29|F|47384.71|1992-03-25|1-URGENT|Clerk#000000182|0|. slyly final packages integrate carefully acro|
+1765|73|O|36551.43|1995-12-03|4-NOT SPECIFIED|Clerk#000000490|0| regular excuses wake slyly|
+1766|139|O|41032.81|1996-10-12|2-HIGH|Clerk#000000983|0|unusual deposits affix quickly beyond the carefully s|
+1767|25|P|136582.60|1995-03-14|2-HIGH|Clerk#000000327|0|eposits use carefully carefully regular platelets. quickly regular packages al|
+1792|49|F|107919.86|1993-11-09|5-LOW|Clerk#000000102|0|ructions haggle along the pending packages. carefully speci|
+1793|19|F|82504.56|1992-07-12|4-NOT SPECIFIED|Clerk#000000291|0|regular packages cajole. blithely special packages according to the final d|
+1794|140|O|179462.21|1997-09-28|1-URGENT|Clerk#000000686|0|ally silent pinto beans. regular package|
+1795|94|F|146849.33|1994-03-19|2-HIGH|Clerk#000000815|0| quickly final packages! blithely dogged accounts c|
+1796|47|F|33755.47|1992-11-21|2-HIGH|Clerk#000000245|0|eans use furiously around th|
+1797|125|O|51494.47|1996-05-07|3-MEDIUM|Clerk#000000508|0|quiet platelets haggle since the quickly ironic instructi|
+1798|52|O|46393.97|1997-07-28|1-URGENT|Clerk#000000741|0|al foxes are blithe|
+1799|61|F|46815.93|1994-03-07|4-NOT SPECIFIED|Clerk#000000339|0|ns sleep furiously final waters. blithely regular instructions h|
+1824|49|F|81351.53|1994-05-05|1-URGENT|Clerk#000000972|0|e blithely fluffily|
+1825|148|F|150582.77|1993-12-05|3-MEDIUM|Clerk#000000345|0|ironic, final accou|
+1826|82|F|124719.97|1992-04-16|4-NOT SPECIFIED|Clerk#000000718|0|the even asymptotes dazzle fluffily slyly regular asymptotes. final, unu|
+1827|106|O|210113.88|1996-06-22|4-NOT SPECIFIED|Clerk#000000369|0|luffily even requests haggle sly|
+1828|32|F|137369.50|1994-04-18|3-MEDIUM|Clerk#000000840|0|y quickly bold packag|
+1829|112|F|127532.20|1994-05-08|2-HIGH|Clerk#000000537|0| accounts wake above the furiously unusual requests. pending package|
+1830|133|F|85122.24|1995-02-23|1-URGENT|Clerk#000000045|0|according to the even,|
+1831|71|F|58032.77|1993-12-02|1-URGENT|Clerk#000000854|0| accounts. carefully even accounts boost furiously. regular ideas engage. |
+1856|106|F|189361.42|1992-03-20|4-NOT SPECIFIED|Clerk#000000952|0|. special pinto beans run acr|
+1857|133|F|102793.59|1993-01-13|2-HIGH|Clerk#000000083|0|hely final ideas slee|
+1858|143|O|30457.91|1997-12-13|1-URGENT|Clerk#000000389|0|thely. slyly final deposits sleep|
+1859|61|O|105094.09|1997-04-11|4-NOT SPECIFIED|Clerk#000000949|0| the foxes. bravely special excuses nag carefully special r|
+1860|10|O|9103.40|1996-04-04|3-MEDIUM|Clerk#000000556|0|osits. quickly bold deposits according to |
+1861|70|F|95063.41|1994-01-03|3-MEDIUM|Clerk#000000847|0|r the fluffily close sauternes. furio|
+1862|34|O|97981.06|1998-02-24|5-LOW|Clerk#000000348|0|ts snooze ironically abou|
+1863|74|F|96359.65|1993-09-23|4-NOT SPECIFIED|Clerk#000000658|0|old sentiments. careful, |
+1888|121|F|224724.11|1993-10-31|4-NOT SPECIFIED|Clerk#000000659|0|olites. pinto beans cajole. regular deposits affix. slyly regular|
+1889|25|O|96431.77|1997-03-16|1-URGENT|Clerk#000000854|0|p around the regular notornis. unusual deposits|
+1890|10|O|202364.58|1996-12-18|4-NOT SPECIFIED|Clerk#000000627|0|romise final, regular deposits. regular fox|
+1891|61|F|76848.96|1994-12-15|5-LOW|Clerk#000000495|0|unusual foxes sleep regular deposits. requests wake special pac|
+1892|25|F|133273.64|1994-03-26|5-LOW|Clerk#000000733|0|sts. slyly regular dependencies use slyly. ironic, spec|
+1893|125|O|116792.13|1997-10-30|2-HIGH|Clerk#000000111|0|olites. silent, special deposits eat slyly quickly express packages; hockey p|
+1894|76|F|44387.23|1992-03-30|1-URGENT|Clerk#000000626|0|e furiously. furiously even accounts are slyly final accounts. closely speci|
+1895|7|F|44429.81|1994-05-30|3-MEDIUM|Clerk#000000878|0|ress accounts. bold accounts cajole. slyly final pinto beans poach regul|
+1920|110|O|119605.91|1998-06-24|5-LOW|Clerk#000000018|0|hely; furiously regular excuses|
+1921|88|F|57584.12|1994-01-18|3-MEDIUM|Clerk#000000293|0|counts. slyly quiet requests along the ruthlessly regular accounts are |
+1922|56|O|11575.77|1996-07-13|3-MEDIUM|Clerk#000000984|0|side of the blithely final re|
+1923|136|O|171128.10|1997-07-07|1-URGENT|Clerk#000000471|0| express dolphins. |
+1924|76|O|169756.19|1996-09-07|4-NOT SPECIFIED|Clerk#000000823|0| of the ironic accounts. instructions near the final instr|
+1925|17|F|146382.71|1992-03-05|1-URGENT|Clerk#000000986|0|e slyly regular deposits. furiously |
+1926|94|O|100035.03|1996-01-31|2-HIGH|Clerk#000000568|0|cajole. even warhorses sleep carefully. |
+1927|140|O|23327.88|1995-09-30|3-MEDIUM|Clerk#000000616|0|riously special packages. permanent pearls wake furiously. even packages alo|
+1952|67|F|12896.25|1994-03-16|2-HIGH|Clerk#000000254|0| silent accounts boost |
+1953|149|F|57213.18|1993-11-30|3-MEDIUM|Clerk#000000891|0| fluffily along the quickly even packages. |
+1954|56|O|158853.63|1997-05-31|4-NOT SPECIFIED|Clerk#000000104|0| unusual excuses cajole according to the blithely regular theodolites.|
+1955|13|F|103085.13|1992-04-20|1-URGENT|Clerk#000000792|0|ly special ideas. sometimes final |
+1956|127|F|88704.26|1992-09-20|4-NOT SPECIFIED|Clerk#000000600|0|ironic ideas are silent ideas. furiously final deposits sleep slyly carefu|
+1957|31|O|77482.87|1998-07-21|2-HIGH|Clerk#000000639|0|nding excuses about the |
+1958|53|O|176294.34|1995-09-22|5-LOW|Clerk#000000343|0| haggle blithely. flu|
+1959|43|O|62277.18|1997-01-13|4-NOT SPECIFIED|Clerk#000000631|0| cajole about the blithely express requests. even excuses mold bl|
+1984|52|O|79230.47|1998-04-01|1-URGENT|Clerk#000000416|0| slyly special instructions. unusual foxes use packages. carefully regular req|
+1985|7|F|171522.54|1994-09-02|4-NOT SPECIFIED|Clerk#000000741|0|slyly slyly even pains. slyly reg|
+1986|149|F|34269.96|1994-05-05|2-HIGH|Clerk#000000609|0|across the theodolites. quick|
+1987|100|F|6406.29|1994-04-30|2-HIGH|Clerk#000000652|0|gular platelets alongside |
+1988|109|O|117132.72|1995-10-06|4-NOT SPECIFIED|Clerk#000000011|0|ly ironic dolphins serve quickly busy accounts. bu|
+1989|118|F|39263.28|1994-03-16|4-NOT SPECIFIED|Clerk#000000747|0|ely bold pinto beans ha|
+1990|119|F|48781.39|1994-12-16|2-HIGH|Clerk#000000114|0|e bold patterns. always regul|
+1991|19|F|139854.41|1992-09-07|4-NOT SPECIFIED|Clerk#000000854|0|ing accounts can haggle at the carefully final Tiresias-- pending, regular|
+2016|8|O|24347.36|1996-08-16|3-MEDIUM|Clerk#000000641|0|the carefully ironic foxes. requests nag bold, r|
+2017|101|O|70529.27|1998-05-13|3-MEDIUM|Clerk#000000427|0|nusual requests. blit|
+2018|19|P|25007.95|1995-04-05|4-NOT SPECIFIED|Clerk#000000920|0|gular accounts wake fur|
+2019|136|F|43789.14|1992-10-23|1-URGENT|Clerk#000000565|0| furiously bold packages. fluffily fi|
+2020|73|F|136162.13|1993-06-21|3-MEDIUM|Clerk#000000192|0|es. furiously regular packages above the furiously special theodolites are a|
+2021|70|O|27016.74|1995-07-15|1-URGENT|Clerk#000000155|0|ong the furiously regular requests. unusual deposits wake fluffily inside|
+2022|62|F|206742.11|1992-03-15|1-URGENT|Clerk#000000268|0| dependencies sleep fluffily even, ironic deposits. express, silen|
+2023|118|F|144123.37|1992-05-06|5-LOW|Clerk#000000137|0|ular courts engage according to the|
+2048|17|F|33401.77|1993-11-15|1-URGENT|Clerk#000000934|0|s cajole after the blithely final accounts. f|
+2049|31|O|153048.74|1995-12-07|2-HIGH|Clerk#000000859|0|ly regular requests thrash blithely about the fluffily even theodolites. r|
+2050|28|F|208517.98|1994-06-02|4-NOT SPECIFIED|Clerk#000000821|0|d accounts against the furiously regular packages use bli|
+2051|40|O|87988.34|1996-03-18|4-NOT SPECIFIED|Clerk#000000333|0|ctions sleep blithely. blithely regu|
+2052|91|F|141822.19|1992-04-13|2-HIGH|Clerk#000000767|0| requests sleep around the even, even courts. ironic theodolites affix furious|
+2053|142|F|125125.57|1995-02-07|1-URGENT|Clerk#000000717|0|ar requests: blithely sly accounts boost carefully across t|
+2054|41|F|144335.16|1992-06-08|4-NOT SPECIFIED|Clerk#000000103|0|l requests affix carefully about the furiously special|
+2055|97|F|57092.26|1993-09-04|1-URGENT|Clerk#000000067|0|. warhorses affix slyly blithely express instructions? fur|
+2080|95|F|45767.69|1993-06-18|5-LOW|Clerk#000000190|0|ironic, pending theodolites are carefully about the quickly regular theodolite|
+2081|121|O|145654.97|1997-07-05|2-HIGH|Clerk#000000136|0|ong the regular theo|
+2082|49|F|46753.63|1995-01-10|2-HIGH|Clerk#000000354|0|cial accounts. ironic, express dolphins nod slyly sometimes final reques|
+2083|101|F|31795.52|1993-07-14|3-MEDIUM|Clerk#000000361|0|al patterns. bold, final foxes nag bravely about the furiously express|
+2084|80|F|190652.53|1993-03-17|2-HIGH|Clerk#000000048|0|zle furiously final, careful packages. slyly ironic ideas amo|
+2085|49|F|45311.07|1993-11-21|3-MEDIUM|Clerk#000000818|0|ress, express ideas haggle|
+2086|142|F|188985.18|1994-10-19|1-URGENT|Clerk#000000046|0| permanently regular|
+2087|50|O|53581.41|1998-01-31|2-HIGH|Clerk#000000626|0|e always regular packages nod against the furiously spec|
+2112|64|O|17986.15|1997-02-05|2-HIGH|Clerk#000000351|0|against the slyly even id|
+2113|32|O|65678.21|1997-11-08|2-HIGH|Clerk#000000527|0|slyly regular instruct|
+2114|79|F|106446.02|1995-01-16|5-LOW|Clerk#000000751|0|r, unusual accounts haggle across the busy platelets. carefully |
+2115|106|O|134814.65|1998-05-23|4-NOT SPECIFIED|Clerk#000000101|0|odolites boost. carefully regular excuses cajole. quickly ironic pinto be|
+2116|23|F|60887.90|1994-08-26|1-URGENT|Clerk#000000197|0|efully after the asymptotes. furiously sp|
+2117|22|O|145713.03|1997-04-26|2-HIGH|Clerk#000000887|0|ely even dependencies. regular foxes use blithely.|
+2118|134|O|38974.67|1996-10-09|1-URGENT|Clerk#000000196|0|ial requests wake carefully special packages. f|
+2119|64|O|34632.57|1996-08-20|2-HIGH|Clerk#000000434|0|uickly pending escapades. fluffily ir|
+2144|136|F|119917.28|1994-03-29|3-MEDIUM|Clerk#000000546|0|t. carefully quick requests across the deposits wake regu|
+2145|134|F|18885.35|1992-10-03|1-URGENT|Clerk#000000886|0|sts would snooze blithely alongside of th|
+2146|118|F|179686.07|1992-09-14|4-NOT SPECIFIED|Clerk#000000476|0|ven packages. dependencies wake slyl|
+2147|100|F|91513.79|1992-09-06|4-NOT SPECIFIED|Clerk#000000424|0| haggle carefully furiously final foxes. pending escapades thrash. bold theod|
+2148|130|F|19612.03|1995-04-19|4-NOT SPECIFIED|Clerk#000000517|0|ross the furiously unusual theodolites. always expre|
+2149|101|F|105145.40|1993-03-13|5-LOW|Clerk#000000555|0|nusual accounts nag furiously special reques|
+2150|82|F|166961.06|1994-06-03|3-MEDIUM|Clerk#000000154|0|ect slyly against the even, final packages. quickly regular pinto beans wake c|
+2151|58|O|124608.69|1996-11-11|3-MEDIUM|Clerk#000000996|0|c requests. ironic platelets cajole across the quickly fluffy deposits.|
+2176|104|F|87248.17|1992-11-10|1-URGENT|Clerk#000000195|0|s haggle regularly accor|
+2177|136|O|183493.42|1997-01-20|3-MEDIUM|Clerk#000000161|0|ove the blithely unusual packages cajole carefully fluffily special request|
+2178|8|O|79594.68|1996-12-12|3-MEDIUM|Clerk#000000656|0|thely according to the instructions. furious|
+2179|41|O|77487.09|1996-09-07|2-HIGH|Clerk#000000935|0|ounts alongside of the furiously unusual braids cajol|
+2180|76|O|208481.57|1996-09-14|4-NOT SPECIFIED|Clerk#000000650|0|xpress, unusual pains. furiously ironic excu|
+2181|76|O|100954.64|1995-09-13|3-MEDIUM|Clerk#000000814|0|y against the ironic, even|
+2182|23|F|116003.11|1994-04-05|2-HIGH|Clerk#000000071|0|ccounts. quickly bold deposits across the excuses sl|
+2183|113|O|49841.12|1996-06-22|1-URGENT|Clerk#000000287|0| among the express, ironic packages. slyly ironic platelets integrat|
+2208|68|P|245388.06|1995-05-01|4-NOT SPECIFIED|Clerk#000000900|0|symptotes wake slyly blithely unusual packages.|
+2209|91|F|129086.93|1992-07-10|2-HIGH|Clerk#000000056|0|er above the slyly silent requests. furiously reg|
+2210|32|F|31689.46|1992-01-16|2-HIGH|Clerk#000000941|0| believe carefully quickly express pinto beans. deposi|
+2211|92|F|140031.23|1994-06-30|2-HIGH|Clerk#000000464|0|ffily bold courts e|
+2212|118|F|17231.05|1994-03-23|3-MEDIUM|Clerk#000000954|0|structions above the unusual requests use fur|
+2213|122|F|146136.10|1993-01-15|4-NOT SPECIFIED|Clerk#000000598|0|osits are carefully reg|
+2214|115|O|150345.63|1998-05-05|3-MEDIUM|Clerk#000000253|0|packages. fluffily even accounts haggle blithely. carefully ironic depen|
+2215|40|O|108239.46|1996-06-16|4-NOT SPECIFIED|Clerk#000000817|0|le final, final foxes. quickly regular gifts are carefully deposit|
+2240|56|F|174090.30|1992-03-06|4-NOT SPECIFIED|Clerk#000000622|0|accounts against the slyly express foxes are after the slyly regular |
+2241|103|F|165219.08|1993-05-11|1-URGENT|Clerk#000000081|0|y about the silent excuses. furiously ironic instructions along the sil|
+2242|82|O|15082.82|1997-07-20|4-NOT SPECIFIED|Clerk#000000360|0| pending multipliers. carefully express asymptotes use quickl|
+2243|49|O|10451.97|1995-06-10|2-HIGH|Clerk#000000813|0|ously regular deposits integrate s|
+2244|127|F|21207.08|1993-01-09|1-URGENT|Clerk#000001000|0|ckages. ironic, ironic accounts haggle blithely express excuses. |
+2245|58|F|150585.73|1993-04-28|3-MEDIUM|Clerk#000000528|0|ake carefully. braids haggle slyly quickly b|
+2246|113|O|85755.84|1996-05-27|4-NOT SPECIFIED|Clerk#000000739|0| final gifts sleep |
+2247|95|F|13491.31|1992-08-02|4-NOT SPECIFIED|Clerk#000000947|0|furiously regular packages. final brai|
+2272|139|F|127934.71|1993-04-13|2-HIGH|Clerk#000000449|0|s. bold, ironic pinto beans wake. silently specia|
+2273|136|O|142291.79|1996-12-14|5-LOW|Clerk#000000155|0|uickly express foxes haggle quickly against|
+2274|104|F|58273.89|1993-09-04|4-NOT SPECIFIED|Clerk#000000258|0|nstructions try to hag|
+2275|149|F|37398.90|1992-10-22|4-NOT SPECIFIED|Clerk#000000206|0| furiously furious platelets. slyly final packa|
+2276|43|O|141159.63|1996-04-29|4-NOT SPECIFIED|Clerk#000000821|0|ecial requests. fox|
+2277|89|F|79270.23|1995-01-02|4-NOT SPECIFIED|Clerk#000000385|0|accounts cajole. even i|
+2278|142|O|101878.46|1998-04-25|3-MEDIUM|Clerk#000000186|0|r pinto beans integrate after the carefully even deposits. blit|
+2279|80|F|142322.33|1993-02-23|3-MEDIUM|Clerk#000000898|0|de of the quickly unusual instructio|
+2304|46|F|93769.28|1994-01-07|4-NOT SPECIFIED|Clerk#000000415|0|onic platelets. ironic packages haggle. packages nag doggedly according to|
+2305|43|F|122964.66|1993-01-26|2-HIGH|Clerk#000000440|0|ove the furiously even acco|
+2306|28|O|244704.23|1995-07-26|2-HIGH|Clerk#000000975|0| wake furiously requests. permanent requests affix. final packages caj|
+2307|106|F|59417.76|1993-06-29|5-LOW|Clerk#000000952|0|furiously even asymptotes? carefully regular accounts|
+2308|25|F|58546.02|1992-10-25|4-NOT SPECIFIED|Clerk#000000609|0|ts. slyly final depo|
+2309|100|O|146933.07|1995-09-04|5-LOW|Clerk#000000803|0|he carefully pending packages. fluffily stealthy foxes engage carefully|
+2310|31|O|82928.12|1996-09-20|5-LOW|Clerk#000000917|0|wake carefully. unusual instructions nag ironic, regular excuse|
+2311|73|P|153233.93|1995-05-02|2-HIGH|Clerk#000000761|0|ly pending asymptotes-- furiously bold excus|
+2336|142|O|22294.51|1996-01-07|4-NOT SPECIFIED|Clerk#000000902|0|c, final excuses sleep furiously among the even theodolites. f|
+2337|142|O|45704.96|1997-06-18|4-NOT SPECIFIED|Clerk#000000754|0| quickly. final accounts haggle. carefully final acco|
+2338|140|O|28155.92|1997-09-15|2-HIGH|Clerk#000000951|0|riously final dugouts. final, ironic packages wake express, ironic id|
+2339|109|F|63470.78|1993-12-15|5-LOW|Clerk#000000847|0| against the regular |
+2340|65|O|30778.78|1996-01-12|1-URGENT|Clerk#000000964|0|ter the deposits sleep according to the slyly regular packages. carefully |
+2341|82|F|55950.21|1993-05-30|5-LOW|Clerk#000000443|0|sts-- blithely bold dolphins through the deposits nag blithely carefully re|
+2342|37|O|104038.78|1996-06-09|1-URGENT|Clerk#000000615|0|oost carefully across the regular accounts. blithely final d|
+2343|73|O|85381.00|1995-08-21|3-MEDIUM|Clerk#000000170|0|fluffily over the slyly special deposits. quickl|
+2368|13|F|101240.96|1993-08-20|1-URGENT|Clerk#000000830|0|t the bold instructions. carefully unusual |
+2369|110|O|73517.91|1996-12-24|2-HIGH|Clerk#000000752|0|iously even requests are dogged, express |
+2370|142|F|73924.21|1994-01-17|1-URGENT|Clerk#000000231|0|lyly final packages. quickly final deposits haggl|
+2371|19|O|193857.67|1998-01-07|1-URGENT|Clerk#000000028|0|ckages haggle at th|
+2372|31|O|104927.66|1997-11-21|5-LOW|Clerk#000000342|0|s: deposits haggle along the final ideas. careful|
+2373|28|F|55211.04|1994-03-12|4-NOT SPECIFIED|Clerk#000000306|0| even, special courts grow quickly. pending,|
+2374|4|F|115219.88|1993-10-29|4-NOT SPECIFIED|Clerk#000000081|0| blithely regular packages. blithely unusua|
+2375|5|O|106612.48|1996-11-20|3-MEDIUM|Clerk#000000197|0|unusual, pending theodolites cajole carefully |
+2400|37|O|92798.66|1998-07-25|5-LOW|Clerk#000000782|0|nusual courts nag against the carefully unusual pinto b|
+2401|148|O|88448.24|1997-07-29|4-NOT SPECIFIED|Clerk#000000531|0|ully unusual instructions boost carefully silently regular requests. |
+2402|67|O|70403.62|1996-09-06|4-NOT SPECIFIED|Clerk#000000162|0|slyly final sheaves sleep slyly. q|
+2403|55|O|111020.79|1998-04-11|3-MEDIUM|Clerk#000000820|0|furiously regular deposits use. furiously unusual accounts wake along the |
+2404|77|O|109077.69|1997-03-13|4-NOT SPECIFIED|Clerk#000000409|0|deposits breach furiously. ironic foxes haggle carefully bold packag|
+2405|73|O|115929.14|1996-12-23|3-MEDIUM|Clerk#000000535|0|ular, regular asympto|
+2406|7|O|182516.77|1996-10-28|5-LOW|Clerk#000000561|0|blithely regular accounts u|
+2407|55|O|112843.52|1998-06-19|2-HIGH|Clerk#000000068|0|uests affix slyly among the slyly regular depos|
+2432|103|O|62661.93|1996-07-13|1-URGENT|Clerk#000000115|0|re. slyly even deposits wake bra|
+2433|31|F|147071.86|1994-08-22|4-NOT SPECIFIED|Clerk#000000324|0|ess patterns are slyly. packages haggle carefu|
+2434|25|O|123956.25|1997-04-27|3-MEDIUM|Clerk#000000190|0|s. quickly ironic dolphins impress final deposits. blithel|
+2435|73|F|122490.66|1993-02-21|5-LOW|Clerk#000000112|0|es are carefully along the carefully final instructions. pe|
+2436|125|O|73990.08|1995-09-11|4-NOT SPECIFIED|Clerk#000000549|0|arefully. blithely bold deposits affix special accounts. final foxes nag. spe|
+2437|85|F|143411.69|1993-04-21|4-NOT SPECIFIED|Clerk#000000578|0|. theodolites wake slyly-- ironic, pending platelets above the carefully exp|
+2438|13|F|214494.39|1993-07-15|2-HIGH|Clerk#000000744|0|the final, regular warhorses. regularly |
+2439|55|O|41811.12|1997-03-15|2-HIGH|Clerk#000000819|0|lithely after the car|
+2464|145|O|30495.65|1997-11-23|5-LOW|Clerk#000000633|0|le about the instructions. courts wake carefully even|
+2465|34|O|180737.75|1995-06-24|1-URGENT|Clerk#000000078|0|al pinto beans. final, bold packages wake quickly|
+2466|19|F|161625.50|1994-03-06|1-URGENT|Clerk#000000424|0|c pinto beans. express deposits wake quickly. even, final courts nag. package|
+2467|35|O|7231.91|1995-07-16|4-NOT SPECIFIED|Clerk#000000914|0|pades sleep furiously. sometimes regular packages again|
+2468|112|O|160627.01|1997-06-09|4-NOT SPECIFIED|Clerk#000000260|0|ickly regular packages. slyly ruthless requests snooze quickly blithe|
+2469|124|O|192074.23|1996-11-26|5-LOW|Clerk#000000730|0| sleep closely regular instructions. furiously ironic instructi|
+2470|58|O|104966.33|1997-04-19|3-MEDIUM|Clerk#000000452|0|to the furiously final packages? pa|
+2471|89|O|34936.31|1998-03-12|4-NOT SPECIFIED|Clerk#000000860|0|carefully blithely regular pac|
+2496|136|F|140390.60|1994-01-09|2-HIGH|Clerk#000000142|0|slyly. pending instructions sleep. quic|
+2497|47|F|171326.48|1992-08-27|1-URGENT|Clerk#000000977|0|ily ironic pinto beans. furiously final platelets alongside of t|
+2498|97|F|45514.27|1993-11-08|5-LOW|Clerk#000000373|0|g the slyly special pinto beans. |
+2499|121|O|147243.86|1995-09-24|1-URGENT|Clerk#000000277|0|r the quickly bold foxes. bold instructi|
+2500|133|F|131122.82|1992-08-15|2-HIGH|Clerk#000000447|0|integrate slyly pending deposits. furiously ironic accounts across the s|
+2501|67|O|79380.51|1997-05-25|5-LOW|Clerk#000000144|0|ickly special theodolite|
+2502|70|F|33470.40|1993-05-28|4-NOT SPECIFIED|Clerk#000000914|0|lyly: carefully pending ideas affix again|
+2503|7|F|183671.08|1993-06-20|3-MEDIUM|Clerk#000000294|0|ly even packages was. ironic, regular deposits unwind furiously across the p|
+2528|55|F|92069.62|1994-11-20|1-URGENT|Clerk#000000789|0|ular dependencies? regular frays kindle according to the blith|
+2529|136|O|4104.30|1996-08-20|2-HIGH|Clerk#000000511|0|posits across the silent instructions wake blithely across |
+2530|128|F|58853.11|1994-03-21|3-MEDIUM|Clerk#000000291|0|ular instructions about the quic|
+2531|44|O|143212.85|1996-05-06|4-NOT SPECIFIED|Clerk#000000095|0|even accounts. furiously ironic excuses sleep fluffily. carefully silen|
+2532|94|O|116093.49|1995-10-11|2-HIGH|Clerk#000000498|0|the blithely pending accounts. regular, regular excuses boost aro|
+2533|50|O|168495.03|1997-03-24|1-URGENT|Clerk#000000594|0|ecial instructions. spec|
+2534|76|O|202784.54|1996-07-17|3-MEDIUM|Clerk#000000332|0|packages cajole ironic requests. furiously regular|
+2535|121|F|67018.30|1993-05-25|5-LOW|Clerk#000000296|0|phins cajole beneath the fluffily express asymptotes. c|
+2560|131|F|153426.79|1992-09-05|1-URGENT|Clerk#000000538|0|atelets; quickly sly requests|
+2561|58|O|137473.58|1997-11-14|1-URGENT|Clerk#000000861|0|ual requests. unusual deposits cajole furiously pending, regular platelets. |
+2562|10|F|136360.37|1992-08-01|1-URGENT|Clerk#000000467|0|elets. pending dolphins promise slyly. bo|
+2563|62|F|168952.10|1993-11-19|4-NOT SPECIFIED|Clerk#000000150|0|sly even packages after the furio|
+2564|77|F|3967.47|1994-09-09|2-HIGH|Clerk#000000718|0|usly regular pinto beans. orbits wake carefully. slyly e|
+2565|56|O|204438.57|1998-02-28|3-MEDIUM|Clerk#000000032|0|x-ray blithely along|
+2566|86|F|89992.48|1992-10-10|3-MEDIUM|Clerk#000000414|0|ructions boost bold ideas. idly ironic accounts use according to th|
+2567|70|O|263411.29|1998-02-27|2-HIGH|Clerk#000000031|0|detect. furiously ironic requests|
+2592|101|F|8225.96|1993-03-05|4-NOT SPECIFIED|Clerk#000000524|0|ts nag fluffily. quickly stealthy theodolite|
+2593|92|F|134726.09|1993-09-04|2-HIGH|Clerk#000000468|0|r the carefully final|
+2594|79|F|94866.39|1992-12-17|1-URGENT|Clerk#000000550|0|ests. theodolites above the blithely even accounts detect furio|
+2595|74|O|173130.20|1995-12-14|4-NOT SPECIFIED|Clerk#000000222|0|arefully ironic requests nag carefully ideas. |
+2596|43|O|74940.13|1996-08-17|1-URGENT|Clerk#000000242|0|requests. ironic, bold theodolites wak|
+2597|104|F|21964.66|1993-02-04|2-HIGH|Clerk#000000757|0|iously ruthless exc|
+2598|112|O|84871.50|1996-03-05|3-MEDIUM|Clerk#000000391|0| ironic notornis according to the blithely final requests should |
+2599|149|O|62807.13|1996-11-07|2-HIGH|Clerk#000000722|0|ts. slyly regular theodolites wake sil|
+2624|52|O|27148.63|1996-11-28|5-LOW|Clerk#000000930|0|ic, regular packages|
+2625|40|F|39382.74|1992-10-14|4-NOT SPECIFIED|Clerk#000000386|0| final deposits. blithely ironic ideas |
+2626|139|O|84314.51|1995-09-08|4-NOT SPECIFIED|Clerk#000000289|0|gside of the carefully special packages are furiously after the slyly express |
+2627|149|F|26798.65|1992-03-24|3-MEDIUM|Clerk#000000181|0|s. silent, ruthless requests|
+2628|56|F|165655.99|1993-10-22|5-LOW|Clerk#000000836|0|ajole across the blithely careful accounts. blithely silent deposits sl|
+2629|139|O|96458.03|1998-04-06|5-LOW|Clerk#000000680|0|uches dazzle carefully even, express excuses. ac|
+2630|85|F|127132.51|1992-10-24|5-LOW|Clerk#000000712|0|inal theodolites. ironic instructions s|
+2631|37|F|63103.32|1993-09-24|5-LOW|Clerk#000000833|0| quickly unusual deposits doubt around |
+2656|77|F|105492.37|1993-05-04|1-URGENT|Clerk#000000307|0|elets. slyly final accou|
+2657|25|O|148176.06|1995-10-17|2-HIGH|Clerk#000000160|0| foxes-- slyly final dependencies around the slyly final theodo|
+2658|14|O|163834.46|1995-09-23|3-MEDIUM|Clerk#000000400|0|bout the slyly regular accounts. ironic, |
+2659|83|F|79785.52|1993-12-18|4-NOT SPECIFIED|Clerk#000000758|0|cross the pending requests maintain |
+2660|127|O|16922.51|1995-08-05|5-LOW|Clerk#000000480|0|ly finally regular deposits. ironic theodolites cajole|
+2661|74|O|106036.84|1997-01-04|3-MEDIUM|Clerk#000000217|0|al, regular pinto beans. silently final deposits should have t|
+2662|37|O|87689.88|1996-08-21|3-MEDIUM|Clerk#000000589|0|bold pinto beans above the slyly final accounts affix furiously deposits. pac|
+2663|95|O|35131.80|1995-09-06|1-URGENT|Clerk#000000950|0|ar requests. furiously final dolphins along the fluffily spe|
+2688|98|F|181077.36|1992-01-24|2-HIGH|Clerk#000000720|0|have to nag according to the pending theodolites. sly|
+2689|103|F|41552.78|1992-04-09|4-NOT SPECIFIED|Clerk#000000698|0|press pains wake. furiously express theodolites alongsid|
+2690|94|O|224674.27|1996-03-31|3-MEDIUM|Clerk#000000760|0|ravely even theodolites |
+2691|7|F|30137.17|1992-04-30|5-LOW|Clerk#000000439|0|es at the regular deposits sleep slyly by the fluffy requests. eve|
+2692|62|O|24265.24|1997-12-02|3-MEDIUM|Clerk#000000878|0|es. regular asymptotes cajole above t|
+2693|19|O|66158.13|1996-09-04|1-URGENT|Clerk#000000370|0|ndle never. blithely regular packages nag carefully enticing platelets. ca|
+2694|121|O|102807.59|1996-03-14|5-LOW|Clerk#000000722|0| requests. bold deposits above the theodol|
+2695|58|O|138584.20|1996-08-20|1-URGENT|Clerk#000000697|0|ven deposits around the quickly regular packa|
+2720|31|F|161307.05|1993-06-08|1-URGENT|Clerk#000000948|0|quickly. special asymptotes are fluffily ironi|
+2721|79|O|59180.25|1996-01-27|2-HIGH|Clerk#000000401|0| ideas eat even, unusual ideas. theodolites are carefully|
+2722|35|F|50328.84|1994-04-09|5-LOW|Clerk#000000638|0|rding to the carefully quick deposits. bli|
+2723|61|O|104759.25|1995-10-06|5-LOW|Clerk#000000836|0|nts must have to cajo|
+2724|137|F|116069.66|1994-09-14|2-HIGH|Clerk#000000217|0| sleep blithely. blithely idle |
+2725|89|F|75144.68|1994-05-21|4-NOT SPECIFIED|Clerk#000000835|0|ular deposits. spec|
+2726|7|F|47753.00|1992-11-27|5-LOW|Clerk#000000470|0| blithely even dinos sleep care|
+2727|74|O|3089.42|1998-04-19|4-NOT SPECIFIED|Clerk#000000879|0|sual theodolites cajole enticingly above the furiously fin|
+2752|59|F|187932.30|1993-11-19|2-HIGH|Clerk#000000648|0| carefully regular foxes are quickly quickl|
+2753|16|F|159720.39|1993-11-30|2-HIGH|Clerk#000000380|0|ending instructions. unusual deposits|
+2754|145|F|25985.52|1994-04-03|2-HIGH|Clerk#000000960|0|cies detect slyly. |
+2755|118|F|101202.18|1992-02-07|4-NOT SPECIFIED|Clerk#000000177|0|ously according to the sly foxes. blithely regular pinto bean|
+2756|118|F|142323.38|1994-04-18|1-URGENT|Clerk#000000537|0|arefully special warho|
+2757|76|O|89792.48|1995-07-20|2-HIGH|Clerk#000000216|0| regular requests subl|
+2758|43|O|36671.88|1998-07-12|5-LOW|Clerk#000000863|0|s cajole according to the carefully special |
+2759|116|F|89731.10|1993-11-25|4-NOT SPECIFIED|Clerk#000000071|0|ts. regular, pending pinto beans sleep ab|
+2784|95|O|106635.21|1998-01-07|1-URGENT|Clerk#000000540|0|g deposits alongside of the silent requests s|
+2785|148|O|132854.79|1995-07-21|2-HIGH|Clerk#000000098|0|iously pending packages sleep according to the blithely unusual foxe|
+2786|79|F|178254.66|1992-03-22|2-HIGH|Clerk#000000976|0|al platelets cajole blithely ironic requests. ironic re|
+2787|103|O|3726.14|1995-09-30|1-URGENT|Clerk#000000906|0|he ironic, regular |
+2788|124|F|17172.66|1994-09-22|1-URGENT|Clerk#000000641|0|nts wake across the fluffily bold accoun|
+2789|37|O|219123.27|1998-03-14|2-HIGH|Clerk#000000972|0|gular patterns boost. carefully even re|
+2790|25|F|177458.97|1994-08-19|2-HIGH|Clerk#000000679|0| the carefully express deposits sleep slyly |
+2791|121|F|156697.55|1994-10-10|2-HIGH|Clerk#000000662|0|as. slyly ironic accounts play furiously bl|
+2816|58|F|42225.53|1994-09-20|2-HIGH|Clerk#000000289|0|kages at the final deposits cajole furious foxes. quickly |
+2817|40|F|71453.85|1994-04-19|3-MEDIUM|Clerk#000000982|0|ic foxes haggle upon the daringly even pinto beans. slyly|
+2818|49|F|120086.84|1994-12-12|3-MEDIUM|Clerk#000000413|0|eep furiously special ideas. express |
+2819|103|F|66927.16|1994-05-05|1-URGENT|Clerk#000000769|0|ngside of the blithely ironic dolphins. furio|
+2820|19|F|143813.39|1994-05-20|3-MEDIUM|Clerk#000000807|0|equests are furiously. carefu|
+2821|118|F|36592.48|1993-08-09|3-MEDIUM|Clerk#000000323|0|ng requests. even instructions are quickly express, silent instructi|
+2822|79|F|40142.15|1993-07-26|2-HIGH|Clerk#000000510|0|furiously against the accounts. unusual accounts aft|
+2823|79|O|171894.45|1995-09-09|2-HIGH|Clerk#000000567|0|encies. carefully fluffy accounts m|
+2848|70|F|116258.53|1992-03-10|1-URGENT|Clerk#000000256|0|ly fluffy foxes sleep furiously across the slyly regu|
+2849|46|O|180054.29|1996-04-30|2-HIGH|Clerk#000000659|0|al packages are after the quickly bold requests. carefully special |
+2850|100|O|122969.79|1996-10-02|2-HIGH|Clerk#000000392|0|, regular deposits. furiously pending packages hinder carefully carefully u|
+2851|145|O|7859.36|1997-09-07|5-LOW|Clerk#000000566|0|Tiresias wake quickly quickly even|
+2852|91|F|99050.81|1993-01-16|1-URGENT|Clerk#000000740|0|ruthless deposits against the final instructions use quickly al|
+2853|94|F|103641.15|1994-05-05|2-HIGH|Clerk#000000878|0|the carefully even packages.|
+2854|139|F|153568.02|1994-06-27|1-URGENT|Clerk#000000010|0| furiously ironic tithes use furiously |
+2855|49|F|48419.58|1993-04-04|4-NOT SPECIFIED|Clerk#000000973|0| silent, regular packages sleep |
+2880|8|F|145761.99|1992-03-15|2-HIGH|Clerk#000000756|0|ves maintain doggedly spec|
+2881|100|F|45695.84|1992-05-10|5-LOW|Clerk#000000864|0|uriously. slyly express requests according to the silent dol|
+2882|121|O|172872.37|1995-08-22|2-HIGH|Clerk#000000891|0|pending deposits. carefully eve|
+2883|121|F|170360.27|1995-01-23|5-LOW|Clerk#000000180|0|uses. carefully ironic accounts lose fluffil|
+2884|92|O|71683.84|1997-10-12|3-MEDIUM|Clerk#000000780|0|efully express instructions sleep against|
+2885|7|F|146896.72|1992-09-19|4-NOT SPECIFIED|Clerk#000000280|0|ly sometimes special excuses. final requests are |
+2886|109|F|94527.23|1994-11-13|4-NOT SPECIFIED|Clerk#000000619|0|uctions. ironic packages sle|
+2887|109|O|28571.39|1997-05-26|5-LOW|Clerk#000000566|0|slyly even pinto beans. slyly bold epitaphs cajole blithely above t|
+2912|94|F|27727.52|1992-03-12|5-LOW|Clerk#000000186|0|jole blithely above the quickly regular packages. carefully regular pinto bean|
+2913|43|O|130702.19|1997-07-12|3-MEDIUM|Clerk#000000118|0|mptotes doubt furiously slyly regu|
+2914|109|F|60867.14|1993-03-03|3-MEDIUM|Clerk#000000543|0|he slyly regular theodolites are furiously sile|
+2915|94|F|96015.13|1994-03-31|5-LOW|Clerk#000000410|0|ld packages. bold deposits boost blithely. ironic, unusual theodoli|
+2916|8|O|20182.22|1995-12-27|2-HIGH|Clerk#000000681|0|ithely blithe deposits sleep beyond the|
+2917|91|O|100714.13|1997-12-09|4-NOT SPECIFIED|Clerk#000000061|0| special dugouts among the special deposi|
+2918|118|O|21760.09|1996-09-08|3-MEDIUM|Clerk#000000439|0|ular deposits across th|
+2919|53|F|137223.14|1993-12-10|2-HIGH|Clerk#000000209|0|es. pearls wake quietly slyly ironic instructions--|
+2944|14|O|146581.14|1997-09-24|4-NOT SPECIFIED|Clerk#000000740|0|deas. permanently special foxes haggle carefully ab|
+2945|29|O|223507.72|1996-01-03|2-HIGH|Clerk#000000499|0|ons are carefully toward the permanent, bold pinto beans. regu|
+2946|125|O|102226.59|1996-02-05|5-LOW|Clerk#000000329|0|g instructions about the regular accounts sleep carefully along the pen|
+2947|70|P|43360.95|1995-04-26|1-URGENT|Clerk#000000464|0|ronic accounts. accounts run furiously d|
+2948|44|F|100758.71|1994-08-23|5-LOW|Clerk#000000701|0| deposits according to the blithely pending |
+2949|137|F|94231.71|1994-04-12|2-HIGH|Clerk#000000184|0|y ironic accounts use. quickly blithe accou|
+2950|136|O|183620.33|1997-07-06|1-URGENT|Clerk#000000833|0| dolphins around the furiously |
+2951|74|O|125509.17|1996-02-06|2-HIGH|Clerk#000000680|0|gular deposits above the finally regular ideas integrate idly stealthil|
+2976|29|F|145768.47|1993-12-10|4-NOT SPECIFIED|Clerk#000000159|0|. furiously ironic asymptotes haggle ruthlessly silently regular r|
+2977|73|O|25170.88|1996-08-27|3-MEDIUM|Clerk#000000252|0|quickly special platelets are furio|
+2978|44|P|139542.14|1995-05-03|1-URGENT|Clerk#000000135|0|d. even platelets are. ironic dependencies cajole slow, e|
+2979|133|O|116789.98|1996-03-23|3-MEDIUM|Clerk#000000820|0|even, ironic foxes sleep along|
+2980|4|O|187514.11|1996-09-14|3-MEDIUM|Clerk#000000661|0|y quick pinto beans wake. slyly re|
+2981|49|O|37776.79|1998-07-29|5-LOW|Clerk#000000299|0|hely among the express foxes. blithely stealthy requests cajole boldly. regu|
+2982|85|F|55582.94|1995-03-19|2-HIGH|Clerk#000000402|0|lyly. express theodolites affix slyly after the slyly speci|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders-part2.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders-part2.tbl
new file mode 100644
index 0000000..7e35045
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders-part2.tbl
@@ -0,0 +1,750 @@
+2983|62|F|58168.07|1992-01-07|1-URGENT|Clerk#000000278|0|r the even requests. accounts maintain. regular accounts|
+3008|40|O|156018.74|1995-11-08|3-MEDIUM|Clerk#000000701|0|ze quickly. blithely regular packages above the slyly bold foxes shall|
+3009|55|O|108424.94|1997-02-28|1-URGENT|Clerk#000000205|0|r ideas. carefully pe|
+3010|8|O|141647.08|1996-01-26|2-HIGH|Clerk#000000931|0| blithely final requests. special deposits are slyl|
+3011|91|F|46418.85|1992-01-14|5-LOW|Clerk#000000515|0|onic deposits kindle slyly. dependencies around the quickly iro|
+3012|32|F|91678.66|1993-05-05|1-URGENT|Clerk#000000414|0|ts after the regular pinto beans impress blithely s|
+3013|143|O|156407.40|1997-02-05|5-LOW|Clerk#000000591|0|the furiously pendin|
+3014|29|F|194159.59|1992-10-30|4-NOT SPECIFIED|Clerk#000000476|0|ep blithely according to the blith|
+3015|103|F|110826.83|1992-09-27|5-LOW|Clerk#000000013|0|ously regular deposits affix carefully. furiousl|
+3040|112|F|119201.64|1993-04-12|3-MEDIUM|Clerk#000000544|0|carefully special packages. blithe|
+3041|113|O|23039.46|1997-06-03|5-LOW|Clerk#000000092|0|s. unusual, pending deposits use carefully. thinly final|
+3042|20|F|104523.03|1994-11-21|3-MEDIUM|Clerk#000000573|0| the slyly ironic depo|
+3043|44|F|78221.69|1992-04-25|5-LOW|Clerk#000000137|0|cajole blithely furiously fina|
+3044|53|O|52433.54|1996-04-03|2-HIGH|Clerk#000000008|0|cajole final courts. ironic deposits about the quickly final re|
+3045|50|O|85822.67|1995-09-27|1-URGENT|Clerk#000000405|0| express courts sleep quickly special asymptotes. |
+3046|32|O|117817.52|1995-11-30|2-HIGH|Clerk#000000522|0|r deposits. platelets use furi|
+3047|25|O|37881.31|1997-03-21|1-URGENT|Clerk#000000962|0|as. slyly express deposits are dogged pearls. silent ide|
+3072|23|F|87475.82|1994-01-30|4-NOT SPECIFIED|Clerk#000000370|0|ely final deposits cajole carefully. ironic, re|
+3073|136|F|151419.50|1994-01-08|3-MEDIUM|Clerk#000000404|0|kly slyly bold accounts. express courts near the regular ideas sleep bli|
+3074|67|F|85861.93|1992-11-01|5-LOW|Clerk#000000546|0|yly even asymptotes shall have to haggle fluffily. deposits are|
+3075|127|F|37696.70|1994-05-07|3-MEDIUM|Clerk#000000433|0|ackages: carefully unusual reques|
+3076|92|F|93828.15|1993-07-23|2-HIGH|Clerk#000000099|0|busy foxes. deposits affix quickly ironic, pending pint|
+3077|121|O|99290.01|1997-08-06|2-HIGH|Clerk#000000228|0|kly. fluffily ironic requests use qui|
+3078|49|F|46310.83|1993-02-12|2-HIGH|Clerk#000000110|0|ounts are alongside of the blith|
+3079|100|O|148299.05|1997-09-12|5-LOW|Clerk#000000505|0|lly ironic accounts|
+3104|70|F|102693.61|1993-09-16|3-MEDIUM|Clerk#000000871|0|ges boost-- regular accounts are furiousl|
+3105|137|O|125396.80|1996-11-13|4-NOT SPECIFIED|Clerk#000000772|0|s. blithely final ins|
+3106|145|O|132494.97|1997-01-12|3-MEDIUM|Clerk#000000729|0|its use slyly final theodolites; regular dolphins hang above t|
+3107|26|O|107406.26|1997-08-21|1-URGENT|Clerk#000000669|0|ously even deposits acr|
+3108|85|F|63278.00|1993-08-05|1-URGENT|Clerk#000000574|0|s packages haggle furiously am|
+3109|124|F|216104.85|1993-07-24|5-LOW|Clerk#000000936|0|bold requests sleep quickly according to the slyly final|
+3110|88|F|115161.29|1994-12-17|2-HIGH|Clerk#000000564|0|round the fluffy instructions. carefully silent packages cajol|
+3111|133|O|154383.37|1995-08-25|5-LOW|Clerk#000000922|0|slyly regular theodolites. furious deposits cajole deposits. ironic theodoli|
+3136|23|F|145426.11|1994-08-10|4-NOT SPECIFIED|Clerk#000000891|0|tructions sleep slyly. pending di|
+3137|136|O|8958.65|1995-07-26|3-MEDIUM|Clerk#000000063|0|ymptotes wake carefully above t|
+3138|139|F|139579.18|1994-02-09|4-NOT SPECIFIED|Clerk#000000650|0|e fluffily final theodolites. even dependencies wake along the quickly ir|
+3139|17|F|40975.96|1992-01-02|3-MEDIUM|Clerk#000000855|0|ounts against the ruthlessly unusual dolphins|
+3140|145|F|54356.10|1992-04-09|1-URGENT|Clerk#000000670|0|carefully ironic deposits use furiously. blith|
+3141|26|O|115959.96|1995-11-10|1-URGENT|Clerk#000000475|0|es. furiously bold instructions after the carefully final p|
+3142|8|F|16030.15|1992-06-28|3-MEDIUM|Clerk#000000043|0|usual accounts about the carefully special requests sleep slyly quickly regul|
+3143|107|F|135647.68|1993-02-17|1-URGENT|Clerk#000000519|0| are final, ironic accounts. ironic |
+3168|136|F|69412.71|1992-01-30|5-LOW|Clerk#000000352|0|s sleep slyly? ironic, furious instructions detect. quickly final i|
+3169|19|F|126804.90|1993-12-21|3-MEDIUM|Clerk#000000252|0| even pinto beans are blithely special, special multip|
+3170|5|O|190142.17|1997-11-09|1-URGENT|Clerk#000000288|0|requests. furiously bold|
+3171|47|F|84405.78|1993-04-06|5-LOW|Clerk#000000940|0|ar deposits. idly r|
+3172|89|F|121360.83|1992-06-03|4-NOT SPECIFIED|Clerk#000000771|0|es. slyly ironic packages x-ra|
+3173|148|O|64892.73|1996-08-10|5-LOW|Clerk#000000516|0|ial requests lose along t|
+3174|127|O|92856.91|1995-11-15|5-LOW|Clerk#000000663|0|rts. silent, regular pinto beans are blithely regular packages. furiousl|
+3175|44|F|205282.63|1994-07-15|5-LOW|Clerk#000000629|0| across the slyly even realms use carefully ironic deposits: sl|
+3200|13|O|131103.31|1996-02-07|1-URGENT|Clerk#000000020|0| regular dependencies impress evenly even excuses. blithely |
+3201|97|F|90755.31|1993-07-02|4-NOT SPECIFIED|Clerk#000000738|0|. busy, express instruction|
+3202|88|F|50601.01|1992-12-24|5-LOW|Clerk#000000067|0|fluffily express requests affix carefully around th|
+3203|127|O|49357.72|1997-11-05|2-HIGH|Clerk#000000493|0|e furiously silent warhorses. slyly silent deposits wake bli|
+3204|10|F|41573.42|1992-12-26|1-URGENT|Clerk#000000693|0|ess somas cajole slyly. pending accounts cajole|
+3205|148|F|153637.79|1992-04-11|5-LOW|Clerk#000000803|0|e furiously. quickly regular dinos about the final pinto be|
+3206|122|O|64344.86|1996-08-09|1-URGENT|Clerk#000000755|0|ntegrate furiously final, express |
+3207|22|O|133038.59|1998-02-16|1-URGENT|Clerk#000000695|0|uriously accounts. fluffily i|
+3232|82|F|55619.01|1992-10-09|1-URGENT|Clerk#000000314|0|yly final accounts. packages agains|
+3233|140|F|54121.92|1994-10-24|5-LOW|Clerk#000000470|0|ly ironic epitaphs use stealthy, express deposits. quickly regular instruct|
+3234|14|O|147343.68|1996-04-05|4-NOT SPECIFIED|Clerk#000000367|0|ents according to the dependencies will sleep after the blithely even p|
+3235|46|O|104695.09|1995-11-15|5-LOW|Clerk#000000349|0| quickly pinto beans. ironi|
+3236|142|O|39470.39|1996-11-06|4-NOT SPECIFIED|Clerk#000000553|0|ithely slyly pending req|
+3237|19|F|10508.12|1992-06-03|1-URGENT|Clerk#000000606|0|inal requests. slyly even foxes detect about the furiously exp|
+3238|61|F|41375.69|1993-02-21|5-LOW|Clerk#000000818|0|lly express deposits are. furiously unusual ideas wake carefully somas. instr|
+3239|35|O|156802.80|1998-01-12|4-NOT SPECIFIED|Clerk#000000619|0| cajole carefully along the furiously pending deposits. |
+3264|94|O|162634.53|1996-11-02|5-LOW|Clerk#000000244|0|carefully. express, bold|
+3265|53|F|43315.15|1992-06-27|1-URGENT|Clerk#000000265|0|re quickly quickly pe|
+3266|4|P|68309.28|1995-03-17|5-LOW|Clerk#000000545|0|refully ironic instructions. slyly final pi|
+3267|112|O|33998.90|1997-01-07|5-LOW|Clerk#000000484|0| the packages. regular decoys about the bold dependencies grow fi|
+3268|142|F|36024.96|1994-06-25|5-LOW|Clerk#000000746|0|y brave requests unwind furiously accordin|
+3269|17|O|218697.85|1996-03-01|3-MEDIUM|Clerk#000000378|0|ts. accounts wake carefully. carefully dogged accounts wake slyly slyly i|
+3270|38|O|166669.86|1997-05-28|1-URGENT|Clerk#000000375|0|uffily pending courts ca|
+3271|34|F|86534.05|1992-01-01|1-URGENT|Clerk#000000421|0|s. furiously regular requests|
+3296|148|F|187553.35|1994-10-19|3-MEDIUM|Clerk#000000991|0|as! carefully final requests wake. furiously even|
+3297|139|F|9679.45|1992-11-03|2-HIGH|Clerk#000000220|0| after the theodolites cajole carefully according to the finally|
+3298|116|O|62716.67|1996-04-17|5-LOW|Clerk#000000241|0|even accounts boost |
+3299|91|F|42867.92|1993-12-26|3-MEDIUM|Clerk#000000853|0|bold deposits. special instructions sleep care|
+3300|118|O|27049.22|1995-07-15|5-LOW|Clerk#000000198|0|ses. carefully unusual instructions must have to detect about the blithel|
+3301|133|F|48497.09|1994-09-04|4-NOT SPECIFIED|Clerk#000000325|0|ular gifts impress enticingly carefully express deposits; instructions boo|
+3302|34|O|38330.42|1995-11-14|2-HIGH|Clerk#000000367|0|eep blithely ironic requests. quickly even courts haggle slyly|
+3303|145|O|97758.28|1997-12-14|4-NOT SPECIFIED|Clerk#000000661|0|nto beans sleep furiously above the carefully ironic |
+3328|7|F|139580.85|1992-11-19|5-LOW|Clerk#000000384|0|ake among the express accounts? carefully ironic packages cajole never.|
+3329|4|O|46107.70|1995-07-03|2-HIGH|Clerk#000000236|0|old deposits. special accounts haggle furiousl|
+3330|7|F|43255.19|1994-12-19|1-URGENT|Clerk#000000124|0|kages use. carefully regular deposits cajole carefully about |
+3331|91|F|65189.17|1993-05-21|2-HIGH|Clerk#000000901|0|uffily carefully sly accounts. blithely unu|
+3332|143|F|73739.06|1994-11-05|1-URGENT|Clerk#000000840|0|ans detect carefully furiously final deposits: regular accoun|
+3333|92|F|197973.22|1992-09-16|4-NOT SPECIFIED|Clerk#000000157|0|ctions boost slyly quickly even accounts. deposits along|
+3334|76|O|28930.68|1996-02-18|5-LOW|Clerk#000000532|0|ounts maintain carefully. furiously close request|
+3335|49|O|112603.34|1995-10-15|3-MEDIUM|Clerk#000000694|0| deposits poach. ironic ideas about the carefully ironi|
+3360|103|O|168750.48|1998-01-23|5-LOW|Clerk#000000254|0| the deposits. fluffily bold requests cajole regula|
+3361|49|F|75026.51|1992-08-23|4-NOT SPECIFIED|Clerk#000000577|0|unts detect furiously instructions. slow deposi|
+3362|140|O|183176.60|1995-07-29|5-LOW|Clerk#000000011|0|the quickly pending deposits. silent, ev|
+3363|52|O|91017.61|1995-09-23|2-HIGH|Clerk#000000615|0|posits. ironic, final deposits are furiously slyly pending |
+3364|46|O|108412.57|1997-06-21|1-URGENT|Clerk#000000280|0|y even foxes? blithely stea|
+3365|82|F|174634.12|1994-11-09|2-HIGH|Clerk#000000126|0|he slyly regular foxes nag about the accounts. fluffily |
+3366|52|O|13603.08|1997-05-18|1-URGENT|Clerk#000000160|0| pinto beans upon the quickly expres|
+3367|73|F|101339.68|1992-12-31|4-NOT SPECIFIED|Clerk#000000029|0|efully blithely ironic pinto beans. carefully close |
+3392|74|O|96057.42|1995-10-28|1-URGENT|Clerk#000000325|0|es thrash blithely depths. bold multipliers wake f|
+3393|98|O|183104.71|1995-07-04|2-HIGH|Clerk#000000076|0|even requests. excuses are carefully deposits. fluf|
+3394|149|O|162165.94|1996-05-05|4-NOT SPECIFIED|Clerk#000000105|0| blithely among the attainments. carefully final accounts nag blit|
+3395|149|F|141486.77|1994-10-30|4-NOT SPECIFIED|Clerk#000000682|0|ideas haggle beside the ev|
+3396|149|F|196443.16|1994-05-21|3-MEDIUM|Clerk#000000868|0|uffily regular platelet|
+3397|130|F|80084.61|1994-06-23|3-MEDIUM|Clerk#000000048|0|yly. final deposits wake f|
+3398|67|O|1147.42|1996-09-23|1-URGENT|Clerk#000000818|0|uthless, special courts atop the unusual accounts grow fur|
+3399|122|P|56938.16|1995-02-28|4-NOT SPECIFIED|Clerk#000000575|0|the carefully sly accounts. regular, pending theodolites wa|
+3424|103|O|42410.57|1996-08-21|1-URGENT|Clerk#000000190|0|ven requests are quickly pending accounts. blithely furious requests |
+3425|115|O|157040.57|1996-03-31|4-NOT SPECIFIED|Clerk#000000188|0|ions. deposits nag blithely alongside of the carefully f|
+3426|53|O|91929.93|1996-10-16|3-MEDIUM|Clerk#000000283|0|alongside of the slyly|
+3427|4|O|133451.14|1997-05-29|4-NOT SPECIFIED|Clerk#000000404|0|y final pinto beans snooze fluffily bold asymptot|
+3428|10|O|88047.04|1996-04-07|5-LOW|Clerk#000000953|0|lar excuses. slyly pending ideas detect p|
+3429|146|O|141902.54|1997-01-06|4-NOT SPECIFIED|Clerk#000000737|0|l deposits cajole furiously enticing deposits. blithe packages haggle careful|
+3430|113|F|161066.22|1994-12-12|4-NOT SPECIFIED|Clerk#000000664|0| regular attainments are at the final foxes. final packages along the blithe|
+3431|47|F|45536.27|1993-08-22|1-URGENT|Clerk#000000439|0| sleep. slyly busy Tiresias a|
+3456|46|F|32796.35|1993-06-01|5-LOW|Clerk#000000924|0|es promise slyly. ironicall|
+3457|25|P|174223.20|1995-04-27|4-NOT SPECIFIED|Clerk#000000849|0|ely thin asymptotes. deposits kindle. pending|
+3458|95|F|153069.14|1994-12-22|2-HIGH|Clerk#000000392|0|rges snooze. slyly unusua|
+3459|119|F|127134.05|1994-07-28|4-NOT SPECIFIED|Clerk#000000777|0|n instructions? carefully regular excuses are blithely. silent, ironi|
+3460|82|O|245976.74|1995-10-03|2-HIGH|Clerk#000000078|0|ans integrate carefu|
+3461|100|F|190960.69|1993-01-31|1-URGENT|Clerk#000000504|0|al, bold deposits cajole fluffily fluffily final foxes. pending ideas beli|
+3462|133|O|63590.17|1997-05-17|3-MEDIUM|Clerk#000000657|0|uriously express asympto|
+3463|89|F|85255.56|1993-08-18|1-URGENT|Clerk#000000545|0|ding to the carefully ironic deposits|
+3488|148|F|92716.17|1995-01-08|3-MEDIUM|Clerk#000000694|0|cording to the carefully regular deposits. re|
+3489|109|F|62453.97|1993-07-29|3-MEDIUM|Clerk#000000307|0|s detect. carefully even platelets across the fur|
+3490|91|O|100106.96|1997-05-26|5-LOW|Clerk#000000703|0|gular ideas. furiously silent deposits across the unusual accounts boost i|
+3491|83|O|50287.06|1998-06-24|1-URGENT|Clerk#000000560|0|nic orbits believe carefully across the |
+3492|103|F|168721.45|1994-11-24|5-LOW|Clerk#000000066|0|packages along the regular foxes lose final dependencie|
+3493|82|F|41686.10|1993-08-24|2-HIGH|Clerk#000000887|0|lyly special accounts use blithely across the furiously sil|
+3494|49|F|136058.70|1993-04-04|5-LOW|Clerk#000000559|0|r instructions haggle. accounts cajole. carefully final requests at the |
+3495|31|O|58666.79|1996-02-26|2-HIGH|Clerk#000000441|0|nticing excuses are carefully|
+3520|125|O|151233.65|1997-08-04|1-URGENT|Clerk#000000023|0|hely. ideas nag; even, even fo|
+3521|7|F|142029.67|1992-10-26|5-LOW|Clerk#000000812|0|y even instructions cajole carefully above the bli|
+3522|26|F|151515.08|1994-09-26|5-LOW|Clerk#000000250|0|deposits-- slyly stealthy requests boost caref|
+3523|149|O|129657.08|1998-04-07|2-HIGH|Clerk#000000688|0|are on the carefully even depe|
+3524|94|F|22767.49|1992-05-03|2-HIGH|Clerk#000000607|0|efully unusual tithes among the foxes use blithely daringly bold deposits. re|
+3525|109|O|100749.60|1995-12-22|4-NOT SPECIFIED|Clerk#000000084|0|s nag among the blithely e|
+3526|56|F|53827.34|1995-03-16|5-LOW|Clerk#000000364|0|to the quickly special deposits print agai|
+3527|56|O|145232.09|1997-06-21|5-LOW|Clerk#000000874|0|regular ideas across the quickly bold theodo|
+3552|35|O|103656.44|1997-04-23|2-HIGH|Clerk#000000973|0| the ironic packages. furiously |
+3553|91|F|119838.14|1994-05-18|3-MEDIUM|Clerk#000000270|0|counts mold furiously. slyly i|
+3554|44|O|98335.61|1995-06-17|5-LOW|Clerk#000000931|0|hely ironic requests haggl|
+3555|46|O|134442.37|1996-07-07|5-LOW|Clerk#000000585|0|s nag carefully regular, even pinto be|
+3556|16|F|114681.55|1992-09-23|4-NOT SPECIFIED|Clerk#000000140|0|e. dependencies need to haggle alongs|
+3557|121|F|85477.89|1992-11-09|2-HIGH|Clerk#000000291|0|ithely courts. furi|
+3558|28|O|112912.00|1996-02-29|1-URGENT|Clerk#000000841|0|around the furiously even requests. quickl|
+3559|106|F|30722.49|1992-10-24|3-MEDIUM|Clerk#000000634|0|sly deposits. fluffily final ideas cajole careful|
+3584|13|O|80487.97|1997-08-11|1-URGENT|Clerk#000000760|0|fully bold packages. fluffily final braids haggle final, ironic dolphins. b|
+3585|139|F|159015.39|1994-11-23|2-HIGH|Clerk#000000988|0|regular asymptotes. bold pains above the carefully pending asymptot|
+3586|121|F|112845.04|1993-12-05|2-HIGH|Clerk#000000438|0|he quickly final courts. carefully regular requests nag unusua|
+3587|79|O|174798.97|1996-05-10|4-NOT SPECIFIED|Clerk#000000443|0|ular patterns detect |
+3588|119|F|207925.83|1995-03-19|4-NOT SPECIFIED|Clerk#000000316|0|ong the pains. evenly unusual |
+3589|31|F|39103.37|1994-05-26|2-HIGH|Clerk#000000023|0|ithe deposits nag furiously. furiously pending packages sleep f|
+3590|149|P|218482.70|1995-05-13|5-LOW|Clerk#000000986|0|lyly final deposits.|
+3591|136|F|98140.86|1993-12-08|3-MEDIUM|Clerk#000000144|0|ual foxes haggle! unusual request|
+3616|128|F|60933.29|1994-02-16|4-NOT SPECIFIED|Clerk#000000268|0|uickly about the quickly final requests. fluffily final packages wake evenly|
+3617|40|O|126205.42|1996-03-19|3-MEDIUM|Clerk#000000886|0|the carefully regular platelets ha|
+3618|10|O|136954.81|1997-12-13|3-MEDIUM|Clerk#000000894|0|. ideas run carefully. thin, pending |
+3619|149|O|222274.54|1996-11-20|2-HIGH|Clerk#000000211|0|uests mold after the blithely ironic excuses. slyly pending pa|
+3620|44|O|59291.75|1997-03-07|5-LOW|Clerk#000000124|0|le quickly against the epitaphs. requests sleep slyly according to the|
+3621|142|F|106150.05|1993-05-06|3-MEDIUM|Clerk#000000643|0|kly unusual deposits. qu|
+3622|91|O|109202.90|1995-11-27|5-LOW|Clerk#000000012|0|c deposits are fluffily about the blithely final theo|
+3623|4|O|175017.68|1996-12-26|1-URGENT|Clerk#000000184|0|- ironic excuses boost quickly in place |
+3648|125|F|180417.11|1993-06-17|5-LOW|Clerk#000000717|0|foxes. unusual deposits boost quickly. slyly regular asymptotes across t|
+3649|40|F|124470.32|1994-07-06|5-LOW|Clerk#000000349|0|taphs boost above the final p|
+3650|46|F|189547.57|1992-05-28|4-NOT SPECIFIED|Clerk#000000454|0|kages sleep fluffily slyly|
+3651|100|O|113191.45|1998-04-27|1-URGENT|Clerk#000000222|0|ly unusual deposits thrash quickly after the ideas.|
+3652|107|O|107732.23|1997-02-25|4-NOT SPECIFIED|Clerk#000000024|0|sly even requests after the |
+3653|40|F|142866.39|1994-03-27|1-URGENT|Clerk#000000402|0| pearls. bold accounts are along the ironic,|
+3654|7|F|222653.54|1992-06-03|5-LOW|Clerk#000000475|0|s cajole slyly carefully special theodolites. even deposits haggl|
+3655|49|F|74882.22|1992-10-06|1-URGENT|Clerk#000000815|0|er the carefully unusual deposits sleep quickly according to|
+3680|127|F|124402.59|1992-12-10|4-NOT SPECIFIED|Clerk#000000793|0|ular platelets. carefully regular packages cajole blithely al|
+3681|52|F|36889.65|1992-04-04|1-URGENT|Clerk#000000566|0|. ironic deposits against the ironic, regular frets use pending plat|
+3682|32|O|67525.43|1997-01-22|2-HIGH|Clerk#000000001|0|es haggle carefully. decoys nag |
+3683|88|F|99960.46|1993-03-04|2-HIGH|Clerk#000000248|0|ze across the express foxes. carefully special acco|
+3684|23|F|89509.91|1993-07-20|2-HIGH|Clerk#000000835|0|bold accounts affix along the carefully ironic requ|
+3685|16|F|154958.89|1992-01-17|3-MEDIUM|Clerk#000000954|0| sleep fluffily special ide|
+3686|40|O|82190.77|1998-07-07|2-HIGH|Clerk#000000175|0|s. furiously final pinto beans poach carefully among |
+3687|43|F|99851.38|1993-02-03|1-URGENT|Clerk#000000585|0|gular accounts. slyly regular instructions can are final ide|
+3712|64|F|127527.05|1992-01-02|2-HIGH|Clerk#000000032|0| promise according |
+3713|149|O|215342.63|1998-05-07|3-MEDIUM|Clerk#000000325|0|s haggle quickly. ironic, regular Tiresi|
+3714|40|O|84493.55|1998-05-01|3-MEDIUM|Clerk#000000595|0|nding accounts. ironic pinto beans wake slyly. furiously pendin|
+3715|65|O|64000.93|1996-03-18|1-URGENT|Clerk#000000463|0| always silent requests wake pinto beans. slyly pending foxes are aga|
+3716|43|O|146221.66|1997-08-19|4-NOT SPECIFIED|Clerk#000000748|0| pending ideas haggle. ironic,|
+3717|28|O|176525.53|1998-06-03|4-NOT SPECIFIED|Clerk#000000974|0|t the carefully even ideas use sp|
+3718|31|O|63195.54|1996-10-23|2-HIGH|Clerk#000000016|0|refully. furiously final packages use carefully slyly pending deposits! final,|
+3719|118|O|139902.71|1997-02-16|2-HIGH|Clerk#000000034|0|, enticing accounts are blithely among the daringly final asymptotes. furious|
+3744|65|F|33085.68|1992-01-10|3-MEDIUM|Clerk#000000765|0|osits sublate about the regular requests. fluffily unusual accou|
+3745|112|F|19405.73|1993-09-29|5-LOW|Clerk#000000181|0|ckages poach slyly against the foxes. slyly ironic instructi|
+3746|74|F|80018.54|1994-09-11|4-NOT SPECIFIED|Clerk#000000188|0|. express, special requests nag quic|
+3747|149|O|204355.65|1996-08-20|1-URGENT|Clerk#000000226|0|refully across the final theodolites. carefully bold accounts cajol|
+3748|53|O|83804.38|1998-02-28|1-URGENT|Clerk#000000156|0|slyly special packages|
+3749|38|P|87073.89|1995-02-24|3-MEDIUM|Clerk#000000639|0|y regular instructions haggle blithel|
+3750|97|P|177181.67|1995-04-30|3-MEDIUM|Clerk#000000885|0|y. express, even packages wake after the ide|
+3751|10|F|202917.72|1994-04-27|4-NOT SPECIFIED|Clerk#000000925|0|sheaves. express, unusual t|
+3776|85|F|150349.92|1992-11-20|2-HIGH|Clerk#000000698|0|efully even platelets slee|
+3777|28|F|82467.29|1994-04-08|3-MEDIUM|Clerk#000000941|0| regular, special dolphins cajole enticingly ca|
+3778|106|F|221036.31|1993-05-26|1-URGENT|Clerk#000000187|0| above the express requests. packages maintain fluffily according to|
+3779|74|O|31538.94|1997-01-05|4-NOT SPECIFIED|Clerk#000000670|0| against the deposits. quickly bold instructions x-ray. pending fox|
+3780|41|O|65385.42|1996-04-13|5-LOW|Clerk#000000967|0| around the brave, pendin|
+3781|139|O|133864.82|1996-06-20|1-URGENT|Clerk#000000394|0|yly after the ruthless packages. pinto beans use slyly: never ironic dependenc|
+3782|65|O|145096.17|1996-08-24|1-URGENT|Clerk#000000121|0|counts are. pending, regular asym|
+3783|44|F|155017.92|1993-12-06|4-NOT SPECIFIED|Clerk#000000614|0| along the pinto beans. special packages use. regular theo|
+3808|79|F|228054.01|1994-04-24|1-URGENT|Clerk#000000717|0|odolites. blithely ironic cour|
+3809|148|O|143070.70|1996-05-01|5-LOW|Clerk#000000646|0| regular excuses. even theodolites are fluffily according to t|
+3810|100|F|124675.27|1992-09-17|1-URGENT|Clerk#000000660|0|ters sleep across the carefully final |
+3811|80|O|154967.89|1998-04-16|3-MEDIUM|Clerk#000000290|0|sits wake slyly abo|
+3812|41|O|70502.52|1996-08-13|3-MEDIUM|Clerk#000000727|0|al, final requests cajole|
+3813|146|O|77247.05|1998-06-29|1-URGENT|Clerk#000000531|0|g the furiously regular instructions|
+3814|118|P|149451.88|1995-02-22|5-LOW|Clerk#000000669|0| the furiously pending theodo|
+3815|104|O|14275.01|1997-08-26|1-URGENT|Clerk#000000249|0|es snooze carefully stealth|
+3840|100|O|187156.38|1998-07-17|4-NOT SPECIFIED|Clerk#000000713|0|yly slow theodolites. enticingly |
+3841|58|F|129033.13|1994-10-05|4-NOT SPECIFIED|Clerk#000000018|0| bold requests sleep quickly ironic packages. sometimes regular deposits nag |
+3842|28|F|131447.03|1992-04-09|5-LOW|Clerk#000000418|0|silent ideas. final deposits use furiously. blithely express excuses cajole fu|
+3843|10|O|34035.17|1997-01-04|4-NOT SPECIFIED|Clerk#000000693|0|eodolites; slyly unusual accounts nag boldly |
+3844|79|F|6793.45|1994-12-29|1-URGENT|Clerk#000000686|0|r dolphins. slyly ironic theodolites ag|
+3845|89|F|134333.33|1992-04-26|1-URGENT|Clerk#000000404|0|es among the pending, regular accounts sleep blithely blithely even de|
+3846|49|O|123120.06|1998-02-05|2-HIGH|Clerk#000000877|0|y alongside of the slyl|
+3847|34|F|7014.31|1993-03-12|5-LOW|Clerk#000000338|0|uriously even deposits. furiously pe|
+3872|134|O|198538.68|1996-09-06|5-LOW|Clerk#000000943|0|counts boost slyly against the ironic platelets-- blithely p|
+3873|55|O|95291.79|1998-03-30|4-NOT SPECIFIED|Clerk#000000791|0|express deposits-- even ideas |
+3874|119|F|66455.34|1993-06-09|3-MEDIUM|Clerk#000000208|0|ular asymptotes sleep blithely ironic ideas. blithel|
+3875|118|O|74483.95|1997-09-10|1-URGENT|Clerk#000000587|0| solve among the fluffily even |
+3876|29|O|95126.32|1996-08-02|5-LOW|Clerk#000000708|0|into beans. blithely|
+3877|17|F|178492.01|1993-05-21|5-LOW|Clerk#000000652|0|foxes. thinly bold reques|
+3878|88|O|59989.66|1997-03-23|1-URGENT|Clerk#000000314|0|e carefully regular platelets. special, express dependencies slee|
+3879|142|O|80274.22|1995-11-23|1-URGENT|Clerk#000000231|0|sts along the quickly ironic sentiments cajole carefully according to t|
+3904|149|O|39338.44|1997-11-15|4-NOT SPECIFIED|Clerk#000000883|0|sits haggle furiously across the requests. theodolites ha|
+3905|22|F|56227.04|1993-12-21|4-NOT SPECIFIED|Clerk#000000573|0|usly even accounts lose quietly above the slyly express p|
+3906|46|F|145630.76|1992-05-28|3-MEDIUM|Clerk#000000867|0|ironic theodolites haggle blithely above the final re|
+3907|67|F|240457.56|1992-08-19|3-MEDIUM|Clerk#000000084|0|gular pinto beans sleep f|
+3908|43|F|57127.71|1993-03-09|3-MEDIUM|Clerk#000000490|0|ounts cajole. regularly|
+3909|22|O|82746.74|1998-07-27|1-URGENT|Clerk#000000980|0|nic, special theodolites sleep furiously! furiously |
+3910|64|O|47272.67|1996-08-26|3-MEDIUM|Clerk#000000270|0|ickly. furiously final packag|
+3911|10|P|35019.95|1995-03-17|4-NOT SPECIFIED|Clerk#000000818|0|he fluffily final forges haggle slyly according to the blithely|
+3936|32|O|168618.39|1996-11-07|2-HIGH|Clerk#000000200|0|iously express packages engage slyly fina|
+3937|94|O|187516.29|1997-11-30|4-NOT SPECIFIED|Clerk#000000189|0|ckages boost carefully blithely q|
+3938|31|F|46918.22|1993-03-03|1-URGENT|Clerk#000000199|0|. unusual, final foxes haggle|
+3939|70|O|8720.45|1996-01-11|5-LOW|Clerk#000000647|0|ly ruthlessly silent requests. blithely regular requests haggle blithely wh|
+3940|149|O|129012.84|1996-02-14|5-LOW|Clerk#000000363|0|e above the ideas. quickly even dependencies along the blithely ir|
+3941|136|O|95453.80|1996-08-29|2-HIGH|Clerk#000000503|0|gular theodolites integrate quickly |
+3942|76|F|38596.81|1993-06-28|4-NOT SPECIFIED|Clerk#000000608|0|eas cajole bold requests. idly silent instructions |
+3943|40|O|60314.97|1996-10-09|5-LOW|Clerk#000000482|0|se alongside of the final pinto beans. regular packages boost across the ca|
+3968|25|O|121704.45|1997-02-17|4-NOT SPECIFIED|Clerk#000000431|0| the slyly special accounts; |
+3969|52|O|169797.40|1997-05-14|2-HIGH|Clerk#000000731|0|uriously final dependencies slee|
+3970|76|F|163709.85|1992-03-27|3-MEDIUM|Clerk#000000190|0|luffily furiously regular deposits. blithely special requests cajole blithely|
+3971|104|O|47925.47|1996-06-28|5-LOW|Clerk#000000287|0|alongside of the instructions ought to are |
+3972|124|F|1861.19|1994-04-21|3-MEDIUM|Clerk#000000049|0|y regular requests haggle quickly. pending, express acco|
+3973|103|F|91541.48|1992-03-24|4-NOT SPECIFIED|Clerk#000000114|0|somas according to the quickly even instructions wake fu|
+3974|94|O|56779.06|1996-03-05|4-NOT SPECIFIED|Clerk#000000938|0|deposits are furiously beneath the bl|
+3975|118|O|37804.43|1995-04-11|3-MEDIUM|Clerk#000000016|0|ts. regular, regular Tiresias play furiously. ironi|
+4000|70|F|84053.93|1992-01-04|5-LOW|Clerk#000000339|0|le carefully closely even pinto beans. regular, ironic foxes against the|
+4001|115|O|95929.46|1997-05-15|3-MEDIUM|Clerk#000000878|0|detect. asymptotes sleep furio|
+4002|104|O|76518.11|1997-04-08|5-LOW|Clerk#000000097|0| regular braids are. furiously even patterns agains|
+4003|112|F|17603.01|1993-01-27|1-URGENT|Clerk#000000177|0| blithe theodolites are slyly. slyly silent accounts toward|
+4004|70|F|220715.14|1993-05-07|3-MEDIUM|Clerk#000000273|0|accounts among the blithely regular sentiments |
+4005|140|O|129062.13|1996-11-20|2-HIGH|Clerk#000000341|0|ily according to the slyly iron|
+4006|35|F|70557.05|1995-01-04|3-MEDIUM|Clerk#000000765|0|ly ironic packages integrate. regular requests alongside of |
+4007|8|F|116193.97|1993-06-18|2-HIGH|Clerk#000000623|0|ecial packages. slyly regular accounts integrate |
+4032|10|O|62497.51|1998-02-26|3-MEDIUM|Clerk#000000686|0|iresias sleep slyly regular ideas. quickly unusual|
+4033|83|F|57740.74|1993-06-02|5-LOW|Clerk#000000181|0|ously bold instructions haggle furiously above the fluf|
+4034|94|F|186912.51|1993-11-14|4-NOT SPECIFIED|Clerk#000000548|0|ts x-ray. express requests affix fluffily regular theodolites. pending, fina|
+4035|118|F|22840.21|1992-02-19|5-LOW|Clerk#000000097|0|he ironic deposits sleep blith|
+4036|47|O|82563.10|1997-04-26|3-MEDIUM|Clerk#000000398|0|ly express deposits nag slyly. ironic, final asymptotes boost bra|
+4037|121|F|36389.43|1993-03-24|2-HIGH|Clerk#000000384|0|t carefully above the unusual the|
+4038|94|O|155045.39|1996-01-06|1-URGENT|Clerk#000000272|0|re slyly. silent requests wake quickly. regular packages play quickly |
+4039|29|O|143753.01|1997-11-16|1-URGENT|Clerk#000000358|0|ly ironic deposits. ironic reques|
+4064|130|O|148500.71|1996-10-10|4-NOT SPECIFIED|Clerk#000000598|0|ccounts. furiously unusual theodolites wake carefully about|
+4065|80|F|156345.64|1994-06-09|1-URGENT|Clerk#000000131|0|even foxes! slyly final deposits agai|
+4066|32|O|176911.21|1997-01-27|4-NOT SPECIFIED|Clerk#000000286|0|yly ironic dinos. quickly regular accounts haggle. requests wa|
+4067|16|F|136517.34|1992-10-07|2-HIGH|Clerk#000000027|0|tes boost furiously quick asymptotes. final deposits of the dolphins solv|
+4068|125|O|71852.67|1996-09-18|3-MEDIUM|Clerk#000000203|0|lly even accounts wake furiously across the unusual platelets. unusu|
+4069|73|F|198816.13|1992-05-13|3-MEDIUM|Clerk#000000359|0|deposits: slyly bold ideas detect furiously. f|
+4070|29|O|98275.37|1995-06-12|2-HIGH|Clerk#000000713|0|xpress ideas poach ab|
+4071|148|O|67789.42|1996-09-15|4-NOT SPECIFIED|Clerk#000000486|0|nal deposits. pending deposits d|
+4096|139|F|81089.61|1992-07-03|4-NOT SPECIFIED|Clerk#000000706|0|sits. quickly thin deposits x-ray blith|
+4097|10|O|134308.04|1996-05-24|1-URGENT|Clerk#000000475|0|ickly under the even accounts. even packages after the furiously express|
+4098|23|O|48478.54|1996-11-05|4-NOT SPECIFIED|Clerk#000000491|0|otes. quickly final requests after the stealthily ironic pinto bean|
+4099|17|F|207364.80|1992-08-21|1-URGENT|Clerk#000000379|0|r platelets. slyly regular requests cajole carefully against the|
+4100|4|O|3892.77|1996-03-12|3-MEDIUM|Clerk#000000429|0|posits. carefully unusual packages use pending deposits. regular she|
+4101|142|F|21640.10|1993-11-22|4-NOT SPECIFIED|Clerk#000000704|0|y around the express, careful epitaphs. accounts use fluffily. quickly p|
+4102|22|O|128786.57|1996-03-17|1-URGENT|Clerk#000000675|0|nding dependencies was slyly about the bl|
+4103|106|F|38164.23|1992-07-03|5-LOW|Clerk#000000679|0|fully ironic dependencies.|
+4128|139|O|5472.17|1995-10-07|4-NOT SPECIFIED|Clerk#000000635|0|ctions. dependencies from the slyly regular accounts nag slyly fu|
+4129|32|F|67226.28|1993-06-26|3-MEDIUM|Clerk#000000541|0|nwind. quickly final theodolites use packages. accounts|
+4130|104|O|47823.04|1996-03-10|5-LOW|Clerk#000000609|0|omise alongside of the carefully final foxes. blithel|
+4131|44|O|145971.60|1998-01-30|1-URGENT|Clerk#000000612|0| above the foxes hang |
+4132|19|P|65601.08|1995-05-29|4-NOT SPECIFIED|Clerk#000000158|0|ld asymptotes solve alongside of the express, final packages. fluffily fi|
+4133|101|F|31693.88|1992-08-07|4-NOT SPECIFIED|Clerk#000000268|0|al, express foxes. quickly pending deposits might cajole alongsi|
+4134|97|F|125191.12|1995-01-12|1-URGENT|Clerk#000000171|0|fully even deposits. regular de|
+4135|37|O|99577.55|1997-03-10|3-MEDIUM|Clerk#000000627|0|ly quietly even ideas. deposits haggle blithely|
+4160|55|O|82493.07|1996-08-20|5-LOW|Clerk#000000283|0|the carefully special accounts. furiously regular dugouts alongs|
+4161|118|F|198995.21|1993-08-21|5-LOW|Clerk#000000047|0|nts. fluffily regular foxes above the quickly daring reques|
+4162|22|F|72359.55|1992-02-10|5-LOW|Clerk#000000179|0|r packages are slyly accounts. furiously special foxes detect carefully re|
+4163|64|F|11493.80|1992-12-21|2-HIGH|Clerk#000000268|0| integrate furiously slyly regular depende|
+4164|94|O|8709.16|1998-07-03|2-HIGH|Clerk#000000720|0| regularly busy theodolites boost furiously quickly bold packages. express, s|
+4165|4|O|11405.40|1997-07-25|3-MEDIUM|Clerk#000000621|0|special foxes affix never blithely ironic pinto beans; blithely |
+4166|43|F|100671.06|1993-02-28|5-LOW|Clerk#000000757|0|quickly sly forges impress. careful foxes across the blithely even a|
+4167|28|O|62108.45|1998-06-17|1-URGENT|Clerk#000000917|0|kly furiously even deposits. unu|
+4192|146|O|197192.95|1998-04-19|1-URGENT|Clerk#000000369|0|equests above the slyly regular pinto beans unwi|
+4193|4|F|143191.54|1994-01-09|2-HIGH|Clerk#000000201|0|ng accounts haggle quickly. packages use fluffily ironic excu|
+4194|106|F|62972.29|1994-10-16|3-MEDIUM|Clerk#000000385|0| instructions are quickly even pinto beans. courts boost furiously regular, ev|
+4195|104|F|54478.95|1993-05-29|4-NOT SPECIFIED|Clerk#000000777|0| pinto beans cajole furiously theodolites-- slyly regular deposits doub|
+4196|106|O|201455.98|1998-05-15|3-MEDIUM|Clerk#000000532|0|affix carefully. quickly final requests |
+4197|92|O|217709.03|1996-08-13|4-NOT SPECIFIED|Clerk#000000264|0| pinto beans according|
+4198|143|O|105789.01|1997-06-16|3-MEDIUM|Clerk#000000583|0|g the special packages haggle pen|
+4199|5|F|30494.62|1992-02-13|1-URGENT|Clerk#000000309|0|e blithely. special deposits haggle slyly final foxes. carefully even|
+4224|70|O|150655.44|1997-07-14|1-URGENT|Clerk#000000034|0|jole quickly final dolphins. slyly pending foxes wake furiously bold pl|
+4225|128|O|72533.07|1997-06-03|3-MEDIUM|Clerk#000000992|0|r the platelets nag among the special deposits. ironic, ironic re|
+4226|92|F|29827.44|1993-03-09|5-LOW|Clerk#000000203|0|phins wake slyly regular packages. deposits haggle slowl|
+4227|133|F|92261.08|1995-02-24|1-URGENT|Clerk#000000063|0|ng the requests; ideas haggle fluffily. slyly unusual ideas c|
+4228|110|O|22072.16|1997-03-28|5-LOW|Clerk#000000309|0|pecial requests aft|
+4229|14|O|75145.87|1998-03-03|1-URGENT|Clerk#000000301|0|p furiously: final excuses hagg|
+4230|140|F|219709.60|1992-03-04|1-URGENT|Clerk#000000364|0|lly ironic deposits integrate carefully about the fu|
+4231|86|O|111403.66|1997-11-20|4-NOT SPECIFIED|Clerk#000000630|0|ly final accounts cajole furiously accounts. bravely ironic platelets am|
+4256|118|F|23067.48|1992-04-05|4-NOT SPECIFIED|Clerk#000000043|0|y alongside of the fluffily iro|
+4257|17|P|41723.86|1995-03-25|3-MEDIUM|Clerk#000000682|0|r ideas cajole along the blithely regular gifts.|
+4258|92|O|133829.35|1996-10-27|4-NOT SPECIFIED|Clerk#000000364|0|efully final platelets around the blit|
+4259|104|O|12918.70|1997-10-09|5-LOW|Clerk#000000781|0|es snooze slyly against the furiously unusual ideas. furious|
+4260|142|F|18566.14|1992-05-16|4-NOT SPECIFIED|Clerk#000000919|0|e among the fluffily bold accounts.|
+4261|118|F|83665.20|1992-10-03|1-URGENT|Clerk#000000662|0| about the even, pending packages. slyly bold deposits boost|
+4262|88|O|176278.57|1996-08-04|3-MEDIUM|Clerk#000000239|0| of the furious accounts. furiously regular accounts w|
+4263|4|O|158885.83|1998-03-16|1-URGENT|Clerk#000000265|0|sly ruthless deposits. final packages are instructions. fu|
+4288|34|F|75030.81|1992-12-04|4-NOT SPECIFIED|Clerk#000000823|0|usly carefully even theodolites: slyly express pac|
+4289|125|F|20752.62|1993-10-07|3-MEDIUM|Clerk#000000912|0|e carefully close instructions. slyly special reques|
+4290|41|F|26128.99|1995-01-15|3-MEDIUM|Clerk#000000688|0| slyly quickly bold requests. final deposits haggle pending ideas! som|
+4291|89|F|71822.86|1993-11-29|3-MEDIUM|Clerk#000000655|0| sleep fluffily between the bold packages. bold|
+4292|25|F|145906.24|1992-01-09|3-MEDIUM|Clerk#000000794|0| ruthlessly. slyly bo|
+4293|103|O|198322.91|1996-08-20|2-HIGH|Clerk#000000750|0|ly packages. regular packages nag according to t|
+4294|49|F|232194.74|1992-08-15|3-MEDIUM|Clerk#000000407|0|ng pinto beans breach. slyly express requests bo|
+4295|5|O|77754.62|1996-02-10|3-MEDIUM|Clerk#000000023|0|e boldly bold dependencies|
+4320|115|O|67049.37|1996-12-08|4-NOT SPECIFIED|Clerk#000000223|0|ages haggle after the slowly bold se|
+4321|16|F|118896.95|1994-07-18|3-MEDIUM|Clerk#000000041|0|ending deposits are carefully carefully regular packa|
+4322|142|O|149671.92|1998-03-13|3-MEDIUM|Clerk#000000433|0|totes nag across the fluffily special instructions. quickly silent hockey |
+4323|104|F|27598.17|1994-01-23|2-HIGH|Clerk#000000282|0|lve after the slyly regular multipliers. even, regular excus|
+4324|73|O|178249.05|1995-07-17|1-URGENT|Clerk#000000800|0|ccounts. slyly stealthy requests shall have t|
+4325|130|O|20214.49|1996-07-18|2-HIGH|Clerk#000000591|0|y around the always ev|
+4326|29|O|39048.94|1996-10-27|4-NOT SPECIFIED|Clerk#000000869|0|packages. carefully express deposit|
+4327|146|P|126235.35|1995-03-16|2-HIGH|Clerk#000000571|0|yly pending braids. final requests abo|
+4352|14|O|18653.09|1997-11-26|2-HIGH|Clerk#000000620|0|ly final platelets integrate carefully even requ|
+4353|73|O|21815.30|1997-12-12|2-HIGH|Clerk#000000790|0|uickly even ideas cajole|
+4354|145|F|179827.12|1994-09-30|4-NOT SPECIFIED|Clerk#000000046|0|pending notornis. requests serve |
+4355|4|O|186370.23|1996-11-16|1-URGENT|Clerk#000000362|0|ndencies use furiously across the regular |
+4356|97|F|39828.51|1994-04-11|5-LOW|Clerk#000000956|0| asymptotes sleep blithely. asymptotes sleep. blithely regul|
+4357|47|O|67045.94|1997-10-23|4-NOT SPECIFIED|Clerk#000000031|0|ages nag between the|
+4358|25|O|46298.53|1997-08-12|1-URGENT|Clerk#000000692|0|according to the fluffily special asymptotes |
+4359|16|F|107824.40|1993-03-03|1-URGENT|Clerk#000000393|0|sts. special, unusual deposits across the ironic theodo|
+4384|25|F|52562.16|1992-07-13|1-URGENT|Clerk#000000192|0|onic platelets. furiously regular asymptotes according to the special pac|
+4385|122|O|39190.62|1996-08-06|2-HIGH|Clerk#000000597|0|ully final requests. ironic, even dolphins above the regular |
+4386|61|O|134413.58|1998-02-06|5-LOW|Clerk#000000070|0| dolphins. silent, idle pinto beans |
+4387|110|O|116740.67|1995-10-23|1-URGENT|Clerk#000000025|0|ter the regular pinto beans. special, final gifts above the requests wi|
+4388|10|O|69668.22|1996-03-28|2-HIGH|Clerk#000000715|0|ts wake against the carefully final accounts. sly|
+4389|55|F|120324.82|1994-05-05|3-MEDIUM|Clerk#000000403|0|wly express excuses after the permanently even instructions are|
+4390|7|P|140608.69|1995-05-23|1-URGENT|Clerk#000000691|0|inal pinto beans. exp|
+4391|38|F|48284.06|1992-02-18|2-HIGH|Clerk#000000880|0|regular accounts. even depo|
+4416|149|F|76067.10|1992-06-30|5-LOW|Clerk#000000391|0| deposits. ideas cajole express theodolites: |
+4417|67|O|60868.39|1998-07-09|1-URGENT|Clerk#000000365|0|ideas are alongside of the blithely final reque|
+4418|61|F|47099.71|1993-03-25|3-MEDIUM|Clerk#000000731|0|pecial pinto beans. close foxes affix iron|
+4419|104|O|94030.43|1996-06-12|4-NOT SPECIFIED|Clerk#000000410|0|ages wake furiously slyly thin theodolit|
+4420|109|F|6088.41|1994-06-18|1-URGENT|Clerk#000000706|0|lly bold deposits along the bold, pending foxes detect blithely after the acco|
+4421|10|O|258779.02|1997-04-04|3-MEDIUM|Clerk#000000246|0|t the pending warhorses. express waters a|
+4422|70|P|107140.22|1995-05-22|3-MEDIUM|Clerk#000000938|0|ly bold accounts sleep special, regular foxes. doggedly regular in|
+4423|64|F|4913.06|1995-02-17|5-LOW|Clerk#000000888|0|excuses are ruthless|
+4448|70|O|127191.47|1998-05-21|2-HIGH|Clerk#000000428|0|. deposits haggle around the silent packages; slyly unusual packages|
+4449|10|O|48206.14|1998-02-08|5-LOW|Clerk#000000035|0|ourts are carefully even deposits. pending |
+4450|106|O|110194.31|1997-07-15|1-URGENT|Clerk#000000867|0|quests boost. furiously even realms are blithely bold requests. bl|
+4451|4|F|92851.80|1994-10-01|1-URGENT|Clerk#000000181|0|. carefully final foxes along the quickly express T|
+4452|13|F|64838.66|1994-06-21|5-LOW|Clerk#000000985|0|oxes are slyly. express, ironic pinto beans wake after the quickly pending re|
+4453|65|O|137030.40|1997-04-01|3-MEDIUM|Clerk#000000603|0|ages could have to nag slyly furiously even asymptotes! slowly regular |
+4454|142|F|159578.94|1994-02-02|5-LOW|Clerk#000000411|0|uriously regular pint|
+4455|19|F|102534.63|1993-10-11|3-MEDIUM|Clerk#000000924|0|even requests. bravely regular foxes according to the carefully unusual |
+4480|85|F|28658.26|1994-03-31|4-NOT SPECIFIED|Clerk#000000534|0|press, bold deposits boost blit|
+4481|148|O|77705.40|1996-03-30|5-LOW|Clerk#000000443|0|press sheaves cajole furio|
+4482|82|P|63535.56|1995-05-15|4-NOT SPECIFIED|Clerk#000000534|0|ravely bold accounts. furiously ironic instructions affix quickly. pend|
+4483|52|F|126597.21|1992-03-07|3-MEDIUM|Clerk#000000615|0|its. blithely idle accounts run; theodolites wake carefully around the fi|
+4484|131|O|237947.61|1996-12-24|1-URGENT|Clerk#000000392|0|ct across the pinto beans. quickly pending excuses engage furiously.|
+4485|53|F|182432.17|1994-11-13|3-MEDIUM|Clerk#000000038|0|es wake slyly even packages. blithely brave requests nag above the regul|
+4486|37|O|135613.18|1998-03-03|2-HIGH|Clerk#000000656|0|ffily according to the carefully pending acc|
+4487|46|F|109469.90|1993-02-23|3-MEDIUM|Clerk#000000017|0|s up the never pending excuses wake furiously special pinto beans. furiously i|
+4512|70|O|148682.82|1995-10-25|5-LOW|Clerk#000000393|0|ending instructions maintain fu|
+4513|85|O|119820.38|1996-03-15|5-LOW|Clerk#000000154|0|ests. final, final ideas|
+4514|97|F|143899.85|1994-04-30|3-MEDIUM|Clerk#000000074|0|deposits according to the carefull|
+4515|140|F|161745.44|1992-03-17|1-URGENT|Clerk#000000191|0|quests among the accounts sleep boldly about the regular f|
+4516|130|F|35949.14|1994-03-29|3-MEDIUM|Clerk#000000739|0|ing packages sleep slyly regular attainments|
+4517|113|O|47614.08|1998-03-07|4-NOT SPECIFIED|Clerk#000000231|0|uriously final deposits doze furiously furiously reg|
+4518|125|O|25861.74|1997-05-01|3-MEDIUM|Clerk#000000187|0|luffily against the spec|
+4519|136|F|68885.66|1993-03-30|4-NOT SPECIFIED|Clerk#000000938|0|ccording to the final |
+4544|112|O|151148.81|1997-08-07|3-MEDIUM|Clerk#000000435|0|g dependencies dazzle slyly ironic somas. carefu|
+4545|59|F|143276.28|1993-01-17|4-NOT SPECIFIED|Clerk#000000303|0|ep. requests use sly|
+4546|43|O|39906.87|1995-07-29|5-LOW|Clerk#000000373|0|ns sleep. regular, regular instructions maintai|
+4547|109|F|52114.01|1993-08-23|3-MEDIUM|Clerk#000000519|0|uctions thrash platelets. slyly final foxes wake slyly against th|
+4548|127|O|139915.23|1996-06-28|5-LOW|Clerk#000000798|0| in place of the blithely express sentiments haggle slyly r|
+4549|64|O|43889.17|1998-03-05|4-NOT SPECIFIED|Clerk#000000965|0|ully even deposits dazzle. fluffily pending ideas against the requests|
+4550|118|F|27461.48|1994-12-29|2-HIGH|Clerk#000000748|0|s haggle carefully acco|
+4551|109|O|82824.14|1996-02-09|2-HIGH|Clerk#000000462|0|ts. slyly quick theodolite|
+4576|139|O|56936.10|1996-08-14|5-LOW|Clerk#000000798|0|e pending deposits. |
+4577|79|O|104259.88|1998-05-02|5-LOW|Clerk#000000409|0|ly. unusual platelets are alw|
+4578|91|F|95761.93|1992-09-13|5-LOW|Clerk#000000121|0| to the furiously ironic instructions? furiou|
+4579|106|O|85927.85|1995-12-01|2-HIGH|Clerk#000000951|0|its wake quickly blithely specia|
+4580|82|F|118464.65|1993-11-15|4-NOT SPECIFIED|Clerk#000000086|0|rs wake blithely regular requests. fluffily ev|
+4581|79|F|89592.11|1992-09-04|4-NOT SPECIFIED|Clerk#000000687|0|ges. carefully pending accounts use furiously abo|
+4582|19|O|18247.86|1996-07-04|1-URGENT|Clerk#000000638|0|g the furiously regular pac|
+4583|22|F|206495.43|1994-09-25|3-MEDIUM|Clerk#000000240|0|equests. slyly even platelets was qui|
+4608|80|F|157767.86|1994-06-17|1-URGENT|Clerk#000000259|0|y even instructions detect slyly asymptotes. blithely final packa|
+4609|133|O|70462.84|1996-12-05|3-MEDIUM|Clerk#000000239|0|hang slyly slyly expre|
+4610|26|F|135934.60|1993-06-18|5-LOW|Clerk#000000616|0|e carefully express pinto|
+4611|29|F|166506.22|1993-01-10|2-HIGH|Clerk#000000152|0|. furiously regular instructions haggle dolphins. even instructions det|
+4612|61|F|82598.87|1993-09-20|3-MEDIUM|Clerk#000000397|0|bove the deposits. even deposits dazzle. slyly express packages haggle sl|
+4613|133|O|212339.55|1998-03-05|3-MEDIUM|Clerk#000000541|0|furiously blithely pending dependen|
+4614|61|O|151801.06|1996-04-22|1-URGENT|Clerk#000000974|0| sauternes wake thinly special accounts. fur|
+4615|29|F|10500.27|1993-08-27|3-MEDIUM|Clerk#000000982|0|jole after the fluffily pending foxes. packages affix carefully acco|
+4640|97|O|81138.17|1996-01-01|5-LOW|Clerk#000000902|0|requests. deposits do detect above the blithely iron|
+4641|134|F|98485.21|1993-01-20|4-NOT SPECIFIED|Clerk#000000755|0|ronic, final requests integrate slyly: specia|
+4642|148|F|117537.87|1995-02-27|1-URGENT|Clerk#000000295|0|cial requests wake carefully around the regular, unusual ideas. furi|
+4643|67|O|52414.19|1995-06-30|2-HIGH|Clerk#000000292|0|ously regular packages. unusual, special platel|
+4644|94|O|85901.70|1998-01-17|5-LOW|Clerk#000000961|0|requests. fluffily even ideas bo|
+4645|44|F|231012.22|1994-09-20|1-URGENT|Clerk#000000764|0|fully even instructions. final gifts sublate quickly final requests. bl|
+4646|83|O|124637.19|1996-06-18|1-URGENT|Clerk#000000036|0|n place of the blithely qu|
+4647|28|F|110958.36|1994-05-14|3-MEDIUM|Clerk#000000626|0|out the deposits. slyly final pinto beans haggle idly. slyly s|
+4672|79|O|199593.71|1995-11-07|1-URGENT|Clerk#000000475|0|lyly final dependencies caj|
+4673|82|O|58094.75|1996-08-13|4-NOT SPECIFIED|Clerk#000000914|0|c deposits are slyly. bravely ironic deposits cajole carefully after the |
+4674|37|F|115411.37|1994-04-19|1-URGENT|Clerk#000000122|0|careful hockey players. carefully pending deposits caj|
+4675|86|F|68817.08|1993-11-25|4-NOT SPECIFIED|Clerk#000000741|0|al deposits haggle slyly final|
+4676|14|O|182025.95|1995-09-01|2-HIGH|Clerk#000000407|0|s. slyly bold accounts sleep furiously special|
+4677|40|O|25661.87|1998-02-21|3-MEDIUM|Clerk#000000245|0|ly pending deposits after the carefully regular foxes sleep blithely after t|
+4678|88|O|131752.07|1998-08-02|4-NOT SPECIFIED|Clerk#000000175|0|side of the bold platelets detect slyly blithely ironic e|
+4679|88|F|7211.59|1993-01-20|2-HIGH|Clerk#000000905|0|ely regular accounts affix slyly. final dolphins are. furiously final de|
+4704|2|O|63873.14|1996-08-16|4-NOT SPECIFIED|Clerk#000000256|0|lithely final requests about the fluffily regular |
+4705|98|F|173340.09|1992-03-22|4-NOT SPECIFIED|Clerk#000000522|0| special instructions poa|
+4706|25|F|101709.52|1992-12-29|4-NOT SPECIFIED|Clerk#000000722|0| packages above the never regular packages nag packages. deposits c|
+4707|91|F|61052.10|1995-02-27|2-HIGH|Clerk#000000943|0|ully enticing accounts behind the regular|
+4708|85|F|56998.36|1994-10-01|1-URGENT|Clerk#000000383|0|ly thinly even accounts. unusu|
+4709|26|O|49903.57|1996-01-08|3-MEDIUM|Clerk#000000785|0|he furiously even deposits! ironic theodolites haggle blithely. r|
+4710|100|F|88966.68|1994-12-08|4-NOT SPECIFIED|Clerk#000000734|0|the final, regular foxes. carefully ironic pattern|
+4711|142|O|129546.56|1998-05-06|1-URGENT|Clerk#000000818|0|mptotes. unusual packages wake furiously qui|
+4736|139|O|67572.73|1995-11-20|2-HIGH|Clerk#000000563|0|blithely regular courts affix into the carefully ironic deposits. slyly exp|
+4737|79|F|62014.51|1993-03-11|4-NOT SPECIFIED|Clerk#000000275|0|ents use slyly among the unusual, ironic pearls. furiously pending |
+4738|5|F|149466.62|1992-04-08|2-HIGH|Clerk#000000150|0|deposits. thin acco|
+4739|148|F|68255.82|1993-02-21|5-LOW|Clerk#000000872|0|ing to the pending attainments: pending, express account|
+4740|68|O|42579.40|1996-07-05|2-HIGH|Clerk#000000420|0| dependencies haggle about the|
+4741|127|F|180692.90|1992-07-07|4-NOT SPECIFIED|Clerk#000000983|0|ly bold deposits are slyly about the r|
+4742|64|P|155356.80|1995-03-23|3-MEDIUM|Clerk#000000058|0|n packages. quickly regular ideas cajole blithely|
+4743|97|F|65702.39|1993-03-31|5-LOW|Clerk#000000048|0|pinto beans above the bold, even idea|
+4768|136|F|4820.55|1993-11-22|2-HIGH|Clerk#000000875|0|ctions snooze idly beneath the quick waters. fluffily u|
+4769|121|P|136765.03|1995-04-14|4-NOT SPECIFIED|Clerk#000000116|0|pon the asymptotes. idle, final account|
+4770|59|O|72150.68|1995-06-20|2-HIGH|Clerk#000000461|0|cial instructions believe carefully. |
+4771|95|F|49625.21|1992-12-14|1-URGENT|Clerk#000000571|0|lly express deposits serve furiously along the f|
+4772|28|F|64102.93|1994-09-14|1-URGENT|Clerk#000000708|0|es sleep. regular requests haggle furiously slyly |
+4773|122|O|196080.26|1995-12-23|1-URGENT|Clerk#000000327|0|ptotes was slyly along the|
+4774|52|F|124380.73|1993-04-20|3-MEDIUM|Clerk#000000299|0|eposits use blithely bold deposits. carefully regular gifts about the fin|
+4775|128|O|112444.42|1995-08-13|4-NOT SPECIFIED|Clerk#000000609|0|s integrate slyly slyly final instructions. carefully bold pack|
+4800|37|F|91795.13|1992-01-06|5-LOW|Clerk#000000625|0|ggle furiously along the pending pinto beans. deposits use: final foxe|
+4801|88|O|108353.08|1996-01-25|1-URGENT|Clerk#000000553|0|r the final sentiments. pending theodolites sleep doggedly across t|
+4802|130|O|5978.65|1997-01-23|3-MEDIUM|Clerk#000000400|0| ironic, thin packages wake furiously ironic, ironic deposits. the|
+4803|124|O|158776.68|1996-02-08|5-LOW|Clerk#000000892|0|lly unusual courts are ironic|
+4804|37|F|111547.31|1992-01-28|2-HIGH|Clerk#000000614|0|ly final accounts. blithely unusual theodolite|
+4805|16|F|172102.96|1992-04-25|4-NOT SPECIFIED|Clerk#000000514|0|even accounts wake furiously slyly final accounts; blithel|
+4806|7|F|35390.15|1993-04-21|5-LOW|Clerk#000000625|0|ave accounts. furiously pending wa|
+4807|53|O|138902.23|1997-01-09|3-MEDIUM|Clerk#000000310|0|kly. slyly special accounts|
+4832|34|O|84954.79|1997-12-04|3-MEDIUM|Clerk#000000548|0|final accounts sleep among the blithe|
+4833|133|O|84800.44|1996-05-12|3-MEDIUM|Clerk#000000256|0|r deposits against the slyly final excuses slee|
+4834|19|O|124539.00|1996-09-12|2-HIGH|Clerk#000000284|0|lar accounts. furiously ironic accounts haggle slyly |
+4835|146|F|70857.51|1994-10-25|1-URGENT|Clerk#000000250|0|s integrate furiously blithely expr|
+4836|65|O|78711.40|1996-12-18|1-URGENT|Clerk#000000691|0|c packages cajole carefully through the accounts. careful|
+4837|130|O|68519.84|1998-04-24|4-NOT SPECIFIED|Clerk#000000517|0|n accounts are regular, bold accounts. even instructions use request|
+4838|44|F|61811.33|1992-08-02|1-URGENT|Clerk#000000569|0|ffily bold sentiments. carefully close dolphins cajole across the |
+4839|25|F|71241.63|1994-05-10|1-URGENT|Clerk#000000925|0| even somas. slyly express ideas lose carefully. blithely unusu|
+4864|88|F|149614.34|1992-11-11|5-LOW|Clerk#000000423|0|ests nag within the quickly ironic asymptotes. ironic|
+4865|85|O|162113.46|1997-06-07|3-MEDIUM|Clerk#000000418|0|sits boost stealthily above the bl|
+4866|53|O|25767.07|1997-08-07|2-HIGH|Clerk#000000663|0|kages. unusual packages nag fluffily. qui|
+4867|10|F|9741.03|1992-05-21|1-URGENT|Clerk#000000891|0|ss the slyly regular dependencies. fluffily regular deposits within the car|
+4868|76|O|159005.35|1997-03-02|5-LOW|Clerk#000000729|0|regular asymptotes. regular packages sublate carefully al|
+4869|58|F|175422.13|1994-09-26|5-LOW|Clerk#000000802|0|boost! ironic packages un|
+4870|103|F|94534.07|1994-08-06|3-MEDIUM|Clerk#000000911|0|nto beans about the blithely regular d|
+4871|46|O|129636.99|1995-06-12|1-URGENT|Clerk#000000531|0|ven, special instructions across t|
+4896|85|F|93206.35|1992-08-22|1-URGENT|Clerk#000000622|0|sly pending deposits. final accounts boost above the sly, even|
+4897|80|F|115688.85|1992-09-17|5-LOW|Clerk#000000184|0|s. bold pinto beans sleep. evenly final accounts daz|
+4898|14|F|40572.64|1994-07-11|4-NOT SPECIFIED|Clerk#000000841|0|final patterns. special theodolites haggle ruthlessly at the blithely spec|
+4899|61|F|12291.83|1993-10-18|4-NOT SPECIFIED|Clerk#000000348|0| instructions. furiously even packages are furiously speci|
+4900|137|F|221320.76|1992-06-30|4-NOT SPECIFIED|Clerk#000000878|0|sleep quickly unusual |
+4901|79|O|146298.28|1997-12-31|4-NOT SPECIFIED|Clerk#000000980|0|inal dependencies cajole furiously. carefully express accounts na|
+4902|139|O|26011.20|1998-07-04|3-MEDIUM|Clerk#000000874|0| the slyly express dolphins. |
+4903|92|F|34363.63|1992-03-22|4-NOT SPECIFIED|Clerk#000000907|0|yly. multipliers within the fo|
+4928|4|F|59931.42|1993-10-04|4-NOT SPECIFIED|Clerk#000000952|0|slyly brave instructions after the ironic excuses haggle ruthlessly about|
+4929|149|O|135187.33|1996-02-29|3-MEDIUM|Clerk#000000109|0|uests. furiously special ideas poach. pending |
+4930|149|F|176867.34|1994-05-06|5-LOW|Clerk#000000593|0| haggle slyly quietly final theodolites. packages are furious|
+4931|50|F|115759.13|1994-11-17|1-URGENT|Clerk#000000356|0|leep. slyly express dolphins nag slyly. furiously regular s|
+4932|122|F|42927.07|1993-08-10|1-URGENT|Clerk#000000830|0|onic foxes. enticingly reg|
+4933|94|O|42945.82|1995-07-14|3-MEDIUM|Clerk#000000848|0|y special sauternes integr|
+4934|40|O|180478.16|1997-02-17|1-URGENT|Clerk#000000372|0|nes cajole; carefully special accounts haggle. special pinto beans nag |
+4935|40|F|162088.30|1993-05-25|4-NOT SPECIFIED|Clerk#000000601|0|c foxes. fluffily pendin|
+4960|124|F|153259.41|1995-02-26|5-LOW|Clerk#000000229|0|uriously even excuses. fluffily regular instructions along the furiously ironi|
+4961|58|O|89224.24|1998-04-06|3-MEDIUM|Clerk#000000731|0| braids. furiously even theodolites |
+4962|104|F|44781.32|1993-07-28|3-MEDIUM|Clerk#000000008|0| breach never ironic |
+4963|34|O|54175.35|1996-11-07|3-MEDIUM|Clerk#000000754|0|ully unusual epitaphs nod s|
+4964|101|O|204163.10|1997-07-28|4-NOT SPECIFIED|Clerk#000000144|0|ithely final theodolites. blithely regu|
+4965|52|F|110626.82|1993-10-21|5-LOW|Clerk#000000638|0|dependencies poach packages. sometim|
+4966|70|O|59186.02|1996-09-07|2-HIGH|Clerk#000000243|0|accounts. blithely ironic courts wake boldly furiously express |
+4967|98|O|103814.27|1997-02-17|3-MEDIUM|Clerk#000000397|0|e theodolites; furiously b|
+4992|62|F|203904.80|1992-05-10|1-URGENT|Clerk#000000166|0|telets nag carefully am|
+4993|13|F|145730.19|1994-08-04|4-NOT SPECIFIED|Clerk#000000258|0|ing instructions nag furiously. un|
+4994|43|O|216071.76|1996-06-29|4-NOT SPECIFIED|Clerk#000000868|0|oxes wake above the asymptotes. bold requests sleep br|
+4995|40|O|189651.76|1996-01-06|4-NOT SPECIFIED|Clerk#000000748|0|s. even deposits boost along the express, even theodolites. stealthily ir|
+4996|133|F|100750.67|1992-09-14|3-MEDIUM|Clerk#000000433|0|foxes. carefully special packages haggle quickly fluffi|
+4997|47|O|122611.05|1998-03-18|5-LOW|Clerk#000000040|0|egrate final pinto beans. fluffily special notornis use blith|
+4998|32|F|129096.80|1992-01-11|4-NOT SPECIFIED|Clerk#000000054|0|alongside of the quickly final requests hang always|
+4999|85|F|98643.17|1993-06-26|2-HIGH|Clerk#000000504|0| dolphins cajole blithely above the sly |
+5024|124|O|116127.69|1996-10-25|3-MEDIUM|Clerk#000000659|0|r foxes. regular excuses are about the quickly regular theodolites. regular, |
+5025|121|O|20099.43|1997-02-03|5-LOW|Clerk#000000805|0|ackages are slyly about the quickly |
+5026|28|O|13197.78|1997-09-06|1-URGENT|Clerk#000000955|0|y final requests us|
+5027|148|O|181346.56|1997-08-30|2-HIGH|Clerk#000000751|0|e-- final, pending requests along t|
+5028|13|F|30755.69|1992-04-17|2-HIGH|Clerk#000000180|0|ickly blithely express deposits. b|
+5029|11|F|19811.69|1992-11-14|3-MEDIUM|Clerk#000000469|0|. regular accounts haggle slyly. regul|
+5030|106|O|71781.23|1998-05-25|4-NOT SPECIFIED|Clerk#000000564|0| wake slyly furiously thin requests. ironic pinto beans ha|
+5031|139|F|91438.59|1994-12-02|3-MEDIUM|Clerk#000000788|0|lar instructions haggle blithely pending foxes? sometimes final excuses h|
+5056|52|O|62258.18|1997-02-15|5-LOW|Clerk#000000828|0|lithely above the express ideas. blithely final deposits are fluffily spec|
+5057|64|O|76164.41|1997-08-03|1-URGENT|Clerk#000000955|0|r ironic requests of the carefully ironic dependencies wake slyly a|
+5058|119|O|17031.01|1998-03-23|1-URGENT|Clerk#000000367|0| the pending packages wake after the quickly speci|
+5059|43|F|67173.82|1993-11-10|2-HIGH|Clerk#000000058|0|latelets. final, regular accounts cajole furiously ironic pinto beans? do|
+5060|112|F|65218.47|1992-07-07|4-NOT SPECIFIED|Clerk#000000333|0|e according to the excuses. express theodo|
+5061|101|F|52190.52|1993-08-14|1-URGENT|Clerk#000000009|0|e packages use fluffily according to the carefully ironic deposits. bol|
+5062|61|F|109247.00|1992-10-08|3-MEDIUM|Clerk#000000012|0|ithely. blithely bold theodolites affix. blithely final deposits haggle ac|
+5063|23|O|98753.57|1997-05-17|2-HIGH|Clerk#000000745|0|lyly after the pending foxes. express theodolites breach across t|
+5088|130|F|101616.44|1993-01-06|5-LOW|Clerk#000000930|0|ole slyly since the quickly ironic br|
+5089|130|F|109246.54|1992-07-29|1-URGENT|Clerk#000000677|0|cial platelets. quiet, final ideas cajole carefully. unusu|
+5090|89|O|132838.49|1997-03-09|1-URGENT|Clerk#000000953|0|ress accounts affix silently carefully quick accounts. carefully f|
+5091|148|O|47852.06|1998-05-21|3-MEDIUM|Clerk#000000311|0|egular decoys mold carefully fluffily unus|
+5092|22|O|195834.96|1995-10-30|5-LOW|Clerk#000000194|0|are blithely along the pin|
+5093|79|F|190693.92|1993-09-03|3-MEDIUM|Clerk#000000802|0|ully ironic theodolites sleep above the furiously ruthless instructions. bli|
+5094|106|F|74892.08|1993-03-29|4-NOT SPECIFIED|Clerk#000000406|0|uickly pending deposits haggle quickly ide|
+5095|97|F|184583.99|1992-04-22|2-HIGH|Clerk#000000964|0|accounts are carefully! slyly even packages wake slyly a|
+5120|16|O|28007.73|1996-06-05|1-URGENT|Clerk#000000332|0|against the slyly express requests. furiousl|
+5121|133|F|150334.57|1992-05-11|4-NOT SPECIFIED|Clerk#000000736|0|gular requests. furiously final pearls against the permanent, thin courts s|
+5122|70|O|79863.84|1996-02-10|5-LOW|Clerk#000000780|0|blithely. slyly ironic deposits nag. excuses s|
+5123|10|O|11850.45|1998-02-10|1-URGENT|Clerk#000000776|0|ic requests. furiously ironic packages grow above the express, ironic inst|
+5124|25|O|159170.80|1997-04-04|4-NOT SPECIFIED|Clerk#000000749|0|kly even courts. bold packages solve. |
+5125|28|O|38065.28|1998-02-07|5-LOW|Clerk#000000834|0|ructions. dolphins wake slowly carefully unusual |
+5126|112|F|92123.32|1992-10-12|4-NOT SPECIFIED|Clerk#000000270|0|s. unusual deposits |
+5127|73|O|48024.99|1997-01-15|5-LOW|Clerk#000000829|0|fully express pinto beans. slyly final accounts along the ironic dugouts use s|
+5152|44|O|60568.34|1997-01-04|3-MEDIUM|Clerk#000000963|0| for the blithely reg|
+5153|113|O|193832.28|1995-08-26|1-URGENT|Clerk#000000954|0| the furiously ironic foxes. express packages shall cajole carefully across|
+5154|8|O|28070.86|1997-04-13|3-MEDIUM|Clerk#000000316|0|inal requests. slyly regular deposits nag. even deposits haggle agains|
+5155|77|F|70183.29|1994-06-12|2-HIGH|Clerk#000000108|0|y pending deposits are ag|
+5156|125|O|59439.44|1996-11-04|5-LOW|Clerk#000000117|0|ngside of the multipliers solve slyly requests. regu|
+5157|142|O|167056.34|1997-07-06|4-NOT SPECIFIED|Clerk#000000689|0|closely above the unusual deposits. furiously|
+5158|76|O|240284.95|1997-01-21|1-URGENT|Clerk#000000541|0| regular foxes. even foxes wake blithely |
+5159|106|O|147543.26|1996-09-25|1-URGENT|Clerk#000000303|0|tegrate slyly around the slyly sly sauternes. final pa|
+5184|85|O|209155.48|1998-07-20|5-LOW|Clerk#000000250|0|nding accounts detect final, even|
+5185|148|O|206179.68|1997-07-25|3-MEDIUM|Clerk#000000195|0| regular ideas about the even ex|
+5186|52|O|208892.63|1996-08-03|1-URGENT|Clerk#000000332|0|pecial platelets. slyly final ac|
+5187|55|O|46380.69|1997-07-16|3-MEDIUM|Clerk#000000682|0|ckly according to t|
+5188|140|P|66268.86|1995-03-02|4-NOT SPECIFIED|Clerk#000000029|0|counts. finally ironic requests ab|
+5189|71|F|184172.31|1993-11-26|5-LOW|Clerk#000000940|0|e after the pending accounts. asymptotes boost. re|
+5190|58|F|89684.31|1992-04-26|5-LOW|Clerk#000000888|0|equests. slyly unusual|
+5191|77|F|119910.04|1994-12-11|4-NOT SPECIFIED|Clerk#000000318|0|ing, regular deposits alongside of the deposits boost fluffily quickly ev|
+5216|59|O|16763.95|1997-08-14|3-MEDIUM|Clerk#000000418|0|des boost across the platelets. slyly busy theodolit|
+5217|35|O|135745.58|1995-10-13|2-HIGH|Clerk#000000873|0|ons might wake quickly according to th|
+5218|82|F|73882.37|1992-07-30|4-NOT SPECIFIED|Clerk#000000683|0|y ruthless packages according to the bold, ironic package|
+5219|88|O|21267.72|1997-02-27|1-URGENT|Clerk#000000510|0|aggle always. foxes above the ironic deposits |
+5220|10|F|24844.39|1992-07-30|2-HIGH|Clerk#000000051|0| final packages. ideas detect slyly around|
+5221|13|O|71968.10|1995-06-09|4-NOT SPECIFIED|Clerk#000000324|0|lar accounts above the sl|
+5222|80|F|1051.15|1994-05-27|4-NOT SPECIFIED|Clerk#000000613|0|along the bold ideas. furiously final foxes snoo|
+5223|149|F|105561.21|1994-06-30|1-URGENT|Clerk#000000745|0|e. theodolites serve blithely unusual, final foxes. carefully pending packag|
+5248|70|P|86958.28|1995-04-15|2-HIGH|Clerk#000000737|0|theodolites cajole according to the silent packages. quickly ironic packages a|
+5249|103|F|123586.03|1994-09-06|3-MEDIUM|Clerk#000000019|0|refully bold accounts |
+5250|97|O|29673.73|1995-07-16|2-HIGH|Clerk#000000307|0|. carefully final instructions sleep among the finally regular dependen|
+5251|34|O|34004.48|1995-04-12|3-MEDIUM|Clerk#000000687|0| ironic dugouts detect. reque|
+5252|91|O|173145.37|1996-02-17|1-URGENT|Clerk#000000724|0| ironic accounts among the silent asym|
+5253|148|P|108361.46|1995-04-11|2-HIGH|Clerk#000000275|0|egular requests! blithely regular deposits alongside of t|
+5254|112|F|196989.09|1992-07-26|4-NOT SPECIFIED|Clerk#000000527|0|he express, even ideas cajole blithely special requests|
+5255|64|O|75074.07|1996-07-12|5-LOW|Clerk#000000591|0|ly slow forges. express foxes haggle. regular, even asymp|
+5280|34|O|68052.70|1997-12-03|3-MEDIUM|Clerk#000000604|0|riously ironic instructions. ironic ideas according to the accounts boost fur|
+5281|124|O|179418.31|1995-11-02|2-HIGH|Clerk#000000158|0|ackages haggle slyly a|
+5282|50|O|94446.69|1998-01-30|1-URGENT|Clerk#000000030|0|rding to the unusual, bold accounts. regular instructions|
+5283|131|F|18594.66|1994-06-04|3-MEDIUM|Clerk#000000579|0|ests. even, final ideas alongside of t|
+5284|61|O|40548.99|1995-07-09|4-NOT SPECIFIED|Clerk#000000155|0| careful dependencies use sly|
+5285|70|F|99377.51|1994-01-18|2-HIGH|Clerk#000000976|0|p across the furiously ironic deposits.|
+5286|116|O|79646.89|1997-09-26|5-LOW|Clerk#000000606|0|structions are furiously quickly ironic asymptotes. quickly iro|
+5287|25|F|30045.95|1993-12-22|5-LOW|Clerk#000000406|0|regular packages. bold instructions sleep always. carefully final p|
+5312|65|F|66697.95|1995-02-24|2-HIGH|Clerk#000000690|0|ter the even, bold foxe|
+5313|13|O|159870.44|1997-06-17|4-NOT SPECIFIED|Clerk#000000896|0|le. final courts haggle furiously according to the |
+5314|34|O|26999.83|1995-06-02|2-HIGH|Clerk#000000617|0|ions across the quickly special d|
+5315|139|F|55554.97|1992-10-29|4-NOT SPECIFIED|Clerk#000000035|0| furiously. quickly unusual packages use. sly|
+5316|100|F|62316.61|1994-01-31|1-URGENT|Clerk#000000734|0| requests haggle across the regular, pending deposits. furiously regular requ|
+5317|37|F|228002.51|1994-09-09|5-LOW|Clerk#000000687|0|jole quickly at the slyly pend|
+5318|59|F|106935.19|1993-04-04|2-HIGH|Clerk#000000663|0|efully regular dolphins. even ideas nag fluffily furiously even packa|
+5319|98|O|68619.29|1996-01-21|1-URGENT|Clerk#000000237|0|lent requests. quickly pe|
+5344|109|O|88216.32|1998-06-21|3-MEDIUM|Clerk#000000569|0|s. ironic excuses cajole across the|
+5345|31|O|111924.56|1997-08-24|1-URGENT|Clerk#000000057|0|r the slyly silent packages. pending, even pinto b|
+5346|37|F|149536.20|1993-12-26|2-HIGH|Clerk#000000220|0|gly close packages against the even, regular escapades boost evenly accordi|
+5347|49|F|173024.71|1995-02-22|3-MEDIUM|Clerk#000000180|0|onic, regular deposits. packag|
+5348|53|O|119164.96|1997-11-08|5-LOW|Clerk#000000497|0|totes. accounts after the furiously|
+5349|67|O|38038.84|1996-09-01|1-URGENT|Clerk#000000960|0|le along the carefully bold dolphins. carefully special packa|
+5350|76|F|113417.03|1993-10-10|5-LOW|Clerk#000000604|0|ccounts after the carefully pending requests believe |
+5351|122|O|76799.25|1998-05-11|1-URGENT|Clerk#000000443|0|to beans sleep furiously after the carefully even|
+5376|149|F|98422.83|1994-07-04|5-LOW|Clerk#000000392|0|. quickly ironic deposits integrate along|
+5377|64|O|117728.37|1997-04-24|2-HIGH|Clerk#000000917|0|ons nag blithely furiously regula|
+5378|43|F|101899.93|1992-10-25|1-URGENT|Clerk#000000520|0|n ideas. regular accounts haggle. ironic ideas use along the bold ideas. blith|
+5379|89|O|47010.15|1995-08-08|2-HIGH|Clerk#000000503|0|he unusual accounts. carefully special instructi|
+5380|148|O|123014.83|1997-10-12|1-URGENT|Clerk#000000481|0|le slyly about the slyly final dolphins. fu|
+5381|32|F|223995.46|1993-01-29|5-LOW|Clerk#000000531|0|arefully bold packages are slyly furiously ironic foxes. fluffil|
+5382|35|F|138423.03|1992-01-13|5-LOW|Clerk#000000809|0|lent deposits are according to the reg|
+5383|31|O|11474.95|1995-05-26|5-LOW|Clerk#000000409|0|ly bold requests hang furiously furiously unusual accounts. evenly unusu|
+5408|23|F|123477.05|1992-07-21|5-LOW|Clerk#000000735|0|egular requests according to the|
+5409|13|F|145040.38|1992-01-09|5-LOW|Clerk#000000171|0|eans. regular accounts are regul|
+5410|22|O|139104.17|1998-07-28|4-NOT SPECIFIED|Clerk#000000117|0|final deposits: pending excuses boost. ironic theodolites cajole furi|
+5411|61|O|62541.27|1997-05-16|3-MEDIUM|Clerk#000000800|0|equests cajole slyly furious|
+5412|142|O|109979.71|1998-01-20|2-HIGH|Clerk#000000151|0|ets boost furiously regular accounts. regular foxes above th|
+5413|94|O|224382.57|1997-10-17|1-URGENT|Clerk#000000066|0|e even excuses. always final depen|
+5414|100|F|167017.39|1993-03-25|4-NOT SPECIFIED|Clerk#000000242|0|lent dependencies? carefully express requests sleep furiously ac|
+5415|23|F|176864.83|1992-08-05|3-MEDIUM|Clerk#000000998|0|ly even ideas nag blithely above the final instructions|
+5440|130|O|3223.17|1997-01-12|1-URGENT|Clerk#000000154|0|posits boost regularly ironic packages. regular, ironic deposits wak|
+5441|41|F|131891.05|1994-07-21|4-NOT SPECIFIED|Clerk#000000257|0|after the furiously ironic |
+5442|43|O|139332.94|1998-01-13|4-NOT SPECIFIED|Clerk#000000954|0|ully. quickly express accounts against the|
+5443|131|O|124950.79|1996-10-10|4-NOT SPECIFIED|Clerk#000000492|0|al foxes could detect. blithely stealthy asymptotes kind|
+5444|130|P|172908.01|1995-03-18|1-URGENT|Clerk#000000677|0| asymptotes. asymptotes cajole quickly quickly bo|
+5445|115|F|114990.63|1993-07-26|5-LOW|Clerk#000000623|0|s. even, special requests cajole furiously even, |
+5446|7|F|29920.80|1994-06-21|5-LOW|Clerk#000000304|0| furiously final pac|
+5447|13|O|29029.84|1996-03-16|3-MEDIUM|Clerk#000000597|0|uternes around the furiously bold accounts wake after |
+5472|70|F|221636.83|1993-04-11|5-LOW|Clerk#000000552|0|counts. deposits about the slyly dogged pinto beans cajole slyly|
+5473|65|F|63041.33|1992-03-25|4-NOT SPECIFIED|Clerk#000000306|0|te the quickly stealthy ideas. even, regular deposits above|
+5474|55|F|131079.52|1992-06-01|4-NOT SPECIFIED|Clerk#000000487|0|gle blithely enticing ideas. final, exp|
+5475|139|O|10645.48|1996-07-07|5-LOW|Clerk#000000856|0|es shall boost slyly. furiously even deposits lose. instruc|
+5476|91|O|26906.38|1997-11-06|1-URGENT|Clerk#000000189|0|furiously final ideas. furiously bold dependencies sleep care|
+5477|107|O|130125.64|1997-12-30|5-LOW|Clerk#000000689|0|ckages. ironic deposits caj|
+5478|116|O|97502.23|1996-05-17|1-URGENT|Clerk#000000272|0|ckages. quickly pending deposits thrash furiously: bl|
+5479|70|F|70553.45|1993-12-22|3-MEDIUM|Clerk#000000335|0|ng asymptotes. pinto beans sleep care|
+5504|19|F|41492.25|1993-01-06|2-HIGH|Clerk#000000221|0|y pending packages. furiousl|
+5505|95|O|147329.51|1997-10-04|5-LOW|Clerk#000000719|0| final, regular packages according to the slyly ironic accounts nag ironica|
+5506|91|F|8413.31|1993-11-08|1-URGENT|Clerk#000000292|0|nusual theodolites. sly|
+5507|2|O|140363.70|1998-05-28|5-LOW|Clerk#000000692|0|the carefully ironic instructions are quickly iro|
+5508|56|O|3808.05|1996-06-21|1-URGENT|Clerk#000000128|0|y express packages cajole furiously. slyly unusual requests |
+5509|80|F|135335.96|1994-04-08|5-LOW|Clerk#000000164|0|usual deposits use packages. furiously final requests wake slyly about th|
+5510|37|F|126948.81|1993-01-08|3-MEDIUM|Clerk#000000819|0| nag slyly. carefully eve|
+5511|79|F|151089.96|1994-11-29|1-URGENT|Clerk#000000438|0|ng instructions integrate fluffily among the fluffily silent accounts. bli|
+5536|116|O|108196.56|1998-03-16|4-NOT SPECIFIED|Clerk#000000076|0| carefully final dolphins. ironic, ironic deposits lose. bold, |
+5537|118|O|102207.20|1996-10-03|3-MEDIUM|Clerk#000000742|0|ng to the daring, final |
+5538|139|F|90981.28|1993-12-25|1-URGENT|Clerk#000000992|0|ttainments. slyly final ideas are about the furiously silent excuses.|
+5539|119|F|39397.60|1994-07-31|5-LOW|Clerk#000000675|0|structions. slyly regular patterns solve above the carefully expres|
+5540|130|O|90707.58|1996-10-12|4-NOT SPECIFIED|Clerk#000000120|0|y ironic packages cajole blithely|
+5541|143|O|37526.68|1997-09-30|3-MEDIUM|Clerk#000000217|0|encies among the silent accounts sleep slyly quickly pending deposits|
+5542|49|O|6402.41|1996-04-20|4-NOT SPECIFIED|Clerk#000000100|0|riously among the regularly regular pac|
+5543|115|F|118201.53|1993-09-25|3-MEDIUM|Clerk#000000644|0|ckly regular epitaphs. carefully bold accounts haggle furiously|
+5568|31|O|105421.09|1995-06-07|3-MEDIUM|Clerk#000000491|0| nag. fluffily pending de|
+5569|109|F|126113.32|1993-04-30|4-NOT SPECIFIED|Clerk#000000759|0|e regular dependencies. furiously unusual ideas b|
+5570|112|O|78567.55|1996-08-12|2-HIGH|Clerk#000000795|0|eans. ironic, even requests doze |
+5571|103|F|79248.35|1992-12-19|4-NOT SPECIFIED|Clerk#000000184|0|ts cajole furiously carefully regular sheaves. un|
+5572|8|F|182966.39|1994-07-17|2-HIGH|Clerk#000000163|0|e fluffily express deposits cajole slyly across th|
+5573|37|O|158479.37|1996-08-15|3-MEDIUM|Clerk#000000055|0|lites. slyly final pinto beans about the carefully regul|
+5574|28|F|129803.03|1992-03-10|4-NOT SPECIFIED|Clerk#000000002|0|n deposits. special, regular t|
+5575|103|O|51839.94|1995-07-24|5-LOW|Clerk#000000948|0|uriously express frays breach|
+5600|95|O|53649.35|1997-02-08|4-NOT SPECIFIED|Clerk#000000019|0|lly regular deposits. car|
+5601|11|F|118570.79|1992-01-06|2-HIGH|Clerk#000000827|0|gular deposits wake platelets? blithe|
+5602|130|O|67979.49|1997-07-30|3-MEDIUM|Clerk#000000395|0|onic asymptotes haggl|
+5603|71|F|145100.47|1992-06-20|4-NOT SPECIFIED|Clerk#000000535|0| asymptotes. fluffily ironic instructions are. pending pinto bean|
+5604|46|O|98987.51|1998-04-14|4-NOT SPECIFIED|Clerk#000000123|0|ously across the blithely ironic pinto beans. sile|
+5605|35|O|172899.84|1996-08-22|2-HIGH|Clerk#000000538|0|sleep carefully final packages. dependencies wake slyly. theodol|
+5606|149|O|219959.08|1996-11-12|5-LOW|Clerk#000000688|0|uriously express pinto beans. packages sh|
+5607|92|F|24660.06|1992-01-01|4-NOT SPECIFIED|Clerk#000000137|0|c requests promise quickly fluffily ironic deposits. caref|
+5632|79|O|89503.11|1996-02-05|1-URGENT|Clerk#000000508|0|ons. blithely pending pinto beans thrash. furiously busy theodoli|
+5633|79|O|207119.83|1998-05-31|3-MEDIUM|Clerk#000000841|0|cial deposits wake final, final|
+5634|68|O|99494.67|1996-07-31|3-MEDIUM|Clerk#000000915|0|out the accounts. carefully ironic ideas are slyly. sheaves could h|
+5635|70|F|192217.86|1992-08-16|3-MEDIUM|Clerk#000000734|0|nal platelets sleep daringly. idle, final accounts about |
+5636|122|F|143350.75|1995-02-16|3-MEDIUM|Clerk#000000916|0|. boldly even Tiresias sleep. blithely ironic packages among the ca|
+5637|103|O|128776.90|1996-06-17|3-MEDIUM|Clerk#000000183|0|nic dolphins are regular packages. ironic pinto beans hagg|
+5638|109|F|79197.77|1994-01-17|1-URGENT|Clerk#000000355|0|enly bold deposits eat. special realms play against the regular, speci|
+5639|145|F|9669.46|1994-06-02|3-MEDIUM|Clerk#000000005|0|ending packages use after the blithely regular accounts. regular package|
+5664|119|O|186215.81|1998-07-23|2-HIGH|Clerk#000000789|0|the quickly ironic dolp|
+5665|100|F|129821.09|1993-06-28|4-NOT SPECIFIED|Clerk#000000513|0| carefully special instructions. ironic pinto beans nag slyly blithe|
+5666|14|F|121663.68|1994-02-02|2-HIGH|Clerk#000000396|0|mptotes. quickly final instructions are |
+5667|44|O|37301.25|1995-08-10|1-URGENT|Clerk#000000358|0|s print upon the quickly ironic packa|
+5668|109|F|13679.32|1995-03-22|4-NOT SPECIFIED|Clerk#000000047|0|p slyly slyly express accoun|
+5669|74|O|113156.30|1996-05-06|1-URGENT|Clerk#000000336|0|ng packages nag fluffily furio|
+5670|7|F|101429.61|1993-04-21|5-LOW|Clerk#000000922|0|he carefully final packages. deposits are slyly among the requests. |
+5671|43|O|176647.54|1998-02-06|2-HIGH|Clerk#000000838|0|k dependencies. slyly |
+5696|142|P|198723.30|1995-05-04|1-URGENT|Clerk#000000447|0|e quickly unusual pack|
+5697|55|F|99177.69|1992-10-05|1-URGENT|Clerk#000000112|0|pendencies impress furiously. bold, final requests solve ab|
+5698|95|F|154936.43|1994-05-21|3-MEDIUM|Clerk#000000455|0|he furiously silent accounts haggle blithely against the carefully unusual|
+5699|142|F|226314.91|1992-07-30|5-LOW|Clerk#000000311|0|o beans. ironic asymptotes boost. blithe, final courts integrate|
+5700|143|O|79901.18|1997-12-25|1-URGENT|Clerk#000000618|0|ly pending dolphins sleep carefully slyly pending i|
+5701|43|O|16689.19|1997-02-07|5-LOW|Clerk#000000798|0| blithely final pinto beans. blit|
+5702|97|F|153024.28|1993-09-07|4-NOT SPECIFIED|Clerk#000000743|0|ironic accounts. final accounts wake express deposits. final pac|
+5703|121|F|1816.28|1993-05-16|3-MEDIUM|Clerk#000000647|0|ly special instructions. slyly even reque|
+5728|80|F|85397.04|1994-12-11|4-NOT SPECIFIED|Clerk#000000426|0|furiously express pin|
+5729|44|F|88080.33|1994-10-10|2-HIGH|Clerk#000000843|0|uffily sly accounts about|
+5730|11|O|10934.84|1997-12-18|1-URGENT|Clerk#000000181|0|l platelets. ironic pinto beans wake slyly. quickly b|
+5731|8|O|57823.37|1997-05-17|5-LOW|Clerk#000000841|0| silent excuses among the express accounts wake |
+5732|37|O|28330.42|1997-08-03|1-URGENT|Clerk#000000910|0|he quickly bold asymptotes: final platelets wake quickly. blithely final pinto|
+5733|101|F|38545.97|1993-03-17|2-HIGH|Clerk#000000873|0|osits. pending accounts boost quickly. furiously permanent acco|
+5734|94|O|45860.94|1997-10-12|3-MEDIUM|Clerk#000000084|0|efully even braids detect blithely alo|
+5735|40|F|39358.51|1994-12-11|3-MEDIUM|Clerk#000000600|0| bold realms cajole slyly fu|
+5760|25|F|59404.77|1994-05-25|4-NOT SPECIFIED|Clerk#000000498|0|s among the blithely regular frays haggle ironically bold theodolites. al|
+5761|16|O|130345.90|1998-07-06|3-MEDIUM|Clerk#000000208|0|s asymptotes cajole boldly. regular, |
+5762|49|O|165019.32|1997-02-14|1-URGENT|Clerk#000000901|0|ly bold packages: slyly ironic deposits sleep quietly foxes. express a|
+5763|8|O|140838.11|1998-06-26|4-NOT SPECIFIED|Clerk#000000633|0|according to the furiously regular pinto beans. even accounts wake fu|
+5764|131|F|53212.95|1993-10-03|4-NOT SPECIFIED|Clerk#000000363|0| furiously regular deposits haggle fluffily around th|
+5765|52|F|249900.42|1994-12-15|5-LOW|Clerk#000000959|0|longside of the quickly final packages. instructions so|
+5766|49|F|47940.51|1993-09-27|5-LOW|Clerk#000000753|0|. quickly final packages print slyly. fu|
+5767|118|F|135643.87|1992-04-29|2-HIGH|Clerk#000000225|0|ts wake fluffily above the r|
+5792|26|F|158991.89|1993-04-04|2-HIGH|Clerk#000000731|0|packages. doggedly bold deposits integrate furiously across the|
+5793|37|O|119887.47|1997-07-13|2-HIGH|Clerk#000000294|0|thely. fluffily even instructi|
+5794|8|F|122823.78|1993-04-05|5-LOW|Clerk#000000855|0|t accounts kindle about the gifts. as|
+5795|37|F|35514.45|1992-05-05|2-HIGH|Clerk#000000581|0| even instructions x-ray ironic req|
+5796|149|O|23280.61|1996-01-23|3-MEDIUM|Clerk#000000326|0|eodolites. slyly ironic pinto beans at the silent, special request|
+5797|122|O|15313.61|1997-10-15|4-NOT SPECIFIED|Clerk#000000381|0|ng! packages against the blithely b|
+5798|106|O|125011.92|1998-03-30|5-LOW|Clerk#000000343|0|lent accounts affix quickly! platelets run slyly slyly final packages. f|
+5799|26|O|71381.21|1995-08-03|1-URGENT|Clerk#000000238|0| unusual deposits sleep blithely along the carefully even requests. care|
+5824|56|O|169107.85|1996-12-03|2-HIGH|Clerk#000000171|0|unusual packages. even ideas along the even requests are along th|
+5825|61|F|23020.62|1995-02-21|5-LOW|Clerk#000000494|0|regular packages use bravely.|
+5826|22|O|21119.86|1998-06-13|1-URGENT|Clerk#000000087|0|even, regular dependenc|
+5827|31|O|137297.71|1998-07-23|3-MEDIUM|Clerk#000000660|0|hely furiously blithe dolphins. slyly |
+5828|127|F|62172.34|1994-03-06|5-LOW|Clerk#000000377|0|ages boost never during the final packa|
+5829|125|O|183734.56|1997-01-11|1-URGENT|Clerk#000000196|0|gular accounts. bold accounts are blithely furiously ironic r|
+5830|85|F|28223.57|1993-03-25|3-MEDIUM|Clerk#000000233|0|lites haggle. ironic, ironic instructions maintain blit|
+5831|139|O|113505.19|1996-11-17|5-LOW|Clerk#000000585|0|s final, final pinto beans. unusual depos|
+5856|37|F|71460.49|1994-11-06|2-HIGH|Clerk#000000634|0|special excuses. slyly final theodolites cajole blithely furiou|
+5857|124|O|158345.31|1997-11-06|4-NOT SPECIFIED|Clerk#000000267|0|gage blithely. quickly special ac|
+5858|64|F|181320.50|1992-07-14|4-NOT SPECIFIED|Clerk#000000580|0|lyly pending dugouts believe through the ironic deposits. silent s|
+5859|5|O|210643.96|1997-04-23|1-URGENT|Clerk#000000993|0|requests boost. asymptotes across the deposits solve slyly furiously pendin|
+5860|13|F|9495.28|1992-02-20|4-NOT SPECIFIED|Clerk#000000079|0| beans. bold, special foxes sleep about the ir|
+5861|139|O|41450.19|1997-04-10|3-MEDIUM|Clerk#000000094|0|rthogs cajole slyly. express packages sleep blithely final |
+5862|64|O|30550.90|1997-02-20|1-URGENT|Clerk#000000039|0|leep beneath the quickly busy excuses. ironic theodolit|
+5863|65|F|67941.54|1993-11-22|3-MEDIUM|Clerk#000000774|0|ets about the slyly pending ideas sleep according to the blithely |
+5888|46|O|67167.19|1996-09-28|3-MEDIUM|Clerk#000000748|0|quickly against the furiously final requests. evenly fi|
+5889|22|O|15417.57|1995-05-23|5-LOW|Clerk#000000690|0|ites wake across the slyly ironic|
+5890|49|F|41162.24|1992-11-04|2-HIGH|Clerk#000000013|0|packages. final, final reques|
+5891|46|F|41760.00|1992-12-29|3-MEDIUM|Clerk#000000302|0|ounts haggle furiously abo|
+5892|101|P|92340.77|1995-05-09|5-LOW|Clerk#000000639|0| pending instruction|
+5893|2|F|44777.63|1992-07-08|4-NOT SPECIFIED|Clerk#000000560|0|final sentiments. instructions boost above the never speci|
+5894|71|F|70377.31|1994-08-13|2-HIGH|Clerk#000000776|0|regular deposits wake|
+5895|64|O|201419.83|1997-01-01|4-NOT SPECIFIED|Clerk#000000747|0| ironic, unusual requests cajole blithely special, special deposits. s|
+5920|119|F|142767.26|1994-11-20|2-HIGH|Clerk#000000081|0|ns: even ideas cajole slyly among the packages. never ironic patterns|
+5921|58|F|152940.00|1994-04-07|5-LOW|Clerk#000000125|0|kly special requests breach.|
+5922|143|O|142494.99|1996-11-14|5-LOW|Clerk#000000625|0| ironic instructions haggle furiously blithely regular accounts: even platele|
+5923|101|O|157968.27|1997-05-27|2-HIGH|Clerk#000000304|0|o beans haggle slyly above the regular, even dependencies|
+5924|31|O|106823.97|1995-10-10|3-MEDIUM|Clerk#000000433|0|arefully after the pains. blithely ironic pinto |
+5925|146|O|242588.87|1995-11-13|5-LOW|Clerk#000000602|0|ourts. boldly regular foxes might sleep. slyly express tithes against |
+5926|76|F|105770.53|1994-05-20|5-LOW|Clerk#000000071|0| carefully after the furiously even re|
+5927|116|O|84983.90|1997-08-28|4-NOT SPECIFIED|Clerk#000000972|0|endencies according to the slyly ironic foxes detect furiously about the furio|
+5952|148|O|128624.99|1997-04-14|3-MEDIUM|Clerk#000000950|0| regular, final pla|
+5953|7|F|95312.81|1992-03-28|1-URGENT|Clerk#000000049|0|ages are furiously. slowly bold requests|
+5954|28|F|167262.34|1992-12-03|1-URGENT|Clerk#000000968|0|requests along the blith|
+5955|94|P|67944.38|1995-03-27|5-LOW|Clerk#000000340|0|deas integrate. fluffily regular pa|
+5956|22|O|118036.54|1998-05-18|1-URGENT|Clerk#000000587|0|le even, express platelets.|
+5957|89|F|230949.45|1993-12-27|2-HIGH|Clerk#000000020|0| dependencies are slyly. bold accounts according to the carefully regular r|
+5958|115|O|145060.41|1995-09-16|3-MEDIUM|Clerk#000000787|0|e final requests detect alongside of the qu|
+5959|23|F|195515.26|1992-05-15|3-MEDIUM|Clerk#000000913|0|into beans use ironic, unusual foxes. carefully regular excuses boost caref|
+5984|70|F|83413.30|1994-06-18|5-LOW|Clerk#000000023|0|ickly final pains haggle along the furiously ironic pinto bea|
+5985|143|F|3942.73|1995-01-12|3-MEDIUM|Clerk#000000417|0|as nag fluffily slyly permanent accounts. regular depo|
+5986|115|F|92187.80|1992-04-22|2-HIGH|Clerk#000000674|0|iously unusual notornis are |
+5987|64|O|98956.82|1996-08-03|1-URGENT|Clerk#000000464|0| ideas. quietly final accounts haggle blithely pending escapade|
+5988|31|F|41655.51|1993-11-22|4-NOT SPECIFIED|Clerk#000000867|0|fully express accounts. final pi|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders.tbl
new file mode 100644
index 0000000..1ebd663
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/orders.tbl
@@ -0,0 +1,1500 @@
+1|37|O|131251.81|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |
+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 sil|
+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 reques|
+802|137|F|156381.95|1995-01-05|1-URGENT|Clerk#000000516|0|posits. ironic, pending requests cajole. even theodol|
+803|16|O|27629.66|1997-04-29|5-LOW|Clerk#000000260|0|ic instructions. even deposits haggle furiously at the deposits-- regular de|
+804|50|F|94400.43|1993-03-12|3-MEDIUM|Clerk#000000931|0|s. blithely final foxes are about the packag|
+805|127|O|90042.41|1995-07-05|4-NOT SPECIFIED|Clerk#000000856|0|y according to the fluffily |
+806|131|O|26839.16|1996-06-20|2-HIGH|Clerk#000000240|0| the ironic packages wake carefully fina|
+807|145|F|222392.53|1993-11-24|3-MEDIUM|Clerk#000000012|0|refully special tithes. blithely regular accoun|
+832|29|F|68494.08|1992-04-19|5-LOW|Clerk#000000495|0|xes. bravely regular packages sleep up the furiously bold accou|
+833|56|F|49033.69|1994-02-13|3-MEDIUM|Clerk#000000437|0|ts haggle quickly across the slyl|
+834|43|F|46459.92|1994-05-23|3-MEDIUM|Clerk#000000805|0| sleep. quickly even foxes are boldly. slyly express requests use slyly|
+835|65|O|62430.67|1995-10-08|4-NOT SPECIFIED|Clerk#000000416|0|s about the carefully special foxes haggle quickly about the|
+836|70|O|72843.48|1996-11-25|4-NOT SPECIFIED|Clerk#000000729|0|ely bold excuses sleep regular ideas. furiously unusual ideas wake furiou|
+837|116|F|60918.41|1994-06-15|4-NOT SPECIFIED|Clerk#000000563|0|kages sleep slyly above the ironic, final orbits|
+838|17|O|82918.36|1998-01-29|5-LOW|Clerk#000000213|0| slyly around the slyly even|
+839|28|O|70182.63|1995-08-08|1-URGENT|Clerk#000000951|0|the carefully even platelets. furiously unusual fo|
+864|139|O|74710.74|1997-08-17|1-URGENT|Clerk#000000036|0|ly after the slyly regular deposits. express, regular asymptotes nag ca|
+865|4|F|70430.54|1993-05-04|3-MEDIUM|Clerk#000000337|0|. special packages wake after the carefully final accounts. express pinto be|
+866|40|F|4766.19|1992-11-28|3-MEDIUM|Clerk#000000718|0|ins after the even, even accounts nod blithel|
+867|26|F|7471.75|1993-11-16|3-MEDIUM|Clerk#000000877|0|pades nag quickly final, |
+868|104|F|127345.45|1992-06-09|4-NOT SPECIFIED|Clerk#000000782|0|onic theodolites print carefully. blithely dogge|
+869|136|O|58932.19|1997-01-12|2-HIGH|Clerk#000000245|0|ar sheaves are slowly. slyly even attainments boost theodolites. furiously|
+870|34|F|40492.37|1993-06-20|4-NOT SPECIFIED|Clerk#000000123|0|blithely ironic ideas nod. sly, r|
+871|16|O|172861.58|1995-11-15|5-LOW|Clerk#000000882|0|oss the ironic theodolites.|
+896|2|F|169847.63|1993-03-09|1-URGENT|Clerk#000000187|0|inal packages eat blithely according to the warhorses. furiously quiet de|
+897|49|P|57697.44|1995-03-20|1-URGENT|Clerk#000000316|0| wake quickly against |
+898|55|F|101020.75|1993-06-03|2-HIGH|Clerk#000000611|0|. unusual pinto beans haggle quickly across |
+899|109|O|125562.09|1998-04-08|5-LOW|Clerk#000000575|0|rts engage carefully final theodolites.|
+900|46|F|120073.51|1994-10-01|4-NOT SPECIFIED|Clerk#000000060|0| fluffily express deposits nag furiousl|
+901|13|O|81826.12|1998-07-21|4-NOT SPECIFIED|Clerk#000000929|0|lyly even foxes are furious, silent requests. requests about the quickly |
+902|10|F|37348.62|1994-07-27|4-NOT SPECIFIED|Clerk#000000811|0|yly final requests over the furiously regula|
+903|11|O|109351.87|1995-07-07|4-NOT SPECIFIED|Clerk#000000793|0|e slyly about the final pl|
+928|67|F|228136.49|1995-03-02|5-LOW|Clerk#000000450|0|ithely express pinto beans. |
+929|83|F|109301.02|1992-10-02|2-HIGH|Clerk#000000160|0|its. furiously even foxes affix carefully finally silent accounts. express req|
+930|131|F|199102.23|1994-12-17|1-URGENT|Clerk#000000004|0| accounts nag slyly. ironic, ironic accounts wake blithel|
+931|103|F|117909.23|1992-12-07|1-URGENT|Clerk#000000881|0|ss packages haggle furiously express, regular deposits. even, e|
+932|41|O|40234.50|1997-05-16|2-HIGH|Clerk#000000218|0|ly express instructions boost furiously reg|
+933|97|F|71349.30|1992-08-05|4-NOT SPECIFIED|Clerk#000000752|0|ial courts wake permanently against the furiously regular ideas. unusual |
+934|52|O|17213.59|1996-07-03|1-URGENT|Clerk#000000229|0|ts integrate carefully. sly, regular deposits af|
+935|50|O|97733.87|1997-09-24|5-LOW|Clerk#000000180|0|iously final deposits cajole. blithely even packages |
+960|35|F|63537.13|1994-09-21|3-MEDIUM|Clerk#000000120|0|regular accounts. requests|
+961|56|P|158893.16|1995-06-04|4-NOT SPECIFIED|Clerk#000000720|0|ons nag furiously among the quickl|
+962|37|F|98258.73|1994-05-06|5-LOW|Clerk#000000463|0|ments nag deposits. fluffily ironic a|
+963|26|F|53287.25|1994-05-26|3-MEDIUM|Clerk#000000497|0|uses haggle carefully. slyly even dependencies after the packages ha|
+964|76|O|131146.47|1995-05-20|3-MEDIUM|Clerk#000000657|0|print blithely ironic, careful theodolit|
+965|70|P|41758.44|1995-05-15|5-LOW|Clerk#000000218|0|iously special packages. slyly pending requests are carefully |
+966|14|O|120516.93|1998-04-30|2-HIGH|Clerk#000000239|0|special deposits. furious|
+967|110|F|179287.95|1992-06-21|3-MEDIUM|Clerk#000000167|0|excuses engage quickly bold dep|
+992|55|O|133665.12|1997-11-11|3-MEDIUM|Clerk#000000875|0|ts. regular pinto beans thrash carefully sl|
+993|80|O|198238.65|1995-09-10|3-MEDIUM|Clerk#000000894|0|quickly express accounts among the furiously bol|
+994|2|F|41433.48|1994-04-20|5-LOW|Clerk#000000497|0|ole. slyly bold excuses nag caref|
+995|116|P|135157.92|1995-05-31|3-MEDIUM|Clerk#000000439|0|deas. blithely final deposits play. express accounts wake blithely caref|
+996|71|O|47447.63|1997-12-29|1-URGENT|Clerk#000000497|0|arefully final packages into the slyly final requests affix blit|
+997|109|O|27561.82|1997-05-19|2-HIGH|Clerk#000000651|0|ly express depths. furiously final requests haggle furiously. carefu|
+998|32|F|65269.38|1994-11-26|4-NOT SPECIFIED|Clerk#000000956|0|ronic dolphins. ironic, bold ideas haggle furiously furious|
+999|61|F|145249.13|1993-09-05|5-LOW|Clerk#000000464|0|pitaphs sleep. regular accounts use. f|
+1024|4|O|176084.63|1997-12-23|5-LOW|Clerk#000000903|0| blithely. even, express theodolites cajole slyly across|
+1025|103|F|82034.03|1995-05-05|2-HIGH|Clerk#000000376|0|ross the slyly final pa|
+1026|73|O|36464.76|1997-06-04|5-LOW|Clerk#000000223|0|s wake blithely. special acco|
+1027|128|F|112770.89|1992-06-03|3-MEDIUM|Clerk#000000241|0|equests cajole. slyly final pinto bean|
+1028|70|F|153864.67|1994-01-01|2-HIGH|Clerk#000000131|0|ts are. final, silent deposits are among the fl|
+1029|130|F|47440.91|1994-06-21|2-HIGH|Clerk#000000700|0|quests sleep. slyly even foxes wake quickly final theodolites. clo|
+1030|134|F|16346.94|1994-06-15|5-LOW|Clerk#000000422|0|ully ironic accounts sleep carefully. requests are carefully alongside of the |
+1031|4|F|128024.71|1994-09-01|3-MEDIUM|Clerk#000000448|0|s; ironic theodolites along the carefully ex|
+1056|28|F|38446.39|1995-02-11|1-URGENT|Clerk#000000125|0|t, even deposits hang about the slyly special i|
+1057|76|F|108107.42|1992-02-20|1-URGENT|Clerk#000000124|0|cuses dazzle carefully careful, ironic pinto beans. carefully even theod|
+1058|53|F|89359.11|1993-04-26|3-MEDIUM|Clerk#000000373|0|kly pending courts haggle. blithely regular sheaves integrate carefully fi|
+1059|127|F|198360.22|1994-02-27|1-URGENT|Clerk#000000104|0|en accounts. carefully bold packages cajole daringly special depende|
+1060|140|F|121994.04|1993-02-21|3-MEDIUM|Clerk#000000989|0|l platelets sleep quickly slyly special requests. furiously |
+1061|103|O|166947.75|1998-05-15|5-LOW|Clerk#000000576|0|uests sleep at the packages. fur|
+1062|106|O|39805.04|1997-01-15|1-URGENT|Clerk#000000152|0|eposits use blithely |
+1063|37|F|41392.31|1994-04-02|2-HIGH|Clerk#000000024|0|deposits nag quickly regular deposits. quickl|
+1088|148|F|47120.41|1992-05-21|5-LOW|Clerk#000000347|0|counts are blithely. platelets print. carefully |
+1089|49|O|103192.74|1996-05-04|4-NOT SPECIFIED|Clerk#000000226|0|ns haggle ruthlessly. even requests are quickly abov|
+1090|19|O|32929.30|1997-11-15|2-HIGH|Clerk#000000300|0| furiously regular platelets haggle along the slyly unusual foxes! |
+1091|83|O|35795.22|1996-08-27|1-URGENT|Clerk#000000549|0| even pinto beans haggle quickly alongside of the eve|
+1092|124|P|85552.21|1995-03-04|3-MEDIUM|Clerk#000000006|0|re quickly along the blithe|
+1093|101|O|79189.58|1997-07-31|4-NOT SPECIFIED|Clerk#000000159|0| after the carefully ironic requests. carefully ironic packages wake fluffil|
+1094|145|O|9006.25|1997-12-24|3-MEDIUM|Clerk#000000570|0|beans affix furiously about the pending, even deposits. finally pendi|
+1095|145|O|178491.24|1995-08-22|3-MEDIUM|Clerk#000000709|0|sly bold requests cajole carefully according to|
+1120|140|O|107958.62|1997-11-07|3-MEDIUM|Clerk#000000319|0|lly special requests. slyly pending platelets are quickly pending requ|
+1121|29|O|241837.88|1997-01-13|3-MEDIUM|Clerk#000000541|0|r escapades. deposits above the fluffily bold requests hag|
+1122|121|O|179747.47|1997-01-10|1-URGENT|Clerk#000000083|0|uffily carefully final theodolites. furiously express packages affix|
+1123|73|O|93259.93|1996-08-03|3-MEDIUM|Clerk#000000929|0|uriously pending requests. slyly regular instruction|
+1124|80|O|141858.97|1998-07-30|5-LOW|Clerk#000000326|0|regular pinto beans along the fluffily silent packages|
+1125|25|F|80438.38|1994-10-27|2-HIGH|Clerk#000000510|0|ithely final requests. i|
+1126|145|O|59982.31|1998-01-28|4-NOT SPECIFIED|Clerk#000000928|0|d slyly regular ideas: special ideas believe slyly. slyly ironic sheaves w|
+1127|58|O|103320.91|1995-09-19|4-NOT SPECIFIED|Clerk#000000397|0|usly silent, regular pinto beans. blithely express requests boos|
+1152|49|F|51775.54|1994-08-14|4-NOT SPECIFIED|Clerk#000000496|0|equests. deposits ab|
+1153|121|O|220727.97|1996-04-18|5-LOW|Clerk#000000059|0| across the pending deposi|
+1154|37|F|192417.85|1992-02-15|1-URGENT|Clerk#000000268|0|old asymptotes are special requests. blithely even deposits sleep furiously|
+1155|149|O|126902.81|1997-10-06|2-HIGH|Clerk#000000164|0|c deposits haggle among the ironic, even requests. carefully ironic sheaves n|
+1156|133|O|217682.81|1996-10-19|1-URGENT|Clerk#000000200|0| blithely ironic dolphins. furiously pendi|
+1157|97|O|85394.06|1998-01-14|4-NOT SPECIFIED|Clerk#000000207|0|out the regular excuses boost carefully against the furio|
+1158|142|O|31075.51|1996-06-30|2-HIGH|Clerk#000000549|0|integrate slyly furiously ironic deposit|
+1159|70|F|55553.68|1992-09-18|3-MEDIUM|Clerk#000000992|0|ts may sleep. requests according to the|
+1184|89|O|39700.29|1997-10-26|5-LOW|Clerk#000000777|0|iously even packages haggle fluffily care|
+1185|74|F|47033.21|1992-08-24|5-LOW|Clerk#000000344|0| even escapades are. package|
+1186|59|O|82026.18|1996-08-15|4-NOT SPECIFIED|Clerk#000000798|0|ingly regular pinto beans: instructi|
+1187|134|F|85948.02|1992-11-20|3-MEDIUM|Clerk#000000047|0|s after the furiously final deposits boost slyly under the|
+1188|20|O|54655.07|1996-04-11|2-HIGH|Clerk#000000256|0|ully ironic deposits. slyl|
+1189|46|F|71017.99|1994-04-09|1-URGENT|Clerk#000000243|0|f the even accounts. courts print blithely ironic accounts. sile|
+1190|13|O|31043.39|1997-03-16|5-LOW|Clerk#000000575|0|ccounts above the foxes integrate carefully after the |
+1191|112|O|28623.04|1995-11-07|3-MEDIUM|Clerk#000000011|0|uests nag furiously. carefully even requests|
+1216|122|F|68056.57|1992-12-07|5-LOW|Clerk#000000918|0|nal foxes around the e|
+1217|7|F|40982.08|1992-04-26|4-NOT SPECIFIED|Clerk#000000538|0| foxes nag quickly. ironic excuses nod. blithely pending|
+1218|10|F|99834.47|1994-06-20|4-NOT SPECIFIED|Clerk#000000994|0|s cajole. special, silent deposits about the theo|
+1219|28|O|10163.56|1995-10-05|3-MEDIUM|Clerk#000000800|0|od carefully. slyly final dependencies across the even fray|
+1220|49|O|122157.14|1996-08-29|1-URGENT|Clerk#000000712|0|inal theodolites wake. fluffily ironic asymptotes cajol|
+1221|14|F|117397.16|1992-04-19|4-NOT SPECIFIED|Clerk#000000852|0| detect against the silent, even deposits. carefully ironic|
+1222|10|F|47623.94|1993-02-05|3-MEDIUM|Clerk#000000811|0|theodolites use quickly even accounts. carefully final asympto|
+1223|10|O|26714.67|1996-05-25|4-NOT SPECIFIED|Clerk#000000238|0|posits was blithely fr|
+1248|49|F|210713.88|1992-01-02|1-URGENT|Clerk#000000890|0|t the carefully regular dugouts. s|
+1249|149|F|45889.09|1994-01-05|1-URGENT|Clerk#000000095|0|al ideas sleep above the pending pin|
+1250|37|F|12907.62|1992-09-29|4-NOT SPECIFIED|Clerk#000000652|0|ts after the fluffily pending instructions use slyly about the s|
+1251|38|O|109536.55|1997-10-30|1-URGENT|Clerk#000000276|0|, brave sauternes. deposits boost fluffily.|
+1252|149|O|93403.05|1997-08-04|5-LOW|Clerk#000000348|0|ng the slyly regular excuses. special courts nag furiously blithely e|
+1253|115|F|92730.74|1993-01-26|1-URGENT|Clerk#000000775|0| requests sleep furiously even foxes. ruthless packag|
+1254|70|O|94649.25|1995-12-22|1-URGENT|Clerk#000000607|0| pinto beans. carefully regular request|
+1255|122|F|62518.31|1994-05-30|4-NOT SPECIFIED|Clerk#000000798|0|ct slyly regular accounts. quick|
+1280|97|F|91664.85|1993-01-11|5-LOW|Clerk#000000160|0|posits thrash quickly after the theodolites. furiously iro|
+1281|62|F|165454.51|1994-12-11|1-URGENT|Clerk#000000430|0|counts. carefully pending accounts eat |
+1282|116|F|61297.42|1992-02-29|4-NOT SPECIFIED|Clerk#000000168|0|he quickly special packages. furiously final re|
+1283|118|O|202623.92|1996-08-30|4-NOT SPECIFIED|Clerk#000000260|0| pinto beans boost slyly ac|
+1284|134|O|106122.38|1996-01-07|2-HIGH|Clerk#000000492|0|s. blithely silent deposits s|
+1285|11|F|139124.72|1992-06-01|1-URGENT|Clerk#000000423|0|cial deposits cajole after the ironic requests. p|
+1286|109|F|207291.83|1993-05-14|4-NOT SPECIFIED|Clerk#000000939|0| deposits use carefully from the excuses. slyly bold p|
+1287|19|F|131432.42|1994-07-05|2-HIGH|Clerk#000000288|0|ly ironic dolphins integrate furiously among the final packages. st|
+1312|112|F|58111.00|1994-05-19|3-MEDIUM|Clerk#000000538|0|n, express accounts across the ironic|
+1313|148|F|46598.65|1994-09-13|1-URGENT|Clerk#000000774|0|ld accounts. regular deposits cajole. ironically pending theodolites use car|
+1314|143|F|56207.66|1994-05-13|3-MEDIUM|Clerk#000000485|0|ickly blithe packages nod ideas. furiously bold braids boost around the car|
+1315|22|O|121935.23|1998-03-22|5-LOW|Clerk#000000840|0|final theodolites alongside of the carefu|
+1316|16|F|163746.47|1993-12-03|1-URGENT|Clerk#000000857|0|ully bold theodolites? pending, bold pin|
+1317|100|P|139714.71|1995-05-19|2-HIGH|Clerk#000000373|0|sts. furiously special deposits lose fur|
+1318|128|O|81663.65|1998-06-27|3-MEDIUM|Clerk#000000581|0|s hang bold requests. pending, re|
+1319|32|O|31103.83|1996-09-27|2-HIGH|Clerk#000000257|0|y across the ruthlessly ironic accounts. unusu|
+1344|17|F|43809.37|1992-04-16|5-LOW|Clerk#000000178|0|omise close, silent requests. pending theodolites boost pending |
+1345|95|F|111207.93|1992-10-28|5-LOW|Clerk#000000447|0| regular tithes. quickly fluffy de|
+1346|76|F|171975.62|1992-06-18|2-HIGH|Clerk#000000374|0|ges sleep quickly-- even pint|
+1347|41|O|173444.60|1997-06-20|5-LOW|Clerk#000000977|0|he furiously even foxes use carefully express req|
+1348|19|O|94135.77|1998-04-18|5-LOW|Clerk#000000206|0|tly. quickly even deposi|
+1349|64|O|46376.09|1997-10-26|1-URGENT|Clerk#000000543|0|yly! blithely special theodolites cajole. unusual, reg|
+1350|52|F|49305.98|1993-08-24|1-URGENT|Clerk#000000635|0|iously about the blithely special a|
+1351|106|O|24637.96|1998-04-20|1-URGENT|Clerk#000000012|0| cajole. regular, special re|
+1376|47|O|23984.88|1997-05-04|4-NOT SPECIFIED|Clerk#000000730|0|der furiously final, final frets. carefull|
+1377|20|O|108334.30|1998-04-24|4-NOT SPECIFIED|Clerk#000000625|0|lly across the blithely express accounts. ironic excuses promise carefully de|
+1378|20|O|118495.12|1996-03-09|4-NOT SPECIFIED|Clerk#000000705|0| furiously even tithes cajole slyly among the quick|
+1379|65|O|84627.76|1998-05-25|5-LOW|Clerk#000000861|0|y deposits are caref|
+1380|137|O|94969.41|1996-07-07|3-MEDIUM|Clerk#000000969|0|inal deposits wake slyly daringly even requests. bold, even foxe|
+1381|127|O|58212.22|1998-05-25|3-MEDIUM|Clerk#000000107|0|even requests breach after the bold, ironic instructions. slyly even|
+1382|133|F|173522.71|1993-08-17|5-LOW|Clerk#000000241|0|fully final packages sl|
+1383|121|F|34797.72|1993-04-27|2-HIGH|Clerk#000000785|0|ts. express requests sleep blithel|
+1408|55|O|183965.61|1997-12-26|4-NOT SPECIFIED|Clerk#000000942|0|t the quickly final asymptotes. unusual|
+1409|143|F|72440.52|1992-12-31|4-NOT SPECIFIED|Clerk#000000065|0|ructions. furiously unusual excuses are regular, unusual theodolites. fin|
+1410|113|O|114879.19|1997-04-12|5-LOW|Clerk#000000123|0|iously along the bravely regular dolphins. pinto beans cajole furiously sp|
+1411|95|F|164462.61|1994-12-21|2-HIGH|Clerk#000000566|0|s. furiously special excuses across the pending pinto beans haggle sp|
+1412|53|F|78676.54|1993-03-13|4-NOT SPECIFIED|Clerk#000000083|0|uffily daring theodolit|
+1413|91|O|75733.58|1997-06-14|3-MEDIUM|Clerk#000000342|0|, ironic instructions. carefully even packages dazzle|
+1414|77|O|38057.81|1995-08-16|1-URGENT|Clerk#000000883|0|ccounts. ironic foxes haggle car|
+1415|79|F|24654.79|1994-05-29|4-NOT SPECIFIED|Clerk#000000601|0|rays. blithely final ideas affix quickl|
+1440|98|O|50201.16|1995-08-10|5-LOW|Clerk#000000956|0| pending requests. closely s|
+1441|122|O|156477.94|1997-03-06|4-NOT SPECIFIED|Clerk#000000156|0|ter the excuses. ironic dependencies m|
+1442|112|F|7108.12|1994-07-05|4-NOT SPECIFIED|Clerk#000000935|0|nal pinto beans. slyly ironic ideas cajol|
+1443|44|O|44672.03|1996-12-16|5-LOW|Clerk#000000185|0|x blithely against the carefully final somas. even asymptotes are. quickly spe|
+1444|134|F|207907.60|1994-12-06|3-MEDIUM|Clerk#000000783|0|ove the bold accounts cajole fluffily about|
+1445|115|F|154653.32|1995-01-10|3-MEDIUM|Clerk#000000211|0|even packages wake fluffily |
+1446|41|O|27663.16|1998-02-16|5-LOW|Clerk#000000274|0|lly regular notornis above the requests sleep final accounts! |
+1447|91|F|108171.38|1992-10-15|2-HIGH|Clerk#000000880|0|inly against the blithely pending excuses. regular, pe|
+1472|149|O|65331.05|1996-10-06|5-LOW|Clerk#000000303|0|y special dolphins around the final dependencies wake quick|
+1473|94|O|80624.38|1997-03-17|3-MEDIUM|Clerk#000000960|0|furiously close accoun|
+1474|70|F|51697.18|1995-01-09|1-URGENT|Clerk#000000438|0|detect quickly above the carefully even |
+1475|5|O|185496.66|1997-11-12|2-HIGH|Clerk#000000972|0|cally final packages boost. blithely ironic packa|
+1476|145|O|18795.62|1996-06-27|2-HIGH|Clerk#000000673|0|ding accounts hinder alongside of the quickly pending requests. fluf|
+1477|76|O|231831.35|1997-08-24|5-LOW|Clerk#000000612|0|ly bold foxes. final ideas would cajo|
+1478|50|O|20791.50|1997-08-03|2-HIGH|Clerk#000000827|0|lessly. carefully express|
+1479|16|O|31471.04|1995-12-16|4-NOT SPECIFIED|Clerk#000000697|0|he furiously even foxes. thinly bold deposits|
+1504|2|F|89399.40|1992-08-28|3-MEDIUM|Clerk#000000381|0|, brave deposits. bold de|
+1505|37|F|55892.35|1992-08-21|2-HIGH|Clerk#000000544|0|s. slyly ironic packages cajole. carefully regular packages haggle |
+1506|148|F|195844.84|1992-09-21|3-MEDIUM|Clerk#000000620|0| dependencies. accounts affix blithely slowly unusual deposits. slyly regular |
+1507|121|F|96166.92|1993-10-14|3-MEDIUM|Clerk#000000305|0|stealthy, ironic de|
+1508|103|O|151282.65|1998-04-10|5-LOW|Clerk#000000117|0| after the furiously regular pinto beans hang slyly quickly ironi|
+1509|64|F|180455.98|1993-07-08|5-LOW|Clerk#000000770|0|the regular ideas. regul|
+1510|53|O|154590.05|1996-09-17|5-LOW|Clerk#000000128|0|ld carefully. furiously final asymptotes haggle furiously|
+1511|79|O|59651.38|1996-12-22|4-NOT SPECIFIED|Clerk#000000386|0|ts above the depend|
+1536|94|O|5184.26|1997-01-26|3-MEDIUM|Clerk#000000117|0|ges are! furiously final deposits cajole iron|
+1537|109|F|108317.51|1992-02-15|4-NOT SPECIFIED|Clerk#000000862|0|g to the even deposits. ironic, final packages |
+1538|29|O|179554.41|1995-06-18|4-NOT SPECIFIED|Clerk#000000258|0| instructions. regular theod|
+1539|112|F|39612.63|1995-03-10|5-LOW|Clerk#000000840|0|nstructions boost pa|
+1540|16|F|128014.15|1992-08-05|2-HIGH|Clerk#000000927|0|r ideas hinder blithe|
+1541|94|P|47286.32|1995-05-18|1-URGENT|Clerk#000000906|0|y. slyly ironic warhorses around the furiously regul|
+1542|143|F|132972.24|1993-09-15|3-MEDIUM|Clerk#000000435|0|t the furiously close deposits do was f|
+1543|52|O|139047.22|1997-02-20|1-URGENT|Clerk#000000398|0|unts. furiously pend|
+1568|17|O|76119.72|1997-01-30|4-NOT SPECIFIED|Clerk#000000554|0|d notornis. carefully |
+1569|104|O|87803.55|1998-04-02|5-LOW|Clerk#000000786|0|orbits. fluffily even decoys serve blithely. furiously furious realms nag acro|
+1570|124|O|35589.57|1998-03-16|1-URGENT|Clerk#000000745|0|pinto beans haggle furiousl|
+1571|103|F|151404.78|1992-12-05|2-HIGH|Clerk#000000565|0|ously furiously bold warthogs. slyly ironic instructions are quickly a|
+1572|11|O|47232.79|1996-02-24|2-HIGH|Clerk#000000994|0|fluffily ironic accounts haggle blithely final platelets! slyly regular foxes|
+1573|148|F|86918.57|1992-12-28|2-HIGH|Clerk#000000940|0|ess, ironic deposits use along the carefu|
+1574|134|O|179923.54|1996-12-12|3-MEDIUM|Clerk#000000809|0| ideas hinder after the carefully unusual |
+1575|145|O|197031.52|1995-09-13|3-MEDIUM|Clerk#000000497|0|. furiously regular dep|
+1600|94|F|130515.61|1993-03-03|3-MEDIUM|Clerk#000000627|0|tions cajole quietly above the regular, silent requests. slyly fin|
+1601|53|F|73962.95|1994-08-27|5-LOW|Clerk#000000469|0|ent deposits are ca|
+1602|1|F|4225.26|1993-08-05|5-LOW|Clerk#000000660|0|deposits. busily silent instructions haggle furiously. fin|
+1603|2|F|29305.47|1993-07-31|4-NOT SPECIFIED|Clerk#000000869|0|s. slyly silent deposits boo|
+1604|113|F|107139.29|1993-07-17|5-LOW|Clerk#000000512|0|lithely silent waters. blithely unusual packages alongside |
+1605|58|O|130687.64|1998-04-24|4-NOT SPECIFIED|Clerk#000000616|0|sleep furiously? ruthless, even pinto beans |
+1606|53|O|115877.40|1997-04-17|4-NOT SPECIFIED|Clerk#000000550|0|r requests. quickly even platelets breach before the ironically|
+1607|149|O|166335.03|1995-12-16|2-HIGH|Clerk#000000498|0| bold, pending foxes haggle. slyly silent |
+1632|67|O|183286.33|1997-01-08|3-MEDIUM|Clerk#000000351|0|onic requests are accounts. bold a|
+1633|16|O|52359.51|1995-10-14|2-HIGH|Clerk#000000666|0|y silent accounts sl|
+1634|70|O|145898.47|1996-09-10|1-URGENT|Clerk#000000360|0|arefully blithely ironic requests. slyly unusual instructions alongside|
+1635|4|O|70232.26|1997-02-13|3-MEDIUM|Clerk#000000958|0|s. slyly ironic requests affix slyly |
+1636|79|O|172021.87|1997-06-17|3-MEDIUM|Clerk#000000457|0|ding requests. slyly ironic courts wake quickl|
+1637|73|F|180912.15|1995-02-08|4-NOT SPECIFIED|Clerk#000000189|0| final accounts. blithely silent ideas cajole bravely. carefully express |
+1638|139|O|172436.30|1997-08-13|2-HIGH|Clerk#000000643|0|he fluffily regular asymp|
+1639|5|O|104166.56|1995-08-20|4-NOT SPECIFIED|Clerk#000000939|0|haggle furiously. final requests detect furious|
+1664|64|O|178060.22|1996-03-03|1-URGENT|Clerk#000000090|0|y quickly even asymptotes. furiously regular packages haggle quickly fin|
+1665|76|F|4819.91|1994-05-08|2-HIGH|Clerk#000000920|0|ly regular packages are fluffily even ideas. fluffily final|
+1666|95|O|128367.97|1995-10-18|1-URGENT|Clerk#000000849|0|ffily pending dependencies wake fluffily. pending, final accounts |
+1667|5|O|125030.37|1997-10-10|2-HIGH|Clerk#000000103|0|e accounts. slyly express accounts must are a|
+1668|142|O|137576.19|1997-07-12|4-NOT SPECIFIED|Clerk#000000148|0|eodolites. carefully dogged dolphins haggle q|
+1669|2|O|24362.39|1997-06-09|3-MEDIUM|Clerk#000000663|0|er ironic requests detect furiously blithely sp|
+1670|25|O|89999.72|1997-05-24|2-HIGH|Clerk#000000320|0|unusual dependencies. furiously special platelets main|
+1671|35|O|104391.11|1996-07-27|4-NOT SPECIFIED|Clerk#000000275|0|ly. slyly pending requests was above the |
+1696|4|O|102665.03|1998-01-08|4-NOT SPECIFIED|Clerk#000000041|0|bravely bold accounts above the quickly bold|
+1697|76|O|122621.31|1996-10-07|1-URGENT|Clerk#000000815|0|o x-ray blithely. pl|
+1698|40|O|141118.87|1997-04-23|2-HIGH|Clerk#000000432|0|slyly. carefully express deposit|
+1699|85|F|66408.29|1993-12-30|1-URGENT|Clerk#000000125|0|jole blithely. furiously un|
+1700|65|O|89143.36|1996-06-15|3-MEDIUM|Clerk#000000328|0|ely final dolphins wake sometimes above the quietly regular deposits. fur|
+1701|130|F|72835.95|1992-05-19|2-HIGH|Clerk#000000395|0|furiously. regular, close theodoli|
+1702|67|P|194119.31|1995-05-07|2-HIGH|Clerk#000000300|0|around the carefully final deposits cajole carefully according to the b|
+1703|134|F|121220.59|1993-01-28|3-MEDIUM|Clerk#000000463|0| pinto beans poach. bold courts boost. regular, express deposits at|
+1728|64|O|131604.34|1996-05-22|2-HIGH|Clerk#000000711|0|beans. slyly regular instructions sleep! slyly final packages|
+1729|133|F|12137.76|1992-05-19|2-HIGH|Clerk#000000158|0|pending foxes wake. accounts|
+1730|124|O|150886.49|1998-07-24|5-LOW|Clerk#000000794|0| fluffily pending deposits serve. furiously even requests wake furiou|
+1731|128|O|190490.78|1996-01-06|1-URGENT|Clerk#000000268|0|lithely regular, final instructions. ironic, express packages are above|
+1732|146|F|179854.51|1993-11-29|5-LOW|Clerk#000000903|0|inal requests integrate dolph|
+1733|148|O|165489.52|1996-05-12|2-HIGH|Clerk#000000789|0|e carefully according to the accounts. furiously pending instructions sleep|
+1734|7|F|44002.53|1994-06-11|2-HIGH|Clerk#000000722|0| final ideas haggle. blithely quick foxes sleep busily bold ideas. i|
+1735|22|F|98541.95|1992-12-27|1-URGENT|Clerk#000000458|0|ully idle requests wake qu|
+1760|115|O|82151.12|1996-05-17|5-LOW|Clerk#000000917|0| deposits. busily regular deposits wake blithely along the furiously even re|
+1761|106|F|211925.95|1993-12-24|2-HIGH|Clerk#000000817|0|efully slyly bold frets. packages boost b|
+1762|77|F|202227.17|1994-08-20|4-NOT SPECIFIED|Clerk#000000653|0|ly ironic packages. furi|
+1763|121|O|140685.01|1996-10-29|2-HIGH|Clerk#000000321|0|es. bold dependencies haggle furiously along |
+1764|29|F|47384.71|1992-03-25|1-URGENT|Clerk#000000182|0|. slyly final packages integrate carefully acro|
+1765|73|O|36551.43|1995-12-03|4-NOT SPECIFIED|Clerk#000000490|0| regular excuses wake slyly|
+1766|139|O|41032.81|1996-10-12|2-HIGH|Clerk#000000983|0|unusual deposits affix quickly beyond the carefully s|
+1767|25|P|136582.60|1995-03-14|2-HIGH|Clerk#000000327|0|eposits use carefully carefully regular platelets. quickly regular packages al|
+1792|49|F|107919.86|1993-11-09|5-LOW|Clerk#000000102|0|ructions haggle along the pending packages. carefully speci|
+1793|19|F|82504.56|1992-07-12|4-NOT SPECIFIED|Clerk#000000291|0|regular packages cajole. blithely special packages according to the final d|
+1794|140|O|179462.21|1997-09-28|1-URGENT|Clerk#000000686|0|ally silent pinto beans. regular package|
+1795|94|F|146849.33|1994-03-19|2-HIGH|Clerk#000000815|0| quickly final packages! blithely dogged accounts c|
+1796|47|F|33755.47|1992-11-21|2-HIGH|Clerk#000000245|0|eans use furiously around th|
+1797|125|O|51494.47|1996-05-07|3-MEDIUM|Clerk#000000508|0|quiet platelets haggle since the quickly ironic instructi|
+1798|52|O|46393.97|1997-07-28|1-URGENT|Clerk#000000741|0|al foxes are blithe|
+1799|61|F|46815.93|1994-03-07|4-NOT SPECIFIED|Clerk#000000339|0|ns sleep furiously final waters. blithely regular instructions h|
+1824|49|F|81351.53|1994-05-05|1-URGENT|Clerk#000000972|0|e blithely fluffily|
+1825|148|F|150582.77|1993-12-05|3-MEDIUM|Clerk#000000345|0|ironic, final accou|
+1826|82|F|124719.97|1992-04-16|4-NOT SPECIFIED|Clerk#000000718|0|the even asymptotes dazzle fluffily slyly regular asymptotes. final, unu|
+1827|106|O|210113.88|1996-06-22|4-NOT SPECIFIED|Clerk#000000369|0|luffily even requests haggle sly|
+1828|32|F|137369.50|1994-04-18|3-MEDIUM|Clerk#000000840|0|y quickly bold packag|
+1829|112|F|127532.20|1994-05-08|2-HIGH|Clerk#000000537|0| accounts wake above the furiously unusual requests. pending package|
+1830|133|F|85122.24|1995-02-23|1-URGENT|Clerk#000000045|0|according to the even,|
+1831|71|F|58032.77|1993-12-02|1-URGENT|Clerk#000000854|0| accounts. carefully even accounts boost furiously. regular ideas engage. |
+1856|106|F|189361.42|1992-03-20|4-NOT SPECIFIED|Clerk#000000952|0|. special pinto beans run acr|
+1857|133|F|102793.59|1993-01-13|2-HIGH|Clerk#000000083|0|hely final ideas slee|
+1858|143|O|30457.91|1997-12-13|1-URGENT|Clerk#000000389|0|thely. slyly final deposits sleep|
+1859|61|O|105094.09|1997-04-11|4-NOT SPECIFIED|Clerk#000000949|0| the foxes. bravely special excuses nag carefully special r|
+1860|10|O|9103.40|1996-04-04|3-MEDIUM|Clerk#000000556|0|osits. quickly bold deposits according to |
+1861|70|F|95063.41|1994-01-03|3-MEDIUM|Clerk#000000847|0|r the fluffily close sauternes. furio|
+1862|34|O|97981.06|1998-02-24|5-LOW|Clerk#000000348|0|ts snooze ironically abou|
+1863|74|F|96359.65|1993-09-23|4-NOT SPECIFIED|Clerk#000000658|0|old sentiments. careful, |
+1888|121|F|224724.11|1993-10-31|4-NOT SPECIFIED|Clerk#000000659|0|olites. pinto beans cajole. regular deposits affix. slyly regular|
+1889|25|O|96431.77|1997-03-16|1-URGENT|Clerk#000000854|0|p around the regular notornis. unusual deposits|
+1890|10|O|202364.58|1996-12-18|4-NOT SPECIFIED|Clerk#000000627|0|romise final, regular deposits. regular fox|
+1891|61|F|76848.96|1994-12-15|5-LOW|Clerk#000000495|0|unusual foxes sleep regular deposits. requests wake special pac|
+1892|25|F|133273.64|1994-03-26|5-LOW|Clerk#000000733|0|sts. slyly regular dependencies use slyly. ironic, spec|
+1893|125|O|116792.13|1997-10-30|2-HIGH|Clerk#000000111|0|olites. silent, special deposits eat slyly quickly express packages; hockey p|
+1894|76|F|44387.23|1992-03-30|1-URGENT|Clerk#000000626|0|e furiously. furiously even accounts are slyly final accounts. closely speci|
+1895|7|F|44429.81|1994-05-30|3-MEDIUM|Clerk#000000878|0|ress accounts. bold accounts cajole. slyly final pinto beans poach regul|
+1920|110|O|119605.91|1998-06-24|5-LOW|Clerk#000000018|0|hely; furiously regular excuses|
+1921|88|F|57584.12|1994-01-18|3-MEDIUM|Clerk#000000293|0|counts. slyly quiet requests along the ruthlessly regular accounts are |
+1922|56|O|11575.77|1996-07-13|3-MEDIUM|Clerk#000000984|0|side of the blithely final re|
+1923|136|O|171128.10|1997-07-07|1-URGENT|Clerk#000000471|0| express dolphins. |
+1924|76|O|169756.19|1996-09-07|4-NOT SPECIFIED|Clerk#000000823|0| of the ironic accounts. instructions near the final instr|
+1925|17|F|146382.71|1992-03-05|1-URGENT|Clerk#000000986|0|e slyly regular deposits. furiously |
+1926|94|O|100035.03|1996-01-31|2-HIGH|Clerk#000000568|0|cajole. even warhorses sleep carefully. |
+1927|140|O|23327.88|1995-09-30|3-MEDIUM|Clerk#000000616|0|riously special packages. permanent pearls wake furiously. even packages alo|
+1952|67|F|12896.25|1994-03-16|2-HIGH|Clerk#000000254|0| silent accounts boost |
+1953|149|F|57213.18|1993-11-30|3-MEDIUM|Clerk#000000891|0| fluffily along the quickly even packages. |
+1954|56|O|158853.63|1997-05-31|4-NOT SPECIFIED|Clerk#000000104|0| unusual excuses cajole according to the blithely regular theodolites.|
+1955|13|F|103085.13|1992-04-20|1-URGENT|Clerk#000000792|0|ly special ideas. sometimes final |
+1956|127|F|88704.26|1992-09-20|4-NOT SPECIFIED|Clerk#000000600|0|ironic ideas are silent ideas. furiously final deposits sleep slyly carefu|
+1957|31|O|77482.87|1998-07-21|2-HIGH|Clerk#000000639|0|nding excuses about the |
+1958|53|O|176294.34|1995-09-22|5-LOW|Clerk#000000343|0| haggle blithely. flu|
+1959|43|O|62277.18|1997-01-13|4-NOT SPECIFIED|Clerk#000000631|0| cajole about the blithely express requests. even excuses mold bl|
+1984|52|O|79230.47|1998-04-01|1-URGENT|Clerk#000000416|0| slyly special instructions. unusual foxes use packages. carefully regular req|
+1985|7|F|171522.54|1994-09-02|4-NOT SPECIFIED|Clerk#000000741|0|slyly slyly even pains. slyly reg|
+1986|149|F|34269.96|1994-05-05|2-HIGH|Clerk#000000609|0|across the theodolites. quick|
+1987|100|F|6406.29|1994-04-30|2-HIGH|Clerk#000000652|0|gular platelets alongside |
+1988|109|O|117132.72|1995-10-06|4-NOT SPECIFIED|Clerk#000000011|0|ly ironic dolphins serve quickly busy accounts. bu|
+1989|118|F|39263.28|1994-03-16|4-NOT SPECIFIED|Clerk#000000747|0|ely bold pinto beans ha|
+1990|119|F|48781.39|1994-12-16|2-HIGH|Clerk#000000114|0|e bold patterns. always regul|
+1991|19|F|139854.41|1992-09-07|4-NOT SPECIFIED|Clerk#000000854|0|ing accounts can haggle at the carefully final Tiresias-- pending, regular|
+2016|8|O|24347.36|1996-08-16|3-MEDIUM|Clerk#000000641|0|the carefully ironic foxes. requests nag bold, r|
+2017|101|O|70529.27|1998-05-13|3-MEDIUM|Clerk#000000427|0|nusual requests. blit|
+2018|19|P|25007.95|1995-04-05|4-NOT SPECIFIED|Clerk#000000920|0|gular accounts wake fur|
+2019|136|F|43789.14|1992-10-23|1-URGENT|Clerk#000000565|0| furiously bold packages. fluffily fi|
+2020|73|F|136162.13|1993-06-21|3-MEDIUM|Clerk#000000192|0|es. furiously regular packages above the furiously special theodolites are a|
+2021|70|O|27016.74|1995-07-15|1-URGENT|Clerk#000000155|0|ong the furiously regular requests. unusual deposits wake fluffily inside|
+2022|62|F|206742.11|1992-03-15|1-URGENT|Clerk#000000268|0| dependencies sleep fluffily even, ironic deposits. express, silen|
+2023|118|F|144123.37|1992-05-06|5-LOW|Clerk#000000137|0|ular courts engage according to the|
+2048|17|F|33401.77|1993-11-15|1-URGENT|Clerk#000000934|0|s cajole after the blithely final accounts. f|
+2049|31|O|153048.74|1995-12-07|2-HIGH|Clerk#000000859|0|ly regular requests thrash blithely about the fluffily even theodolites. r|
+2050|28|F|208517.98|1994-06-02|4-NOT SPECIFIED|Clerk#000000821|0|d accounts against the furiously regular packages use bli|
+2051|40|O|87988.34|1996-03-18|4-NOT SPECIFIED|Clerk#000000333|0|ctions sleep blithely. blithely regu|
+2052|91|F|141822.19|1992-04-13|2-HIGH|Clerk#000000767|0| requests sleep around the even, even courts. ironic theodolites affix furious|
+2053|142|F|125125.57|1995-02-07|1-URGENT|Clerk#000000717|0|ar requests: blithely sly accounts boost carefully across t|
+2054|41|F|144335.16|1992-06-08|4-NOT SPECIFIED|Clerk#000000103|0|l requests affix carefully about the furiously special|
+2055|97|F|57092.26|1993-09-04|1-URGENT|Clerk#000000067|0|. warhorses affix slyly blithely express instructions? fur|
+2080|95|F|45767.69|1993-06-18|5-LOW|Clerk#000000190|0|ironic, pending theodolites are carefully about the quickly regular theodolite|
+2081|121|O|145654.97|1997-07-05|2-HIGH|Clerk#000000136|0|ong the regular theo|
+2082|49|F|46753.63|1995-01-10|2-HIGH|Clerk#000000354|0|cial accounts. ironic, express dolphins nod slyly sometimes final reques|
+2083|101|F|31795.52|1993-07-14|3-MEDIUM|Clerk#000000361|0|al patterns. bold, final foxes nag bravely about the furiously express|
+2084|80|F|190652.53|1993-03-17|2-HIGH|Clerk#000000048|0|zle furiously final, careful packages. slyly ironic ideas amo|
+2085|49|F|45311.07|1993-11-21|3-MEDIUM|Clerk#000000818|0|ress, express ideas haggle|
+2086|142|F|188985.18|1994-10-19|1-URGENT|Clerk#000000046|0| permanently regular|
+2087|50|O|53581.41|1998-01-31|2-HIGH|Clerk#000000626|0|e always regular packages nod against the furiously spec|
+2112|64|O|17986.15|1997-02-05|2-HIGH|Clerk#000000351|0|against the slyly even id|
+2113|32|O|65678.21|1997-11-08|2-HIGH|Clerk#000000527|0|slyly regular instruct|
+2114|79|F|106446.02|1995-01-16|5-LOW|Clerk#000000751|0|r, unusual accounts haggle across the busy platelets. carefully |
+2115|106|O|134814.65|1998-05-23|4-NOT SPECIFIED|Clerk#000000101|0|odolites boost. carefully regular excuses cajole. quickly ironic pinto be|
+2116|23|F|60887.90|1994-08-26|1-URGENT|Clerk#000000197|0|efully after the asymptotes. furiously sp|
+2117|22|O|145713.03|1997-04-26|2-HIGH|Clerk#000000887|0|ely even dependencies. regular foxes use blithely.|
+2118|134|O|38974.67|1996-10-09|1-URGENT|Clerk#000000196|0|ial requests wake carefully special packages. f|
+2119|64|O|34632.57|1996-08-20|2-HIGH|Clerk#000000434|0|uickly pending escapades. fluffily ir|
+2144|136|F|119917.28|1994-03-29|3-MEDIUM|Clerk#000000546|0|t. carefully quick requests across the deposits wake regu|
+2145|134|F|18885.35|1992-10-03|1-URGENT|Clerk#000000886|0|sts would snooze blithely alongside of th|
+2146|118|F|179686.07|1992-09-14|4-NOT SPECIFIED|Clerk#000000476|0|ven packages. dependencies wake slyl|
+2147|100|F|91513.79|1992-09-06|4-NOT SPECIFIED|Clerk#000000424|0| haggle carefully furiously final foxes. pending escapades thrash. bold theod|
+2148|130|F|19612.03|1995-04-19|4-NOT SPECIFIED|Clerk#000000517|0|ross the furiously unusual theodolites. always expre|
+2149|101|F|105145.40|1993-03-13|5-LOW|Clerk#000000555|0|nusual accounts nag furiously special reques|
+2150|82|F|166961.06|1994-06-03|3-MEDIUM|Clerk#000000154|0|ect slyly against the even, final packages. quickly regular pinto beans wake c|
+2151|58|O|124608.69|1996-11-11|3-MEDIUM|Clerk#000000996|0|c requests. ironic platelets cajole across the quickly fluffy deposits.|
+2176|104|F|87248.17|1992-11-10|1-URGENT|Clerk#000000195|0|s haggle regularly accor|
+2177|136|O|183493.42|1997-01-20|3-MEDIUM|Clerk#000000161|0|ove the blithely unusual packages cajole carefully fluffily special request|
+2178|8|O|79594.68|1996-12-12|3-MEDIUM|Clerk#000000656|0|thely according to the instructions. furious|
+2179|41|O|77487.09|1996-09-07|2-HIGH|Clerk#000000935|0|ounts alongside of the furiously unusual braids cajol|
+2180|76|O|208481.57|1996-09-14|4-NOT SPECIFIED|Clerk#000000650|0|xpress, unusual pains. furiously ironic excu|
+2181|76|O|100954.64|1995-09-13|3-MEDIUM|Clerk#000000814|0|y against the ironic, even|
+2182|23|F|116003.11|1994-04-05|2-HIGH|Clerk#000000071|0|ccounts. quickly bold deposits across the excuses sl|
+2183|113|O|49841.12|1996-06-22|1-URGENT|Clerk#000000287|0| among the express, ironic packages. slyly ironic platelets integrat|
+2208|68|P|245388.06|1995-05-01|4-NOT SPECIFIED|Clerk#000000900|0|symptotes wake slyly blithely unusual packages.|
+2209|91|F|129086.93|1992-07-10|2-HIGH|Clerk#000000056|0|er above the slyly silent requests. furiously reg|
+2210|32|F|31689.46|1992-01-16|2-HIGH|Clerk#000000941|0| believe carefully quickly express pinto beans. deposi|
+2211|92|F|140031.23|1994-06-30|2-HIGH|Clerk#000000464|0|ffily bold courts e|
+2212|118|F|17231.05|1994-03-23|3-MEDIUM|Clerk#000000954|0|structions above the unusual requests use fur|
+2213|122|F|146136.10|1993-01-15|4-NOT SPECIFIED|Clerk#000000598|0|osits are carefully reg|
+2214|115|O|150345.63|1998-05-05|3-MEDIUM|Clerk#000000253|0|packages. fluffily even accounts haggle blithely. carefully ironic depen|
+2215|40|O|108239.46|1996-06-16|4-NOT SPECIFIED|Clerk#000000817|0|le final, final foxes. quickly regular gifts are carefully deposit|
+2240|56|F|174090.30|1992-03-06|4-NOT SPECIFIED|Clerk#000000622|0|accounts against the slyly express foxes are after the slyly regular |
+2241|103|F|165219.08|1993-05-11|1-URGENT|Clerk#000000081|0|y about the silent excuses. furiously ironic instructions along the sil|
+2242|82|O|15082.82|1997-07-20|4-NOT SPECIFIED|Clerk#000000360|0| pending multipliers. carefully express asymptotes use quickl|
+2243|49|O|10451.97|1995-06-10|2-HIGH|Clerk#000000813|0|ously regular deposits integrate s|
+2244|127|F|21207.08|1993-01-09|1-URGENT|Clerk#000001000|0|ckages. ironic, ironic accounts haggle blithely express excuses. |
+2245|58|F|150585.73|1993-04-28|3-MEDIUM|Clerk#000000528|0|ake carefully. braids haggle slyly quickly b|
+2246|113|O|85755.84|1996-05-27|4-NOT SPECIFIED|Clerk#000000739|0| final gifts sleep |
+2247|95|F|13491.31|1992-08-02|4-NOT SPECIFIED|Clerk#000000947|0|furiously regular packages. final brai|
+2272|139|F|127934.71|1993-04-13|2-HIGH|Clerk#000000449|0|s. bold, ironic pinto beans wake. silently specia|
+2273|136|O|142291.79|1996-12-14|5-LOW|Clerk#000000155|0|uickly express foxes haggle quickly against|
+2274|104|F|58273.89|1993-09-04|4-NOT SPECIFIED|Clerk#000000258|0|nstructions try to hag|
+2275|149|F|37398.90|1992-10-22|4-NOT SPECIFIED|Clerk#000000206|0| furiously furious platelets. slyly final packa|
+2276|43|O|141159.63|1996-04-29|4-NOT SPECIFIED|Clerk#000000821|0|ecial requests. fox|
+2277|89|F|79270.23|1995-01-02|4-NOT SPECIFIED|Clerk#000000385|0|accounts cajole. even i|
+2278|142|O|101878.46|1998-04-25|3-MEDIUM|Clerk#000000186|0|r pinto beans integrate after the carefully even deposits. blit|
+2279|80|F|142322.33|1993-02-23|3-MEDIUM|Clerk#000000898|0|de of the quickly unusual instructio|
+2304|46|F|93769.28|1994-01-07|4-NOT SPECIFIED|Clerk#000000415|0|onic platelets. ironic packages haggle. packages nag doggedly according to|
+2305|43|F|122964.66|1993-01-26|2-HIGH|Clerk#000000440|0|ove the furiously even acco|
+2306|28|O|244704.23|1995-07-26|2-HIGH|Clerk#000000975|0| wake furiously requests. permanent requests affix. final packages caj|
+2307|106|F|59417.76|1993-06-29|5-LOW|Clerk#000000952|0|furiously even asymptotes? carefully regular accounts|
+2308|25|F|58546.02|1992-10-25|4-NOT SPECIFIED|Clerk#000000609|0|ts. slyly final depo|
+2309|100|O|146933.07|1995-09-04|5-LOW|Clerk#000000803|0|he carefully pending packages. fluffily stealthy foxes engage carefully|
+2310|31|O|82928.12|1996-09-20|5-LOW|Clerk#000000917|0|wake carefully. unusual instructions nag ironic, regular excuse|
+2311|73|P|153233.93|1995-05-02|2-HIGH|Clerk#000000761|0|ly pending asymptotes-- furiously bold excus|
+2336|142|O|22294.51|1996-01-07|4-NOT SPECIFIED|Clerk#000000902|0|c, final excuses sleep furiously among the even theodolites. f|
+2337|142|O|45704.96|1997-06-18|4-NOT SPECIFIED|Clerk#000000754|0| quickly. final accounts haggle. carefully final acco|
+2338|140|O|28155.92|1997-09-15|2-HIGH|Clerk#000000951|0|riously final dugouts. final, ironic packages wake express, ironic id|
+2339|109|F|63470.78|1993-12-15|5-LOW|Clerk#000000847|0| against the regular |
+2340|65|O|30778.78|1996-01-12|1-URGENT|Clerk#000000964|0|ter the deposits sleep according to the slyly regular packages. carefully |
+2341|82|F|55950.21|1993-05-30|5-LOW|Clerk#000000443|0|sts-- blithely bold dolphins through the deposits nag blithely carefully re|
+2342|37|O|104038.78|1996-06-09|1-URGENT|Clerk#000000615|0|oost carefully across the regular accounts. blithely final d|
+2343|73|O|85381.00|1995-08-21|3-MEDIUM|Clerk#000000170|0|fluffily over the slyly special deposits. quickl|
+2368|13|F|101240.96|1993-08-20|1-URGENT|Clerk#000000830|0|t the bold instructions. carefully unusual |
+2369|110|O|73517.91|1996-12-24|2-HIGH|Clerk#000000752|0|iously even requests are dogged, express |
+2370|142|F|73924.21|1994-01-17|1-URGENT|Clerk#000000231|0|lyly final packages. quickly final deposits haggl|
+2371|19|O|193857.67|1998-01-07|1-URGENT|Clerk#000000028|0|ckages haggle at th|
+2372|31|O|104927.66|1997-11-21|5-LOW|Clerk#000000342|0|s: deposits haggle along the final ideas. careful|
+2373|28|F|55211.04|1994-03-12|4-NOT SPECIFIED|Clerk#000000306|0| even, special courts grow quickly. pending,|
+2374|4|F|115219.88|1993-10-29|4-NOT SPECIFIED|Clerk#000000081|0| blithely regular packages. blithely unusua|
+2375|5|O|106612.48|1996-11-20|3-MEDIUM|Clerk#000000197|0|unusual, pending theodolites cajole carefully |
+2400|37|O|92798.66|1998-07-25|5-LOW|Clerk#000000782|0|nusual courts nag against the carefully unusual pinto b|
+2401|148|O|88448.24|1997-07-29|4-NOT SPECIFIED|Clerk#000000531|0|ully unusual instructions boost carefully silently regular requests. |
+2402|67|O|70403.62|1996-09-06|4-NOT SPECIFIED|Clerk#000000162|0|slyly final sheaves sleep slyly. q|
+2403|55|O|111020.79|1998-04-11|3-MEDIUM|Clerk#000000820|0|furiously regular deposits use. furiously unusual accounts wake along the |
+2404|77|O|109077.69|1997-03-13|4-NOT SPECIFIED|Clerk#000000409|0|deposits breach furiously. ironic foxes haggle carefully bold packag|
+2405|73|O|115929.14|1996-12-23|3-MEDIUM|Clerk#000000535|0|ular, regular asympto|
+2406|7|O|182516.77|1996-10-28|5-LOW|Clerk#000000561|0|blithely regular accounts u|
+2407|55|O|112843.52|1998-06-19|2-HIGH|Clerk#000000068|0|uests affix slyly among the slyly regular depos|
+2432|103|O|62661.93|1996-07-13|1-URGENT|Clerk#000000115|0|re. slyly even deposits wake bra|
+2433|31|F|147071.86|1994-08-22|4-NOT SPECIFIED|Clerk#000000324|0|ess patterns are slyly. packages haggle carefu|
+2434|25|O|123956.25|1997-04-27|3-MEDIUM|Clerk#000000190|0|s. quickly ironic dolphins impress final deposits. blithel|
+2435|73|F|122490.66|1993-02-21|5-LOW|Clerk#000000112|0|es are carefully along the carefully final instructions. pe|
+2436|125|O|73990.08|1995-09-11|4-NOT SPECIFIED|Clerk#000000549|0|arefully. blithely bold deposits affix special accounts. final foxes nag. spe|
+2437|85|F|143411.69|1993-04-21|4-NOT SPECIFIED|Clerk#000000578|0|. theodolites wake slyly-- ironic, pending platelets above the carefully exp|
+2438|13|F|214494.39|1993-07-15|2-HIGH|Clerk#000000744|0|the final, regular warhorses. regularly |
+2439|55|O|41811.12|1997-03-15|2-HIGH|Clerk#000000819|0|lithely after the car|
+2464|145|O|30495.65|1997-11-23|5-LOW|Clerk#000000633|0|le about the instructions. courts wake carefully even|
+2465|34|O|180737.75|1995-06-24|1-URGENT|Clerk#000000078|0|al pinto beans. final, bold packages wake quickly|
+2466|19|F|161625.50|1994-03-06|1-URGENT|Clerk#000000424|0|c pinto beans. express deposits wake quickly. even, final courts nag. package|
+2467|35|O|7231.91|1995-07-16|4-NOT SPECIFIED|Clerk#000000914|0|pades sleep furiously. sometimes regular packages again|
+2468|112|O|160627.01|1997-06-09|4-NOT SPECIFIED|Clerk#000000260|0|ickly regular packages. slyly ruthless requests snooze quickly blithe|
+2469|124|O|192074.23|1996-11-26|5-LOW|Clerk#000000730|0| sleep closely regular instructions. furiously ironic instructi|
+2470|58|O|104966.33|1997-04-19|3-MEDIUM|Clerk#000000452|0|to the furiously final packages? pa|
+2471|89|O|34936.31|1998-03-12|4-NOT SPECIFIED|Clerk#000000860|0|carefully blithely regular pac|
+2496|136|F|140390.60|1994-01-09|2-HIGH|Clerk#000000142|0|slyly. pending instructions sleep. quic|
+2497|47|F|171326.48|1992-08-27|1-URGENT|Clerk#000000977|0|ily ironic pinto beans. furiously final platelets alongside of t|
+2498|97|F|45514.27|1993-11-08|5-LOW|Clerk#000000373|0|g the slyly special pinto beans. |
+2499|121|O|147243.86|1995-09-24|1-URGENT|Clerk#000000277|0|r the quickly bold foxes. bold instructi|
+2500|133|F|131122.82|1992-08-15|2-HIGH|Clerk#000000447|0|integrate slyly pending deposits. furiously ironic accounts across the s|
+2501|67|O|79380.51|1997-05-25|5-LOW|Clerk#000000144|0|ickly special theodolite|
+2502|70|F|33470.40|1993-05-28|4-NOT SPECIFIED|Clerk#000000914|0|lyly: carefully pending ideas affix again|
+2503|7|F|183671.08|1993-06-20|3-MEDIUM|Clerk#000000294|0|ly even packages was. ironic, regular deposits unwind furiously across the p|
+2528|55|F|92069.62|1994-11-20|1-URGENT|Clerk#000000789|0|ular dependencies? regular frays kindle according to the blith|
+2529|136|O|4104.30|1996-08-20|2-HIGH|Clerk#000000511|0|posits across the silent instructions wake blithely across |
+2530|128|F|58853.11|1994-03-21|3-MEDIUM|Clerk#000000291|0|ular instructions about the quic|
+2531|44|O|143212.85|1996-05-06|4-NOT SPECIFIED|Clerk#000000095|0|even accounts. furiously ironic excuses sleep fluffily. carefully silen|
+2532|94|O|116093.49|1995-10-11|2-HIGH|Clerk#000000498|0|the blithely pending accounts. regular, regular excuses boost aro|
+2533|50|O|168495.03|1997-03-24|1-URGENT|Clerk#000000594|0|ecial instructions. spec|
+2534|76|O|202784.54|1996-07-17|3-MEDIUM|Clerk#000000332|0|packages cajole ironic requests. furiously regular|
+2535|121|F|67018.30|1993-05-25|5-LOW|Clerk#000000296|0|phins cajole beneath the fluffily express asymptotes. c|
+2560|131|F|153426.79|1992-09-05|1-URGENT|Clerk#000000538|0|atelets; quickly sly requests|
+2561|58|O|137473.58|1997-11-14|1-URGENT|Clerk#000000861|0|ual requests. unusual deposits cajole furiously pending, regular platelets. |
+2562|10|F|136360.37|1992-08-01|1-URGENT|Clerk#000000467|0|elets. pending dolphins promise slyly. bo|
+2563|62|F|168952.10|1993-11-19|4-NOT SPECIFIED|Clerk#000000150|0|sly even packages after the furio|
+2564|77|F|3967.47|1994-09-09|2-HIGH|Clerk#000000718|0|usly regular pinto beans. orbits wake carefully. slyly e|
+2565|56|O|204438.57|1998-02-28|3-MEDIUM|Clerk#000000032|0|x-ray blithely along|
+2566|86|F|89992.48|1992-10-10|3-MEDIUM|Clerk#000000414|0|ructions boost bold ideas. idly ironic accounts use according to th|
+2567|70|O|263411.29|1998-02-27|2-HIGH|Clerk#000000031|0|detect. furiously ironic requests|
+2592|101|F|8225.96|1993-03-05|4-NOT SPECIFIED|Clerk#000000524|0|ts nag fluffily. quickly stealthy theodolite|
+2593|92|F|134726.09|1993-09-04|2-HIGH|Clerk#000000468|0|r the carefully final|
+2594|79|F|94866.39|1992-12-17|1-URGENT|Clerk#000000550|0|ests. theodolites above the blithely even accounts detect furio|
+2595|74|O|173130.20|1995-12-14|4-NOT SPECIFIED|Clerk#000000222|0|arefully ironic requests nag carefully ideas. |
+2596|43|O|74940.13|1996-08-17|1-URGENT|Clerk#000000242|0|requests. ironic, bold theodolites wak|
+2597|104|F|21964.66|1993-02-04|2-HIGH|Clerk#000000757|0|iously ruthless exc|
+2598|112|O|84871.50|1996-03-05|3-MEDIUM|Clerk#000000391|0| ironic notornis according to the blithely final requests should |
+2599|149|O|62807.13|1996-11-07|2-HIGH|Clerk#000000722|0|ts. slyly regular theodolites wake sil|
+2624|52|O|27148.63|1996-11-28|5-LOW|Clerk#000000930|0|ic, regular packages|
+2625|40|F|39382.74|1992-10-14|4-NOT SPECIFIED|Clerk#000000386|0| final deposits. blithely ironic ideas |
+2626|139|O|84314.51|1995-09-08|4-NOT SPECIFIED|Clerk#000000289|0|gside of the carefully special packages are furiously after the slyly express |
+2627|149|F|26798.65|1992-03-24|3-MEDIUM|Clerk#000000181|0|s. silent, ruthless requests|
+2628|56|F|165655.99|1993-10-22|5-LOW|Clerk#000000836|0|ajole across the blithely careful accounts. blithely silent deposits sl|
+2629|139|O|96458.03|1998-04-06|5-LOW|Clerk#000000680|0|uches dazzle carefully even, express excuses. ac|
+2630|85|F|127132.51|1992-10-24|5-LOW|Clerk#000000712|0|inal theodolites. ironic instructions s|
+2631|37|F|63103.32|1993-09-24|5-LOW|Clerk#000000833|0| quickly unusual deposits doubt around |
+2656|77|F|105492.37|1993-05-04|1-URGENT|Clerk#000000307|0|elets. slyly final accou|
+2657|25|O|148176.06|1995-10-17|2-HIGH|Clerk#000000160|0| foxes-- slyly final dependencies around the slyly final theodo|
+2658|14|O|163834.46|1995-09-23|3-MEDIUM|Clerk#000000400|0|bout the slyly regular accounts. ironic, |
+2659|83|F|79785.52|1993-12-18|4-NOT SPECIFIED|Clerk#000000758|0|cross the pending requests maintain |
+2660|127|O|16922.51|1995-08-05|5-LOW|Clerk#000000480|0|ly finally regular deposits. ironic theodolites cajole|
+2661|74|O|106036.84|1997-01-04|3-MEDIUM|Clerk#000000217|0|al, regular pinto beans. silently final deposits should have t|
+2662|37|O|87689.88|1996-08-21|3-MEDIUM|Clerk#000000589|0|bold pinto beans above the slyly final accounts affix furiously deposits. pac|
+2663|95|O|35131.80|1995-09-06|1-URGENT|Clerk#000000950|0|ar requests. furiously final dolphins along the fluffily spe|
+2688|98|F|181077.36|1992-01-24|2-HIGH|Clerk#000000720|0|have to nag according to the pending theodolites. sly|
+2689|103|F|41552.78|1992-04-09|4-NOT SPECIFIED|Clerk#000000698|0|press pains wake. furiously express theodolites alongsid|
+2690|94|O|224674.27|1996-03-31|3-MEDIUM|Clerk#000000760|0|ravely even theodolites |
+2691|7|F|30137.17|1992-04-30|5-LOW|Clerk#000000439|0|es at the regular deposits sleep slyly by the fluffy requests. eve|
+2692|62|O|24265.24|1997-12-02|3-MEDIUM|Clerk#000000878|0|es. regular asymptotes cajole above t|
+2693|19|O|66158.13|1996-09-04|1-URGENT|Clerk#000000370|0|ndle never. blithely regular packages nag carefully enticing platelets. ca|
+2694|121|O|102807.59|1996-03-14|5-LOW|Clerk#000000722|0| requests. bold deposits above the theodol|
+2695|58|O|138584.20|1996-08-20|1-URGENT|Clerk#000000697|0|ven deposits around the quickly regular packa|
+2720|31|F|161307.05|1993-06-08|1-URGENT|Clerk#000000948|0|quickly. special asymptotes are fluffily ironi|
+2721|79|O|59180.25|1996-01-27|2-HIGH|Clerk#000000401|0| ideas eat even, unusual ideas. theodolites are carefully|
+2722|35|F|50328.84|1994-04-09|5-LOW|Clerk#000000638|0|rding to the carefully quick deposits. bli|
+2723|61|O|104759.25|1995-10-06|5-LOW|Clerk#000000836|0|nts must have to cajo|
+2724|137|F|116069.66|1994-09-14|2-HIGH|Clerk#000000217|0| sleep blithely. blithely idle |
+2725|89|F|75144.68|1994-05-21|4-NOT SPECIFIED|Clerk#000000835|0|ular deposits. spec|
+2726|7|F|47753.00|1992-11-27|5-LOW|Clerk#000000470|0| blithely even dinos sleep care|
+2727|74|O|3089.42|1998-04-19|4-NOT SPECIFIED|Clerk#000000879|0|sual theodolites cajole enticingly above the furiously fin|
+2752|59|F|187932.30|1993-11-19|2-HIGH|Clerk#000000648|0| carefully regular foxes are quickly quickl|
+2753|16|F|159720.39|1993-11-30|2-HIGH|Clerk#000000380|0|ending instructions. unusual deposits|
+2754|145|F|25985.52|1994-04-03|2-HIGH|Clerk#000000960|0|cies detect slyly. |
+2755|118|F|101202.18|1992-02-07|4-NOT SPECIFIED|Clerk#000000177|0|ously according to the sly foxes. blithely regular pinto bean|
+2756|118|F|142323.38|1994-04-18|1-URGENT|Clerk#000000537|0|arefully special warho|
+2757|76|O|89792.48|1995-07-20|2-HIGH|Clerk#000000216|0| regular requests subl|
+2758|43|O|36671.88|1998-07-12|5-LOW|Clerk#000000863|0|s cajole according to the carefully special |
+2759|116|F|89731.10|1993-11-25|4-NOT SPECIFIED|Clerk#000000071|0|ts. regular, pending pinto beans sleep ab|
+2784|95|O|106635.21|1998-01-07|1-URGENT|Clerk#000000540|0|g deposits alongside of the silent requests s|
+2785|148|O|132854.79|1995-07-21|2-HIGH|Clerk#000000098|0|iously pending packages sleep according to the blithely unusual foxe|
+2786|79|F|178254.66|1992-03-22|2-HIGH|Clerk#000000976|0|al platelets cajole blithely ironic requests. ironic re|
+2787|103|O|3726.14|1995-09-30|1-URGENT|Clerk#000000906|0|he ironic, regular |
+2788|124|F|17172.66|1994-09-22|1-URGENT|Clerk#000000641|0|nts wake across the fluffily bold accoun|
+2789|37|O|219123.27|1998-03-14|2-HIGH|Clerk#000000972|0|gular patterns boost. carefully even re|
+2790|25|F|177458.97|1994-08-19|2-HIGH|Clerk#000000679|0| the carefully express deposits sleep slyly |
+2791|121|F|156697.55|1994-10-10|2-HIGH|Clerk#000000662|0|as. slyly ironic accounts play furiously bl|
+2816|58|F|42225.53|1994-09-20|2-HIGH|Clerk#000000289|0|kages at the final deposits cajole furious foxes. quickly |
+2817|40|F|71453.85|1994-04-19|3-MEDIUM|Clerk#000000982|0|ic foxes haggle upon the daringly even pinto beans. slyly|
+2818|49|F|120086.84|1994-12-12|3-MEDIUM|Clerk#000000413|0|eep furiously special ideas. express |
+2819|103|F|66927.16|1994-05-05|1-URGENT|Clerk#000000769|0|ngside of the blithely ironic dolphins. furio|
+2820|19|F|143813.39|1994-05-20|3-MEDIUM|Clerk#000000807|0|equests are furiously. carefu|
+2821|118|F|36592.48|1993-08-09|3-MEDIUM|Clerk#000000323|0|ng requests. even instructions are quickly express, silent instructi|
+2822|79|F|40142.15|1993-07-26|2-HIGH|Clerk#000000510|0|furiously against the accounts. unusual accounts aft|
+2823|79|O|171894.45|1995-09-09|2-HIGH|Clerk#000000567|0|encies. carefully fluffy accounts m|
+2848|70|F|116258.53|1992-03-10|1-URGENT|Clerk#000000256|0|ly fluffy foxes sleep furiously across the slyly regu|
+2849|46|O|180054.29|1996-04-30|2-HIGH|Clerk#000000659|0|al packages are after the quickly bold requests. carefully special |
+2850|100|O|122969.79|1996-10-02|2-HIGH|Clerk#000000392|0|, regular deposits. furiously pending packages hinder carefully carefully u|
+2851|145|O|7859.36|1997-09-07|5-LOW|Clerk#000000566|0|Tiresias wake quickly quickly even|
+2852|91|F|99050.81|1993-01-16|1-URGENT|Clerk#000000740|0|ruthless deposits against the final instructions use quickly al|
+2853|94|F|103641.15|1994-05-05|2-HIGH|Clerk#000000878|0|the carefully even packages.|
+2854|139|F|153568.02|1994-06-27|1-URGENT|Clerk#000000010|0| furiously ironic tithes use furiously |
+2855|49|F|48419.58|1993-04-04|4-NOT SPECIFIED|Clerk#000000973|0| silent, regular packages sleep |
+2880|8|F|145761.99|1992-03-15|2-HIGH|Clerk#000000756|0|ves maintain doggedly spec|
+2881|100|F|45695.84|1992-05-10|5-LOW|Clerk#000000864|0|uriously. slyly express requests according to the silent dol|
+2882|121|O|172872.37|1995-08-22|2-HIGH|Clerk#000000891|0|pending deposits. carefully eve|
+2883|121|F|170360.27|1995-01-23|5-LOW|Clerk#000000180|0|uses. carefully ironic accounts lose fluffil|
+2884|92|O|71683.84|1997-10-12|3-MEDIUM|Clerk#000000780|0|efully express instructions sleep against|
+2885|7|F|146896.72|1992-09-19|4-NOT SPECIFIED|Clerk#000000280|0|ly sometimes special excuses. final requests are |
+2886|109|F|94527.23|1994-11-13|4-NOT SPECIFIED|Clerk#000000619|0|uctions. ironic packages sle|
+2887|109|O|28571.39|1997-05-26|5-LOW|Clerk#000000566|0|slyly even pinto beans. slyly bold epitaphs cajole blithely above t|
+2912|94|F|27727.52|1992-03-12|5-LOW|Clerk#000000186|0|jole blithely above the quickly regular packages. carefully regular pinto bean|
+2913|43|O|130702.19|1997-07-12|3-MEDIUM|Clerk#000000118|0|mptotes doubt furiously slyly regu|
+2914|109|F|60867.14|1993-03-03|3-MEDIUM|Clerk#000000543|0|he slyly regular theodolites are furiously sile|
+2915|94|F|96015.13|1994-03-31|5-LOW|Clerk#000000410|0|ld packages. bold deposits boost blithely. ironic, unusual theodoli|
+2916|8|O|20182.22|1995-12-27|2-HIGH|Clerk#000000681|0|ithely blithe deposits sleep beyond the|
+2917|91|O|100714.13|1997-12-09|4-NOT SPECIFIED|Clerk#000000061|0| special dugouts among the special deposi|
+2918|118|O|21760.09|1996-09-08|3-MEDIUM|Clerk#000000439|0|ular deposits across th|
+2919|53|F|137223.14|1993-12-10|2-HIGH|Clerk#000000209|0|es. pearls wake quietly slyly ironic instructions--|
+2944|14|O|146581.14|1997-09-24|4-NOT SPECIFIED|Clerk#000000740|0|deas. permanently special foxes haggle carefully ab|
+2945|29|O|223507.72|1996-01-03|2-HIGH|Clerk#000000499|0|ons are carefully toward the permanent, bold pinto beans. regu|
+2946|125|O|102226.59|1996-02-05|5-LOW|Clerk#000000329|0|g instructions about the regular accounts sleep carefully along the pen|
+2947|70|P|43360.95|1995-04-26|1-URGENT|Clerk#000000464|0|ronic accounts. accounts run furiously d|
+2948|44|F|100758.71|1994-08-23|5-LOW|Clerk#000000701|0| deposits according to the blithely pending |
+2949|137|F|94231.71|1994-04-12|2-HIGH|Clerk#000000184|0|y ironic accounts use. quickly blithe accou|
+2950|136|O|183620.33|1997-07-06|1-URGENT|Clerk#000000833|0| dolphins around the furiously |
+2951|74|O|125509.17|1996-02-06|2-HIGH|Clerk#000000680|0|gular deposits above the finally regular ideas integrate idly stealthil|
+2976|29|F|145768.47|1993-12-10|4-NOT SPECIFIED|Clerk#000000159|0|. furiously ironic asymptotes haggle ruthlessly silently regular r|
+2977|73|O|25170.88|1996-08-27|3-MEDIUM|Clerk#000000252|0|quickly special platelets are furio|
+2978|44|P|139542.14|1995-05-03|1-URGENT|Clerk#000000135|0|d. even platelets are. ironic dependencies cajole slow, e|
+2979|133|O|116789.98|1996-03-23|3-MEDIUM|Clerk#000000820|0|even, ironic foxes sleep along|
+2980|4|O|187514.11|1996-09-14|3-MEDIUM|Clerk#000000661|0|y quick pinto beans wake. slyly re|
+2981|49|O|37776.79|1998-07-29|5-LOW|Clerk#000000299|0|hely among the express foxes. blithely stealthy requests cajole boldly. regu|
+2982|85|F|55582.94|1995-03-19|2-HIGH|Clerk#000000402|0|lyly. express theodolites affix slyly after the slyly speci|
+2983|62|F|58168.07|1992-01-07|1-URGENT|Clerk#000000278|0|r the even requests. accounts maintain. regular accounts|
+3008|40|O|156018.74|1995-11-08|3-MEDIUM|Clerk#000000701|0|ze quickly. blithely regular packages above the slyly bold foxes shall|
+3009|55|O|108424.94|1997-02-28|1-URGENT|Clerk#000000205|0|r ideas. carefully pe|
+3010|8|O|141647.08|1996-01-26|2-HIGH|Clerk#000000931|0| blithely final requests. special deposits are slyl|
+3011|91|F|46418.85|1992-01-14|5-LOW|Clerk#000000515|0|onic deposits kindle slyly. dependencies around the quickly iro|
+3012|32|F|91678.66|1993-05-05|1-URGENT|Clerk#000000414|0|ts after the regular pinto beans impress blithely s|
+3013|143|O|156407.40|1997-02-05|5-LOW|Clerk#000000591|0|the furiously pendin|
+3014|29|F|194159.59|1992-10-30|4-NOT SPECIFIED|Clerk#000000476|0|ep blithely according to the blith|
+3015|103|F|110826.83|1992-09-27|5-LOW|Clerk#000000013|0|ously regular deposits affix carefully. furiousl|
+3040|112|F|119201.64|1993-04-12|3-MEDIUM|Clerk#000000544|0|carefully special packages. blithe|
+3041|113|O|23039.46|1997-06-03|5-LOW|Clerk#000000092|0|s. unusual, pending deposits use carefully. thinly final|
+3042|20|F|104523.03|1994-11-21|3-MEDIUM|Clerk#000000573|0| the slyly ironic depo|
+3043|44|F|78221.69|1992-04-25|5-LOW|Clerk#000000137|0|cajole blithely furiously fina|
+3044|53|O|52433.54|1996-04-03|2-HIGH|Clerk#000000008|0|cajole final courts. ironic deposits about the quickly final re|
+3045|50|O|85822.67|1995-09-27|1-URGENT|Clerk#000000405|0| express courts sleep quickly special asymptotes. |
+3046|32|O|117817.52|1995-11-30|2-HIGH|Clerk#000000522|0|r deposits. platelets use furi|
+3047|25|O|37881.31|1997-03-21|1-URGENT|Clerk#000000962|0|as. slyly express deposits are dogged pearls. silent ide|
+3072|23|F|87475.82|1994-01-30|4-NOT SPECIFIED|Clerk#000000370|0|ely final deposits cajole carefully. ironic, re|
+3073|136|F|151419.50|1994-01-08|3-MEDIUM|Clerk#000000404|0|kly slyly bold accounts. express courts near the regular ideas sleep bli|
+3074|67|F|85861.93|1992-11-01|5-LOW|Clerk#000000546|0|yly even asymptotes shall have to haggle fluffily. deposits are|
+3075|127|F|37696.70|1994-05-07|3-MEDIUM|Clerk#000000433|0|ackages: carefully unusual reques|
+3076|92|F|93828.15|1993-07-23|2-HIGH|Clerk#000000099|0|busy foxes. deposits affix quickly ironic, pending pint|
+3077|121|O|99290.01|1997-08-06|2-HIGH|Clerk#000000228|0|kly. fluffily ironic requests use qui|
+3078|49|F|46310.83|1993-02-12|2-HIGH|Clerk#000000110|0|ounts are alongside of the blith|
+3079|100|O|148299.05|1997-09-12|5-LOW|Clerk#000000505|0|lly ironic accounts|
+3104|70|F|102693.61|1993-09-16|3-MEDIUM|Clerk#000000871|0|ges boost-- regular accounts are furiousl|
+3105|137|O|125396.80|1996-11-13|4-NOT SPECIFIED|Clerk#000000772|0|s. blithely final ins|
+3106|145|O|132494.97|1997-01-12|3-MEDIUM|Clerk#000000729|0|its use slyly final theodolites; regular dolphins hang above t|
+3107|26|O|107406.26|1997-08-21|1-URGENT|Clerk#000000669|0|ously even deposits acr|
+3108|85|F|63278.00|1993-08-05|1-URGENT|Clerk#000000574|0|s packages haggle furiously am|
+3109|124|F|216104.85|1993-07-24|5-LOW|Clerk#000000936|0|bold requests sleep quickly according to the slyly final|
+3110|88|F|115161.29|1994-12-17|2-HIGH|Clerk#000000564|0|round the fluffy instructions. carefully silent packages cajol|
+3111|133|O|154383.37|1995-08-25|5-LOW|Clerk#000000922|0|slyly regular theodolites. furious deposits cajole deposits. ironic theodoli|
+3136|23|F|145426.11|1994-08-10|4-NOT SPECIFIED|Clerk#000000891|0|tructions sleep slyly. pending di|
+3137|136|O|8958.65|1995-07-26|3-MEDIUM|Clerk#000000063|0|ymptotes wake carefully above t|
+3138|139|F|139579.18|1994-02-09|4-NOT SPECIFIED|Clerk#000000650|0|e fluffily final theodolites. even dependencies wake along the quickly ir|
+3139|17|F|40975.96|1992-01-02|3-MEDIUM|Clerk#000000855|0|ounts against the ruthlessly unusual dolphins|
+3140|145|F|54356.10|1992-04-09|1-URGENT|Clerk#000000670|0|carefully ironic deposits use furiously. blith|
+3141|26|O|115959.96|1995-11-10|1-URGENT|Clerk#000000475|0|es. furiously bold instructions after the carefully final p|
+3142|8|F|16030.15|1992-06-28|3-MEDIUM|Clerk#000000043|0|usual accounts about the carefully special requests sleep slyly quickly regul|
+3143|107|F|135647.68|1993-02-17|1-URGENT|Clerk#000000519|0| are final, ironic accounts. ironic |
+3168|136|F|69412.71|1992-01-30|5-LOW|Clerk#000000352|0|s sleep slyly? ironic, furious instructions detect. quickly final i|
+3169|19|F|126804.90|1993-12-21|3-MEDIUM|Clerk#000000252|0| even pinto beans are blithely special, special multip|
+3170|5|O|190142.17|1997-11-09|1-URGENT|Clerk#000000288|0|requests. furiously bold|
+3171|47|F|84405.78|1993-04-06|5-LOW|Clerk#000000940|0|ar deposits. idly r|
+3172|89|F|121360.83|1992-06-03|4-NOT SPECIFIED|Clerk#000000771|0|es. slyly ironic packages x-ra|
+3173|148|O|64892.73|1996-08-10|5-LOW|Clerk#000000516|0|ial requests lose along t|
+3174|127|O|92856.91|1995-11-15|5-LOW|Clerk#000000663|0|rts. silent, regular pinto beans are blithely regular packages. furiousl|
+3175|44|F|205282.63|1994-07-15|5-LOW|Clerk#000000629|0| across the slyly even realms use carefully ironic deposits: sl|
+3200|13|O|131103.31|1996-02-07|1-URGENT|Clerk#000000020|0| regular dependencies impress evenly even excuses. blithely |
+3201|97|F|90755.31|1993-07-02|4-NOT SPECIFIED|Clerk#000000738|0|. busy, express instruction|
+3202|88|F|50601.01|1992-12-24|5-LOW|Clerk#000000067|0|fluffily express requests affix carefully around th|
+3203|127|O|49357.72|1997-11-05|2-HIGH|Clerk#000000493|0|e furiously silent warhorses. slyly silent deposits wake bli|
+3204|10|F|41573.42|1992-12-26|1-URGENT|Clerk#000000693|0|ess somas cajole slyly. pending accounts cajole|
+3205|148|F|153637.79|1992-04-11|5-LOW|Clerk#000000803|0|e furiously. quickly regular dinos about the final pinto be|
+3206|122|O|64344.86|1996-08-09|1-URGENT|Clerk#000000755|0|ntegrate furiously final, express |
+3207|22|O|133038.59|1998-02-16|1-URGENT|Clerk#000000695|0|uriously accounts. fluffily i|
+3232|82|F|55619.01|1992-10-09|1-URGENT|Clerk#000000314|0|yly final accounts. packages agains|
+3233|140|F|54121.92|1994-10-24|5-LOW|Clerk#000000470|0|ly ironic epitaphs use stealthy, express deposits. quickly regular instruct|
+3234|14|O|147343.68|1996-04-05|4-NOT SPECIFIED|Clerk#000000367|0|ents according to the dependencies will sleep after the blithely even p|
+3235|46|O|104695.09|1995-11-15|5-LOW|Clerk#000000349|0| quickly pinto beans. ironi|
+3236|142|O|39470.39|1996-11-06|4-NOT SPECIFIED|Clerk#000000553|0|ithely slyly pending req|
+3237|19|F|10508.12|1992-06-03|1-URGENT|Clerk#000000606|0|inal requests. slyly even foxes detect about the furiously exp|
+3238|61|F|41375.69|1993-02-21|5-LOW|Clerk#000000818|0|lly express deposits are. furiously unusual ideas wake carefully somas. instr|
+3239|35|O|156802.80|1998-01-12|4-NOT SPECIFIED|Clerk#000000619|0| cajole carefully along the furiously pending deposits. |
+3264|94|O|162634.53|1996-11-02|5-LOW|Clerk#000000244|0|carefully. express, bold|
+3265|53|F|43315.15|1992-06-27|1-URGENT|Clerk#000000265|0|re quickly quickly pe|
+3266|4|P|68309.28|1995-03-17|5-LOW|Clerk#000000545|0|refully ironic instructions. slyly final pi|
+3267|112|O|33998.90|1997-01-07|5-LOW|Clerk#000000484|0| the packages. regular decoys about the bold dependencies grow fi|
+3268|142|F|36024.96|1994-06-25|5-LOW|Clerk#000000746|0|y brave requests unwind furiously accordin|
+3269|17|O|218697.85|1996-03-01|3-MEDIUM|Clerk#000000378|0|ts. accounts wake carefully. carefully dogged accounts wake slyly slyly i|
+3270|38|O|166669.86|1997-05-28|1-URGENT|Clerk#000000375|0|uffily pending courts ca|
+3271|34|F|86534.05|1992-01-01|1-URGENT|Clerk#000000421|0|s. furiously regular requests|
+3296|148|F|187553.35|1994-10-19|3-MEDIUM|Clerk#000000991|0|as! carefully final requests wake. furiously even|
+3297|139|F|9679.45|1992-11-03|2-HIGH|Clerk#000000220|0| after the theodolites cajole carefully according to the finally|
+3298|116|O|62716.67|1996-04-17|5-LOW|Clerk#000000241|0|even accounts boost |
+3299|91|F|42867.92|1993-12-26|3-MEDIUM|Clerk#000000853|0|bold deposits. special instructions sleep care|
+3300|118|O|27049.22|1995-07-15|5-LOW|Clerk#000000198|0|ses. carefully unusual instructions must have to detect about the blithel|
+3301|133|F|48497.09|1994-09-04|4-NOT SPECIFIED|Clerk#000000325|0|ular gifts impress enticingly carefully express deposits; instructions boo|
+3302|34|O|38330.42|1995-11-14|2-HIGH|Clerk#000000367|0|eep blithely ironic requests. quickly even courts haggle slyly|
+3303|145|O|97758.28|1997-12-14|4-NOT SPECIFIED|Clerk#000000661|0|nto beans sleep furiously above the carefully ironic |
+3328|7|F|139580.85|1992-11-19|5-LOW|Clerk#000000384|0|ake among the express accounts? carefully ironic packages cajole never.|
+3329|4|O|46107.70|1995-07-03|2-HIGH|Clerk#000000236|0|old deposits. special accounts haggle furiousl|
+3330|7|F|43255.19|1994-12-19|1-URGENT|Clerk#000000124|0|kages use. carefully regular deposits cajole carefully about |
+3331|91|F|65189.17|1993-05-21|2-HIGH|Clerk#000000901|0|uffily carefully sly accounts. blithely unu|
+3332|143|F|73739.06|1994-11-05|1-URGENT|Clerk#000000840|0|ans detect carefully furiously final deposits: regular accoun|
+3333|92|F|197973.22|1992-09-16|4-NOT SPECIFIED|Clerk#000000157|0|ctions boost slyly quickly even accounts. deposits along|
+3334|76|O|28930.68|1996-02-18|5-LOW|Clerk#000000532|0|ounts maintain carefully. furiously close request|
+3335|49|O|112603.34|1995-10-15|3-MEDIUM|Clerk#000000694|0| deposits poach. ironic ideas about the carefully ironi|
+3360|103|O|168750.48|1998-01-23|5-LOW|Clerk#000000254|0| the deposits. fluffily bold requests cajole regula|
+3361|49|F|75026.51|1992-08-23|4-NOT SPECIFIED|Clerk#000000577|0|unts detect furiously instructions. slow deposi|
+3362|140|O|183176.60|1995-07-29|5-LOW|Clerk#000000011|0|the quickly pending deposits. silent, ev|
+3363|52|O|91017.61|1995-09-23|2-HIGH|Clerk#000000615|0|posits. ironic, final deposits are furiously slyly pending |
+3364|46|O|108412.57|1997-06-21|1-URGENT|Clerk#000000280|0|y even foxes? blithely stea|
+3365|82|F|174634.12|1994-11-09|2-HIGH|Clerk#000000126|0|he slyly regular foxes nag about the accounts. fluffily |
+3366|52|O|13603.08|1997-05-18|1-URGENT|Clerk#000000160|0| pinto beans upon the quickly expres|
+3367|73|F|101339.68|1992-12-31|4-NOT SPECIFIED|Clerk#000000029|0|efully blithely ironic pinto beans. carefully close |
+3392|74|O|96057.42|1995-10-28|1-URGENT|Clerk#000000325|0|es thrash blithely depths. bold multipliers wake f|
+3393|98|O|183104.71|1995-07-04|2-HIGH|Clerk#000000076|0|even requests. excuses are carefully deposits. fluf|
+3394|149|O|162165.94|1996-05-05|4-NOT SPECIFIED|Clerk#000000105|0| blithely among the attainments. carefully final accounts nag blit|
+3395|149|F|141486.77|1994-10-30|4-NOT SPECIFIED|Clerk#000000682|0|ideas haggle beside the ev|
+3396|149|F|196443.16|1994-05-21|3-MEDIUM|Clerk#000000868|0|uffily regular platelet|
+3397|130|F|80084.61|1994-06-23|3-MEDIUM|Clerk#000000048|0|yly. final deposits wake f|
+3398|67|O|1147.42|1996-09-23|1-URGENT|Clerk#000000818|0|uthless, special courts atop the unusual accounts grow fur|
+3399|122|P|56938.16|1995-02-28|4-NOT SPECIFIED|Clerk#000000575|0|the carefully sly accounts. regular, pending theodolites wa|
+3424|103|O|42410.57|1996-08-21|1-URGENT|Clerk#000000190|0|ven requests are quickly pending accounts. blithely furious requests |
+3425|115|O|157040.57|1996-03-31|4-NOT SPECIFIED|Clerk#000000188|0|ions. deposits nag blithely alongside of the carefully f|
+3426|53|O|91929.93|1996-10-16|3-MEDIUM|Clerk#000000283|0|alongside of the slyly|
+3427|4|O|133451.14|1997-05-29|4-NOT SPECIFIED|Clerk#000000404|0|y final pinto beans snooze fluffily bold asymptot|
+3428|10|O|88047.04|1996-04-07|5-LOW|Clerk#000000953|0|lar excuses. slyly pending ideas detect p|
+3429|146|O|141902.54|1997-01-06|4-NOT SPECIFIED|Clerk#000000737|0|l deposits cajole furiously enticing deposits. blithe packages haggle careful|
+3430|113|F|161066.22|1994-12-12|4-NOT SPECIFIED|Clerk#000000664|0| regular attainments are at the final foxes. final packages along the blithe|
+3431|47|F|45536.27|1993-08-22|1-URGENT|Clerk#000000439|0| sleep. slyly busy Tiresias a|
+3456|46|F|32796.35|1993-06-01|5-LOW|Clerk#000000924|0|es promise slyly. ironicall|
+3457|25|P|174223.20|1995-04-27|4-NOT SPECIFIED|Clerk#000000849|0|ely thin asymptotes. deposits kindle. pending|
+3458|95|F|153069.14|1994-12-22|2-HIGH|Clerk#000000392|0|rges snooze. slyly unusua|
+3459|119|F|127134.05|1994-07-28|4-NOT SPECIFIED|Clerk#000000777|0|n instructions? carefully regular excuses are blithely. silent, ironi|
+3460|82|O|245976.74|1995-10-03|2-HIGH|Clerk#000000078|0|ans integrate carefu|
+3461|100|F|190960.69|1993-01-31|1-URGENT|Clerk#000000504|0|al, bold deposits cajole fluffily fluffily final foxes. pending ideas beli|
+3462|133|O|63590.17|1997-05-17|3-MEDIUM|Clerk#000000657|0|uriously express asympto|
+3463|89|F|85255.56|1993-08-18|1-URGENT|Clerk#000000545|0|ding to the carefully ironic deposits|
+3488|148|F|92716.17|1995-01-08|3-MEDIUM|Clerk#000000694|0|cording to the carefully regular deposits. re|
+3489|109|F|62453.97|1993-07-29|3-MEDIUM|Clerk#000000307|0|s detect. carefully even platelets across the fur|
+3490|91|O|100106.96|1997-05-26|5-LOW|Clerk#000000703|0|gular ideas. furiously silent deposits across the unusual accounts boost i|
+3491|83|O|50287.06|1998-06-24|1-URGENT|Clerk#000000560|0|nic orbits believe carefully across the |
+3492|103|F|168721.45|1994-11-24|5-LOW|Clerk#000000066|0|packages along the regular foxes lose final dependencie|
+3493|82|F|41686.10|1993-08-24|2-HIGH|Clerk#000000887|0|lyly special accounts use blithely across the furiously sil|
+3494|49|F|136058.70|1993-04-04|5-LOW|Clerk#000000559|0|r instructions haggle. accounts cajole. carefully final requests at the |
+3495|31|O|58666.79|1996-02-26|2-HIGH|Clerk#000000441|0|nticing excuses are carefully|
+3520|125|O|151233.65|1997-08-04|1-URGENT|Clerk#000000023|0|hely. ideas nag; even, even fo|
+3521|7|F|142029.67|1992-10-26|5-LOW|Clerk#000000812|0|y even instructions cajole carefully above the bli|
+3522|26|F|151515.08|1994-09-26|5-LOW|Clerk#000000250|0|deposits-- slyly stealthy requests boost caref|
+3523|149|O|129657.08|1998-04-07|2-HIGH|Clerk#000000688|0|are on the carefully even depe|
+3524|94|F|22767.49|1992-05-03|2-HIGH|Clerk#000000607|0|efully unusual tithes among the foxes use blithely daringly bold deposits. re|
+3525|109|O|100749.60|1995-12-22|4-NOT SPECIFIED|Clerk#000000084|0|s nag among the blithely e|
+3526|56|F|53827.34|1995-03-16|5-LOW|Clerk#000000364|0|to the quickly special deposits print agai|
+3527|56|O|145232.09|1997-06-21|5-LOW|Clerk#000000874|0|regular ideas across the quickly bold theodo|
+3552|35|O|103656.44|1997-04-23|2-HIGH|Clerk#000000973|0| the ironic packages. furiously |
+3553|91|F|119838.14|1994-05-18|3-MEDIUM|Clerk#000000270|0|counts mold furiously. slyly i|
+3554|44|O|98335.61|1995-06-17|5-LOW|Clerk#000000931|0|hely ironic requests haggl|
+3555|46|O|134442.37|1996-07-07|5-LOW|Clerk#000000585|0|s nag carefully regular, even pinto be|
+3556|16|F|114681.55|1992-09-23|4-NOT SPECIFIED|Clerk#000000140|0|e. dependencies need to haggle alongs|
+3557|121|F|85477.89|1992-11-09|2-HIGH|Clerk#000000291|0|ithely courts. furi|
+3558|28|O|112912.00|1996-02-29|1-URGENT|Clerk#000000841|0|around the furiously even requests. quickl|
+3559|106|F|30722.49|1992-10-24|3-MEDIUM|Clerk#000000634|0|sly deposits. fluffily final ideas cajole careful|
+3584|13|O|80487.97|1997-08-11|1-URGENT|Clerk#000000760|0|fully bold packages. fluffily final braids haggle final, ironic dolphins. b|
+3585|139|F|159015.39|1994-11-23|2-HIGH|Clerk#000000988|0|regular asymptotes. bold pains above the carefully pending asymptot|
+3586|121|F|112845.04|1993-12-05|2-HIGH|Clerk#000000438|0|he quickly final courts. carefully regular requests nag unusua|
+3587|79|O|174798.97|1996-05-10|4-NOT SPECIFIED|Clerk#000000443|0|ular patterns detect |
+3588|119|F|207925.83|1995-03-19|4-NOT SPECIFIED|Clerk#000000316|0|ong the pains. evenly unusual |
+3589|31|F|39103.37|1994-05-26|2-HIGH|Clerk#000000023|0|ithe deposits nag furiously. furiously pending packages sleep f|
+3590|149|P|218482.70|1995-05-13|5-LOW|Clerk#000000986|0|lyly final deposits.|
+3591|136|F|98140.86|1993-12-08|3-MEDIUM|Clerk#000000144|0|ual foxes haggle! unusual request|
+3616|128|F|60933.29|1994-02-16|4-NOT SPECIFIED|Clerk#000000268|0|uickly about the quickly final requests. fluffily final packages wake evenly|
+3617|40|O|126205.42|1996-03-19|3-MEDIUM|Clerk#000000886|0|the carefully regular platelets ha|
+3618|10|O|136954.81|1997-12-13|3-MEDIUM|Clerk#000000894|0|. ideas run carefully. thin, pending |
+3619|149|O|222274.54|1996-11-20|2-HIGH|Clerk#000000211|0|uests mold after the blithely ironic excuses. slyly pending pa|
+3620|44|O|59291.75|1997-03-07|5-LOW|Clerk#000000124|0|le quickly against the epitaphs. requests sleep slyly according to the|
+3621|142|F|106150.05|1993-05-06|3-MEDIUM|Clerk#000000643|0|kly unusual deposits. qu|
+3622|91|O|109202.90|1995-11-27|5-LOW|Clerk#000000012|0|c deposits are fluffily about the blithely final theo|
+3623|4|O|175017.68|1996-12-26|1-URGENT|Clerk#000000184|0|- ironic excuses boost quickly in place |
+3648|125|F|180417.11|1993-06-17|5-LOW|Clerk#000000717|0|foxes. unusual deposits boost quickly. slyly regular asymptotes across t|
+3649|40|F|124470.32|1994-07-06|5-LOW|Clerk#000000349|0|taphs boost above the final p|
+3650|46|F|189547.57|1992-05-28|4-NOT SPECIFIED|Clerk#000000454|0|kages sleep fluffily slyly|
+3651|100|O|113191.45|1998-04-27|1-URGENT|Clerk#000000222|0|ly unusual deposits thrash quickly after the ideas.|
+3652|107|O|107732.23|1997-02-25|4-NOT SPECIFIED|Clerk#000000024|0|sly even requests after the |
+3653|40|F|142866.39|1994-03-27|1-URGENT|Clerk#000000402|0| pearls. bold accounts are along the ironic,|
+3654|7|F|222653.54|1992-06-03|5-LOW|Clerk#000000475|0|s cajole slyly carefully special theodolites. even deposits haggl|
+3655|49|F|74882.22|1992-10-06|1-URGENT|Clerk#000000815|0|er the carefully unusual deposits sleep quickly according to|
+3680|127|F|124402.59|1992-12-10|4-NOT SPECIFIED|Clerk#000000793|0|ular platelets. carefully regular packages cajole blithely al|
+3681|52|F|36889.65|1992-04-04|1-URGENT|Clerk#000000566|0|. ironic deposits against the ironic, regular frets use pending plat|
+3682|32|O|67525.43|1997-01-22|2-HIGH|Clerk#000000001|0|es haggle carefully. decoys nag |
+3683|88|F|99960.46|1993-03-04|2-HIGH|Clerk#000000248|0|ze across the express foxes. carefully special acco|
+3684|23|F|89509.91|1993-07-20|2-HIGH|Clerk#000000835|0|bold accounts affix along the carefully ironic requ|
+3685|16|F|154958.89|1992-01-17|3-MEDIUM|Clerk#000000954|0| sleep fluffily special ide|
+3686|40|O|82190.77|1998-07-07|2-HIGH|Clerk#000000175|0|s. furiously final pinto beans poach carefully among |
+3687|43|F|99851.38|1993-02-03|1-URGENT|Clerk#000000585|0|gular accounts. slyly regular instructions can are final ide|
+3712|64|F|127527.05|1992-01-02|2-HIGH|Clerk#000000032|0| promise according |
+3713|149|O|215342.63|1998-05-07|3-MEDIUM|Clerk#000000325|0|s haggle quickly. ironic, regular Tiresi|
+3714|40|O|84493.55|1998-05-01|3-MEDIUM|Clerk#000000595|0|nding accounts. ironic pinto beans wake slyly. furiously pendin|
+3715|65|O|64000.93|1996-03-18|1-URGENT|Clerk#000000463|0| always silent requests wake pinto beans. slyly pending foxes are aga|
+3716|43|O|146221.66|1997-08-19|4-NOT SPECIFIED|Clerk#000000748|0| pending ideas haggle. ironic,|
+3717|28|O|176525.53|1998-06-03|4-NOT SPECIFIED|Clerk#000000974|0|t the carefully even ideas use sp|
+3718|31|O|63195.54|1996-10-23|2-HIGH|Clerk#000000016|0|refully. furiously final packages use carefully slyly pending deposits! final,|
+3719|118|O|139902.71|1997-02-16|2-HIGH|Clerk#000000034|0|, enticing accounts are blithely among the daringly final asymptotes. furious|
+3744|65|F|33085.68|1992-01-10|3-MEDIUM|Clerk#000000765|0|osits sublate about the regular requests. fluffily unusual accou|
+3745|112|F|19405.73|1993-09-29|5-LOW|Clerk#000000181|0|ckages poach slyly against the foxes. slyly ironic instructi|
+3746|74|F|80018.54|1994-09-11|4-NOT SPECIFIED|Clerk#000000188|0|. express, special requests nag quic|
+3747|149|O|204355.65|1996-08-20|1-URGENT|Clerk#000000226|0|refully across the final theodolites. carefully bold accounts cajol|
+3748|53|O|83804.38|1998-02-28|1-URGENT|Clerk#000000156|0|slyly special packages|
+3749|38|P|87073.89|1995-02-24|3-MEDIUM|Clerk#000000639|0|y regular instructions haggle blithel|
+3750|97|P|177181.67|1995-04-30|3-MEDIUM|Clerk#000000885|0|y. express, even packages wake after the ide|
+3751|10|F|202917.72|1994-04-27|4-NOT SPECIFIED|Clerk#000000925|0|sheaves. express, unusual t|
+3776|85|F|150349.92|1992-11-20|2-HIGH|Clerk#000000698|0|efully even platelets slee|
+3777|28|F|82467.29|1994-04-08|3-MEDIUM|Clerk#000000941|0| regular, special dolphins cajole enticingly ca|
+3778|106|F|221036.31|1993-05-26|1-URGENT|Clerk#000000187|0| above the express requests. packages maintain fluffily according to|
+3779|74|O|31538.94|1997-01-05|4-NOT SPECIFIED|Clerk#000000670|0| against the deposits. quickly bold instructions x-ray. pending fox|
+3780|41|O|65385.42|1996-04-13|5-LOW|Clerk#000000967|0| around the brave, pendin|
+3781|139|O|133864.82|1996-06-20|1-URGENT|Clerk#000000394|0|yly after the ruthless packages. pinto beans use slyly: never ironic dependenc|
+3782|65|O|145096.17|1996-08-24|1-URGENT|Clerk#000000121|0|counts are. pending, regular asym|
+3783|44|F|155017.92|1993-12-06|4-NOT SPECIFIED|Clerk#000000614|0| along the pinto beans. special packages use. regular theo|
+3808|79|F|228054.01|1994-04-24|1-URGENT|Clerk#000000717|0|odolites. blithely ironic cour|
+3809|148|O|143070.70|1996-05-01|5-LOW|Clerk#000000646|0| regular excuses. even theodolites are fluffily according to t|
+3810|100|F|124675.27|1992-09-17|1-URGENT|Clerk#000000660|0|ters sleep across the carefully final |
+3811|80|O|154967.89|1998-04-16|3-MEDIUM|Clerk#000000290|0|sits wake slyly abo|
+3812|41|O|70502.52|1996-08-13|3-MEDIUM|Clerk#000000727|0|al, final requests cajole|
+3813|146|O|77247.05|1998-06-29|1-URGENT|Clerk#000000531|0|g the furiously regular instructions|
+3814|118|P|149451.88|1995-02-22|5-LOW|Clerk#000000669|0| the furiously pending theodo|
+3815|104|O|14275.01|1997-08-26|1-URGENT|Clerk#000000249|0|es snooze carefully stealth|
+3840|100|O|187156.38|1998-07-17|4-NOT SPECIFIED|Clerk#000000713|0|yly slow theodolites. enticingly |
+3841|58|F|129033.13|1994-10-05|4-NOT SPECIFIED|Clerk#000000018|0| bold requests sleep quickly ironic packages. sometimes regular deposits nag |
+3842|28|F|131447.03|1992-04-09|5-LOW|Clerk#000000418|0|silent ideas. final deposits use furiously. blithely express excuses cajole fu|
+3843|10|O|34035.17|1997-01-04|4-NOT SPECIFIED|Clerk#000000693|0|eodolites; slyly unusual accounts nag boldly |
+3844|79|F|6793.45|1994-12-29|1-URGENT|Clerk#000000686|0|r dolphins. slyly ironic theodolites ag|
+3845|89|F|134333.33|1992-04-26|1-URGENT|Clerk#000000404|0|es among the pending, regular accounts sleep blithely blithely even de|
+3846|49|O|123120.06|1998-02-05|2-HIGH|Clerk#000000877|0|y alongside of the slyl|
+3847|34|F|7014.31|1993-03-12|5-LOW|Clerk#000000338|0|uriously even deposits. furiously pe|
+3872|134|O|198538.68|1996-09-06|5-LOW|Clerk#000000943|0|counts boost slyly against the ironic platelets-- blithely p|
+3873|55|O|95291.79|1998-03-30|4-NOT SPECIFIED|Clerk#000000791|0|express deposits-- even ideas |
+3874|119|F|66455.34|1993-06-09|3-MEDIUM|Clerk#000000208|0|ular asymptotes sleep blithely ironic ideas. blithel|
+3875|118|O|74483.95|1997-09-10|1-URGENT|Clerk#000000587|0| solve among the fluffily even |
+3876|29|O|95126.32|1996-08-02|5-LOW|Clerk#000000708|0|into beans. blithely|
+3877|17|F|178492.01|1993-05-21|5-LOW|Clerk#000000652|0|foxes. thinly bold reques|
+3878|88|O|59989.66|1997-03-23|1-URGENT|Clerk#000000314|0|e carefully regular platelets. special, express dependencies slee|
+3879|142|O|80274.22|1995-11-23|1-URGENT|Clerk#000000231|0|sts along the quickly ironic sentiments cajole carefully according to t|
+3904|149|O|39338.44|1997-11-15|4-NOT SPECIFIED|Clerk#000000883|0|sits haggle furiously across the requests. theodolites ha|
+3905|22|F|56227.04|1993-12-21|4-NOT SPECIFIED|Clerk#000000573|0|usly even accounts lose quietly above the slyly express p|
+3906|46|F|145630.76|1992-05-28|3-MEDIUM|Clerk#000000867|0|ironic theodolites haggle blithely above the final re|
+3907|67|F|240457.56|1992-08-19|3-MEDIUM|Clerk#000000084|0|gular pinto beans sleep f|
+3908|43|F|57127.71|1993-03-09|3-MEDIUM|Clerk#000000490|0|ounts cajole. regularly|
+3909|22|O|82746.74|1998-07-27|1-URGENT|Clerk#000000980|0|nic, special theodolites sleep furiously! furiously |
+3910|64|O|47272.67|1996-08-26|3-MEDIUM|Clerk#000000270|0|ickly. furiously final packag|
+3911|10|P|35019.95|1995-03-17|4-NOT SPECIFIED|Clerk#000000818|0|he fluffily final forges haggle slyly according to the blithely|
+3936|32|O|168618.39|1996-11-07|2-HIGH|Clerk#000000200|0|iously express packages engage slyly fina|
+3937|94|O|187516.29|1997-11-30|4-NOT SPECIFIED|Clerk#000000189|0|ckages boost carefully blithely q|
+3938|31|F|46918.22|1993-03-03|1-URGENT|Clerk#000000199|0|. unusual, final foxes haggle|
+3939|70|O|8720.45|1996-01-11|5-LOW|Clerk#000000647|0|ly ruthlessly silent requests. blithely regular requests haggle blithely wh|
+3940|149|O|129012.84|1996-02-14|5-LOW|Clerk#000000363|0|e above the ideas. quickly even dependencies along the blithely ir|
+3941|136|O|95453.80|1996-08-29|2-HIGH|Clerk#000000503|0|gular theodolites integrate quickly |
+3942|76|F|38596.81|1993-06-28|4-NOT SPECIFIED|Clerk#000000608|0|eas cajole bold requests. idly silent instructions |
+3943|40|O|60314.97|1996-10-09|5-LOW|Clerk#000000482|0|se alongside of the final pinto beans. regular packages boost across the ca|
+3968|25|O|121704.45|1997-02-17|4-NOT SPECIFIED|Clerk#000000431|0| the slyly special accounts; |
+3969|52|O|169797.40|1997-05-14|2-HIGH|Clerk#000000731|0|uriously final dependencies slee|
+3970|76|F|163709.85|1992-03-27|3-MEDIUM|Clerk#000000190|0|luffily furiously regular deposits. blithely special requests cajole blithely|
+3971|104|O|47925.47|1996-06-28|5-LOW|Clerk#000000287|0|alongside of the instructions ought to are |
+3972|124|F|1861.19|1994-04-21|3-MEDIUM|Clerk#000000049|0|y regular requests haggle quickly. pending, express acco|
+3973|103|F|91541.48|1992-03-24|4-NOT SPECIFIED|Clerk#000000114|0|somas according to the quickly even instructions wake fu|
+3974|94|O|56779.06|1996-03-05|4-NOT SPECIFIED|Clerk#000000938|0|deposits are furiously beneath the bl|
+3975|118|O|37804.43|1995-04-11|3-MEDIUM|Clerk#000000016|0|ts. regular, regular Tiresias play furiously. ironi|
+4000|70|F|84053.93|1992-01-04|5-LOW|Clerk#000000339|0|le carefully closely even pinto beans. regular, ironic foxes against the|
+4001|115|O|95929.46|1997-05-15|3-MEDIUM|Clerk#000000878|0|detect. asymptotes sleep furio|
+4002|104|O|76518.11|1997-04-08|5-LOW|Clerk#000000097|0| regular braids are. furiously even patterns agains|
+4003|112|F|17603.01|1993-01-27|1-URGENT|Clerk#000000177|0| blithe theodolites are slyly. slyly silent accounts toward|
+4004|70|F|220715.14|1993-05-07|3-MEDIUM|Clerk#000000273|0|accounts among the blithely regular sentiments |
+4005|140|O|129062.13|1996-11-20|2-HIGH|Clerk#000000341|0|ily according to the slyly iron|
+4006|35|F|70557.05|1995-01-04|3-MEDIUM|Clerk#000000765|0|ly ironic packages integrate. regular requests alongside of |
+4007|8|F|116193.97|1993-06-18|2-HIGH|Clerk#000000623|0|ecial packages. slyly regular accounts integrate |
+4032|10|O|62497.51|1998-02-26|3-MEDIUM|Clerk#000000686|0|iresias sleep slyly regular ideas. quickly unusual|
+4033|83|F|57740.74|1993-06-02|5-LOW|Clerk#000000181|0|ously bold instructions haggle furiously above the fluf|
+4034|94|F|186912.51|1993-11-14|4-NOT SPECIFIED|Clerk#000000548|0|ts x-ray. express requests affix fluffily regular theodolites. pending, fina|
+4035|118|F|22840.21|1992-02-19|5-LOW|Clerk#000000097|0|he ironic deposits sleep blith|
+4036|47|O|82563.10|1997-04-26|3-MEDIUM|Clerk#000000398|0|ly express deposits nag slyly. ironic, final asymptotes boost bra|
+4037|121|F|36389.43|1993-03-24|2-HIGH|Clerk#000000384|0|t carefully above the unusual the|
+4038|94|O|155045.39|1996-01-06|1-URGENT|Clerk#000000272|0|re slyly. silent requests wake quickly. regular packages play quickly |
+4039|29|O|143753.01|1997-11-16|1-URGENT|Clerk#000000358|0|ly ironic deposits. ironic reques|
+4064|130|O|148500.71|1996-10-10|4-NOT SPECIFIED|Clerk#000000598|0|ccounts. furiously unusual theodolites wake carefully about|
+4065|80|F|156345.64|1994-06-09|1-URGENT|Clerk#000000131|0|even foxes! slyly final deposits agai|
+4066|32|O|176911.21|1997-01-27|4-NOT SPECIFIED|Clerk#000000286|0|yly ironic dinos. quickly regular accounts haggle. requests wa|
+4067|16|F|136517.34|1992-10-07|2-HIGH|Clerk#000000027|0|tes boost furiously quick asymptotes. final deposits of the dolphins solv|
+4068|125|O|71852.67|1996-09-18|3-MEDIUM|Clerk#000000203|0|lly even accounts wake furiously across the unusual platelets. unusu|
+4069|73|F|198816.13|1992-05-13|3-MEDIUM|Clerk#000000359|0|deposits: slyly bold ideas detect furiously. f|
+4070|29|O|98275.37|1995-06-12|2-HIGH|Clerk#000000713|0|xpress ideas poach ab|
+4071|148|O|67789.42|1996-09-15|4-NOT SPECIFIED|Clerk#000000486|0|nal deposits. pending deposits d|
+4096|139|F|81089.61|1992-07-03|4-NOT SPECIFIED|Clerk#000000706|0|sits. quickly thin deposits x-ray blith|
+4097|10|O|134308.04|1996-05-24|1-URGENT|Clerk#000000475|0|ickly under the even accounts. even packages after the furiously express|
+4098|23|O|48478.54|1996-11-05|4-NOT SPECIFIED|Clerk#000000491|0|otes. quickly final requests after the stealthily ironic pinto bean|
+4099|17|F|207364.80|1992-08-21|1-URGENT|Clerk#000000379|0|r platelets. slyly regular requests cajole carefully against the|
+4100|4|O|3892.77|1996-03-12|3-MEDIUM|Clerk#000000429|0|posits. carefully unusual packages use pending deposits. regular she|
+4101|142|F|21640.10|1993-11-22|4-NOT SPECIFIED|Clerk#000000704|0|y around the express, careful epitaphs. accounts use fluffily. quickly p|
+4102|22|O|128786.57|1996-03-17|1-URGENT|Clerk#000000675|0|nding dependencies was slyly about the bl|
+4103|106|F|38164.23|1992-07-03|5-LOW|Clerk#000000679|0|fully ironic dependencies.|
+4128|139|O|5472.17|1995-10-07|4-NOT SPECIFIED|Clerk#000000635|0|ctions. dependencies from the slyly regular accounts nag slyly fu|
+4129|32|F|67226.28|1993-06-26|3-MEDIUM|Clerk#000000541|0|nwind. quickly final theodolites use packages. accounts|
+4130|104|O|47823.04|1996-03-10|5-LOW|Clerk#000000609|0|omise alongside of the carefully final foxes. blithel|
+4131|44|O|145971.60|1998-01-30|1-URGENT|Clerk#000000612|0| above the foxes hang |
+4132|19|P|65601.08|1995-05-29|4-NOT SPECIFIED|Clerk#000000158|0|ld asymptotes solve alongside of the express, final packages. fluffily fi|
+4133|101|F|31693.88|1992-08-07|4-NOT SPECIFIED|Clerk#000000268|0|al, express foxes. quickly pending deposits might cajole alongsi|
+4134|97|F|125191.12|1995-01-12|1-URGENT|Clerk#000000171|0|fully even deposits. regular de|
+4135|37|O|99577.55|1997-03-10|3-MEDIUM|Clerk#000000627|0|ly quietly even ideas. deposits haggle blithely|
+4160|55|O|82493.07|1996-08-20|5-LOW|Clerk#000000283|0|the carefully special accounts. furiously regular dugouts alongs|
+4161|118|F|198995.21|1993-08-21|5-LOW|Clerk#000000047|0|nts. fluffily regular foxes above the quickly daring reques|
+4162|22|F|72359.55|1992-02-10|5-LOW|Clerk#000000179|0|r packages are slyly accounts. furiously special foxes detect carefully re|
+4163|64|F|11493.80|1992-12-21|2-HIGH|Clerk#000000268|0| integrate furiously slyly regular depende|
+4164|94|O|8709.16|1998-07-03|2-HIGH|Clerk#000000720|0| regularly busy theodolites boost furiously quickly bold packages. express, s|
+4165|4|O|11405.40|1997-07-25|3-MEDIUM|Clerk#000000621|0|special foxes affix never blithely ironic pinto beans; blithely |
+4166|43|F|100671.06|1993-02-28|5-LOW|Clerk#000000757|0|quickly sly forges impress. careful foxes across the blithely even a|
+4167|28|O|62108.45|1998-06-17|1-URGENT|Clerk#000000917|0|kly furiously even deposits. unu|
+4192|146|O|197192.95|1998-04-19|1-URGENT|Clerk#000000369|0|equests above the slyly regular pinto beans unwi|
+4193|4|F|143191.54|1994-01-09|2-HIGH|Clerk#000000201|0|ng accounts haggle quickly. packages use fluffily ironic excu|
+4194|106|F|62972.29|1994-10-16|3-MEDIUM|Clerk#000000385|0| instructions are quickly even pinto beans. courts boost furiously regular, ev|
+4195|104|F|54478.95|1993-05-29|4-NOT SPECIFIED|Clerk#000000777|0| pinto beans cajole furiously theodolites-- slyly regular deposits doub|
+4196|106|O|201455.98|1998-05-15|3-MEDIUM|Clerk#000000532|0|affix carefully. quickly final requests |
+4197|92|O|217709.03|1996-08-13|4-NOT SPECIFIED|Clerk#000000264|0| pinto beans according|
+4198|143|O|105789.01|1997-06-16|3-MEDIUM|Clerk#000000583|0|g the special packages haggle pen|
+4199|5|F|30494.62|1992-02-13|1-URGENT|Clerk#000000309|0|e blithely. special deposits haggle slyly final foxes. carefully even|
+4224|70|O|150655.44|1997-07-14|1-URGENT|Clerk#000000034|0|jole quickly final dolphins. slyly pending foxes wake furiously bold pl|
+4225|128|O|72533.07|1997-06-03|3-MEDIUM|Clerk#000000992|0|r the platelets nag among the special deposits. ironic, ironic re|
+4226|92|F|29827.44|1993-03-09|5-LOW|Clerk#000000203|0|phins wake slyly regular packages. deposits haggle slowl|
+4227|133|F|92261.08|1995-02-24|1-URGENT|Clerk#000000063|0|ng the requests; ideas haggle fluffily. slyly unusual ideas c|
+4228|110|O|22072.16|1997-03-28|5-LOW|Clerk#000000309|0|pecial requests aft|
+4229|14|O|75145.87|1998-03-03|1-URGENT|Clerk#000000301|0|p furiously: final excuses hagg|
+4230|140|F|219709.60|1992-03-04|1-URGENT|Clerk#000000364|0|lly ironic deposits integrate carefully about the fu|
+4231|86|O|111403.66|1997-11-20|4-NOT SPECIFIED|Clerk#000000630|0|ly final accounts cajole furiously accounts. bravely ironic platelets am|
+4256|118|F|23067.48|1992-04-05|4-NOT SPECIFIED|Clerk#000000043|0|y alongside of the fluffily iro|
+4257|17|P|41723.86|1995-03-25|3-MEDIUM|Clerk#000000682|0|r ideas cajole along the blithely regular gifts.|
+4258|92|O|133829.35|1996-10-27|4-NOT SPECIFIED|Clerk#000000364|0|efully final platelets around the blit|
+4259|104|O|12918.70|1997-10-09|5-LOW|Clerk#000000781|0|es snooze slyly against the furiously unusual ideas. furious|
+4260|142|F|18566.14|1992-05-16|4-NOT SPECIFIED|Clerk#000000919|0|e among the fluffily bold accounts.|
+4261|118|F|83665.20|1992-10-03|1-URGENT|Clerk#000000662|0| about the even, pending packages. slyly bold deposits boost|
+4262|88|O|176278.57|1996-08-04|3-MEDIUM|Clerk#000000239|0| of the furious accounts. furiously regular accounts w|
+4263|4|O|158885.83|1998-03-16|1-URGENT|Clerk#000000265|0|sly ruthless deposits. final packages are instructions. fu|
+4288|34|F|75030.81|1992-12-04|4-NOT SPECIFIED|Clerk#000000823|0|usly carefully even theodolites: slyly express pac|
+4289|125|F|20752.62|1993-10-07|3-MEDIUM|Clerk#000000912|0|e carefully close instructions. slyly special reques|
+4290|41|F|26128.99|1995-01-15|3-MEDIUM|Clerk#000000688|0| slyly quickly bold requests. final deposits haggle pending ideas! som|
+4291|89|F|71822.86|1993-11-29|3-MEDIUM|Clerk#000000655|0| sleep fluffily between the bold packages. bold|
+4292|25|F|145906.24|1992-01-09|3-MEDIUM|Clerk#000000794|0| ruthlessly. slyly bo|
+4293|103|O|198322.91|1996-08-20|2-HIGH|Clerk#000000750|0|ly packages. regular packages nag according to t|
+4294|49|F|232194.74|1992-08-15|3-MEDIUM|Clerk#000000407|0|ng pinto beans breach. slyly express requests bo|
+4295|5|O|77754.62|1996-02-10|3-MEDIUM|Clerk#000000023|0|e boldly bold dependencies|
+4320|115|O|67049.37|1996-12-08|4-NOT SPECIFIED|Clerk#000000223|0|ages haggle after the slowly bold se|
+4321|16|F|118896.95|1994-07-18|3-MEDIUM|Clerk#000000041|0|ending deposits are carefully carefully regular packa|
+4322|142|O|149671.92|1998-03-13|3-MEDIUM|Clerk#000000433|0|totes nag across the fluffily special instructions. quickly silent hockey |
+4323|104|F|27598.17|1994-01-23|2-HIGH|Clerk#000000282|0|lve after the slyly regular multipliers. even, regular excus|
+4324|73|O|178249.05|1995-07-17|1-URGENT|Clerk#000000800|0|ccounts. slyly stealthy requests shall have t|
+4325|130|O|20214.49|1996-07-18|2-HIGH|Clerk#000000591|0|y around the always ev|
+4326|29|O|39048.94|1996-10-27|4-NOT SPECIFIED|Clerk#000000869|0|packages. carefully express deposit|
+4327|146|P|126235.35|1995-03-16|2-HIGH|Clerk#000000571|0|yly pending braids. final requests abo|
+4352|14|O|18653.09|1997-11-26|2-HIGH|Clerk#000000620|0|ly final platelets integrate carefully even requ|
+4353|73|O|21815.30|1997-12-12|2-HIGH|Clerk#000000790|0|uickly even ideas cajole|
+4354|145|F|179827.12|1994-09-30|4-NOT SPECIFIED|Clerk#000000046|0|pending notornis. requests serve |
+4355|4|O|186370.23|1996-11-16|1-URGENT|Clerk#000000362|0|ndencies use furiously across the regular |
+4356|97|F|39828.51|1994-04-11|5-LOW|Clerk#000000956|0| asymptotes sleep blithely. asymptotes sleep. blithely regul|
+4357|47|O|67045.94|1997-10-23|4-NOT SPECIFIED|Clerk#000000031|0|ages nag between the|
+4358|25|O|46298.53|1997-08-12|1-URGENT|Clerk#000000692|0|according to the fluffily special asymptotes |
+4359|16|F|107824.40|1993-03-03|1-URGENT|Clerk#000000393|0|sts. special, unusual deposits across the ironic theodo|
+4384|25|F|52562.16|1992-07-13|1-URGENT|Clerk#000000192|0|onic platelets. furiously regular asymptotes according to the special pac|
+4385|122|O|39190.62|1996-08-06|2-HIGH|Clerk#000000597|0|ully final requests. ironic, even dolphins above the regular |
+4386|61|O|134413.58|1998-02-06|5-LOW|Clerk#000000070|0| dolphins. silent, idle pinto beans |
+4387|110|O|116740.67|1995-10-23|1-URGENT|Clerk#000000025|0|ter the regular pinto beans. special, final gifts above the requests wi|
+4388|10|O|69668.22|1996-03-28|2-HIGH|Clerk#000000715|0|ts wake against the carefully final accounts. sly|
+4389|55|F|120324.82|1994-05-05|3-MEDIUM|Clerk#000000403|0|wly express excuses after the permanently even instructions are|
+4390|7|P|140608.69|1995-05-23|1-URGENT|Clerk#000000691|0|inal pinto beans. exp|
+4391|38|F|48284.06|1992-02-18|2-HIGH|Clerk#000000880|0|regular accounts. even depo|
+4416|149|F|76067.10|1992-06-30|5-LOW|Clerk#000000391|0| deposits. ideas cajole express theodolites: |
+4417|67|O|60868.39|1998-07-09|1-URGENT|Clerk#000000365|0|ideas are alongside of the blithely final reque|
+4418|61|F|47099.71|1993-03-25|3-MEDIUM|Clerk#000000731|0|pecial pinto beans. close foxes affix iron|
+4419|104|O|94030.43|1996-06-12|4-NOT SPECIFIED|Clerk#000000410|0|ages wake furiously slyly thin theodolit|
+4420|109|F|6088.41|1994-06-18|1-URGENT|Clerk#000000706|0|lly bold deposits along the bold, pending foxes detect blithely after the acco|
+4421|10|O|258779.02|1997-04-04|3-MEDIUM|Clerk#000000246|0|t the pending warhorses. express waters a|
+4422|70|P|107140.22|1995-05-22|3-MEDIUM|Clerk#000000938|0|ly bold accounts sleep special, regular foxes. doggedly regular in|
+4423|64|F|4913.06|1995-02-17|5-LOW|Clerk#000000888|0|excuses are ruthless|
+4448|70|O|127191.47|1998-05-21|2-HIGH|Clerk#000000428|0|. deposits haggle around the silent packages; slyly unusual packages|
+4449|10|O|48206.14|1998-02-08|5-LOW|Clerk#000000035|0|ourts are carefully even deposits. pending |
+4450|106|O|110194.31|1997-07-15|1-URGENT|Clerk#000000867|0|quests boost. furiously even realms are blithely bold requests. bl|
+4451|4|F|92851.80|1994-10-01|1-URGENT|Clerk#000000181|0|. carefully final foxes along the quickly express T|
+4452|13|F|64838.66|1994-06-21|5-LOW|Clerk#000000985|0|oxes are slyly. express, ironic pinto beans wake after the quickly pending re|
+4453|65|O|137030.40|1997-04-01|3-MEDIUM|Clerk#000000603|0|ages could have to nag slyly furiously even asymptotes! slowly regular |
+4454|142|F|159578.94|1994-02-02|5-LOW|Clerk#000000411|0|uriously regular pint|
+4455|19|F|102534.63|1993-10-11|3-MEDIUM|Clerk#000000924|0|even requests. bravely regular foxes according to the carefully unusual |
+4480|85|F|28658.26|1994-03-31|4-NOT SPECIFIED|Clerk#000000534|0|press, bold deposits boost blit|
+4481|148|O|77705.40|1996-03-30|5-LOW|Clerk#000000443|0|press sheaves cajole furio|
+4482|82|P|63535.56|1995-05-15|4-NOT SPECIFIED|Clerk#000000534|0|ravely bold accounts. furiously ironic instructions affix quickly. pend|
+4483|52|F|126597.21|1992-03-07|3-MEDIUM|Clerk#000000615|0|its. blithely idle accounts run; theodolites wake carefully around the fi|
+4484|131|O|237947.61|1996-12-24|1-URGENT|Clerk#000000392|0|ct across the pinto beans. quickly pending excuses engage furiously.|
+4485|53|F|182432.17|1994-11-13|3-MEDIUM|Clerk#000000038|0|es wake slyly even packages. blithely brave requests nag above the regul|
+4486|37|O|135613.18|1998-03-03|2-HIGH|Clerk#000000656|0|ffily according to the carefully pending acc|
+4487|46|F|109469.90|1993-02-23|3-MEDIUM|Clerk#000000017|0|s up the never pending excuses wake furiously special pinto beans. furiously i|
+4512|70|O|148682.82|1995-10-25|5-LOW|Clerk#000000393|0|ending instructions maintain fu|
+4513|85|O|119820.38|1996-03-15|5-LOW|Clerk#000000154|0|ests. final, final ideas|
+4514|97|F|143899.85|1994-04-30|3-MEDIUM|Clerk#000000074|0|deposits according to the carefull|
+4515|140|F|161745.44|1992-03-17|1-URGENT|Clerk#000000191|0|quests among the accounts sleep boldly about the regular f|
+4516|130|F|35949.14|1994-03-29|3-MEDIUM|Clerk#000000739|0|ing packages sleep slyly regular attainments|
+4517|113|O|47614.08|1998-03-07|4-NOT SPECIFIED|Clerk#000000231|0|uriously final deposits doze furiously furiously reg|
+4518|125|O|25861.74|1997-05-01|3-MEDIUM|Clerk#000000187|0|luffily against the spec|
+4519|136|F|68885.66|1993-03-30|4-NOT SPECIFIED|Clerk#000000938|0|ccording to the final |
+4544|112|O|151148.81|1997-08-07|3-MEDIUM|Clerk#000000435|0|g dependencies dazzle slyly ironic somas. carefu|
+4545|59|F|143276.28|1993-01-17|4-NOT SPECIFIED|Clerk#000000303|0|ep. requests use sly|
+4546|43|O|39906.87|1995-07-29|5-LOW|Clerk#000000373|0|ns sleep. regular, regular instructions maintai|
+4547|109|F|52114.01|1993-08-23|3-MEDIUM|Clerk#000000519|0|uctions thrash platelets. slyly final foxes wake slyly against th|
+4548|127|O|139915.23|1996-06-28|5-LOW|Clerk#000000798|0| in place of the blithely express sentiments haggle slyly r|
+4549|64|O|43889.17|1998-03-05|4-NOT SPECIFIED|Clerk#000000965|0|ully even deposits dazzle. fluffily pending ideas against the requests|
+4550|118|F|27461.48|1994-12-29|2-HIGH|Clerk#000000748|0|s haggle carefully acco|
+4551|109|O|82824.14|1996-02-09|2-HIGH|Clerk#000000462|0|ts. slyly quick theodolite|
+4576|139|O|56936.10|1996-08-14|5-LOW|Clerk#000000798|0|e pending deposits. |
+4577|79|O|104259.88|1998-05-02|5-LOW|Clerk#000000409|0|ly. unusual platelets are alw|
+4578|91|F|95761.93|1992-09-13|5-LOW|Clerk#000000121|0| to the furiously ironic instructions? furiou|
+4579|106|O|85927.85|1995-12-01|2-HIGH|Clerk#000000951|0|its wake quickly blithely specia|
+4580|82|F|118464.65|1993-11-15|4-NOT SPECIFIED|Clerk#000000086|0|rs wake blithely regular requests. fluffily ev|
+4581|79|F|89592.11|1992-09-04|4-NOT SPECIFIED|Clerk#000000687|0|ges. carefully pending accounts use furiously abo|
+4582|19|O|18247.86|1996-07-04|1-URGENT|Clerk#000000638|0|g the furiously regular pac|
+4583|22|F|206495.43|1994-09-25|3-MEDIUM|Clerk#000000240|0|equests. slyly even platelets was qui|
+4608|80|F|157767.86|1994-06-17|1-URGENT|Clerk#000000259|0|y even instructions detect slyly asymptotes. blithely final packa|
+4609|133|O|70462.84|1996-12-05|3-MEDIUM|Clerk#000000239|0|hang slyly slyly expre|
+4610|26|F|135934.60|1993-06-18|5-LOW|Clerk#000000616|0|e carefully express pinto|
+4611|29|F|166506.22|1993-01-10|2-HIGH|Clerk#000000152|0|. furiously regular instructions haggle dolphins. even instructions det|
+4612|61|F|82598.87|1993-09-20|3-MEDIUM|Clerk#000000397|0|bove the deposits. even deposits dazzle. slyly express packages haggle sl|
+4613|133|O|212339.55|1998-03-05|3-MEDIUM|Clerk#000000541|0|furiously blithely pending dependen|
+4614|61|O|151801.06|1996-04-22|1-URGENT|Clerk#000000974|0| sauternes wake thinly special accounts. fur|
+4615|29|F|10500.27|1993-08-27|3-MEDIUM|Clerk#000000982|0|jole after the fluffily pending foxes. packages affix carefully acco|
+4640|97|O|81138.17|1996-01-01|5-LOW|Clerk#000000902|0|requests. deposits do detect above the blithely iron|
+4641|134|F|98485.21|1993-01-20|4-NOT SPECIFIED|Clerk#000000755|0|ronic, final requests integrate slyly: specia|
+4642|148|F|117537.87|1995-02-27|1-URGENT|Clerk#000000295|0|cial requests wake carefully around the regular, unusual ideas. furi|
+4643|67|O|52414.19|1995-06-30|2-HIGH|Clerk#000000292|0|ously regular packages. unusual, special platel|
+4644|94|O|85901.70|1998-01-17|5-LOW|Clerk#000000961|0|requests. fluffily even ideas bo|
+4645|44|F|231012.22|1994-09-20|1-URGENT|Clerk#000000764|0|fully even instructions. final gifts sublate quickly final requests. bl|
+4646|83|O|124637.19|1996-06-18|1-URGENT|Clerk#000000036|0|n place of the blithely qu|
+4647|28|F|110958.36|1994-05-14|3-MEDIUM|Clerk#000000626|0|out the deposits. slyly final pinto beans haggle idly. slyly s|
+4672|79|O|199593.71|1995-11-07|1-URGENT|Clerk#000000475|0|lyly final dependencies caj|
+4673|82|O|58094.75|1996-08-13|4-NOT SPECIFIED|Clerk#000000914|0|c deposits are slyly. bravely ironic deposits cajole carefully after the |
+4674|37|F|115411.37|1994-04-19|1-URGENT|Clerk#000000122|0|careful hockey players. carefully pending deposits caj|
+4675|86|F|68817.08|1993-11-25|4-NOT SPECIFIED|Clerk#000000741|0|al deposits haggle slyly final|
+4676|14|O|182025.95|1995-09-01|2-HIGH|Clerk#000000407|0|s. slyly bold accounts sleep furiously special|
+4677|40|O|25661.87|1998-02-21|3-MEDIUM|Clerk#000000245|0|ly pending deposits after the carefully regular foxes sleep blithely after t|
+4678|88|O|131752.07|1998-08-02|4-NOT SPECIFIED|Clerk#000000175|0|side of the bold platelets detect slyly blithely ironic e|
+4679|88|F|7211.59|1993-01-20|2-HIGH|Clerk#000000905|0|ely regular accounts affix slyly. final dolphins are. furiously final de|
+4704|2|O|63873.14|1996-08-16|4-NOT SPECIFIED|Clerk#000000256|0|lithely final requests about the fluffily regular |
+4705|98|F|173340.09|1992-03-22|4-NOT SPECIFIED|Clerk#000000522|0| special instructions poa|
+4706|25|F|101709.52|1992-12-29|4-NOT SPECIFIED|Clerk#000000722|0| packages above the never regular packages nag packages. deposits c|
+4707|91|F|61052.10|1995-02-27|2-HIGH|Clerk#000000943|0|ully enticing accounts behind the regular|
+4708|85|F|56998.36|1994-10-01|1-URGENT|Clerk#000000383|0|ly thinly even accounts. unusu|
+4709|26|O|49903.57|1996-01-08|3-MEDIUM|Clerk#000000785|0|he furiously even deposits! ironic theodolites haggle blithely. r|
+4710|100|F|88966.68|1994-12-08|4-NOT SPECIFIED|Clerk#000000734|0|the final, regular foxes. carefully ironic pattern|
+4711|142|O|129546.56|1998-05-06|1-URGENT|Clerk#000000818|0|mptotes. unusual packages wake furiously qui|
+4736|139|O|67572.73|1995-11-20|2-HIGH|Clerk#000000563|0|blithely regular courts affix into the carefully ironic deposits. slyly exp|
+4737|79|F|62014.51|1993-03-11|4-NOT SPECIFIED|Clerk#000000275|0|ents use slyly among the unusual, ironic pearls. furiously pending |
+4738|5|F|149466.62|1992-04-08|2-HIGH|Clerk#000000150|0|deposits. thin acco|
+4739|148|F|68255.82|1993-02-21|5-LOW|Clerk#000000872|0|ing to the pending attainments: pending, express account|
+4740|68|O|42579.40|1996-07-05|2-HIGH|Clerk#000000420|0| dependencies haggle about the|
+4741|127|F|180692.90|1992-07-07|4-NOT SPECIFIED|Clerk#000000983|0|ly bold deposits are slyly about the r|
+4742|64|P|155356.80|1995-03-23|3-MEDIUM|Clerk#000000058|0|n packages. quickly regular ideas cajole blithely|
+4743|97|F|65702.39|1993-03-31|5-LOW|Clerk#000000048|0|pinto beans above the bold, even idea|
+4768|136|F|4820.55|1993-11-22|2-HIGH|Clerk#000000875|0|ctions snooze idly beneath the quick waters. fluffily u|
+4769|121|P|136765.03|1995-04-14|4-NOT SPECIFIED|Clerk#000000116|0|pon the asymptotes. idle, final account|
+4770|59|O|72150.68|1995-06-20|2-HIGH|Clerk#000000461|0|cial instructions believe carefully. |
+4771|95|F|49625.21|1992-12-14|1-URGENT|Clerk#000000571|0|lly express deposits serve furiously along the f|
+4772|28|F|64102.93|1994-09-14|1-URGENT|Clerk#000000708|0|es sleep. regular requests haggle furiously slyly |
+4773|122|O|196080.26|1995-12-23|1-URGENT|Clerk#000000327|0|ptotes was slyly along the|
+4774|52|F|124380.73|1993-04-20|3-MEDIUM|Clerk#000000299|0|eposits use blithely bold deposits. carefully regular gifts about the fin|
+4775|128|O|112444.42|1995-08-13|4-NOT SPECIFIED|Clerk#000000609|0|s integrate slyly slyly final instructions. carefully bold pack|
+4800|37|F|91795.13|1992-01-06|5-LOW|Clerk#000000625|0|ggle furiously along the pending pinto beans. deposits use: final foxe|
+4801|88|O|108353.08|1996-01-25|1-URGENT|Clerk#000000553|0|r the final sentiments. pending theodolites sleep doggedly across t|
+4802|130|O|5978.65|1997-01-23|3-MEDIUM|Clerk#000000400|0| ironic, thin packages wake furiously ironic, ironic deposits. the|
+4803|124|O|158776.68|1996-02-08|5-LOW|Clerk#000000892|0|lly unusual courts are ironic|
+4804|37|F|111547.31|1992-01-28|2-HIGH|Clerk#000000614|0|ly final accounts. blithely unusual theodolite|
+4805|16|F|172102.96|1992-04-25|4-NOT SPECIFIED|Clerk#000000514|0|even accounts wake furiously slyly final accounts; blithel|
+4806|7|F|35390.15|1993-04-21|5-LOW|Clerk#000000625|0|ave accounts. furiously pending wa|
+4807|53|O|138902.23|1997-01-09|3-MEDIUM|Clerk#000000310|0|kly. slyly special accounts|
+4832|34|O|84954.79|1997-12-04|3-MEDIUM|Clerk#000000548|0|final accounts sleep among the blithe|
+4833|133|O|84800.44|1996-05-12|3-MEDIUM|Clerk#000000256|0|r deposits against the slyly final excuses slee|
+4834|19|O|124539.00|1996-09-12|2-HIGH|Clerk#000000284|0|lar accounts. furiously ironic accounts haggle slyly |
+4835|146|F|70857.51|1994-10-25|1-URGENT|Clerk#000000250|0|s integrate furiously blithely expr|
+4836|65|O|78711.40|1996-12-18|1-URGENT|Clerk#000000691|0|c packages cajole carefully through the accounts. careful|
+4837|130|O|68519.84|1998-04-24|4-NOT SPECIFIED|Clerk#000000517|0|n accounts are regular, bold accounts. even instructions use request|
+4838|44|F|61811.33|1992-08-02|1-URGENT|Clerk#000000569|0|ffily bold sentiments. carefully close dolphins cajole across the |
+4839|25|F|71241.63|1994-05-10|1-URGENT|Clerk#000000925|0| even somas. slyly express ideas lose carefully. blithely unusu|
+4864|88|F|149614.34|1992-11-11|5-LOW|Clerk#000000423|0|ests nag within the quickly ironic asymptotes. ironic|
+4865|85|O|162113.46|1997-06-07|3-MEDIUM|Clerk#000000418|0|sits boost stealthily above the bl|
+4866|53|O|25767.07|1997-08-07|2-HIGH|Clerk#000000663|0|kages. unusual packages nag fluffily. qui|
+4867|10|F|9741.03|1992-05-21|1-URGENT|Clerk#000000891|0|ss the slyly regular dependencies. fluffily regular deposits within the car|
+4868|76|O|159005.35|1997-03-02|5-LOW|Clerk#000000729|0|regular asymptotes. regular packages sublate carefully al|
+4869|58|F|175422.13|1994-09-26|5-LOW|Clerk#000000802|0|boost! ironic packages un|
+4870|103|F|94534.07|1994-08-06|3-MEDIUM|Clerk#000000911|0|nto beans about the blithely regular d|
+4871|46|O|129636.99|1995-06-12|1-URGENT|Clerk#000000531|0|ven, special instructions across t|
+4896|85|F|93206.35|1992-08-22|1-URGENT|Clerk#000000622|0|sly pending deposits. final accounts boost above the sly, even|
+4897|80|F|115688.85|1992-09-17|5-LOW|Clerk#000000184|0|s. bold pinto beans sleep. evenly final accounts daz|
+4898|14|F|40572.64|1994-07-11|4-NOT SPECIFIED|Clerk#000000841|0|final patterns. special theodolites haggle ruthlessly at the blithely spec|
+4899|61|F|12291.83|1993-10-18|4-NOT SPECIFIED|Clerk#000000348|0| instructions. furiously even packages are furiously speci|
+4900|137|F|221320.76|1992-06-30|4-NOT SPECIFIED|Clerk#000000878|0|sleep quickly unusual |
+4901|79|O|146298.28|1997-12-31|4-NOT SPECIFIED|Clerk#000000980|0|inal dependencies cajole furiously. carefully express accounts na|
+4902|139|O|26011.20|1998-07-04|3-MEDIUM|Clerk#000000874|0| the slyly express dolphins. |
+4903|92|F|34363.63|1992-03-22|4-NOT SPECIFIED|Clerk#000000907|0|yly. multipliers within the fo|
+4928|4|F|59931.42|1993-10-04|4-NOT SPECIFIED|Clerk#000000952|0|slyly brave instructions after the ironic excuses haggle ruthlessly about|
+4929|149|O|135187.33|1996-02-29|3-MEDIUM|Clerk#000000109|0|uests. furiously special ideas poach. pending |
+4930|149|F|176867.34|1994-05-06|5-LOW|Clerk#000000593|0| haggle slyly quietly final theodolites. packages are furious|
+4931|50|F|115759.13|1994-11-17|1-URGENT|Clerk#000000356|0|leep. slyly express dolphins nag slyly. furiously regular s|
+4932|122|F|42927.07|1993-08-10|1-URGENT|Clerk#000000830|0|onic foxes. enticingly reg|
+4933|94|O|42945.82|1995-07-14|3-MEDIUM|Clerk#000000848|0|y special sauternes integr|
+4934|40|O|180478.16|1997-02-17|1-URGENT|Clerk#000000372|0|nes cajole; carefully special accounts haggle. special pinto beans nag |
+4935|40|F|162088.30|1993-05-25|4-NOT SPECIFIED|Clerk#000000601|0|c foxes. fluffily pendin|
+4960|124|F|153259.41|1995-02-26|5-LOW|Clerk#000000229|0|uriously even excuses. fluffily regular instructions along the furiously ironi|
+4961|58|O|89224.24|1998-04-06|3-MEDIUM|Clerk#000000731|0| braids. furiously even theodolites |
+4962|104|F|44781.32|1993-07-28|3-MEDIUM|Clerk#000000008|0| breach never ironic |
+4963|34|O|54175.35|1996-11-07|3-MEDIUM|Clerk#000000754|0|ully unusual epitaphs nod s|
+4964|101|O|204163.10|1997-07-28|4-NOT SPECIFIED|Clerk#000000144|0|ithely final theodolites. blithely regu|
+4965|52|F|110626.82|1993-10-21|5-LOW|Clerk#000000638|0|dependencies poach packages. sometim|
+4966|70|O|59186.02|1996-09-07|2-HIGH|Clerk#000000243|0|accounts. blithely ironic courts wake boldly furiously express |
+4967|98|O|103814.27|1997-02-17|3-MEDIUM|Clerk#000000397|0|e theodolites; furiously b|
+4992|62|F|203904.80|1992-05-10|1-URGENT|Clerk#000000166|0|telets nag carefully am|
+4993|13|F|145730.19|1994-08-04|4-NOT SPECIFIED|Clerk#000000258|0|ing instructions nag furiously. un|
+4994|43|O|216071.76|1996-06-29|4-NOT SPECIFIED|Clerk#000000868|0|oxes wake above the asymptotes. bold requests sleep br|
+4995|40|O|189651.76|1996-01-06|4-NOT SPECIFIED|Clerk#000000748|0|s. even deposits boost along the express, even theodolites. stealthily ir|
+4996|133|F|100750.67|1992-09-14|3-MEDIUM|Clerk#000000433|0|foxes. carefully special packages haggle quickly fluffi|
+4997|47|O|122611.05|1998-03-18|5-LOW|Clerk#000000040|0|egrate final pinto beans. fluffily special notornis use blith|
+4998|32|F|129096.80|1992-01-11|4-NOT SPECIFIED|Clerk#000000054|0|alongside of the quickly final requests hang always|
+4999|85|F|98643.17|1993-06-26|2-HIGH|Clerk#000000504|0| dolphins cajole blithely above the sly |
+5024|124|O|116127.69|1996-10-25|3-MEDIUM|Clerk#000000659|0|r foxes. regular excuses are about the quickly regular theodolites. regular, |
+5025|121|O|20099.43|1997-02-03|5-LOW|Clerk#000000805|0|ackages are slyly about the quickly |
+5026|28|O|13197.78|1997-09-06|1-URGENT|Clerk#000000955|0|y final requests us|
+5027|148|O|181346.56|1997-08-30|2-HIGH|Clerk#000000751|0|e-- final, pending requests along t|
+5028|13|F|30755.69|1992-04-17|2-HIGH|Clerk#000000180|0|ickly blithely express deposits. b|
+5029|11|F|19811.69|1992-11-14|3-MEDIUM|Clerk#000000469|0|. regular accounts haggle slyly. regul|
+5030|106|O|71781.23|1998-05-25|4-NOT SPECIFIED|Clerk#000000564|0| wake slyly furiously thin requests. ironic pinto beans ha|
+5031|139|F|91438.59|1994-12-02|3-MEDIUM|Clerk#000000788|0|lar instructions haggle blithely pending foxes? sometimes final excuses h|
+5056|52|O|62258.18|1997-02-15|5-LOW|Clerk#000000828|0|lithely above the express ideas. blithely final deposits are fluffily spec|
+5057|64|O|76164.41|1997-08-03|1-URGENT|Clerk#000000955|0|r ironic requests of the carefully ironic dependencies wake slyly a|
+5058|119|O|17031.01|1998-03-23|1-URGENT|Clerk#000000367|0| the pending packages wake after the quickly speci|
+5059|43|F|67173.82|1993-11-10|2-HIGH|Clerk#000000058|0|latelets. final, regular accounts cajole furiously ironic pinto beans? do|
+5060|112|F|65218.47|1992-07-07|4-NOT SPECIFIED|Clerk#000000333|0|e according to the excuses. express theodo|
+5061|101|F|52190.52|1993-08-14|1-URGENT|Clerk#000000009|0|e packages use fluffily according to the carefully ironic deposits. bol|
+5062|61|F|109247.00|1992-10-08|3-MEDIUM|Clerk#000000012|0|ithely. blithely bold theodolites affix. blithely final deposits haggle ac|
+5063|23|O|98753.57|1997-05-17|2-HIGH|Clerk#000000745|0|lyly after the pending foxes. express theodolites breach across t|
+5088|130|F|101616.44|1993-01-06|5-LOW|Clerk#000000930|0|ole slyly since the quickly ironic br|
+5089|130|F|109246.54|1992-07-29|1-URGENT|Clerk#000000677|0|cial platelets. quiet, final ideas cajole carefully. unusu|
+5090|89|O|132838.49|1997-03-09|1-URGENT|Clerk#000000953|0|ress accounts affix silently carefully quick accounts. carefully f|
+5091|148|O|47852.06|1998-05-21|3-MEDIUM|Clerk#000000311|0|egular decoys mold carefully fluffily unus|
+5092|22|O|195834.96|1995-10-30|5-LOW|Clerk#000000194|0|are blithely along the pin|
+5093|79|F|190693.92|1993-09-03|3-MEDIUM|Clerk#000000802|0|ully ironic theodolites sleep above the furiously ruthless instructions. bli|
+5094|106|F|74892.08|1993-03-29|4-NOT SPECIFIED|Clerk#000000406|0|uickly pending deposits haggle quickly ide|
+5095|97|F|184583.99|1992-04-22|2-HIGH|Clerk#000000964|0|accounts are carefully! slyly even packages wake slyly a|
+5120|16|O|28007.73|1996-06-05|1-URGENT|Clerk#000000332|0|against the slyly express requests. furiousl|
+5121|133|F|150334.57|1992-05-11|4-NOT SPECIFIED|Clerk#000000736|0|gular requests. furiously final pearls against the permanent, thin courts s|
+5122|70|O|79863.84|1996-02-10|5-LOW|Clerk#000000780|0|blithely. slyly ironic deposits nag. excuses s|
+5123|10|O|11850.45|1998-02-10|1-URGENT|Clerk#000000776|0|ic requests. furiously ironic packages grow above the express, ironic inst|
+5124|25|O|159170.80|1997-04-04|4-NOT SPECIFIED|Clerk#000000749|0|kly even courts. bold packages solve. |
+5125|28|O|38065.28|1998-02-07|5-LOW|Clerk#000000834|0|ructions. dolphins wake slowly carefully unusual |
+5126|112|F|92123.32|1992-10-12|4-NOT SPECIFIED|Clerk#000000270|0|s. unusual deposits |
+5127|73|O|48024.99|1997-01-15|5-LOW|Clerk#000000829|0|fully express pinto beans. slyly final accounts along the ironic dugouts use s|
+5152|44|O|60568.34|1997-01-04|3-MEDIUM|Clerk#000000963|0| for the blithely reg|
+5153|113|O|193832.28|1995-08-26|1-URGENT|Clerk#000000954|0| the furiously ironic foxes. express packages shall cajole carefully across|
+5154|8|O|28070.86|1997-04-13|3-MEDIUM|Clerk#000000316|0|inal requests. slyly regular deposits nag. even deposits haggle agains|
+5155|77|F|70183.29|1994-06-12|2-HIGH|Clerk#000000108|0|y pending deposits are ag|
+5156|125|O|59439.44|1996-11-04|5-LOW|Clerk#000000117|0|ngside of the multipliers solve slyly requests. regu|
+5157|142|O|167056.34|1997-07-06|4-NOT SPECIFIED|Clerk#000000689|0|closely above the unusual deposits. furiously|
+5158|76|O|240284.95|1997-01-21|1-URGENT|Clerk#000000541|0| regular foxes. even foxes wake blithely |
+5159|106|O|147543.26|1996-09-25|1-URGENT|Clerk#000000303|0|tegrate slyly around the slyly sly sauternes. final pa|
+5184|85|O|209155.48|1998-07-20|5-LOW|Clerk#000000250|0|nding accounts detect final, even|
+5185|148|O|206179.68|1997-07-25|3-MEDIUM|Clerk#000000195|0| regular ideas about the even ex|
+5186|52|O|208892.63|1996-08-03|1-URGENT|Clerk#000000332|0|pecial platelets. slyly final ac|
+5187|55|O|46380.69|1997-07-16|3-MEDIUM|Clerk#000000682|0|ckly according to t|
+5188|140|P|66268.86|1995-03-02|4-NOT SPECIFIED|Clerk#000000029|0|counts. finally ironic requests ab|
+5189|71|F|184172.31|1993-11-26|5-LOW|Clerk#000000940|0|e after the pending accounts. asymptotes boost. re|
+5190|58|F|89684.31|1992-04-26|5-LOW|Clerk#000000888|0|equests. slyly unusual|
+5191|77|F|119910.04|1994-12-11|4-NOT SPECIFIED|Clerk#000000318|0|ing, regular deposits alongside of the deposits boost fluffily quickly ev|
+5216|59|O|16763.95|1997-08-14|3-MEDIUM|Clerk#000000418|0|des boost across the platelets. slyly busy theodolit|
+5217|35|O|135745.58|1995-10-13|2-HIGH|Clerk#000000873|0|ons might wake quickly according to th|
+5218|82|F|73882.37|1992-07-30|4-NOT SPECIFIED|Clerk#000000683|0|y ruthless packages according to the bold, ironic package|
+5219|88|O|21267.72|1997-02-27|1-URGENT|Clerk#000000510|0|aggle always. foxes above the ironic deposits |
+5220|10|F|24844.39|1992-07-30|2-HIGH|Clerk#000000051|0| final packages. ideas detect slyly around|
+5221|13|O|71968.10|1995-06-09|4-NOT SPECIFIED|Clerk#000000324|0|lar accounts above the sl|
+5222|80|F|1051.15|1994-05-27|4-NOT SPECIFIED|Clerk#000000613|0|along the bold ideas. furiously final foxes snoo|
+5223|149|F|105561.21|1994-06-30|1-URGENT|Clerk#000000745|0|e. theodolites serve blithely unusual, final foxes. carefully pending packag|
+5248|70|P|86958.28|1995-04-15|2-HIGH|Clerk#000000737|0|theodolites cajole according to the silent packages. quickly ironic packages a|
+5249|103|F|123586.03|1994-09-06|3-MEDIUM|Clerk#000000019|0|refully bold accounts |
+5250|97|O|29673.73|1995-07-16|2-HIGH|Clerk#000000307|0|. carefully final instructions sleep among the finally regular dependen|
+5251|34|O|34004.48|1995-04-12|3-MEDIUM|Clerk#000000687|0| ironic dugouts detect. reque|
+5252|91|O|173145.37|1996-02-17|1-URGENT|Clerk#000000724|0| ironic accounts among the silent asym|
+5253|148|P|108361.46|1995-04-11|2-HIGH|Clerk#000000275|0|egular requests! blithely regular deposits alongside of t|
+5254|112|F|196989.09|1992-07-26|4-NOT SPECIFIED|Clerk#000000527|0|he express, even ideas cajole blithely special requests|
+5255|64|O|75074.07|1996-07-12|5-LOW|Clerk#000000591|0|ly slow forges. express foxes haggle. regular, even asymp|
+5280|34|O|68052.70|1997-12-03|3-MEDIUM|Clerk#000000604|0|riously ironic instructions. ironic ideas according to the accounts boost fur|
+5281|124|O|179418.31|1995-11-02|2-HIGH|Clerk#000000158|0|ackages haggle slyly a|
+5282|50|O|94446.69|1998-01-30|1-URGENT|Clerk#000000030|0|rding to the unusual, bold accounts. regular instructions|
+5283|131|F|18594.66|1994-06-04|3-MEDIUM|Clerk#000000579|0|ests. even, final ideas alongside of t|
+5284|61|O|40548.99|1995-07-09|4-NOT SPECIFIED|Clerk#000000155|0| careful dependencies use sly|
+5285|70|F|99377.51|1994-01-18|2-HIGH|Clerk#000000976|0|p across the furiously ironic deposits.|
+5286|116|O|79646.89|1997-09-26|5-LOW|Clerk#000000606|0|structions are furiously quickly ironic asymptotes. quickly iro|
+5287|25|F|30045.95|1993-12-22|5-LOW|Clerk#000000406|0|regular packages. bold instructions sleep always. carefully final p|
+5312|65|F|66697.95|1995-02-24|2-HIGH|Clerk#000000690|0|ter the even, bold foxe|
+5313|13|O|159870.44|1997-06-17|4-NOT SPECIFIED|Clerk#000000896|0|le. final courts haggle furiously according to the |
+5314|34|O|26999.83|1995-06-02|2-HIGH|Clerk#000000617|0|ions across the quickly special d|
+5315|139|F|55554.97|1992-10-29|4-NOT SPECIFIED|Clerk#000000035|0| furiously. quickly unusual packages use. sly|
+5316|100|F|62316.61|1994-01-31|1-URGENT|Clerk#000000734|0| requests haggle across the regular, pending deposits. furiously regular requ|
+5317|37|F|228002.51|1994-09-09|5-LOW|Clerk#000000687|0|jole quickly at the slyly pend|
+5318|59|F|106935.19|1993-04-04|2-HIGH|Clerk#000000663|0|efully regular dolphins. even ideas nag fluffily furiously even packa|
+5319|98|O|68619.29|1996-01-21|1-URGENT|Clerk#000000237|0|lent requests. quickly pe|
+5344|109|O|88216.32|1998-06-21|3-MEDIUM|Clerk#000000569|0|s. ironic excuses cajole across the|
+5345|31|O|111924.56|1997-08-24|1-URGENT|Clerk#000000057|0|r the slyly silent packages. pending, even pinto b|
+5346|37|F|149536.20|1993-12-26|2-HIGH|Clerk#000000220|0|gly close packages against the even, regular escapades boost evenly accordi|
+5347|49|F|173024.71|1995-02-22|3-MEDIUM|Clerk#000000180|0|onic, regular deposits. packag|
+5348|53|O|119164.96|1997-11-08|5-LOW|Clerk#000000497|0|totes. accounts after the furiously|
+5349|67|O|38038.84|1996-09-01|1-URGENT|Clerk#000000960|0|le along the carefully bold dolphins. carefully special packa|
+5350|76|F|113417.03|1993-10-10|5-LOW|Clerk#000000604|0|ccounts after the carefully pending requests believe |
+5351|122|O|76799.25|1998-05-11|1-URGENT|Clerk#000000443|0|to beans sleep furiously after the carefully even|
+5376|149|F|98422.83|1994-07-04|5-LOW|Clerk#000000392|0|. quickly ironic deposits integrate along|
+5377|64|O|117728.37|1997-04-24|2-HIGH|Clerk#000000917|0|ons nag blithely furiously regula|
+5378|43|F|101899.93|1992-10-25|1-URGENT|Clerk#000000520|0|n ideas. regular accounts haggle. ironic ideas use along the bold ideas. blith|
+5379|89|O|47010.15|1995-08-08|2-HIGH|Clerk#000000503|0|he unusual accounts. carefully special instructi|
+5380|148|O|123014.83|1997-10-12|1-URGENT|Clerk#000000481|0|le slyly about the slyly final dolphins. fu|
+5381|32|F|223995.46|1993-01-29|5-LOW|Clerk#000000531|0|arefully bold packages are slyly furiously ironic foxes. fluffil|
+5382|35|F|138423.03|1992-01-13|5-LOW|Clerk#000000809|0|lent deposits are according to the reg|
+5383|31|O|11474.95|1995-05-26|5-LOW|Clerk#000000409|0|ly bold requests hang furiously furiously unusual accounts. evenly unusu|
+5408|23|F|123477.05|1992-07-21|5-LOW|Clerk#000000735|0|egular requests according to the|
+5409|13|F|145040.38|1992-01-09|5-LOW|Clerk#000000171|0|eans. regular accounts are regul|
+5410|22|O|139104.17|1998-07-28|4-NOT SPECIFIED|Clerk#000000117|0|final deposits: pending excuses boost. ironic theodolites cajole furi|
+5411|61|O|62541.27|1997-05-16|3-MEDIUM|Clerk#000000800|0|equests cajole slyly furious|
+5412|142|O|109979.71|1998-01-20|2-HIGH|Clerk#000000151|0|ets boost furiously regular accounts. regular foxes above th|
+5413|94|O|224382.57|1997-10-17|1-URGENT|Clerk#000000066|0|e even excuses. always final depen|
+5414|100|F|167017.39|1993-03-25|4-NOT SPECIFIED|Clerk#000000242|0|lent dependencies? carefully express requests sleep furiously ac|
+5415|23|F|176864.83|1992-08-05|3-MEDIUM|Clerk#000000998|0|ly even ideas nag blithely above the final instructions|
+5440|130|O|3223.17|1997-01-12|1-URGENT|Clerk#000000154|0|posits boost regularly ironic packages. regular, ironic deposits wak|
+5441|41|F|131891.05|1994-07-21|4-NOT SPECIFIED|Clerk#000000257|0|after the furiously ironic |
+5442|43|O|139332.94|1998-01-13|4-NOT SPECIFIED|Clerk#000000954|0|ully. quickly express accounts against the|
+5443|131|O|124950.79|1996-10-10|4-NOT SPECIFIED|Clerk#000000492|0|al foxes could detect. blithely stealthy asymptotes kind|
+5444|130|P|172908.01|1995-03-18|1-URGENT|Clerk#000000677|0| asymptotes. asymptotes cajole quickly quickly bo|
+5445|115|F|114990.63|1993-07-26|5-LOW|Clerk#000000623|0|s. even, special requests cajole furiously even, |
+5446|7|F|29920.80|1994-06-21|5-LOW|Clerk#000000304|0| furiously final pac|
+5447|13|O|29029.84|1996-03-16|3-MEDIUM|Clerk#000000597|0|uternes around the furiously bold accounts wake after |
+5472|70|F|221636.83|1993-04-11|5-LOW|Clerk#000000552|0|counts. deposits about the slyly dogged pinto beans cajole slyly|
+5473|65|F|63041.33|1992-03-25|4-NOT SPECIFIED|Clerk#000000306|0|te the quickly stealthy ideas. even, regular deposits above|
+5474|55|F|131079.52|1992-06-01|4-NOT SPECIFIED|Clerk#000000487|0|gle blithely enticing ideas. final, exp|
+5475|139|O|10645.48|1996-07-07|5-LOW|Clerk#000000856|0|es shall boost slyly. furiously even deposits lose. instruc|
+5476|91|O|26906.38|1997-11-06|1-URGENT|Clerk#000000189|0|furiously final ideas. furiously bold dependencies sleep care|
+5477|107|O|130125.64|1997-12-30|5-LOW|Clerk#000000689|0|ckages. ironic deposits caj|
+5478|116|O|97502.23|1996-05-17|1-URGENT|Clerk#000000272|0|ckages. quickly pending deposits thrash furiously: bl|
+5479|70|F|70553.45|1993-12-22|3-MEDIUM|Clerk#000000335|0|ng asymptotes. pinto beans sleep care|
+5504|19|F|41492.25|1993-01-06|2-HIGH|Clerk#000000221|0|y pending packages. furiousl|
+5505|95|O|147329.51|1997-10-04|5-LOW|Clerk#000000719|0| final, regular packages according to the slyly ironic accounts nag ironica|
+5506|91|F|8413.31|1993-11-08|1-URGENT|Clerk#000000292|0|nusual theodolites. sly|
+5507|2|O|140363.70|1998-05-28|5-LOW|Clerk#000000692|0|the carefully ironic instructions are quickly iro|
+5508|56|O|3808.05|1996-06-21|1-URGENT|Clerk#000000128|0|y express packages cajole furiously. slyly unusual requests |
+5509|80|F|135335.96|1994-04-08|5-LOW|Clerk#000000164|0|usual deposits use packages. furiously final requests wake slyly about th|
+5510|37|F|126948.81|1993-01-08|3-MEDIUM|Clerk#000000819|0| nag slyly. carefully eve|
+5511|79|F|151089.96|1994-11-29|1-URGENT|Clerk#000000438|0|ng instructions integrate fluffily among the fluffily silent accounts. bli|
+5536|116|O|108196.56|1998-03-16|4-NOT SPECIFIED|Clerk#000000076|0| carefully final dolphins. ironic, ironic deposits lose. bold, |
+5537|118|O|102207.20|1996-10-03|3-MEDIUM|Clerk#000000742|0|ng to the daring, final |
+5538|139|F|90981.28|1993-12-25|1-URGENT|Clerk#000000992|0|ttainments. slyly final ideas are about the furiously silent excuses.|
+5539|119|F|39397.60|1994-07-31|5-LOW|Clerk#000000675|0|structions. slyly regular patterns solve above the carefully expres|
+5540|130|O|90707.58|1996-10-12|4-NOT SPECIFIED|Clerk#000000120|0|y ironic packages cajole blithely|
+5541|143|O|37526.68|1997-09-30|3-MEDIUM|Clerk#000000217|0|encies among the silent accounts sleep slyly quickly pending deposits|
+5542|49|O|6402.41|1996-04-20|4-NOT SPECIFIED|Clerk#000000100|0|riously among the regularly regular pac|
+5543|115|F|118201.53|1993-09-25|3-MEDIUM|Clerk#000000644|0|ckly regular epitaphs. carefully bold accounts haggle furiously|
+5568|31|O|105421.09|1995-06-07|3-MEDIUM|Clerk#000000491|0| nag. fluffily pending de|
+5569|109|F|126113.32|1993-04-30|4-NOT SPECIFIED|Clerk#000000759|0|e regular dependencies. furiously unusual ideas b|
+5570|112|O|78567.55|1996-08-12|2-HIGH|Clerk#000000795|0|eans. ironic, even requests doze |
+5571|103|F|79248.35|1992-12-19|4-NOT SPECIFIED|Clerk#000000184|0|ts cajole furiously carefully regular sheaves. un|
+5572|8|F|182966.39|1994-07-17|2-HIGH|Clerk#000000163|0|e fluffily express deposits cajole slyly across th|
+5573|37|O|158479.37|1996-08-15|3-MEDIUM|Clerk#000000055|0|lites. slyly final pinto beans about the carefully regul|
+5574|28|F|129803.03|1992-03-10|4-NOT SPECIFIED|Clerk#000000002|0|n deposits. special, regular t|
+5575|103|O|51839.94|1995-07-24|5-LOW|Clerk#000000948|0|uriously express frays breach|
+5600|95|O|53649.35|1997-02-08|4-NOT SPECIFIED|Clerk#000000019|0|lly regular deposits. car|
+5601|11|F|118570.79|1992-01-06|2-HIGH|Clerk#000000827|0|gular deposits wake platelets? blithe|
+5602|130|O|67979.49|1997-07-30|3-MEDIUM|Clerk#000000395|0|onic asymptotes haggl|
+5603|71|F|145100.47|1992-06-20|4-NOT SPECIFIED|Clerk#000000535|0| asymptotes. fluffily ironic instructions are. pending pinto bean|
+5604|46|O|98987.51|1998-04-14|4-NOT SPECIFIED|Clerk#000000123|0|ously across the blithely ironic pinto beans. sile|
+5605|35|O|172899.84|1996-08-22|2-HIGH|Clerk#000000538|0|sleep carefully final packages. dependencies wake slyly. theodol|
+5606|149|O|219959.08|1996-11-12|5-LOW|Clerk#000000688|0|uriously express pinto beans. packages sh|
+5607|92|F|24660.06|1992-01-01|4-NOT SPECIFIED|Clerk#000000137|0|c requests promise quickly fluffily ironic deposits. caref|
+5632|79|O|89503.11|1996-02-05|1-URGENT|Clerk#000000508|0|ons. blithely pending pinto beans thrash. furiously busy theodoli|
+5633|79|O|207119.83|1998-05-31|3-MEDIUM|Clerk#000000841|0|cial deposits wake final, final|
+5634|68|O|99494.67|1996-07-31|3-MEDIUM|Clerk#000000915|0|out the accounts. carefully ironic ideas are slyly. sheaves could h|
+5635|70|F|192217.86|1992-08-16|3-MEDIUM|Clerk#000000734|0|nal platelets sleep daringly. idle, final accounts about |
+5636|122|F|143350.75|1995-02-16|3-MEDIUM|Clerk#000000916|0|. boldly even Tiresias sleep. blithely ironic packages among the ca|
+5637|103|O|128776.90|1996-06-17|3-MEDIUM|Clerk#000000183|0|nic dolphins are regular packages. ironic pinto beans hagg|
+5638|109|F|79197.77|1994-01-17|1-URGENT|Clerk#000000355|0|enly bold deposits eat. special realms play against the regular, speci|
+5639|145|F|9669.46|1994-06-02|3-MEDIUM|Clerk#000000005|0|ending packages use after the blithely regular accounts. regular package|
+5664|119|O|186215.81|1998-07-23|2-HIGH|Clerk#000000789|0|the quickly ironic dolp|
+5665|100|F|129821.09|1993-06-28|4-NOT SPECIFIED|Clerk#000000513|0| carefully special instructions. ironic pinto beans nag slyly blithe|
+5666|14|F|121663.68|1994-02-02|2-HIGH|Clerk#000000396|0|mptotes. quickly final instructions are |
+5667|44|O|37301.25|1995-08-10|1-URGENT|Clerk#000000358|0|s print upon the quickly ironic packa|
+5668|109|F|13679.32|1995-03-22|4-NOT SPECIFIED|Clerk#000000047|0|p slyly slyly express accoun|
+5669|74|O|113156.30|1996-05-06|1-URGENT|Clerk#000000336|0|ng packages nag fluffily furio|
+5670|7|F|101429.61|1993-04-21|5-LOW|Clerk#000000922|0|he carefully final packages. deposits are slyly among the requests. |
+5671|43|O|176647.54|1998-02-06|2-HIGH|Clerk#000000838|0|k dependencies. slyly |
+5696|142|P|198723.30|1995-05-04|1-URGENT|Clerk#000000447|0|e quickly unusual pack|
+5697|55|F|99177.69|1992-10-05|1-URGENT|Clerk#000000112|0|pendencies impress furiously. bold, final requests solve ab|
+5698|95|F|154936.43|1994-05-21|3-MEDIUM|Clerk#000000455|0|he furiously silent accounts haggle blithely against the carefully unusual|
+5699|142|F|226314.91|1992-07-30|5-LOW|Clerk#000000311|0|o beans. ironic asymptotes boost. blithe, final courts integrate|
+5700|143|O|79901.18|1997-12-25|1-URGENT|Clerk#000000618|0|ly pending dolphins sleep carefully slyly pending i|
+5701|43|O|16689.19|1997-02-07|5-LOW|Clerk#000000798|0| blithely final pinto beans. blit|
+5702|97|F|153024.28|1993-09-07|4-NOT SPECIFIED|Clerk#000000743|0|ironic accounts. final accounts wake express deposits. final pac|
+5703|121|F|1816.28|1993-05-16|3-MEDIUM|Clerk#000000647|0|ly special instructions. slyly even reque|
+5728|80|F|85397.04|1994-12-11|4-NOT SPECIFIED|Clerk#000000426|0|furiously express pin|
+5729|44|F|88080.33|1994-10-10|2-HIGH|Clerk#000000843|0|uffily sly accounts about|
+5730|11|O|10934.84|1997-12-18|1-URGENT|Clerk#000000181|0|l platelets. ironic pinto beans wake slyly. quickly b|
+5731|8|O|57823.37|1997-05-17|5-LOW|Clerk#000000841|0| silent excuses among the express accounts wake |
+5732|37|O|28330.42|1997-08-03|1-URGENT|Clerk#000000910|0|he quickly bold asymptotes: final platelets wake quickly. blithely final pinto|
+5733|101|F|38545.97|1993-03-17|2-HIGH|Clerk#000000873|0|osits. pending accounts boost quickly. furiously permanent acco|
+5734|94|O|45860.94|1997-10-12|3-MEDIUM|Clerk#000000084|0|efully even braids detect blithely alo|
+5735|40|F|39358.51|1994-12-11|3-MEDIUM|Clerk#000000600|0| bold realms cajole slyly fu|
+5760|25|F|59404.77|1994-05-25|4-NOT SPECIFIED|Clerk#000000498|0|s among the blithely regular frays haggle ironically bold theodolites. al|
+5761|16|O|130345.90|1998-07-06|3-MEDIUM|Clerk#000000208|0|s asymptotes cajole boldly. regular, |
+5762|49|O|165019.32|1997-02-14|1-URGENT|Clerk#000000901|0|ly bold packages: slyly ironic deposits sleep quietly foxes. express a|
+5763|8|O|140838.11|1998-06-26|4-NOT SPECIFIED|Clerk#000000633|0|according to the furiously regular pinto beans. even accounts wake fu|
+5764|131|F|53212.95|1993-10-03|4-NOT SPECIFIED|Clerk#000000363|0| furiously regular deposits haggle fluffily around th|
+5765|52|F|249900.42|1994-12-15|5-LOW|Clerk#000000959|0|longside of the quickly final packages. instructions so|
+5766|49|F|47940.51|1993-09-27|5-LOW|Clerk#000000753|0|. quickly final packages print slyly. fu|
+5767|118|F|135643.87|1992-04-29|2-HIGH|Clerk#000000225|0|ts wake fluffily above the r|
+5792|26|F|158991.89|1993-04-04|2-HIGH|Clerk#000000731|0|packages. doggedly bold deposits integrate furiously across the|
+5793|37|O|119887.47|1997-07-13|2-HIGH|Clerk#000000294|0|thely. fluffily even instructi|
+5794|8|F|122823.78|1993-04-05|5-LOW|Clerk#000000855|0|t accounts kindle about the gifts. as|
+5795|37|F|35514.45|1992-05-05|2-HIGH|Clerk#000000581|0| even instructions x-ray ironic req|
+5796|149|O|23280.61|1996-01-23|3-MEDIUM|Clerk#000000326|0|eodolites. slyly ironic pinto beans at the silent, special request|
+5797|122|O|15313.61|1997-10-15|4-NOT SPECIFIED|Clerk#000000381|0|ng! packages against the blithely b|
+5798|106|O|125011.92|1998-03-30|5-LOW|Clerk#000000343|0|lent accounts affix quickly! platelets run slyly slyly final packages. f|
+5799|26|O|71381.21|1995-08-03|1-URGENT|Clerk#000000238|0| unusual deposits sleep blithely along the carefully even requests. care|
+5824|56|O|169107.85|1996-12-03|2-HIGH|Clerk#000000171|0|unusual packages. even ideas along the even requests are along th|
+5825|61|F|23020.62|1995-02-21|5-LOW|Clerk#000000494|0|regular packages use bravely.|
+5826|22|O|21119.86|1998-06-13|1-URGENT|Clerk#000000087|0|even, regular dependenc|
+5827|31|O|137297.71|1998-07-23|3-MEDIUM|Clerk#000000660|0|hely furiously blithe dolphins. slyly |
+5828|127|F|62172.34|1994-03-06|5-LOW|Clerk#000000377|0|ages boost never during the final packa|
+5829|125|O|183734.56|1997-01-11|1-URGENT|Clerk#000000196|0|gular accounts. bold accounts are blithely furiously ironic r|
+5830|85|F|28223.57|1993-03-25|3-MEDIUM|Clerk#000000233|0|lites haggle. ironic, ironic instructions maintain blit|
+5831|139|O|113505.19|1996-11-17|5-LOW|Clerk#000000585|0|s final, final pinto beans. unusual depos|
+5856|37|F|71460.49|1994-11-06|2-HIGH|Clerk#000000634|0|special excuses. slyly final theodolites cajole blithely furiou|
+5857|124|O|158345.31|1997-11-06|4-NOT SPECIFIED|Clerk#000000267|0|gage blithely. quickly special ac|
+5858|64|F|181320.50|1992-07-14|4-NOT SPECIFIED|Clerk#000000580|0|lyly pending dugouts believe through the ironic deposits. silent s|
+5859|5|O|210643.96|1997-04-23|1-URGENT|Clerk#000000993|0|requests boost. asymptotes across the deposits solve slyly furiously pendin|
+5860|13|F|9495.28|1992-02-20|4-NOT SPECIFIED|Clerk#000000079|0| beans. bold, special foxes sleep about the ir|
+5861|139|O|41450.19|1997-04-10|3-MEDIUM|Clerk#000000094|0|rthogs cajole slyly. express packages sleep blithely final |
+5862|64|O|30550.90|1997-02-20|1-URGENT|Clerk#000000039|0|leep beneath the quickly busy excuses. ironic theodolit|
+5863|65|F|67941.54|1993-11-22|3-MEDIUM|Clerk#000000774|0|ets about the slyly pending ideas sleep according to the blithely |
+5888|46|O|67167.19|1996-09-28|3-MEDIUM|Clerk#000000748|0|quickly against the furiously final requests. evenly fi|
+5889|22|O|15417.57|1995-05-23|5-LOW|Clerk#000000690|0|ites wake across the slyly ironic|
+5890|49|F|41162.24|1992-11-04|2-HIGH|Clerk#000000013|0|packages. final, final reques|
+5891|46|F|41760.00|1992-12-29|3-MEDIUM|Clerk#000000302|0|ounts haggle furiously abo|
+5892|101|P|92340.77|1995-05-09|5-LOW|Clerk#000000639|0| pending instruction|
+5893|2|F|44777.63|1992-07-08|4-NOT SPECIFIED|Clerk#000000560|0|final sentiments. instructions boost above the never speci|
+5894|71|F|70377.31|1994-08-13|2-HIGH|Clerk#000000776|0|regular deposits wake|
+5895|64|O|201419.83|1997-01-01|4-NOT SPECIFIED|Clerk#000000747|0| ironic, unusual requests cajole blithely special, special deposits. s|
+5920|119|F|142767.26|1994-11-20|2-HIGH|Clerk#000000081|0|ns: even ideas cajole slyly among the packages. never ironic patterns|
+5921|58|F|152940.00|1994-04-07|5-LOW|Clerk#000000125|0|kly special requests breach.|
+5922|143|O|142494.99|1996-11-14|5-LOW|Clerk#000000625|0| ironic instructions haggle furiously blithely regular accounts: even platele|
+5923|101|O|157968.27|1997-05-27|2-HIGH|Clerk#000000304|0|o beans haggle slyly above the regular, even dependencies|
+5924|31|O|106823.97|1995-10-10|3-MEDIUM|Clerk#000000433|0|arefully after the pains. blithely ironic pinto |
+5925|146|O|242588.87|1995-11-13|5-LOW|Clerk#000000602|0|ourts. boldly regular foxes might sleep. slyly express tithes against |
+5926|76|F|105770.53|1994-05-20|5-LOW|Clerk#000000071|0| carefully after the furiously even re|
+5927|116|O|84983.90|1997-08-28|4-NOT SPECIFIED|Clerk#000000972|0|endencies according to the slyly ironic foxes detect furiously about the furio|
+5952|148|O|128624.99|1997-04-14|3-MEDIUM|Clerk#000000950|0| regular, final pla|
+5953|7|F|95312.81|1992-03-28|1-URGENT|Clerk#000000049|0|ages are furiously. slowly bold requests|
+5954|28|F|167262.34|1992-12-03|1-URGENT|Clerk#000000968|0|requests along the blith|
+5955|94|P|67944.38|1995-03-27|5-LOW|Clerk#000000340|0|deas integrate. fluffily regular pa|
+5956|22|O|118036.54|1998-05-18|1-URGENT|Clerk#000000587|0|le even, express platelets.|
+5957|89|F|230949.45|1993-12-27|2-HIGH|Clerk#000000020|0| dependencies are slyly. bold accounts according to the carefully regular r|
+5958|115|O|145060.41|1995-09-16|3-MEDIUM|Clerk#000000787|0|e final requests detect alongside of the qu|
+5959|23|F|195515.26|1992-05-15|3-MEDIUM|Clerk#000000913|0|into beans use ironic, unusual foxes. carefully regular excuses boost caref|
+5984|70|F|83413.30|1994-06-18|5-LOW|Clerk#000000023|0|ickly final pains haggle along the furiously ironic pinto bea|
+5985|143|F|3942.73|1995-01-12|3-MEDIUM|Clerk#000000417|0|as nag fluffily slyly permanent accounts. regular depo|
+5986|115|F|92187.80|1992-04-22|2-HIGH|Clerk#000000674|0|iously unusual notornis are |
+5987|64|O|98956.82|1996-08-03|1-URGENT|Clerk#000000464|0| ideas. quietly final accounts haggle blithely pending escapade|
+5988|31|F|41655.51|1993-11-22|4-NOT SPECIFIED|Clerk#000000867|0|fully express accounts. final pi|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/part.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/part.tbl
new file mode 100644
index 0000000..691ba9d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/part.tbl
@@ -0,0 +1,200 @@
+1|goldenrod lace spring peru powder|Manufacturer#1|Brand#13|PROMO BURNISHED COPPER|7|JUMBO PKG|901.00|ly. slyly ironi|
+2|blush rosy metallic lemon navajo|Manufacturer#1|Brand#13|LARGE BRUSHED BRASS|1|LG CASE|902.00|lar accounts amo|
+3|dark green antique puff wheat|Manufacturer#4|Brand#42|STANDARD POLISHED BRASS|21|WRAP CASE|903.00|egular deposits hag|
+4|chocolate metallic smoke ghost drab|Manufacturer#3|Brand#34|SMALL PLATED BRASS|14|MED DRUM|904.00|p furiously r|
+5|forest blush chiffon thistle chocolate|Manufacturer#3|Brand#32|STANDARD POLISHED TIN|15|SM PKG|905.00| wake carefully |
+6|white ivory azure firebrick black|Manufacturer#2|Brand#24|PROMO PLATED STEEL|4|MED BAG|906.00|sual a|
+7|blue blanched tan indian olive|Manufacturer#1|Brand#11|SMALL PLATED COPPER|45|SM BAG|907.00|lyly. ex|
+8|ivory khaki cream midnight rosy|Manufacturer#4|Brand#44|PROMO BURNISHED TIN|41|LG DRUM|908.00|eposi|
+9|thistle rose moccasin light floral|Manufacturer#4|Brand#43|SMALL BURNISHED STEEL|12|WRAP CASE|909.00|ironic foxe|
+10|floral moccasin royal powder burnished|Manufacturer#5|Brand#54|LARGE BURNISHED STEEL|44|LG CAN|910.01|ithely final deposit|
+11|chocolate turquoise sandy snow misty|Manufacturer#2|Brand#25|STANDARD BURNISHED NICKEL|43|WRAP BOX|911.01|ng gr|
+12|peru ivory olive powder frosted|Manufacturer#3|Brand#33|MEDIUM ANODIZED STEEL|25|JUMBO CASE|912.01| quickly|
+13|ghost blue olive sky gainsboro|Manufacturer#5|Brand#55|MEDIUM BURNISHED NICKEL|1|JUMBO PACK|913.01|osits.|
+14|linen seashell burnished blue gainsboro|Manufacturer#1|Brand#13|SMALL POLISHED STEEL|28|JUMBO BOX|914.01|kages c|
+15|navajo dark sky turquoise royal|Manufacturer#1|Brand#15|LARGE ANODIZED BRASS|45|LG CASE|915.01|usual ac|
+16|deep brown turquoise dim papaya|Manufacturer#3|Brand#32|PROMO PLATED TIN|2|MED PACK|916.01|unts a|
+17|burnished navy orange dodger cream|Manufacturer#4|Brand#43|ECONOMY BRUSHED STEEL|16|LG BOX|917.01| regular accounts|
+18|spring indian forest khaki midnight|Manufacturer#1|Brand#11|SMALL BURNISHED STEEL|42|JUMBO PACK|918.01|s cajole slyly a|
+19|dodger forest floral cream black|Manufacturer#2|Brand#23|SMALL ANODIZED NICKEL|33|WRAP BOX|919.01| pending acc|
+20|bisque salmon dark blanched linen|Manufacturer#1|Brand#12|LARGE POLISHED NICKEL|48|MED BAG|920.02|are across the asympt|
+21|lemon aquamarine firebrick floral almond|Manufacturer#3|Brand#33|SMALL BURNISHED TIN|31|MED BAG|921.02|ss packages. pendin|
+22|medium floral beige cornsilk olive|Manufacturer#4|Brand#43|PROMO POLISHED BRASS|19|LG DRUM|922.02| even p|
+23|firebrick bisque slate rose blanched|Manufacturer#3|Brand#35|MEDIUM BURNISHED TIN|42|JUMBO JAR|923.02|nic, fina|
+24|saddle dim white honeydew spring|Manufacturer#5|Brand#52|MEDIUM PLATED STEEL|20|MED CASE|924.02| final the|
+25|blush forest magenta metallic turquoise|Manufacturer#5|Brand#55|STANDARD BRUSHED COPPER|3|JUMBO BAG|925.02|requests wake|
+26|lawn forest misty chartreuse snow|Manufacturer#3|Brand#32|SMALL BRUSHED STEEL|32|SM CASE|926.02| instructions i|
+27|plum drab cornflower brown dim|Manufacturer#1|Brand#14|LARGE ANODIZED TIN|20|MED PKG|927.02|s wake. ir|
+28|green honeydew dim cyan bisque|Manufacturer#4|Brand#44|SMALL PLATED COPPER|19|JUMBO PKG|928.02|x-ray pending, iron|
+29|aquamarine sky salmon rose burlywood|Manufacturer#3|Brand#33|PROMO PLATED COPPER|7|LG DRUM|929.02| carefully fluffi|
+30|blush beige steel cyan navy|Manufacturer#4|Brand#42|PROMO ANODIZED TIN|17|LG BOX|930.03|carefully bus|
+31|grey ghost steel maroon moccasin|Manufacturer#5|Brand#53|STANDARD BRUSHED TIN|10|LG BAG|931.03|uriously s|
+32|yellow wheat midnight papaya blue|Manufacturer#4|Brand#42|ECONOMY PLATED BRASS|31|LG CASE|932.03|urts. carefully fin|
+33|maroon beige mint cyan peru|Manufacturer#2|Brand#22|ECONOMY PLATED NICKEL|16|LG PKG|933.03|ly eve|
+34|bisque pink dodger cream goldenrod|Manufacturer#1|Brand#13|LARGE BRUSHED STEEL|8|JUMBO BOX|934.03|riously ironic|
+35|green blue coral antique sandy|Manufacturer#4|Brand#43|MEDIUM ANODIZED BRASS|14|JUMBO PACK|935.03|e carefully furi|
+36|chiffon smoke floral misty navy|Manufacturer#2|Brand#25|SMALL BURNISHED COPPER|3|JUMBO CAN|936.03|olites o|
+37|turquoise ivory orange sandy maroon|Manufacturer#4|Brand#45|LARGE POLISHED TIN|48|JUMBO BOX|937.03|silent |
+38|seashell floral bisque midnight black|Manufacturer#4|Brand#43|ECONOMY ANODIZED BRASS|11|SM JAR|938.03|structions inte|
+39|rose dodger lace peru floral|Manufacturer#5|Brand#53|SMALL POLISHED TIN|43|JUMBO JAR|939.03|se slowly above the fl|
+40|rosy midnight floral misty green|Manufacturer#2|Brand#25|ECONOMY BURNISHED COPPER|27|SM CASE|940.04|! blithely specia|
+41|burlywood ghost antique metallic pale|Manufacturer#2|Brand#23|ECONOMY ANODIZED TIN|7|WRAP JAR|941.04|uriously. furiously cl|
+42|midnight azure bisque turquoise lawn|Manufacturer#5|Brand#52|MEDIUM BURNISHED TIN|45|LG BOX|942.04|the slow|
+43|medium khaki chocolate rosy blush|Manufacturer#4|Brand#44|PROMO POLISHED STEEL|5|WRAP CASE|943.04|e slyly along the ir|
+44|medium maroon wheat lavender blush|Manufacturer#4|Brand#45|MEDIUM PLATED TIN|48|SM PACK|944.04|pinto beans. carefully|
+45|lemon medium frosted indian linen|Manufacturer#4|Brand#43|SMALL BRUSHED NICKEL|9|WRAP BAG|945.04|nts bo|
+46|antique peru lemon sandy steel|Manufacturer#1|Brand#11|STANDARD POLISHED TIN|45|WRAP CASE|946.04|the blithely unusual |
+47|sky firebrick red linen dim|Manufacturer#4|Brand#45|LARGE BURNISHED BRASS|14|JUMBO PACK|947.04| even plate|
+48|slate thistle indian lace drab|Manufacturer#5|Brand#53|STANDARD BRUSHED STEEL|27|JUMBO CASE|948.04|ng to the depo|
+49|blue tan cornsilk burlywood beige|Manufacturer#2|Brand#24|SMALL BURNISHED TIN|31|MED DRUM|949.04|ar pack|
+50|yellow cornflower royal blush almond|Manufacturer#3|Brand#33|LARGE ANODIZED TIN|25|WRAP PKG|950.05|kages m|
+51|lime forest puff deep firebrick|Manufacturer#4|Brand#45|ECONOMY BURNISHED NICKEL|34|JUMBO PACK|951.05|n foxes|
+52|lemon lace khaki powder blanched|Manufacturer#3|Brand#35|STANDARD BURNISHED TIN|25|WRAP CASE|952.05| final deposits. fu|
+53|grey black dodger sandy puff|Manufacturer#2|Brand#23|ECONOMY BURNISHED NICKEL|32|MED BAG|953.05|mptot|
+54|peach smoke yellow pale cornflower|Manufacturer#2|Brand#21|LARGE BURNISHED COPPER|19|WRAP CASE|954.05|e blithely|
+55|antique cream pale tomato rose|Manufacturer#2|Brand#23|ECONOMY BRUSHED COPPER|9|MED BAG|955.05|ly final pac|
+56|midnight goldenrod drab ivory blanched|Manufacturer#1|Brand#12|MEDIUM PLATED STEEL|20|WRAP DRUM|956.05|ts. blithel|
+57|purple blue tan goldenrod medium|Manufacturer#3|Brand#32|MEDIUM BURNISHED BRASS|49|MED PKG|957.05|lly abov|
+58|linen beige coral dim salmon|Manufacturer#5|Brand#53|STANDARD POLISHED TIN|44|LG PACK|958.05| fluffily blithely reg|
+59|misty blush metallic spring blanched|Manufacturer#5|Brand#53|MEDIUM POLISHED TIN|2|LG BAG|959.05|regular exc|
+60|sky burnished salmon navajo hot|Manufacturer#1|Brand#11|LARGE POLISHED COPPER|27|JUMBO CASE|960.06| integ|
+61|drab burnished lime navy slate|Manufacturer#5|Brand#54|SMALL BURNISHED NICKEL|18|WRAP DRUM|961.06|es. blithely en|
+62|tan ivory aquamarine indian burlywood|Manufacturer#3|Brand#35|STANDARD BRUSHED BRASS|39|JUMBO BOX|962.06|ckly across the carefu|
+63|frosted puff hot rose coral|Manufacturer#3|Brand#32|STANDARD BURNISHED NICKEL|10|JUMBO CAN|963.06| quickly |
+64|aquamarine puff papaya hot floral|Manufacturer#2|Brand#21|MEDIUM ANODIZED BRASS|1|JUMBO CAN|964.06|efully regular pi|
+65|slate lavender tomato black floral|Manufacturer#5|Brand#53|MEDIUM BRUSHED COPPER|3|MED CAN|965.06|posits after the quic|
+66|cornflower orchid chocolate lawn light|Manufacturer#3|Brand#35|PROMO ANODIZED NICKEL|46|SM CASE|966.06|haggle blithely iro|
+67|slate cornsilk red orange plum|Manufacturer#2|Brand#21|SMALL BRUSHED TIN|31|WRAP DRUM|967.06| regular, p|
+68|bisque frosted pale puff sandy|Manufacturer#1|Brand#11|PROMO ANODIZED STEEL|10|WRAP BOX|968.06|eposits shall h|
+69|peru green linen blush olive|Manufacturer#5|Brand#52|MEDIUM POLISHED BRASS|2|SM BOX|969.06|ely final depo|
+70|violet midnight dark cornsilk wheat|Manufacturer#1|Brand#11|STANDARD BRUSHED STEEL|42|LG PACK|970.07|inal gifts. sl|
+71|sky green spring lavender white|Manufacturer#3|Brand#33|STANDARD PLATED BRASS|26|WRAP DRUM|971.07| packages alongside|
+72|orange spring burlywood chocolate black|Manufacturer#2|Brand#23|STANDARD ANODIZED TIN|25|JUMBO PACK|972.07|efully final the|
+73|cream misty tomato peach burlywood|Manufacturer#2|Brand#21|SMALL BRUSHED COPPER|35|WRAP DRUM|973.07|ts haggl|
+74|dim linen frosted thistle pale|Manufacturer#5|Brand#55|ECONOMY ANODIZED BRASS|25|JUMBO CASE|974.07|ent foxes|
+75|aquamarine lace wheat salmon blue|Manufacturer#3|Brand#35|SMALL BURNISHED NICKEL|39|SM JAR|975.07|s sleep furiou|
+76|bisque light yellow puff salmon|Manufacturer#3|Brand#34|MEDIUM BRUSHED COPPER|9|SM PKG|976.07|n accounts sleep qu|
+77|slate bisque burnished honeydew beige|Manufacturer#5|Brand#52|STANDARD BRUSHED COPPER|13|MED PKG|977.07|uests.|
+78|blush green slate bisque powder|Manufacturer#1|Brand#14|ECONOMY POLISHED STEEL|24|LG JAR|978.07|icing deposits wake|
+79|puff pink floral steel forest|Manufacturer#4|Brand#45|PROMO ANODIZED BRASS|22|JUMBO BAG|979.07| foxes are slyly regu|
+80|tomato saddle brown bisque light|Manufacturer#4|Brand#44|PROMO PLATED BRASS|28|MED CAN|980.08|unusual dependencies i|
+81|misty salmon cornflower dark frosted|Manufacturer#5|Brand#53|ECONOMY BRUSHED TIN|21|MED BAG|981.08|ove the furiou|
+82|khaki tomato purple indian lemon|Manufacturer#1|Brand#15|ECONOMY POLISHED TIN|12|WRAP BOX|982.08|ial requests haggle |
+83|chocolate green deep lace aquamarine|Manufacturer#1|Brand#12|PROMO BURNISHED NICKEL|47|SM CAN|983.08|ly regul|
+84|salmon peru frosted red chiffon|Manufacturer#4|Brand#45|SMALL ANODIZED NICKEL|26|JUMBO PACK|984.08|ideas nag|
+85|aquamarine cornsilk floral misty light|Manufacturer#5|Brand#55|PROMO ANODIZED NICKEL|16|LG BAG|985.08| silent|
+86|green blanched khaki steel deep|Manufacturer#4|Brand#44|STANDARD PLATED TIN|37|LG CASE|986.08| daring sheaves |
+87|pale khaki sandy antique black|Manufacturer#4|Brand#41|LARGE PLATED STEEL|41|WRAP PACK|987.08|yly final|
+88|blue olive azure burlywood lavender|Manufacturer#4|Brand#44|PROMO PLATED COPPER|16|SM CASE|988.08|e regular packages. |
+89|ghost khaki lawn pale dim|Manufacturer#5|Brand#53|STANDARD BURNISHED STEEL|7|MED JAR|989.08|y final pinto |
+90|ghost goldenrod violet blue purple|Manufacturer#5|Brand#51|ECONOMY POLISHED STEEL|49|JUMBO CAN|990.09|caref|
+91|dark beige metallic ivory burlywood|Manufacturer#2|Brand#21|STANDARD BRUSHED TIN|32|JUMBO PKG|991.09|counts dete|
+92|chiffon dodger turquoise thistle red|Manufacturer#2|Brand#22|STANDARD ANODIZED TIN|35|JUMBO PKG|992.09|he ironic accounts. sp|
+93|blanched yellow cornflower khaki mint|Manufacturer#2|Brand#24|LARGE ANODIZED TIN|2|WRAP DRUM|993.09| platel|
+94|azure pink orange goldenrod grey|Manufacturer#3|Brand#35|STANDARD POLISHED BRASS|32|SM BOX|994.09|s accounts cajo|
+95|coral chocolate wheat smoke magenta|Manufacturer#3|Brand#33|LARGE BRUSHED TIN|36|WRAP DRUM|995.09| final pinto beans |
+96|steel light drab chartreuse honeydew|Manufacturer#5|Brand#53|STANDARD BRUSHED STEEL|32|SM CASE|996.09|ng to the bli|
+97|metallic dodger aquamarine azure drab|Manufacturer#3|Brand#33|MEDIUM POLISHED BRASS|49|WRAP CAN|997.09|ss excuses sleep am|
+98|frosted goldenrod chartreuse dark honeydew|Manufacturer#5|Brand#54|STANDARD ANODIZED BRASS|22|MED JAR|998.09|e the q|
+99|lemon peru chiffon sienna deep|Manufacturer#2|Brand#21|SMALL BURNISHED STEEL|11|JUMBO PKG|999.09|press|
+100|light orange khaki chiffon purple|Manufacturer#3|Brand#33|ECONOMY ANODIZED TIN|4|LG BAG|1000.10|of the steal|
+101|hot yellow turquoise violet floral|Manufacturer#3|Brand#32|LARGE ANODIZED STEEL|26|JUMBO JAR|1001.10|ly even,|
+102|firebrick orchid blanched sienna grey|Manufacturer#3|Brand#31|MEDIUM BURNISHED BRASS|17|SM DRUM|1002.10|ular packa|
+103|navy cream lime orange metallic|Manufacturer#2|Brand#25|MEDIUM PLATED BRASS|45|WRAP DRUM|1003.10|e blithely blith|
+104|dim cream chocolate cornflower orange|Manufacturer#1|Brand#13|MEDIUM ANODIZED STEEL|36|JUMBO BAG|1004.10|ites sleep quickly|
+105|rose puff ivory blanched burnished|Manufacturer#1|Brand#15|SMALL POLISHED COPPER|27|LG DRUM|1005.10|odolites was |
+106|cornsilk beige deep lavender antique|Manufacturer#3|Brand#31|MEDIUM PLATED BRASS|28|WRAP DRUM|1006.10|unts maintain |
+107|cyan coral red slate orange|Manufacturer#5|Brand#53|SMALL BURNISHED TIN|12|MED BOX|1007.10|slyly special depos|
+108|honeydew papaya gainsboro tomato puff|Manufacturer#1|Brand#12|PROMO PLATED NICKEL|41|MED PKG|1008.10|after the carefully |
+109|lemon indian blanched chiffon coral|Manufacturer#3|Brand#33|ECONOMY POLISHED TIN|11|LG PACK|1009.10|instruction|
+110|cornflower azure beige floral black|Manufacturer#3|Brand#33|STANDARD BURNISHED COPPER|46|LG DRUM|1010.11|t quickly a|
+111|sandy almond chocolate moccasin misty|Manufacturer#5|Brand#54|LARGE BRUSHED COPPER|28|JUMBO JAR|1011.11|kly bold epitaphs |
+112|hot burnished tomato chocolate peru|Manufacturer#4|Brand#43|PROMO BRUSHED STEEL|42|JUMBO CAN|1012.11|the express, |
+113|almond coral royal honeydew lawn|Manufacturer#3|Brand#31|PROMO POLISHED TIN|23|LG CAN|1013.11|finally even |
+114|pale bisque antique white royal|Manufacturer#5|Brand#51|MEDIUM POLISHED NICKEL|41|MED PACK|1014.11|ully final foxes. pint|
+115|spring hot sienna azure dark|Manufacturer#4|Brand#45|STANDARD POLISHED STEEL|24|MED CAN|1015.11|counts nag! caref|
+116|thistle cornsilk pale firebrick black|Manufacturer#5|Brand#53|PROMO POLISHED NICKEL|33|SM PACK|1016.11|usly final courts |
+117|tomato grey chiffon burlywood ghost|Manufacturer#1|Brand#14|SMALL BRUSHED TIN|25|LG BAG|1017.11|ages acc|
+118|bisque plum royal burlywood brown|Manufacturer#2|Brand#25|PROMO ANODIZED TIN|31|MED PACK|1018.11|ly ironic pinto|
+119|olive black navajo burlywood ghost|Manufacturer#4|Brand#43|LARGE POLISHED STEEL|30|WRAP CASE|1019.11|out the quickly r|
+120|navy chocolate frosted burnished moccasin|Manufacturer#1|Brand#14|SMALL ANODIZED NICKEL|45|WRAP JAR|1020.12|lly a|
+121|smoke chiffon gainsboro magenta firebrick|Manufacturer#1|Brand#14|ECONOMY BRUSHED COPPER|13|SM PKG|1021.12|deposi|
+122|ghost royal chocolate peach ivory|Manufacturer#2|Brand#21|MEDIUM ANODIZED TIN|8|LG DRUM|1022.12|sts c|
+123|cream almond papaya salmon blanched|Manufacturer#1|Brand#12|SMALL BURNISHED TIN|31|JUMBO PKG|1023.12|ray regula|
+124|wheat azure khaki tan lime|Manufacturer#3|Brand#32|PROMO ANODIZED STEEL|1|LG BOX|1024.12|g the expr|
+125|metallic indian magenta navy medium|Manufacturer#1|Brand#12|STANDARD BRUSHED BRASS|17|WRAP BAG|1025.12|kages against|
+126|burnished navy indian medium deep|Manufacturer#4|Brand#45|MEDIUM BRUSHED NICKEL|4|LG BAG|1026.12|es sleep al|
+127|moccasin green orchid blue blanched|Manufacturer#5|Brand#52|SMALL BURNISHED NICKEL|14|LG JAR|1027.12|lithely expr|
+128|linen brown khaki snow gainsboro|Manufacturer#2|Brand#22|PROMO PLATED TIN|5|SM BAG|1028.12|e of the furiously ex|
+129|turquoise spring sandy tan ivory|Manufacturer#1|Brand#15|LARGE POLISHED TIN|20|SM JAR|1029.12| careful|
+130|gainsboro goldenrod chiffon orchid rose|Manufacturer#2|Brand#23|SMALL PLATED NICKEL|26|LG BOX|1030.13|ake slyly|
+131|tomato moccasin sandy blue frosted|Manufacturer#5|Brand#52|STANDARD ANODIZED BRASS|43|MED DRUM|1031.13|nts wake dar|
+132|seashell papaya thistle light powder|Manufacturer#4|Brand#45|STANDARD BURNISHED BRASS|2|WRAP DRUM|1032.13|ckly expre|
+133|navajo black dim peru deep|Manufacturer#1|Brand#13|SMALL BRUSHED NICKEL|19|LG PKG|1033.13| final pinto beans|
+134|turquoise goldenrod white moccasin spring|Manufacturer#4|Brand#42|SMALL POLISHED STEEL|35|SM PKG|1034.13|es. bold pa|
+135|metallic lime cyan forest burlywood|Manufacturer#2|Brand#21|MEDIUM BURNISHED STEEL|24|JUMBO CASE|1035.13|l frets |
+136|cornsilk blush powder tan rose|Manufacturer#2|Brand#22|SMALL PLATED STEEL|2|WRAP BAG|1036.13|kages print carefully|
+137|saddle dodger purple salmon rosy|Manufacturer#3|Brand#31|ECONOMY PLATED STEEL|25|MED PACK|1037.13|the t|
+138|dark goldenrod ghost hot grey|Manufacturer#1|Brand#13|ECONOMY BURNISHED COPPER|42|JUMBO DRUM|1038.13|ts solve acro|
+139|floral chocolate purple deep coral|Manufacturer#3|Brand#32|MEDIUM BRUSHED STEEL|7|SM BOX|1039.13|ter t|
+140|frosted chiffon thistle sky grey|Manufacturer#5|Brand#53|STANDARD PLATED STEEL|45|SM BOX|1040.14|oss the carefu|
+141|peru sienna salmon snow gainsboro|Manufacturer#3|Brand#35|STANDARD ANODIZED STEEL|23|SM PKG|1041.14|ans nag furiously pen|
+142|chartreuse sky deep slate thistle|Manufacturer#5|Brand#55|STANDARD ANODIZED BRASS|36|MED JAR|1042.14|he accounts. pac|
+143|moccasin dim almond black misty|Manufacturer#3|Brand#34|ECONOMY PLATED TIN|44|MED BAG|1043.14|nts across the|
+144|wheat brown orange almond aquamarine|Manufacturer#1|Brand#14|SMALL ANODIZED TIN|26|SM BOX|1044.14|owly |
+145|indian rosy chartreuse orchid grey|Manufacturer#5|Brand#53|PROMO BRUSHED COPPER|24|SM BAG|1045.14|es wake furiously blit|
+146|azure medium midnight burlywood blue|Manufacturer#3|Brand#34|STANDARD BRUSHED COPPER|11|WRAP PACK|1046.14|unts cajole|
+147|honeydew brown deep light chartreuse|Manufacturer#1|Brand#11|MEDIUM PLATED COPPER|29|JUMBO PKG|1047.14|wake never bold |
+148|yellow white tan khaki saddle|Manufacturer#3|Brand#31|STANDARD PLATED STEEL|20|SM BOX|1048.14|platelets wake fu|
+149|tan cyan forest ghost plum|Manufacturer#2|Brand#24|MEDIUM BURNISHED NICKEL|6|MED PKG|1049.14|leep requests. dog|
+150|antique red moccasin green burlywood|Manufacturer#3|Brand#35|LARGE BRUSHED TIN|21|SM BAG|1050.15|ironic foxes|
+151|chartreuse lime slate white thistle|Manufacturer#3|Brand#34|LARGE PLATED BRASS|45|MED CAN|1051.15|ccounts nag i|
+152|light indian linen dodger blush|Manufacturer#5|Brand#53|MEDIUM POLISHED STEEL|48|MED CASE|1052.15|thely regular t|
+153|powder firebrick chiffon purple khaki|Manufacturer#1|Brand#11|STANDARD PLATED TIN|20|MED BAG|1053.15|thlessly. silen|
+154|peru powder pink orchid lace|Manufacturer#1|Brand#11|ECONOMY ANODIZED TIN|1|JUMBO BAG|1054.15|posits |
+155|beige saddle cornsilk mint sky|Manufacturer#2|Brand#21|SMALL BRUSHED NICKEL|28|WRAP CASE|1055.15|lly ironic, r|
+156|steel gainsboro plum black cornsilk|Manufacturer#4|Brand#43|SMALL POLISHED NICKEL|2|LG PKG|1056.15| pinto beans. eve|
+157|navajo lime cream ghost drab|Manufacturer#1|Brand#11|ECONOMY ANODIZED STEEL|26|JUMBO PACK|1057.15|ial courts. ru|
+158|magenta light rosy almond goldenrod|Manufacturer#4|Brand#45|MEDIUM BURNISHED COPPER|47|LG JAR|1058.15| ideas detect slyl|
+159|linen drab antique steel aquamarine|Manufacturer#4|Brand#43|SMALL ANODIZED BRASS|46|SM BAG|1059.15| ironic requests-- pe|
+160|lace coral cream saddle maroon|Manufacturer#5|Brand#55|STANDARD POLISHED COPPER|47|JUMBO CAN|1060.16|nts are carefully|
+161|hot dodger rosy firebrick aquamarine|Manufacturer#2|Brand#22|STANDARD PLATED TIN|17|SM PACK|1061.16|r the bl|
+162|aquamarine coral black ivory misty|Manufacturer#3|Brand#33|MEDIUM ANODIZED COPPER|35|JUMBO PACK|1062.16|e slyly around th|
+163|blush yellow frosted lace drab|Manufacturer#2|Brand#21|ECONOMY PLATED TIN|34|WRAP DRUM|1063.16|nly s|
+164|brown cream linen frosted blue|Manufacturer#2|Brand#23|LARGE PLATED BRASS|35|JUMBO BAG|1064.16|mong th|
+165|chiffon dim coral sienna seashell|Manufacturer#1|Brand#15|STANDARD PLATED STEEL|24|SM CAN|1065.16| carefully fin|
+166|linen royal chiffon floral misty|Manufacturer#5|Brand#52|LARGE POLISHED COPPER|4|MED BAG|1066.16|ss the|
+167|almond lavender lawn pink ghost|Manufacturer#3|Brand#32|LARGE ANODIZED STEEL|46|WRAP BOX|1067.16|ic ac|
+168|lace frosted brown slate lawn|Manufacturer#1|Brand#13|SMALL BRUSHED COPPER|20|JUMBO DRUM|1068.16|ss package|
+169|tan lime sky almond midnight|Manufacturer#5|Brand#55|STANDARD POLISHED BRASS|10|JUMBO CASE|1069.16|lets alongside of|
+170|peru green hot medium thistle|Manufacturer#3|Brand#33|LARGE POLISHED COPPER|28|LG DRUM|1070.17|yly s|
+171|beige seashell almond linen blush|Manufacturer#1|Brand#11|STANDARD BURNISHED COPPER|40|LG JAR|1071.17| the r|
+172|medium goldenrod tan light brown|Manufacturer#5|Brand#53|PROMO PLATED NICKEL|28|MED CASE|1072.17|quick as|
+173|pale papaya tomato navajo coral|Manufacturer#1|Brand#12|ECONOMY BURNISHED TIN|17|LG CASE|1073.17|sly bold excuses haggl|
+174|plum pink medium saddle orchid|Manufacturer#1|Brand#15|ECONOMY BRUSHED COPPER|25|LG CASE|1074.17| accounts nag ab|
+175|azure blanched chartreuse moccasin ghost|Manufacturer#1|Brand#11|PROMO ANODIZED TIN|45|JUMBO JAR|1075.17|ole against the|
+176|lawn forest indian orchid peru|Manufacturer#2|Brand#24|SMALL ANODIZED STEEL|40|MED CAN|1076.17|blithely. ironic|
+177|indian deep sandy floral yellow|Manufacturer#2|Brand#21|MEDIUM BRUSHED STEEL|42|LG BAG|1077.17|ermanently eve|
+178|blue black cornflower white powder|Manufacturer#1|Brand#13|STANDARD POLISHED TIN|10|LG JAR|1078.17|regular instructions.|
+179|rose lime blush bisque ivory|Manufacturer#4|Brand#43|ECONOMY BRUSHED STEEL|20|LG JAR|1079.17|ely regul|
+180|purple blanched khaki blush aquamarine|Manufacturer#3|Brand#33|STANDARD BURNISHED NICKEL|7|WRAP BAG|1080.18|oss the |
+181|almond burlywood smoke peru dark|Manufacturer#2|Brand#24|MEDIUM PLATED STEEL|19|WRAP CAN|1081.18|al deposits |
+182|beige medium brown chartreuse purple|Manufacturer#3|Brand#31|MEDIUM ANODIZED COPPER|11|JUMBO CAN|1082.18|bits are |
+183|ivory white wheat pale orange|Manufacturer#5|Brand#52|PROMO POLISHED STEEL|35|LG PKG|1083.18|ly regular excus|
+184|burlywood cream cornsilk chocolate purple|Manufacturer#5|Brand#53|SMALL POLISHED TIN|42|LG BOX|1084.18|ding courts. idly iro|
+185|firebrick rose indian spring magenta|Manufacturer#4|Brand#44|ECONOMY POLISHED TIN|4|WRAP BAG|1085.18|even foxe|
+186|papaya cream chartreuse spring blush|Manufacturer#2|Brand#23|ECONOMY BRUSHED TIN|15|JUMBO PKG|1086.18|ly reg|
+187|white brown cyan lavender orange|Manufacturer#4|Brand#45|PROMO ANODIZED BRASS|45|MED CAN|1087.18|leep slyly s|
+188|gainsboro steel rosy dim drab|Manufacturer#5|Brand#54|ECONOMY ANODIZED BRASS|9|MED CAN|1088.18| above the silent p|
+189|dodger misty yellow brown grey|Manufacturer#2|Brand#22|MEDIUM BRUSHED BRASS|13|WRAP DRUM|1089.18|en requests. sauternes|
+190|chartreuse khaki ivory chiffon bisque|Manufacturer#5|Brand#53|LARGE BURNISHED NICKEL|23|WRAP BAG|1090.19| furiously even d|
+191|navajo metallic puff firebrick cornflower|Manufacturer#3|Brand#31|MEDIUM POLISHED BRASS|36|WRAP BOX|1091.19| asymptote|
+192|light seashell firebrick cornflower green|Manufacturer#3|Brand#34|STANDARD BRUSHED COPPER|17|MED BAG|1092.19|uickly regular, expr|
+193|turquoise sienna rose rosy chocolate|Manufacturer#4|Brand#45|ECONOMY BURNISHED BRASS|31|SM PKG|1093.19|final ideas wake furi|
+194|powder bisque cornflower navajo pink|Manufacturer#5|Brand#51|ECONOMY POLISHED STEEL|7|SM CAN|1094.19|y special accoun|
+195|lace floral blush linen cornflower|Manufacturer#4|Brand#41|STANDARD BRUSHED NICKEL|40|MED CASE|1095.19|oxes sleep care|
+196|pale blush lime grey blue|Manufacturer#3|Brand#33|SMALL BURNISHED NICKEL|3|JUMBO JAR|1096.19|uickly special |
+197|blue turquoise cornsilk rosy chartreuse|Manufacturer#5|Brand#52|SMALL ANODIZED COPPER|18|SM JAR|1097.19|lithely after the eve|
+198|orange khaki papaya cyan navajo|Manufacturer#4|Brand#41|PROMO BRUSHED NICKEL|43|SM PACK|1098.19|ackages? carefully re|
+199|royal slate lace olive gainsboro|Manufacturer#3|Brand#31|ECONOMY PLATED STEEL|23|JUMBO DRUM|1099.19|ickly regul|
+200|burnished mint navajo beige dark|Manufacturer#5|Brand#54|MEDIUM POLISHED BRASS|22|LG PKG|1100.20|furiously even depo|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/partsupp.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/partsupp.tbl
new file mode 100644
index 0000000..d8e5856
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/partsupp.tbl
@@ -0,0 +1,800 @@
+1|2|3325|771.64|, even theodolites. regular, final theodolites eat after the carefully pending foxes. furiously regular deposits sleep slyly. carefully bold realms above the ironic dependencies haggle careful|
+1|4|8076|993.49|ven ideas. quickly even packages print. pending multipliers must have to are fluff|
+1|6|3956|337.09|after the fluffily ironic deposits? blithely special dependencies integrate furiously even excuses. blithely silent theodolites could have to haggle pending, express requests; fu|
+1|8|4069|357.84|al, regular dependencies serve carefully after the quickly final pinto beans. furiously even deposits sleep quickly final, silent pinto beans. fluffily reg|
+2|3|8895|378.49|nic accounts. final accounts sleep furiously about the ironic, bold packages. regular, regular accounts|
+2|5|4969|915.27|ptotes. quickly pending dependencies integrate furiously. fluffily ironic ideas impress blithely above the express accounts. furiously even epitaphs need to wak|
+2|7|8539|438.37|blithely bold ideas. furiously stealthy packages sleep fluffily. slyly special deposits snooze furiously carefully regular accounts. regular deposits according to the accounts nag carefully slyl|
+2|9|3025|306.39|olites. deposits wake carefully. even, express requests cajole. carefully regular ex|
+3|4|4651|920.92|ilent foxes affix furiously quickly unusual requests. even packages across the carefully even theodolites nag above the sp|
+3|6|4093|498.13|ending dependencies haggle fluffily. regular deposits boost quickly carefully regular requests. deposits affix furiously around the pinto beans. ironic, unusual platelets across the p|
+3|8|3917|645.40|of the blithely regular theodolites. final theodolites haggle blithely carefully unusual ideas. blithely even f|
+3|10|9942|191.92| unusual, ironic foxes according to the ideas detect furiously alongside of the even, express requests. blithely regular the|
+4|5|1339|113.97| carefully unusual ideas. packages use slyly. blithely final pinto beans cajole along the furiously express requests. regular orbits haggle carefully. care|
+4|7|6377|591.18|ly final courts haggle carefully regular accounts. carefully regular accounts could integrate slyly. slyly express packages about the accounts wake slyly|
+4|9|2694|51.37|g, regular deposits: quick instructions run across the carefully ironic theodolites-- final dependencies haggle into the dependencies. f|
+4|1|2480|444.37|requests sleep quickly regular accounts. theodolites detect. carefully final depths w|
+5|6|3735|255.88|arefully even requests. ironic requests cajole carefully even dolphin|
+5|8|9653|50.52|y stealthy deposits. furiously final pinto beans wake furiou|
+5|10|1329|219.83|iously regular deposits wake deposits. pending pinto beans promise ironic dependencies. even, regular pinto beans integrate|
+5|2|6925|537.98|sits. quickly fluffy packages wake quickly beyond the blithely regular requests. pending requests cajole among the final pinto beans. carefully busy theodolites affix quickly stealthily |
+6|7|8851|130.72|usly final packages. slyly ironic accounts poach across the even, sly requests. carefully pending request|
+6|9|1627|424.25| quick packages. ironic deposits print. furiously silent platelets across the carefully final requests are slyly along the furiously even instructi|
+6|1|3336|642.13|final instructions. courts wake packages. blithely unusual realms along the multipliers nag |
+6|3|6451|175.32| accounts alongside of the slyly even accounts wake carefully final instructions-- ruthless platelets wake carefully ideas. even deposits are quickly final,|
+7|8|7454|763.98|y express tithes haggle furiously even foxes. furiously ironic deposits sleep toward the furiously unusual|
+7|10|2770|149.66|hould have to nag after the blithely final asymptotes. fluffily spe|
+7|2|3377|68.77|usly against the daring asymptotes. slyly regular platelets sleep quickly blithely regular deposits. boldly regular deposits wake blithely ironic accounts|
+7|4|9460|299.58|. furiously final ideas hinder slyly among the ironic, final packages. blithely ironic dependencies cajole pending requests: blithely even packa|
+8|9|6834|249.63|lly ironic accounts solve express, unusual theodolites. special packages use quickly. quickly fin|
+8|1|396|957.34|r accounts. furiously pending dolphins use even, regular platelets. final|
+8|3|9845|220.62|s against the fluffily special packages snooze slyly slyly regular p|
+8|5|8126|916.91|final accounts around the blithely special asymptotes wake carefully beyond the bold dugouts. regular ideas haggle furiously after|
+9|10|7054|84.20|ts boost. evenly regular packages haggle after the quickly careful accounts. |
+9|2|7542|811.84|ate after the final pinto beans. express requests cajole express packages. carefully bold ideas haggle furiously. blithely express accounts eat carefully among the evenly busy accounts. carefully un|
+9|4|9583|381.31|d foxes. final, even braids sleep slyly slyly regular ideas. unusual ideas above|
+9|6|3063|291.84| the blithely ironic instructions. blithely express theodolites nag furiously. carefully bold requests shall have to use slyly pending requests. carefully regular instr|
+10|1|2952|996.12| bold foxes wake quickly even, final asymptotes. blithely even depe|
+10|3|3335|673.27|s theodolites haggle according to the fluffily unusual instructions. silent realms nag carefully ironic theodolites. furiously unusual instructions would detect fu|
+10|5|5691|164.00|r, silent instructions sleep slyly regular pinto beans. furiously unusual gifts use. silently ironic theodolites cajole final deposits! express dugouts are furiously. packages sleep |
+10|7|841|374.02|refully above the ironic packages. quickly regular packages haggle foxes. blithely ironic deposits a|
+11|2|4540|709.87|thely across the blithely unusual requests. slyly regular instructions wake slyly ironic theodolites. requests haggle blithely above the blithely brave p|
+11|5|4729|894.90|ters wake. sometimes bold packages cajole sometimes blithely final instructions. carefully ironic foxes after the furiously unusual foxes cajole carefully acr|
+11|8|3708|818.74|inal accounts nag quickly slyly special frays; bold, final theodolites play slyly after the furiously pending packages. f|
+11|1|3213|471.98|nusual, regular requests use carefully. slyly final packages haggle quickly. slyly express packages impress blithely across the blithely regular ideas. regular depe|
+12|3|3610|659.73|jole bold theodolites. final packages haggle! carefully regular deposits play furiously among the special ideas. quickly ironic packages detect quickly carefully final|
+12|6|7606|332.81|luffily regular courts engage carefully special realms. regular accounts across the blithely special pinto beans use carefully at the silent request|
+12|9|824|337.06|es are unusual deposits. fluffily even deposits across the blithely final theodolites doubt across the unusual accounts. regular, |
+12|2|5454|901.70|s across the carefully regular courts haggle fluffily among the even theodolites. blithely final platelets x-ray even ideas. fluffily express pinto beans sleep slyly. carefully even a|
+13|4|612|169.44|s. furiously even asymptotes use slyly blithely express foxes. pending courts integrate blithely among the ironic requests! blithely pending deposits integrate slyly furiously final packa|
+13|7|7268|862.70|s sleep slyly packages. final theodolites to the express packages haggle quic|
+13|10|864|38.64|s after the slyly pending instructions haggle even, express requests. permanently regular pinto beans are. slyly pending req|
+13|3|9736|327.18|tect after the express instructions. furiously silent ideas sleep blithely special ideas. attainments sleep furiously. carefully bold requests ab|
+14|5|5278|650.07|e quickly among the furiously ironic accounts. special, final sheaves against the|
+14|8|5334|889.50|ss dependencies are furiously silent excuses. blithely ironic pinto beans affix quickly according to the slyly ironic asymptotes. final packag|
+14|1|3676|893.39|sits are according to the fluffily silent asymptotes. final ideas are slyly above the regular instructions. furiousl|
+14|4|4947|310.13| final deposits boost slyly regular packages; carefully pending theodolites |
+15|6|7047|835.70|blithely quick requests sleep carefully fluffily regular pinto beans. ironic pinto beans around the slyly regular foxe|
+15|9|3336|784.55|slyly. fluffily bold accounts cajole furiously. furiously regular dependencies wak|
+15|2|3316|265.89|e express instructions. ironic requests haggle fluffily along the carefully even packages. furiously final acco|
+15|5|5255|458.67|refully bold instructions among the silent grouches must boost against the express deposits:|
+16|7|5282|709.16|lithely ironic theodolites should have to are furiously-- |
+16|10|9412|887.53|ly special accounts wake. fluffily bold ideas believe blith|
+16|3|854|781.91| unusual excuses. requests after the carefully regular pinto |
+16|6|1491|918.51|unts cajole furiously across the fluffily pending instructions. slyly special accounts could have to boost b|
+17|8|8555|995.35|are furiously final accounts. carefully unusual accounts snooze across the requests. carefully special dolphins|
+17|1|7737|648.75|e blithely express accounts. foxes kindle slyly unusual dinos. quickly special f|
+17|4|3123|555.04|ly bold accounts. regular packages use silently. quickly unusual sentiments around the quickly ironic theodolites haggle furiously pending requests. care|
+17|7|3203|64.40|bold packages nag fluffily after the regular accounts. furiously ironic asymptotes sleep quickly enticing pinto beans. carefully pending accounts use about the |
+18|9|1125|664.17|. ironic, regular accounts across the furiously express |
+18|2|8132|52.44| final packages wake quickly across the blithely ironic instructions. regular pains integrate slyly across the deposits. carefully regular pinto beans among the close|
+18|5|3133|568.61|riously bold accounts. packages boost daringly. blithely regular requests cajole. regular foxes wake carefully final accounts. blithely unusual excuses det|
+18|8|6475|386.29|. furiously regular accounts cajole slyly across the pending|
+19|10|1416|144.80|o beans. even packages nag boldly according to the bold, special deposits. ironic packages after the pinto beans nag above the quickly ironic requests. bl|
+19|3|5467|405.70|nstructions use furiously. fluffily regular excuses wake. slyly special grouches are carefully regular Tiresias. regular requests use about the quickly furio|
+19|6|8800|635.66|sual requests sleep carefully. deposits cajole carefully over the regular, regular requests. quickly unusual asymptotes use some|
+19|9|1340|346.92| requests. final, pending realms use carefully; slyly dogged foxes impress fluffily above the blithely regular deposits. ironic, regular courts wake carefully. bold requests impress|
+20|1|2927|675.54|s, ironic deposits haggle across the quickly bold asymptotes. express, ironic pinto beans wake carefully enticingly special foxes. requests are at the c|
+20|4|2723|305.84|nal, bold frets cajole slyly regular, unusual platelets. slyly permanent deposits wake carefully carefully silent accounts. even, even requests wake quickly. furiously pending packages are|
+20|7|5905|546.66|ing deposits use furiously. ironically final pinto bea|
+20|10|4271|115.89|xcuses wake at the deposits. regular pinto beans nag slyly fluffi|
+21|2|6571|944.44|ing instructions impress bold foxes. ironic pinto beans use. thinly even asymptotes cajole ironic packages. quickly ironic pinto beans detect slyly regular deposits. ruthlessly even deposits are. sl|
+21|6|1704|139.05|posits cajole; quickly even requests sleep furiously. ironic theodolites sleep pending, express instructions. stealthily even platelets cajole carefully after the final, ironic p|
+21|10|7153|664.50|blithely enticing instructions use alongside of the carefully thin deposits. blithely bold requests are fluffily|
+21|4|367|584.86|ong the even theodolites. pending, pending accounts sleep-- courts boost quickly at the accounts. quickly fin|
+22|3|4410|786.18|even accounts. final excuses try to sleep regular, even packages. carefully express dolphins cajole; furiously special pinto bea|
+22|7|9779|635.84|l instructions cajole across the blithely special deposits. blithely pending accounts use thinly slyly final requests. instructions haggle. pinto beans sleep along the slyly pen|
+22|1|7834|359.16|sits wake fluffily carefully stealthy accounts. furiously ironic requests x-ray fluffily alongside of the pending asymptotes. slyly silent packages use along the instructions. fu|
+22|5|1434|597.21|ix across the blithely express packages. carefully regular pinto beans boost across the special, pending d|
+23|4|2739|460.12|platelets against the furiously bold Tiresias dazzle quickly into the special, bold courts. silent, regular instructions wake blithely ironic multipliers. ideas|
+23|8|5739|103.13| theodolites need to nag blithely final notornis. slyly idle packages cajole after the furiously stealthy packages. slyly regular accounts use furiously. carefully final accounts affix |
+23|2|9898|233.94|l, express packages wake permanently. quickly even deposits sleep quickly slyly silent id|
+23|6|7035|51.75|xcuses; decoys wake after the pending packages. final instructions are furi|
+24|5|5180|905.41|heodolites above the ironic requests poach fluffily carefully unusual pinto beans. even packages acc|
+24|9|2227|511.20|, silent packages boost around the instructions. special requests sleep slyly against the slyly regular deposits. final, final accounts haggle fluffily among the final requests. regular |
+24|3|7182|582.03| the final, ironic asymptotes. regular requests nag instead of the carefully unusual asymptotes. furiously pending attainments among the slyly final packages boost after th|
+24|7|5318|62.15| careful requests cajole blithely realms. special asymptotes sleep. pinto beans sleep carefully furiously ironic packages. furiously |
+25|6|9029|832.74|fully fluffily regular frets. sometimes even requests after the requests wake slyly at the quickly ruthless requests. a|
+25|10|9062|928.96|he foxes. final, final accounts sleep. boldly ironic excuses thrash quick|
+25|4|9946|694.35|ld, ironic requests. furiously special packages cajole furiously enticing instructions.|
+25|8|7340|746.59|dly final packages haggle blithely according to the pending packages. slyly regula|
+26|7|5020|683.96|es. fluffily express deposits kindle slyly accounts. slyly ironic requests wake blithely bold ideas|
+26|1|6577|892.20|riously pending pinto beans. furiously express instructions detect slyly according to the b|
+26|5|3499|382.11|imes even pinto beans among the busily ironic accounts doubt blithely quickly final courts. furiously fluffy packages despite the carefully even plate|
+26|9|9702|821.89| behind the blithely regular courts impress after the silent sheaves. bravely final ideas haggle |
+27|8|2111|444.01|the even, ironic deposits. theodolites along the ironic, final dolphins cajole slyly quickly bold asymptotes. furiously regular theodolites integrate furiously furiously bold requests. carefully|
+27|2|9080|157.03|ole express, final requests. carefully regular packages lose about the regular pinto beans. blithely re|
+27|6|3407|151.34|ironic theodolites are by the furiously bold ideas. ironic requests shall have to sublate final packages. furiously quick foxes alongside of the express, special deposits was boldly according |
+27|10|4283|348.61|ound the final foxes detect furiously across the even warhorses. quickly t|
+28|9|6643|204.86|y ironic deposits above the slyly final deposits sleep furiously above the final deposits. quickly even i|
+28|3|2452|744.57|ully regular theodolites haggle about the blithely pending packages. carefully ironic sentiments use quickly around the blithely silent requests. slyly ironic frays bo|
+28|7|302|690.30|uickly unusual requests alongside of the final courts integrate slyly |
+28|1|9988|666.53|beans haggle carefully around the slyly ironic acco|
+29|10|3506|799.27|leep fluffily according to the quietly regular requests: accounts integrate carefully bold foxes. carefully silent|
+29|4|8106|981.33|the ironic, bold asymptotes! blithely regular packages hang furiously above the dependencies. blithely permanent dependencies are furiously furiously ironic acco|
+29|8|9193|734.44|ly unusual packages. foxes cajole. theodolites nag|
+29|2|6252|186.21|thely carefully even packages. even, final packages cajole after the quickly bold accounts. fluffily quick accounts in place of the theodolites doze slyly f|
+30|1|4767|989.05|ts. slyly final pinto beans cajole ironic accounts. blithely final accounts use among the request|
+30|5|535|743.26|sual instructions wake carefully blithely even hockey playe|
+30|9|7756|568.86| special foxes across the dependencies cajole quickly against the slyly express packages! furiously unusual pinto beans boost blithely ironic Tir|
+30|3|7945|583.84| sleep. bold, regular deposits hang doggedly furiously bold requests. slyly bold excuses detect busily above the even gifts. blithely express courts are carefully. blithely final packages until th|
+31|2|9685|620.84|he blithely regular ideas. blithely unusual requests haggle fluffily. platelets|
+31|7|1951|120.99|refully regular pinto beans. ironic requests integrate furiously since the quickly ruthless platelets. quickly ironic attainments ha|
+31|2|1402|761.64|r platelets nag blithely regular deposits. ironic, bold requests |
+31|7|137|849.11|blithely ironic accounts. slyly ironic asymptotes sleep ironic, even accounts. regular accounts thrash quickly|
+32|3|2203|406.03|es? slyly enticing dugouts haggle carefully. regular packages alongside of the asymptotes are carefull|
+32|8|467|109.34|ainst the unusual braids nod fluffily packages. regular packages nod among the slyly express|
+32|3|7975|747.14|final foxes boost furiously pending packages. quickly regular depths promise blithely accoun|
+32|8|7938|856.09|s integrate according to the even dependencies. carefully regular reque|
+33|4|4028|891.46|, pending requests affix slyly. slyly ironic deposits wake accounts. express accounts sleep slowly. ironic, express accounts run carefully fluffily final dependencies. furiously unusual ideas|
+33|9|4410|929.05| packages sleep carefully. slyly final instructions boost. slyly even requests among the carefully pending platelets wake along the final accounts. quickly expre|
+33|4|1287|310.76|dolites above the slyly express deposits try to haggle blithely special gifts. blithely ironic reque|
+33|9|6006|327.19|ly. ironic dependencies haggle carefully silent instructions. furiously ironic dolphins are fluffily furiously even theo|
+34|5|9934|848.75|ven instructions besides the gifts are furiously among the slyly regular packages! instructions use carefully. even requests sleep quickl|
+34|10|4749|265.31|ckly regular theodolites eat above the bravely regular courts. ironic requests wake slyly.|
+34|5|5459|824.69|ong the slyly silent requests. express, even requests haggle slyly|
+34|10|5884|609.69|ully final tithes. slyly ironic deposits hang furiously about the regular, regular deposits|
+35|6|2500|451.58|nic packages boost carefully carefully even theodolites. blithely fina|
+35|1|8875|537.72|ully regular deposits: special accounts use. slyly final deposits wake slyly unusual, special ideas. asymptotes |
+35|6|596|669.19|slyly against the daring, pending accounts. fluffily special pinto beans integrate slyly after the carefully unusual packages. slyly bold accounts besides|
+35|1|2025|411.17|s cajole fluffily final deposits. furiously express packages after the blithely special realms boost evenly even requests. slow requests use above the unusual accoun|
+36|7|3907|630.91|al deposits detect fluffily fluffily unusual sauternes. carefully regular requests against the car|
+36|2|174|434.47|permanently express instructions. unusual accounts nag toward the accou|
+36|7|2625|569.91|ctions. pending requests are fluffily across the furiously regular notornis. unusu|
+36|2|8209|289.15|arefully regular requests cajole. special, express foxes sleep slowly. quickly unusual in|
+37|8|7171|824.96|usly into the slyly final requests. ironic accounts are furiously furiously ironic i|
+37|3|5542|126.59|ven deposits. ironic foxes cajole. slyly final deposits are furiously after the furiously even packages. slyly ironic platelets toward the slyl|
+37|8|7113|15.72|re bravely along the furiously express requests. blithely special asymptotes are quickly. fluffily regular packages alo|
+37|3|1449|745.64|y after the ironic accounts. blithely final instructions affix blithely. bold packages sleep carefully regular instructions. regular packages affix carefully. stealthy fo|
+38|9|1226|570.11| slyly even pinto beans. blithely special requests nag slyly about the ironic packages. |
+38|4|4237|662.75|lar warhorses cajole evenly against the attainments. requests cajole furiously furiously express requests. carefully regular platelets use fluffily after the silent, unusual ideas: bl|
+38|9|1135|160.70|express accounts haggle. carefully even pinto beans according to the slyly final foxes nag slyly about the enticingly express dol|
+38|4|3516|847.09|nal accounts. furiously pending hockey players solve slyly after the furiously final dependencies. deposits are blithely. carefully regular packages unwind busily at the deposits. fluffily |
+39|10|3633|463.10|kages are slyly above the slyly pending pinto beans. bold, ironic pinto beans sleep against the blithely regular requests. fluffily even pinto beans use. regular theodolites haggle against the quic|
+39|5|3682|300.43|ng requests are according to the packages. regular packages boost quickly. express Tiresias sleep silently across the even, regular ideas! blithely iro|
+39|10|5475|532.26| beans cajole carefully carefully express requests. instructions sleep furiously bold deposits. furiously regular depos|
+39|5|6259|737.86|y. special, even asymptotes cajole carefully ironic accounts. regular, final pinto beans cajole quickly. regular requests use warhorses. special, special accounts hinder boldly across the|
+40|1|7690|776.13|lets use fluffily carefully final deposits. blithely ironic instructions sublate against the furiously final ideas; slyly bold courts x-ray silent foxes. regular foxes wake blithely. slyl|
+40|6|1704|565.82|riously furiously silent asymptotes. final deposits cajole blithely ironic requests. furiously special pains into the blithely final instru|
+40|1|4521|374.71|ptotes haggle. slyly even requests nag fluffily silent packages. blith|
+40|6|6617|196.64|he slyly unusual epitaphs? ironic deposits at the furiously unusual instructions thrash blithely requests. requests are carefully blithely pending waters.|
+41|2|9040|488.55|ss the dinos wake along the blithely regular theodolites. foxes cajole quickly ironic, final foxes. blithely ironic packages haggle against |
+41|8|5946|391.81| slyly slyly regular requests. final deposits sleep fluffily. blithely bold instructions detect carefully. blithely pending requests are furiously ironically final ideas. regul|
+41|4|1550|916.55| the blithely final ideas. furiously regular asymptotes could cajole furious|
+41|10|560|37.59|special pinto beans against the unusual accounts cajole slyly final foxes. close, ironic|
+42|3|2893|716.81|requests nag. furiously brave packages boost at the furiously even waters. slyly pending ideas nag carefully caref|
+42|9|2927|709.06|g dugouts. carefully careful ideas are fluffily. carefully final pinto beans snooze. ironic deposits wake evenly along |
+42|5|3500|200.00|against the ironic, ironic forges. slyly final deposits wake blithely. ironic courts sleep furiously ab|
+42|1|3662|29.46|es sleep slyly among the slyly final requests. bold theodolites use silently against the final foxes. carefully pending requests use furiously. dogged, unusual asymptotes use |
+43|4|3211|805.78|gular accounts. bold theodolites nag slyly. quickly express excuses use blithely. blithely even ideas boost fluffily! blithely unusual ideas detect bli|
+43|10|6770|493.19|ing to the quickly even theodolites. quickly bold excuses haggle. sometimes unusua|
+43|6|9506|493.65|riously! slyly ironic sauternes affix. ironic theodolites sleep furiously about the express packages. slyly ironic deposits are blithely against the regular package|
+43|2|3232|307.12|counts: express, final platelets use slyly bold ideas. ironic theodolites about the blithely s|
+44|5|486|164.22| final notornis throughout the unusual pinto beans are about the special accounts. bold packages sleep fluffily above the|
+44|1|5310|114.37|quests. quickly unusual requests against the carefully final somas detect slyly bold a|
+44|7|3534|383.01|r the pending pinto beans! requests wake furiously after the special deposits. silent deposits mold quickly along the express, special |
+44|3|4798|833.15| run. ironic, special dolphins according to the even, ironic deposits haggle carefully alongside of the carefully regular excuses. regular frays haggle carefully ironic dependenc|
+45|6|1685|919.63|he doggedly final accounts; carefully regular packages cajole idly regular idea|
+45|2|5202|877.29|ngage blithely after the final requests. bold accounts sleep blithely blithely express dependencies. pinto beans through the carefully regular hockey players wake|
+45|8|5669|532.70|es play carefully doggedly unusual requests. bold grouches against the furiously ironic dugouts sleep furiously qu|
+45|4|1872|155.32| ironic, even pinto beans. bold theodolites haggle after the furiously ironic accounts. slyly bold courts|
+46|7|4171|244.65|lly quiet instructions. furiously express requests among the final ideas cajole carefully bold waters. furiously regular pac|
+46|3|8518|106.80|e unusual instructions shall have to detect slyly blithely ironic foxes. bold requests impress silent foxes. ironic, quiet realms haggle quickly pending, express pinto be|
+46|9|7225|14.78|ously about the fluffily pending accounts. fluffily even dugouts are quickly slyly express platelets; quickly bold pearls sleep slyly even instructions. furiously ironic packages poach quic|
+46|5|1381|985.88|ending platelets are carefully regular accounts. fluffily even accounts against the dependencies nag carefully final, |
+47|8|6989|292.52|even ideas. blithely final requests boost blithely. final, ironic instruct|
+47|4|4458|539.47|; finally enticing theodolites cajole enticing, silent warhorses! slyly bold pains c|
+47|10|2896|74.54|grate final asymptotes. pending requests kindle carefully final frets. ironic deposits above the slyly e|
+47|6|5873|296.63|after the regular dependencies. final, bold pains sleep quickly pend|
+48|9|5052|611.16|posits are blithely blithely final foxes. blithely even deposits haggle fluffily express requests. furiously final theodolites use sl|
+48|5|9451|191.36|ckages cajole never even, special foxes. regular dependencies wake after the blithely ironic instructions. thinly ironic reque|
+48|1|5564|668.19|al pinto beans. furiously final frays use slyly according to the ironic theodolites. regular ideas cajole furiously after the slyly even deposits. |
+48|7|1719|606.16|forges lose. packages cajole regular, bold accounts. never ironic accounts may promise about the permanently bold deposits. always express requests cajole fluffily regular c|
+49|10|9056|35.11| bold deposits? final, bold pinto beans are furiously slyly regular packages. sly|
+49|6|6646|908.15|ts sleep across the fluffily final deposits. carefully express accounts around the regular, express excuses x-ray inside the ironic theodolites. expre|
+49|2|5336|713.25|ld accounts. furiously blithe waters use furiously blithely idle dependencies. pending deposits along the permanently re|
+49|8|597|812.62|n foxes snooze furiously. courts integrate never. carefully unusual requests are carefully. quickly ironic deposits ha|
+50|1|1832|565.54|liers above the dolphins dazzle across the regular foxes. furiously regular packages haggle furiously blithely ironic grouches. ironic, even accounts haggle pending, furious instruction|
+50|7|43|690.87|aggle daringly along the close, express deposits. final requests snooze carefully carefully bold deposits. carefully unusual ideas doze furiously after the furious|
+50|3|6160|301.06|arefully ironic requests use. furiously pending waters play carefully carefully regular platelets. sly requests cajole furiously slyly regular pinto beans. bold packages boost fluffily. furiously i|
+50|9|2104|107.17|t blithely unusual theodolites. quickly final accounts affix fluffily regular requests. c|
+51|2|837|310.74|ly dogged, regular dependencies. express, even packages are |
+51|9|7318|85.03|al foxes. carefully ironic accounts detect carefully-- slyly even accounts use. furiously final platelets shall haggle sometimes after the blithely regu|
+51|6|138|728.95|requests according to the carefully unusual deposits promise slyly ironic packages. slyly ironic dependencies are accordin|
+51|3|8062|901.04|le ruthlessly furiously slow requests. fluffily slow depende|
+52|3|6533|54.92|efully. slyly special deposits haggle along the quick deposits. slyly pending requests use quickly packages. final, final dolphins doubt according to the quickly unusual excuses|
+52|10|1937|210.44|s. never even asymptotes nag carefully! regularly unusual foxes along the unusual requests haggle accounts. fluffily express pinto |
+52|7|4084|628.53| deposits wake slyly pending asymptotes. ironic asymptotes haggle. blithely ironic requests are qui|
+52|4|5524|424.93|cial, ironic packages. even dolphins boost. slyly final deposits integrate. final sheaves along the silent excuses use at the slyly close foxes; bold accounts are finally even packages. ironi|
+53|4|6443|192.78|carefully ironic accounts. blithely bold deposits detect furiously against the flu|
+53|1|5319|563.44|ly. fluffily final pearls boost carefully. special sauternes nod furiously even instructions. carefully regular dependencies across the slyly regular deposits|
+53|8|8200|388.08|fully requests. furiously final accounts cajole express, regular pearls. special deposits wake fluffily express accounts. quic|
+53|5|6929|224.83|xes. carefully ruthless asymptotes impress slyly. fluffily final deposits sleep against the ideas. slyly final packages wake. pending, express packages sleep quickly.|
+54|5|2515|686.51|ly along the packages. blithely close pinto beans are blithely alongside of the unusual packages. carefully even platelets boost alongside of the even foxes. ironic de|
+54|2|7079|798.98|he carefully unusual packages wake according to the ironic dolphins. permanently regular sheaves nag quickly. regular, ironic|
+54|9|2386|23.78|kly ironic foxes. final instructions hinder doggedly. carefull|
+54|6|536|259.24| furiously along the fluffily regular requests. carefully unusual accounts use fluffily final platelets. pending deposits integrate furiou|
+55|6|7874|611.04|ly special packages. furiously even warhorses integrate. silen|
+55|3|8460|236.27|round the special, bold asymptotes cajole alongside of the instructions. qui|
+55|10|8278|134.62|gedly silent pinto beans! furiously regular sentiments was furiously across the silent pinto beans. pending warthogs along the slyly |
+55|7|1289|130.33|ut the blithely final requests. requests nag blithely. |
+56|7|241|855.39|nto beans. finally regular sauternes are. carefully bold deposits according to the blithely express requests wake carefully ironic excuses? furiously final deposit|
+56|4|9104|54.79|tructions above the blithely pending foxes cajole blithely furiously even sentiments. special, exp|
+56|1|1330|52.29|xpress instructions haggle furiously regular deposits. quickly unusual packages sleep furiously final pinto|
+56|8|5799|926.25|ades grow around the dependencies. carefully special ideas cajole furiously across the blithely express requests. unusual tithes are caref|
+57|8|2972|123.11| asymptotes use carefully furiously final deposits. quickly regular deposits are furiously slyly ironic requests. blithely even excuses haggle: blithely special ideas|
+57|5|4721|411.08|instructions. quickly unusual deposits about the furiously special ideas believe among the furiously bold theodolites. unusual, even ideas nag: slow, special theodolites hagg|
+57|2|3788|211.66|ly according to the ironic requests-- slyly final accounts print carefully depths? pending, unusual accounts solve |
+57|9|4583|137.68|ts. blithely bold theodolites can boost carefully carefully even instr|
+58|9|4328|542.52|ven deposits wake requests. quickly bold platelets sleep furiously after the ironic requests. even accounts haggle quickly bold |
+58|6|4307|448.31|quickly carefully ironic foxes. bold platelets nag furiously regular packages. slyly specia|
+58|3|4136|512.24|packages cajole slyly quickly pending depths. special, bold realms cajole slyly. slyly ir|
+58|10|9689|25.09|long the unusual, express asymptotes. ironic ideas boost bold, special deposits? ironic foxes among the fin|
+59|10|8374|357.22|c decoys. carefully even pinto beans wake slyly alongside of the express accounts. regular grouches haggle.|
+59|7|4226|80.98|lar packages. regular depths use slyly after the fluffily regular packages; theodolites around the furiously ironic asy|
+59|4|99|598.55|he special pinto beans. fluffily even accounts cajole. fluffily regular foxes haggle among the|
+59|1|8184|45.50|ependencies. ironic dependencies wake carefully according to the blithely bold packages. quickly unusual ideas about th|
+60|1|6642|800.72| blithely. slyly final realms alongside of the excuses use quickly blithely bold foxes. final theodolites are slyly after the slyly regular excuses. never thin foxes about |
+60|8|5017|314.81| even pinto beans wake carefully. quickly regular deposits hinder along the furiously regular pack|
+60|5|148|504.10|s use fluffily. furiously regular deposits boost furiously against the even instructions. blithely final platelets wake. carefully pending asymptotes sleep blithely. regular, s|
+60|2|5792|92.64|s the carefully pending deposits. slyly regular pinto beans against the furiously regular grouches lose carefully around the enticingly final ideas. furiously express packages cajole bold pa|
+61|2|1540|858.64| could have to use upon the packages. fluffily special packages integrate slyly final theodolites. pending warhorses wake quickly after the blithely final fo|
+61|10|9170|771.26|ly. pinto beans sleep blithely about the patterns. slyly final accounts wake according to the furiously bold requests. slyly regular packages wake according to the ironic packages. requests acros|
+61|8|4762|633.74|final theodolites haggle. fluffily express ideas about the silent theodolites cajole ideas; fluffily special instructions are accordin|
+61|6|7312|153.74|gly final instructions. pending theodolites will wake furiously. slyly bold instructions run. furiously special foxes cajole f|
+62|3|1780|692.42|s around the even ideas cajole furiously somas. silent asym|
+62|1|5896|348.82| final accounts. furious deposits wake slyly. idly regular packages haggle blithely pending grouches. ironic accounts boost blithely. carefully express pa|
+62|9|9127|620.08|totes. unusual requests after the unusual accounts sleep fluffily bold notornis. slowly careful requests use according to the final ideas. pinto beans sleep. foxes are furiously furiously pe|
+62|7|9542|255.78|lly express requests haggle carefully. idle, pending pinto beans are furiously regular excuses. quickly sly attainments are furiously; even accounts are slyly quickl|
+63|4|1804|498.84|leep bravely. final accounts nag. forges sleep against the slyly ironic pa|
+63|2|1998|509.16|yly express theodolites. slyly bold ideas sleep furiously accordi|
+63|10|6839|274.15| among the carefully ironic accounts. carefully even accounts against the regular, final deposits detec|
+63|8|6325|463.69|arly express accounts. express, unusual escapades haggle. special packages must wake. express, regular requests sleep furiously ironic packages|
+64|5|5567|228.61|y even instructions. unusual requests serve slyly. special foxes sleep quickly. fluffily ir|
+64|3|4542|398.92|. quickly final ideas cajole carefully among the blithely silent requests. sometimes ironic accounts nag furiously against the pending instructions. f|
+64|1|9110|602.65| ironic accounts are carefully carefully final accounts. slyly ironic packa|
+64|9|2064|25.77| quickly regular ideas. carefully final requests snooze carefully regular, regular instructions. stealthily final pi|
+65|6|2918|846.26|inal, even foxes cajole. furiously final dolphins hang quickly ironic foxes. furiously special packages alongside of the bold foxes solve above the carefully final instructio|
+65|4|1779|393.63|ully after the quickly regular ideas. ironic, final multipliers above the carefully bold deposits breach slyly furiously express deposits. unusual accounts haggle carefully idea|
+65|2|2054|503.10|e express excuses. ironic, even accounts across the reg|
+65|10|2188|288.73|lent requests nag quickly. blithely silent platelets haggle ironic accounts. slyly bold instructions boost carefully final accounts. carefully even dependencies must nag blithely; qui|
+66|7|3077|809.13|nod carefully besides the furiously final theodolites. slyly final requests haggle. furiously silent excuses detect quickly. ironic deposits detect above the furiously final |
+66|5|1076|785.75|its across the blithely regular theodolites wake furiously among the furiously regular accounts. pains are slyly care|
+66|3|2568|447.08|ously even accounts boost slyly daring requests. even, regular realms kindle blithely. unusual, ironic ins|
+66|1|296|797.27|s nag enticingly outside the furiously final foxes. final accounts haggle fluffily accord|
+67|8|9923|306.37|ly according to the quickly ironic requests. express instructions after the slyly even instructions x-ray blith|
+67|6|7908|546.75|furiously express dolphins integrate carefully regular notor|
+67|4|3368|625.62|le slyly regular requests: regular platelets wake quickly across the quickly regular accounts. reg|
+67|2|5826|397.34|en, ironic deposits affix quickly unusual requests. busily ironic accounts are finally never even sauternes. ironic depos|
+68|9|3444|31.37|es impress furiously pending packages. always silent instructions above the fluffily bold packages haggle slyly blit|
+68|7|6762|5.16|lithely. carefully even grouches along the bold deposits might sleep slyly requests. blithel|
+68|5|8300|80.86|nooze according to the furiously even ideas. blithely regular accounts wake blithely. furiously regular Tiresias cajole regular deposits. regular theodolites eat alongside of the|
+68|3|5399|683.59|. finally final pinto beans play carefully unusual requests. never pending accounts are. regular, final theodolites wake furiously excuses. special request|
+69|10|6197|694.24|eep across the packages. regular, final foxes boost fluffily regular pinto beans. packages sleep along the final requests. bold, unusual packages cajo|
+69|8|8235|846.49|nt fluffily. carefully ironic instructions wake. blithely express foxes cajole slyly. unusual requests sleep quickly. final packages affix slyly according to the spec|
+69|6|9294|386.96|ar packages. blithely regular dependencies are dolphins. slyly ironic excuses nag quickly pending, regular ideas. furiously special sheaves haggle. close, regular pinto beans about the slyly bold|
+69|4|7017|344.28|heodolites. unusual, regular requests boost slyly pending deposits. slyly daring instruct|
+70|1|4536|348.27|ructions. blithely final packages cajole carefully after the express, even requests. furiously final theodolites cajole |
+70|9|8063|452.80|y regular deposits nag about the carefully regular instructions; furiously express accounts along the final, express instruct|
+70|7|2990|940.81|s deposits. unusual foxes are carefully according to the carefully even deposits. carefully ironic foxes cajole fluffily against the carefully pending deposits. slyly special depo|
+70|5|9074|182.58|ions after the fluffily regular foxes wake above the furiously regular requests: slyly regular deposits wake slyly daringly even Tiresias. express, express deposits are. always unusual pa|
+71|2|508|842.21|es cajole carefully around the furiously pending instructions. |
+71|1|8329|239.57|ins sleep carefully slyly express accounts! quickly even accounts boost carefully about the carefully regular excuses. dogged, even dolphins against the sometimes ironic packages believe bl|
+71|10|6768|744.67|ructions. daring requests solve carefully about the furiously pending pinto|
+71|9|5179|329.13|usly at the packages. blithely regular deposits haggle regular packages. quickly special theodolites at the blithely ironic instructions wake|
+72|3|9855|497.26|tithes. quickly pending foxes haggle enticingly according to the accounts. accounts detect slyly: final packages wake. fina|
+72|2|9346|41.04| pending instructions before the even, silent dep|
+72|1|2654|762.61|nusual packages: blithely bold Tiresias sleep furiously. slyly brave accounts according to the final, |
+72|10|4526|154.47|use across the never ironic packages. express, regular accounts above the pending, fluffy deposits are carefully across the slyly even pinto be|
+73|4|9873|947.99|tes use pending packages. final foxes wake final, unusual packages. blithely blithe ideas haggle sometimes slyly express accounts. express instructions nag furiously quickly|
+73|3|7729|920.66|ecial accounts sleep according to the slyly sly accounts. slyly express instructions nag. accounts cajole furiously quickly even foxes. furiously regular requests wake. carefully even frets haggle |
+73|2|5327|108.96| beans are furiously between the regular ideas! unusual pinto beans use. furiously silent requests against the carefully even somas wake care|
+73|1|3928|309.57|longside of the blithely final ideas. carefully ironic courts sleep along the enticingly pending requests. fluffily regular accounts use fluffily bold ideas. slyly ironic packa|
+74|5|3128|345.92|ic theodolites. express deposits haggle blithely pending packages. quickly express foxes could are slyly. deposits sleep deposits. final dependencies sleep ab|
+74|4|2479|930.97|o beans sleep dependencies. regular accounts use blithely asymptotes. u|
+74|3|9473|496.36| haggle carefully alongside of the regular requests. slyly regular accounts belie|
+74|2|6234|849.66| slyly regular foxes. silent accounts integrate. even deposits are quick|
+75|6|7086|624.39|sits are furiously fluffily even courts. furiously pending requests are blithely. pending, regular accounts play carefully slyly unusual platelets. blithely final requests against the ru|
+75|5|6308|759.36|refully ironic dependencies. pinto beans use according to the packages. regular platelets wake around the blithely p|
+75|4|9080|433.59|sits. permanent packages breach. carefully final waters wake. bold, pending foxes haggle furiously evenly express instructions. even deposits about the final|
+75|3|5439|884.01|ding excuses snooze special accounts. tithes alongside of the regular dep|
+76|7|6754|494.83|gular accounts solve. ironic deposits sleep slyly even packages. slyly pending accounts detect slyly express accounts. ironic forges can play furiously carefully express fox|
+76|6|2009|108.97|n packages. blithely even accounts sleep carefully furiously ironic accounts. carefully express requests|
+76|5|6371|552.38|ts use against the quickly ironic ideas. quickly even deposits are carefully a|
+76|4|7986|252.03| packages across the furiously ironic platelets cajole across the regular, ironic accounts. carefully enticing accounts among the blithely regular instructions detect regular pinto be|
+77|8|552|254.92|e after the carefully pending packages. carefully even dependencies cajole pending |
+77|7|8170|875.83|xcuses. blithely even foxes use fluffily. blithely even requests use. slyl|
+77|6|8541|936.13|e slyly express instructions haggle about the sometimes regula|
+77|5|1713|402.14|the even ideas kindle after the requests. regular theodolites cajole carefully about the blithely final ideas. carefully even dependencies at the flu|
+78|9|9915|729.94|around the special excuses. furiously even deposits serve boldly according to the platelets. carefully express accounts at the blithely unusual pinto beans sleep furiously against the u|
+78|8|7246|577.23|regular dependencies cajole doggedly ironic accounts. bold theodolites doze about the accounts. quickly final requests boost slyly final asymptotes. carefully final dolphins ha|
+78|7|1801|434.34|nts kindle furiously according to the even packages. blithely ironic platelets are slyly silent foxes. final, final packages would sleep. pinto beans a|
+78|6|9599|382.82| carefully special theodolites cajole among the quickly even asymptotes. foxes wake blithely across the carefully |
+79|10|4248|765.34|nusual, express asymptotes wake furiously. ironic pinto beans detect above the carefully express theodolites: even, dogged instructions nag. spe|
+79|9|465|28.33|uriously special frays cajole across the finally ironic pinto beans. ironic accounts sleep blithely. fluffily silent accounts are slyly at the slyly unusual ideas. even deposits nag slyly |
+79|8|3309|880.23|tect final, thin accounts? furiously ironic accounts boost regular deposits. carefully ironic attainments sleep. furiously special ins|
+79|7|8627|891.18|r dolphins grow blithely against the slyly ironic packages. deposits about the regular, ironic decoys are slyly around the carefully regular packages. slyly pending excuses sle|
+80|1|8893|127.65|ld accounts detect carefully. carefully bold courts along the regular deposits could have to affix ca|
+80|10|2243|775.79|endencies. bold, regular pinto beans wake furiously above|
+80|9|5385|945.72|cial asymptotes believe after the blithely unusual deposits. furiously silent pinto beans cajole quickly inside the slyly even deposits. regular, f|
+80|8|4034|797.05|ptotes cajole carefully. express ideas cajole carefully even somas. final pinto beans print fluffily across the |
+81|2|1605|550.29|es haggle blithely fluffily final requests. furiously regular foxes use. furiously unusual requests outside the furiously regular requests|
+81|2|5923|220.23|the final, quick accounts are blithely above the s|
+81|2|2942|409.73|accounts boost. fluffily unusual requests cajole fluffily slyly ironic requests. foxes cajole quick|
+81|2|58|492.19| instructions boost furiously across the foxes-- final depo|
+82|3|7793|697.31|he accounts cajole quickly after the even patterns. ironic platelets sublate regular, even asymptotes. quick courts affix according to|
+82|3|7698|585.86|pinto beans. slyly express excuses haggle. blithely even pinto beans about the quick inst|
+82|3|8268|604.25|e after the carefully even theodolites. regular, pending accounts boost. quickly final asymptotes haggle slyly. requests use final, bold pinto beans. bold, ruthle|
+82|3|5532|900.07| slyly? fluffily special dependencies haggle among the slyly special requests. regular, bold packages after the blithely ironic packages are slyly ironic packages. slyly final deposits w|
+83|4|3010|745.51|l foxes along the bold, regular packages integrate carefully express courts! final excuses sleep carefully ironic|
+83|4|8200|399.64|y final platelets are carefully carefully special platelets. carefully ironic requests wake blithely alongside of the slyly even accounts. bold, regular requests sleep |
+83|4|5974|657.22| even packages boost furiously. slyly regular gifts above the accounts are quickly express packages. slyly pending deposits besides the express, even asymptotes haggle after the ironic ins|
+83|4|3890|24.73|deposits. carefully even dependencies across the dependencies haggl|
+84|5|5711|233.61|arefully final platelets cajole blithely; quickly final accounts use furiously. furiously reg|
+84|5|208|469.80|carefully express dolphins nag about the slyly bold requests. slyly even packages wake among the furiously special attainments.|
+84|5|2909|969.44|silent requests cajole slowly bold ideas. special, special deposits according to the always silent packages are against the furiously silent packages. even, blithe accounts sleep slyly across |
+84|5|903|707.77|gly regular dependencies boost. slyly even accounts sleep. furiously final hockey players wake carefully with the reg|
+85|6|2628|608.77|xes wake furiously after the carefully even platelets. blithe theodolites are furi|
+85|6|118|917.83| against the even deposits. furiously bold ideas along the furious requ|
+85|6|2074|491.20|encies-- slyly regular requests about the quiet accounts detect quickly at the |
+85|6|8289|73.81|s cajole slyly along the slyly special accounts. regular, special deposits wake. furiously special foxes boost. blithely even packa|
+86|7|806|65.98|ackages. blithely pending accounts are slyly furiously pending theodolites. furiously eve|
+86|7|2773|250.04|ding accounts. slyly special requests will have to affix carefully along the furiously unusual packages. regular theodol|
+86|7|5546|816.53|s. slyly final requests wake. furious deposits must wake blithely among the blithely ironic instructions. special hockey players try to are bli|
+86|7|1418|332.65|press theodolites sleep carefully about the blithely unusual requests. quickly final deposits breach slyly |
+87|8|5679|688.33|t the carefully regular asymptotes. blithely stealthy pinto beans within the furiously expres|
+87|8|1272|435.42|ronic foxes sleep along the special foxes. final ideas wake quickly about the carefully special theodolites. blithely ironic packages are blithely. regular, regular pint|
+87|8|9041|617.20|furiously final deposits. furiously special dependencies solve across the regular, special ideas. carefully silent requests haggle furiously after the special, specia|
+87|8|1892|868.60|arhorses are. unusual requests use blithely furiously final ideas. final requests sleep theodoli|
+88|9|6116|334.58|ect furiously around the regular deposits. special, final platelets boost furiously. blithely unusu|
+88|9|395|71.50| the regular accounts-- furiously even accounts use quickly after the regular, regular deposits. furiously e|
+88|9|9979|81.82|f the regular, regular requests believe fluffily along the final, quiet decoys. furiously even accounts cajole. carefully express requests wake quickly among the ideas. quickly silent |
+88|9|276|821.43|gular pinto beans. slyly pending excuses breach blithely express accounts. thin deposits sleep slyly around the even accounts; fluffily busy patterns kindle. slyly final deposits along the |
+89|10|3430|744.87| integrate slyly dolphins. bold, final frets use beside the carefully even accounts. slyly close dependencies sleep quickly carefully final pinto beans. foxes promi|
+89|10|8599|776.53|ress packages use furiously. furiously regular packages thrash blithely about the slyly pe|
+89|10|7876|417.61|nstructions: furiously even requests are quietly unusual accounts. regular requests are after the blithely regular deposits. sl|
+89|10|924|920.02|ickly unusual asymptotes after the slyly unusual accounts are carefully doggedly ironic accounts. even, final accounts use furiousl|
+90|1|8037|409.38|eas. unusual, pending packages boost quietly final accounts. slyly final packages serve. slyly even instructions sleep carefully. quickly even foxes wake quickly. |
+90|1|9683|498.43| accounts! fluffily regular deposits x-ray about the unusual, final packages. furiously final deposits alongside of the caref|
+90|1|7849|666.13|carefully ironic accounts are around the slyly bold asymptotes. carefully regular packages use furiously. ironic platelets affix carefully final accounts-- fluffily final pinto beans across the fina|
+90|1|7629|50.84|onic requests wake fluffily unusual packages. furiously even frays after the daringly pending requests wake furiously alongside of the bold requests. fluffily ironic ideas nag. ironic,|
+91|2|7986|528.64|luffily final instructions. furiously unusual foxes haggle |
+91|3|3257|906.20|ackages cajole slyly. blithely bold deposits cajole. blithely |
+91|4|483|823.21|n: slyly ironic foxes nag blithely according to the furiously bold foxes. regular, regular accounts a|
+91|5|1265|703.41| quickly silent deposits use attainments. final requests along the carefully ironic accounts wake blithely about the carefully ironic excuses. furiously bold excuses wake final, final ex|
+92|3|9337|224.01| requests are slyly along the deposits. fluffy pains alongside of the deposits |
+92|4|2246|985.03|jole enticingly regular asymptotes. carefully unusual pinto beans nag carefully ironic ideas. quickly un|
+92|5|3199|91.63|ake carefully: carefully ironic requests sleep careful|
+92|6|1044|854.89|l instructions are fluffily silently regular accounts. quickly final dolphins w|
+93|4|3008|615.98|sits promise blithely fluffily special decoys. slyly regular packages along the slyly final deposits wake accord|
+93|5|5275|376.47|ounts boost fluffily along the thinly regular realms. busily regular a|
+93|6|3869|868.81|ly among the furiously silent accounts. closely regular pinto beans nag slyly! slyly e|
+93|7|7188|805.90|y furiously bold pinto beans. express asymptotes was quickly. carefully final accounts affix slyly! platelets according to the ca|
+94|5|5433|365.56| even excuses wake carefully. quickly unusual requests wake accounts. regularly pending packages are regular |
+94|6|7784|358.08|ironic packages wake slyly carefully regular accounts. quickly regular warhorses against the blithely ironic packages haggle doggedly sly|
+94|7|7232|478.94|y regular requests. carefully final asymptotes haggle carefully against the slyly unusual requests: blithely brave grouches are fu|
+94|8|3261|824.08|quests. enticingly final accounts sleep fluffily. quickly express asymptotes around th|
+95|6|5186|291.03|ites across the blithely pending theodolites do affix across the unusual, bold Tiresias. bold packages|
+95|7|6552|456.36|tes; final, final accounts boost blithely ironic pinto beans. blithely ironic deposits cajole above the quickly pending requests? i|
+95|8|367|987.22| express requests detect furiously. requests cajole carefully|
+95|9|7379|973.74| above the furiously unusual deposits haggle ironic ideas. express, even packages haggle slyly slyly special asymp|
+96|7|5739|202.06|re. slyly regular theodolites breach slyly even dinos. fluffily regular asymptotes haggle slyly. fluffily bold courts affix furiously. regular requests |
+96|8|4942|571.30|e carefully. bold packages sleep against the furiously express requests. express foxes above the dependencies use quickly according to the slyly expres|
+96|9|9985|672.29|ecial instructions-- blithely silent theodolites play. even, silent accounts sleep. blithely silent requests haggle final, f|
+96|10|7250|587.08|efully ironic foxes. regular, final pinto beans boost above the express a|
+97|8|6371|129.77|fluffily unusual accounts. slyly regular theodolites integrate furiou|
+97|9|2390|458.34| carefully unusual pinto beans; even deposits detect furiously|
+97|10|2618|239.34|al theodolites are daringly requests. warhorses sleep blithely requests. special accounts cajole slyly deposits. a|
+97|1|4580|761.41| beans. carefully final deposits alongside of the carefully final requests haggle idly blithely ironic accounts. foxes cajole slyly against the ironic, special packages. furiously brave excuses boo|
+98|9|9486|908.21|usly final deposits mold furiously above the even deposits. carefully ironic packages across the quickly regular dolphins are slyly according to the slyly even|
+98|10|8550|657.16| sleep carefully. bravely bold somas may sleep pendin|
+98|1|3443|139.00|gular pinto beans maintain quickly fluffily regular deposits. express requests sleep. even requests after the regu|
+98|2|3759|811.55|iously. final, express packages are across the ironic dependencies. slyly thin ideas according to the even Tiresias detect furiou|
+99|10|8487|438.38|lphins affix ironic packages. blithely ironic requests nag fluffily after the slyly ironic foxes. bold dependencies boost furiously. special, |
+99|1|7567|496.93|es? permanently even excuses haggle quickly across the dependencies.|
+99|2|7970|365.83|ending accounts cajole furiously. requests promise care|
+99|3|2789|843.88|ending accounts. furiously sly packages above the carefully unusual dolphins sleep after the thinly even deposits. requests wake abo|
+100|1|7885|490.61| accounts nag slyly against the bold excuses. pearls according to the fluffily ironic accounts haggle fluffily along the quickly final platelets|
+100|2|2070|196.73| dolphins. bold deposits along the even theodolites sleep furiously about the final pinto beans. furiously unusual courts cajole about the carefully bold asymptotes. accounts integrate slyly entic|
+100|3|4994|929.57| deposits. accounts are slyly regular ideas. slyly special pinto beans upo|
+100|4|9688|22.00|uctions according to the carefully ironic deposits haggle carefully express ideas? packages across the quickly final requests c|
+101|2|5589|305.40|blithely above the fluffily pending ideas. quickly quick accounts nod ruthlessly above the carefully pending packages. slyly s|
+101|4|8263|218.71|fluffily final requests. carefully even packages wake quickly about the quickly ironic foxes. fluffily even requests hang quickly about the pending, final requests. sp|
+101|6|6324|786.53|olites sleep quickly. slyly ironic theodolites affix. furiously bold accounts integrate among the pinto beans. final ideas hang slyly along the quickly regular packages. instructions cajole.|
+101|8|55|612.09|beans against the carefully express ideas wake quickly along the quickly unusual requests. blithely regular accounts cajole fluffily. enticingly pending theodolites haggle furiously fluffily pendi|
+102|3|1384|876.75|s wake quickly. carefully express deposits wake. silent, regular requests sleep slyly after the furiously ironic deposits. slyly unusual accounts cajole|
+102|5|9137|332.71|telets are final, special deposits. silently ironic deposits wake. pending, eve|
+102|7|9156|618.00| the unusual, ironic pinto beans. theodolites above the foxes sleep slyly car|
+102|9|6942|231.02|tions haggle against the furiously ironic deposits. quickly final asymptotes haggle carefully. regular sentiments might cajole silent courts. blithely bold frays |
+103|4|5913|905.88|e across the theodolites. carefully pending escapades haggle after the ironic theodolites. furiously pending ac|
+103|6|7742|414.42|bout the bold, regular deposits; blithely even accounts are regular, even platelets-- carefully express accounts nag slyly pen|
+103|8|5164|361.48|furiously thin deposits haggle blithely. blithely regular deposits above the carefully regular accounts are slyly carefully regular packages. silent, unusual|
+103|10|429|605.20| theodolites cajole quickly above the asymptotes-- slyly special packages can haggle carefully blithely final instructions. unusual, regular ideas|
+104|5|2368|946.39|packages. final packages wake enticingly. furiously regular asymptotes are always about the carefully regular deposits. slyly regular platelets cajole carefully. final pinto beans must pro|
+104|7|6110|15.57|ending requests. carefully regular deposits use blithely. bold, ironic deposits wake slyly carefully specia|
+104|9|6269|213.89| ideas against the final accounts wake slyly regular notornis. final deposits haggle a|
+104|1|3369|729.38|ong the foxes. foxes sleep quickly? carefully regular accounts sleep. special foxes boost quickl|
+105|6|4602|27.75|lar pearls cajole never carefully even depths. blithely regular ideas are quickly. unusual asymptotes nod carefully carefully regula|
+105|8|269|158.62| unusual courts eat pending excuses. ironic, ironic requests use. bravely |
+105|10|8855|810.86|. slyly special depths sleep. bold packages engage furiously bold packages. fluff|
+105|2|8519|904.17|ding to the furiously careful ideas. dogged theodolites wake fluffily among the slyly bold ideas. blithely brave warthogs above the slyly even theodolit|
+106|7|8649|732.15| slyly ironic instructions are. bold, final accounts cajole slyly ironic pinto beans. fluffily ironic accounts around the quickly special requests use blith|
+106|9|3144|779.68| final deposits along the slyly express theodolites cajole blithely after the ironic pinto beans. furiousl|
+106|1|1411|310.40|al accounts impress. even instructions engage furiously final foxes. silently final deposits wake qui|
+106|3|2297|281.98|inal packages. pending foxes sleep bold hockey players. courts across the blithely regular packages sleep fl|
+107|8|7249|168.03|he fluffily even packages. slyly regular dependencies nag fluffily above the final, unusual foxes. final, pending foxes affix. furiously final deposits cajole quickly blithely|
+107|10|4029|91.31|integrate. requests maintain quickly. carefully regular ideas about the instructions sle|
+107|2|4667|372.94|uctions sleep doggedly final requests. express, final theodolites cajole fluffily furiously silent deposits. blithely regular requests cajole quickly regular instruction|
+107|4|7912|474.77|fluffily across the final, bold accounts. quickly regular deposits grow carefully deposits. regular requests haggle blithely. slyly special platelets boost furiously care|
+108|9|4149|558.85| of the quickly pending theodolites. fluffily unusual frays wake accounts. carefully even foxes wake slyly. carefully special pinto beans |
+108|1|4898|241.47|lthily according to the fluffy deposits. furiously silent ideas according to the furiously special theodolites wake furiously a|
+108|3|5534|626.89|instructions. blithely regular instructions according to the permanent foxes cajole blithely slyly fluffy foxes. slyly regular asymptotes cajole foxes. slyly unusual deposits |
+108|5|3142|922.27|slyly express accounts are fluffily along the blithely unusual packages. pinto beans mold furiously. furiously bold instructions are blithely deposits. quickly special accounts detect t|
+109|10|524|275.19|st the permanently final requests. carefully pending pinto beans haggle quickly slyly ironic dolphins. blithely bold deposits wake blithely. even requests cajole foxes. iro|
+109|2|4470|992.21|ake furiously packages. blithely even foxes haggle furious|
+109|4|8176|936.60|d the express accounts. even theodolites wake quickly up the furiously bold foxes. furiously regular packages use regular, bold|
+109|6|7524|374.49|sual requests. final pinto beans cajole furiously. dependencies integrate slyly even excuses. blithely special requests wake around the slyly final pinto beans. furiously pending requests wake furi|
+110|1|2782|566.46|ly sly deposits. regular deposits are; theodolites haggle furiously bold foxes. final pinto beans wake blithely furiously ironic depths. unusual, regular platelets cajole. final, ironic|
+110|3|8914|183.47|yly even foxes. carefully regular requests doubt. pending, regular requests across the blithely final theodolites hag|
+110|5|1160|932.08|ake blithely. furiously ironic accounts sleep fluffily|
+110|7|4927|196.63|theodolites. instructions haggle even packages. waters impress furiously quickly express courts. furiously ironic depths nod quickly? ironic, ironic requests sle|
+111|2|1890|321.97|st foxes. daring depths use above the furiously special ideas. ironic foxes among the carefully final theodolites are alongside of the regular depths. e|
+111|5|9587|978.65|express instructions against the furiously final grouches haggle across the blithely ironic theodolites. slyly special dependencies in place of the carefully pending |
+111|8|8069|745.33|thely bold requests. unusual packages sleep. quickly pending ideas nag furiously to the carefully final p|
+111|1|2175|496.08|s around the daringly final pinto beans use furiously against the pi|
+112|3|8509|111.42|unts. carefully ironic instructions are final, bold foxes. bold excuses run according to the unusual packages. theodolites cajole carefully according to the fluffily pending deposits? sly|
+112|6|7652|783.52| carefully among the furiously final packages. regular instructions nag. slyly pending ideas hang fluffily blithely ironic instructions. ironic deposits haggle except the quickl|
+112|9|4835|913.75|ach slyly special dependencies. furiously ironic pinto beans sleep slyly pen|
+112|2|1211|815.88|l requests integrate furiously. quickly quiet packages are carefully regular accounts. regular packages eat quickly express, ironic sheaves. quickly dogged accounts o|
+113|4|9981|396.26|ithely express pains lose bravely fluffily pending foxes. blithely ironic sauternes cajole q|
+113|7|3804|860.68|ully about the carefully even accounts: thinly even foxes are carefully. |
+113|10|522|981.41| warthogs use quickly alongside of the furiously unusual requests. final deposits are blithely acro|
+113|3|4692|141.48| blithely unusual gifts snooze against the quickly ironic packages. regular packages across the carefully regular packages bo|
+114|5|7146|447.24|ideas will nag regular accounts! carefully final requests cajole furiously quickly final tithes. furiously express instructions a|
+114|8|3062|555.12|ts. furiously regular requests run carefully thin decoys. ironic platelets sleep alongside of the slyly silent deposits. reg|
+114|1|4519|382.87|nts. ironically express dolphins dazzle blithely. special instructions wake carefully along the ideas. quickly special dolphins sleep. furiously pendi|
+114|4|2113|570.79|o beans sleep among the ironic excuses. furiously even sheaves are. never regular instructions nod.|
+115|6|1817|82.84|uffily final accounts integrate furiously along the carefully busy excuses. slyly even asymptotes doubt quickly. fluffily thin theodoli|
+115|9|983|867.45|kly. requests nag after the blithely bold packages. express requests cajole theodolites. blithely express requests sleep after the furiously regular accounts. fluffily r|
+115|2|7781|861.93|lyly ironic pinto beans affix alongside of the furiously even ideas: quickly bold warhorses sle|
+115|5|3002|81.52|efully after the quickly regular deposits. daringly pending ideas sleep even ideas. silent, re|
+116|7|6580|705.50|iously. slyly regular requests detect slyly. carefully bold packages sleep furiously carefu|
+116|10|4975|413.86|tions. regular excuses detect. ideas haggle slyly about the slyly ironic courts. ironic foxes solve. ideas affix fluffily after the special, even dependencies. final platelets according|
+116|3|8679|866.56|aphs cajole blithely regular accounts. even packages doubt; bold instructions boost quickly. fluffi|
+116|6|5632|37.30|ccounts about the special packages nag across the carefu|
+117|8|5906|706.51|into beans sleep carefully blithely bold packages. even, bold instructions use across the carefully e|
+117|1|1986|322.08|eposits. special pinto beans use fluffily across the furiously regular pinto beans. furiously regular epitaphs nag fluffily packages. special accounts a|
+117|4|2577|761.86|riously. doggedly unusual ideas boost blithely blithely regula|
+117|7|4762|552.88| run furiously ironic accounts. slyly ironic deposits haggle slyly fluffy requests. flu|
+118|9|694|744.73|refully slow requests. requests against the special pac|
+118|2|6326|325.61| packages. express, final frays affix quickly above the final asymptotes. carefully regular requests doubt quickly f|
+118|5|7806|283.27| accounts affix carefully. regular, regular packages among the brave, pendin|
+118|8|4951|976.55|s orbits. even asymptotes above the instructions wake fluffily according to the sly, final excuses. express deposits across the blithely ironic depend|
+119|10|2307|473.64| blithely unusual dolphins boost busy, express ideas. regular requests use carefully furiously ironic deposits. carefully regular packages would sle|
+119|3|1452|676.92|ular instructions was slyly. furiously bold gifts boost f|
+119|6|4955|488.93|ias are along the express requests. fluffily pending ideas nag idly against the fluffily bold instructions? foxes cajole quickly. slyly special deposits haggle slyly e|
+119|9|583|782.47|yly pending requests-- carefully special instructions haggle carefully even instructions. blithely regular theodolites detect blithely final ideas. blithely ironic deposits among the sl|
+120|1|4976|201.21|inal, regular pinto beans haggle carefully! ironic ideas unwind among the slyly regular theodolites. regular platelets kindle blith|
+120|4|7744|365.79|l, special escapades! ideas sleep slyly instructions. carefully bold requests are. even accounts cajole. final accounts use slyly |
+120|7|5329|249.61|s cajole blithely. carefully bold requests believe blithely? brave accounts above the pending, dog|
+120|10|3102|566.34|ctions; realms beside the blithely final theodolites unwind blithely packages. regular dolphins sleep carefully-- carefully express accounts wake quickly. pending depths use never courts.|
+121|2|9741|147.45|ly according to the carefully regular asymptotes. silent excuses cajole carefully against the never silent instructions. furio|
+121|6|4246|850.42|usly final instructions. theodolites are according to the permanently ironic accounts. carefully pending accounts haggle about the pending instructio|
+121|10|7670|449.39|carefully daring packages. express packages use carefully about the quickly unusual packages. special ideas along |
+121|4|8709|655.22| detect carefully along the carefully even pinto beans. gifts haggle: ideas sleep ar|
+122|3|1019|451.29| blithely regular accounts. blithely final pains sleep besides the blithely final warhorses. furiously unusual requests haggle furiously|
+122|7|2490|637.28|efully special excuses grow slyly unusual packages. carefully quiet as|
+122|1|4957|650.84|quests. quickly bold requests nag across the furiously ironic accounts. ironically express instructions detect slyly carefully ironic requests. even, un|
+122|5|2083|739.25|counts. unusual requests alongside of the regular requests are carefully stealthy instructions? regular sauternes cajole. final theodolites breach carefully at the blithely final idea|
+123|4|9881|107.03|fully bold deposits detect slyly pending instructions. slyly special ideas detect blithely. slyly fluffy instructions hinder|
+123|8|5638|818.19|thely even pinto beans. furiously regular asymptotes affix furiously. regular, ironic tithes integrate quickly. blithely regular requests breach finally. decoys alon|
+123|2|2692|217.01|he ironic accounts nag fluffily after the bold, pending theodolites. blithely final ideas sleep carefully according to the blithely ironic foxes. regular requests are. furiousl|
+123|6|5311|149.65|eposits cajole according to the carefully pending packages. furiously final epitaphs solve alongside of the even requests|
+124|5|7102|901.98|ily accounts. furiously busy theodolites above the deposits thrash above the blithely final foxes. express instructions nod slyly furiously busy packages. special asymp|
+124|9|3969|908.64|l epitaphs. packages cajole among the furiously regular requests. closely|
+124|3|9295|882.54|s along the accounts poach quickly ironic deposits. even, final excuses thrash carefully about the express, special pains. carefully careful accounts breach slyly|
+124|7|9416|822.78|ously. theodolites affix around the slyly bold packages. even, ironic packages are carefully pains. furiously unusual requests sleep blith|
+125|6|2263|358.45|e. ironic, regular requests cajole fluffily along the even ideas. final ideas wake blithely. blithely bold |
+125|10|8276|668.65|nd the carefully express requests. slyly regular requests haggle. blithely unusual platelets solve fluffily fluffily regular|
+125|4|2600|455.70|ounts. thinly special accounts cajole carefully. even, special accounts after|
+125|8|5546|806.66| to the unusual courts are deposits! final, final pinto beans solve slyly. ironic accounts boost fluffily. furiously pending d|
+126|7|2647|221.89|lyly final pinto beans across the regular, even courts use slyly slyly pending braids! unusual requests along the furious|
+126|1|2373|194.38|fter the ideas. blithely daring sheaves print furiously among the blithely final packages. iron|
+126|5|1532|451.61|refully alongside of the quickly bold excuses. enticing, bold |
+126|9|5458|929.43|leep to the furiously special accounts. furiously final courts |
+127|8|7658|712.33|al pinto beans! slyly ironic excuses boost after the packages. express foxes integrate carefully. pending, regular theodolites |
+127|2|1467|237.98|regular accounts! quickly ironic packages haggle according to the accounts. carefully ironic |
+127|6|8281|3.14|ts above the furiously pending asymptotes cajole after the deposits. slyly ironi|
+127|10|8894|73.42|fter the sometimes special courts sleep about the slyly unusual reque|
+128|9|6982|425.29|ironic asymptotes. fluffily ironic packages use. ironic, regular ideas are in place of the quickly silent deposits. final, bold gifts across the ironic, regular pac|
+128|3|7602|224.49|xcuses. blithely unusual theodolites use slyly carefully even warthogs. slyly even dugouts haggle slyly final, express pinto beans. furiously bold packages thrash requests? slyly unusual packages |
+128|7|3766|947.16|arefully regular packages boost regularly. accounts are according to the blithely even dependencies. slyly silent accounts doubt slyl|
+128|1|7023|875.78| furiously quickly regular pinto beans. always special requests are. quickly regular deposits are furiously. slyly unusual theodolites haggle evenly; furiously special deposits wa|
+129|10|5721|129.69|ully express requests above the ironic, final requests cajole slyly along the quickly special packages. sl|
+129|4|7242|200.26|es across the furious escapades wake quickly slyly e|
+129|8|5299|330.59|final sentiments affix atop the silent foxes. busy pinto beans cajole. slyly final pinto beans haggle against the carefully expres|
+129|2|1968|27.22|ealthy, ironic deposits. slyly ironic pinto beans are blithely pinto beans. blithely ironic |
+130|1|4928|223.38|ths. slyly even theodolites detect according to the slyly final courts. carefully unusual deposits ar|
+130|5|6909|275.58|lly unusual accounts try to boost along the special packages. furiously bold requests x-ray blithely ironic waters. slyly unusual orbi|
+130|9|4850|442.81|ully regular deposits snooze. slyly silent foxes detect furiously furiously bold requests. slyly regular accounts breach. blithely bli|
+130|3|7387|883.99|aggle furiously. even ideas hinder deposits. even, final ideas are. unusual theodolites after the special, express foxes haggle carefully pending accou|
+131|2|3263|211.70|sits sleep quickly regular multipliers. slyly even platelets cajole after the furiously ironic deposits. slyly ironic requests should have to cajole: bl|
+131|7|125|861.84|l accounts grow quickly-- slyly ironic requests haggle? quickly express pinto bean|
+131|2|5138|572.43|grouches run with the carefully even packages. ironic, even deposits run slyly along the packages. special dependencies among the regular |
+131|7|8945|613.09| are carefully along the quickly final theodolites. packages after the quickly pending package|
+132|3|3092|687.29|y special decoys against the ideas affix against the sly|
+132|8|1904|925.73|the regular foxes wake ironic deposits. ironic, special requests use blithely instructions! final requests hang. blithely regular deposits haggle. ir|
+132|3|7441|357.06|ests. furiously unusual requests wake furiously. quickly unusual depos|
+132|8|5303|353.06|ep blithely after the sly accounts. slyly express dolphins cajole amon|
+133|4|5727|49.17|boost blithely across the ironic, regular instructions. packages use slyly unusual requests. bold accounts above the fu|
+133|9|404|478.18|ly ironic requests run instead of the blithely ironic accounts? regular ideas use fluffily: even, express packages sleep abov|
+133|4|4568|57.48|dolites. ironic accounts are blithely pinto beans. regular pinto beans haggle beneath|
+133|9|2813|277.26|s. pending, final accounts haggle blithely furiously pending deposits! carefully unusual attainments integrate. blithely bo|
+134|5|8879|848.14|lites. slyly final foxes after the bold requests cajole carefu|
+134|10|9013|102.99|pendencies. furiously express warthogs cajole furiously ironic, regular asymptotes. bold deposits boost among the furiously even theodolites. regular instructions integrate carefully |
+134|5|852|927.45| sleep unusual, express packages. unusual sentiments are furio|
+134|10|6270|388.28| to the furiously pending deposits nag along the slyly express asymptotes. slyly silent accounts shal|
+135|6|6940|465.82|ding foxes cajole. even dugouts haggle busily. fluffily pending packages about the express excuses boost slyly final packages. blithely express ideas cajole about the carefu|
+135|1|2443|9.83|atterns. pending, special deposits are furiously. express, regular deposits integrate quickly. unusual gifts cajole blithely stealthily pending deposit|
+135|6|7453|698.42|ven accounts. slyly final instructions nag slyly around the regular, unusual packages. slyly sp|
+135|1|2771|306.43|old deposits. furiously express instructions boost. pending dolphins use requests. slyly regular packages cajole quickly final ideas. pending, regular ideas nag carefully even, express pla|
+136|7|2237|548.19|ond the silent accounts haggle above the blithely regular packages|
+136|2|6068|806.19|structions. ironic theodolites haggle according to the final, daring pearls. carefully ironic somas are silently requests. express pa|
+136|7|8979|387.57|ans. express pinto beans wake carefully among the slyly ironic foxes: carefully final pinto beans haggle blithely. pending, final deposits promise furiously|
+136|2|9617|525.81| across the carefully pending warthogs. close, regular packages are quickly after the never ironic foxes. accounts sleep quickly along the furiously regular re|
+137|8|9057|302.26|slyly about the regular instructions. even, ironic theodolites use carefully around the even decoys. unusual, pending dolphin|
+137|3|4078|441.11|packages. blithely unusual sentiments should are. furiously regular accounts nag quickly carefully special asymptotes! idly ironic requests dazzle bold requests. carefully expres|
+137|8|467|371.85|ly special accounts detect carefully. furiously ironic deposits nag express packages. slyly quiet |
+137|3|7850|187.31|atelets sublate fluffily. enticingly unusual packages boost according to the blithely ironic foxes. pending requests mold sly|
+138|9|133|576.96|regular, final deposits maintain slyly even requests. regularly furious deposits use above the stealthy requests. ironic deposits are. carefully final frays are carefully. carefu|
+138|4|2535|885.35|lar deposits. courts sleep carefully. furiously express ideas boost furiously after the final, regular foxes. furiously bold deposits are. express accounts haggle blithely. |
+138|9|7907|119.83|epitaphs? quickly express foxes use pending accounts. special packages cajole blithely among the quickly unusual accounts? boldly ironic packages across the slyly ironic senti|
+138|4|967|309.03|pendencies integrate against the unusual pains. carefully unusual theodolites wake quickly across the deposits. blithely regular deposits alongside of the carefully regular deposits|
+139|10|2886|285.75|fully ironic requests according to the quickly final idea|
+139|5|9255|684.61|ickly furiously regular excuses. boldly express deposits sleep. ideas nag above the silent dependencies. slyly regular packages wake furiously. requests are carefully. quickly final fox|
+139|10|1042|972.23|gular, regular theodolites. regular asymptotes haggle carefully according to the permanently even deposits. slyly special account|
+139|5|3285|690.00|xpress pains. quickly regular ideas after the special, bold excuses wake furiously final ideas. slyly bold accounts nag packages. ironically regular|
+140|1|2379|501.05|of the silent, bold courts. slyly regular dependencies haggle. fluffily special deposits cajole carefully. quickly ironic depos|
+140|6|3533|781.45|ayers. carefully ironic pinto beans nod carefully furiously regular pinto beans. slyly ironic requests after the carefully regular packages are about the blithel|
+140|1|304|45.84|ing requests. carefully unusual foxes are final requests. slyly regular accounts wake permanently. quickly ironic theodolites hagg|
+140|6|7346|429.52| special pinto beans wake carefully unusual warthogs! furi|
+141|2|6776|293.63|fluffily unusual courts sleep. close pinto beans haggle quickly after the carefully ir|
+141|8|1660|139.18|egular accounts. enticingly bold theodolites eat slyly across the never ironic platelets. theodolites wake bli|
+141|4|7628|838.08|sly about the pinto beans. blithely ironic ideas sleep. foxes are quietly among the pinto beans. carefu|
+141|10|90|810.68|e doggedly regular ideas. foxes haggle slyly. slyly regular theodolites across the carefu|
+142|3|9219|897.49|thlessly special requests sleep blithely about the bold deposits. express, ironic instructions wake. final packages are blithely. deposits are carefully furiously even deposits. furiously regular a|
+142|9|13|334.33|are blithely blithely brave requests. slyly regular theodolites are furiously. blithely ironic dependencies haggle blithely. furiously unu|
+142|5|3076|860.55|gular requests about the pending packages wake furiously dogged accounts. th|
+142|1|3858|854.08|efully special deposits. blithely bold pinto beans haggle. slyly final ideas boost blithely. finally special requests mold along the blithely express packages. entic|
+143|4|7326|960.64|the slyly pending requests cajole quickly blithely regular platelets. even requests boost carefully. ironic, final instructions above the regular courts boost a|
+143|10|3923|741.01|le quickly furiously silent ideas. carefully regular requests ar|
+143|6|7152|772.24|fully furious accounts. final asymptotes cajole regular requests. carefully regular courts are quickly. slyly ironic ideas above the carefully regular requests wake|
+143|2|1952|199.37|l accounts are quickly after the unusual packages. regular accounts wake among the quickly even accounts. even, ironic|
+144|5|6295|457.37| pinto beans promise across the blithely bold packages. express, regular accounts play around the slyly silent deposits. specia|
+144|1|494|849.96|uriously ironic pearls wake idly furiously even pearls. foxes impress slyly busily express requests. carefully slow somas wake quick|
+144|7|1799|713.88|yly final requests. packages are. carefully daring accou|
+144|3|5427|361.83| foxes integrate carefully. deposits cajole fluffily. pending deposits kindle slyly carefully regular packages. even, thin accounts according to th|
+145|6|11|641.67| slyly regular packages are slyly carefully special dolphins. unusual braids use furiously about the final courts. slyly special|
+145|2|3838|568.91|ss, final asymptotes are. furiously express accounts run. furiously express dependencies eat carefully blithely ironic theodolites. closely ironic foxes among the silent asymptotes cajole|
+145|8|1344|422.60| dependencies. even patterns detect slyly after the ironic deposits. ironically even ideas wake slyly. even packages against the blithely express accounts haggle furiously carefully regular|
+145|4|4332|894.57|are carefully above the quickly silent deposits. evenly bold reque|
+146|7|2726|231.15|uriously after the fluffy accounts. furiously bold deposits cajole. requests might engage. quick accounts wake carefu|
+146|3|4628|123.91|ly across the dependencies. daringly ironic deposits are furiously; requests are. quickly regular accounts hang. carefu|
+146|9|5893|858.59|sual instructions believe. fluffily unusual warhorses nag. unusual dependencies sleep. slow sheaves haggle furiously. carefully ironic dependencies cajole slyly against the accounts. |
+146|5|9687|882.37|packages? ideas affix slyly even accounts: express requests wake slyly carefully special depths. ironic in|
+147|8|7442|939.14|sts against the furiously unusual instructions integrate ironic accounts. slyly final pinto beans sleep blithely carefully final asymptotes. slyly ironic warhorses befor|
+147|4|7647|102.19|refully regular orbits about the furiously express asymptotes haggle carefully according to the blithely regular ideas. blithely express excuses around the furiously |
+147|10|1596|466.37|ole. slyly final packages do haggle quickly. unusual accounts across the pending pinto beans was furiously according to the furiously brave deposits. pending deposits along the regular request|
+147|6|4235|235.91|ar pinto beans. regular instructions sleep carefully after the furiously blithe accounts. slowly pending ideas could nag careful, even accounts. attainments use slyly quickly|
+148|9|8007|177.40|final requests-- slyly regular theodolites haggle carefully across the blithely final dependencies. slyly even requests about the carefully even accounts sleep |
+148|5|1904|774.56|y even pinto beans. fluffily ironic packages sleep slyly. permanently brave requests boost furiously packages. boldly ironic deposits across the carefully bold pinto b|
+148|1|5393|264.09|ses. slyly pending packages haggle fluffily fluffily even instructions. fluffily regular packages are carefully about the furiously even asymptot|
+148|7|2957|250.98|. requests boost above the bold, special foxes. blithely regular platelets serve blithely slyly final ideas. carefully special idea|
+149|10|959|679.10|y to sleep carefully ironic requests. even, regular dependencies haggle. slyly unusual foxes haggle along the instructions. quickly even accounts nag furiously special accoun|
+149|6|7283|201.03|usly bold instructions. regular, final deposits alongside of the furiously ironic platelets are slyly even instructions. carefully bold accounts are. ironic, regular requests nag furious|
+149|2|7392|266.53|es detect along the regular instructions. bold ideas boost slyly. quickly unusual accounts doubt. carefully even foxes thrash slyly silent, ironic dolphins: Tiresias must wake |
+149|8|4104|312.37|ly express excuses. bold pinto beans boost blithely across the bold, final pinto beans. final deposits haggle carefully from the|
+150|1|8091|524.71|sleep furiously furiously bold warthogs. furiously express gifts according to the regularly silent sentiments boost within the f|
+150|7|2721|814.79|dependencies. special accounts wake carefully furiously regular accounts. regular accounts haggle along the express instructions. express pinto beans along the express, bold deposits run |
+150|3|3172|33.71| about the silent ideas. fluffily final requests impress. slyly final requests wake carefully about the slyly express foxes. slyly regular warthogs sleep fur|
+150|9|1640|327.34|slyly even deposits alongside of the furiously even accounts detect boldly quickly regular accounts. final accounts kindle carefu|
+151|2|391|281.90|dolites. boldly ironic packages cajole fluffily regular instructions. regular, ironic accounts are blithely. ironic accounts are alongside of th|
+151|9|253|840.06| haggle. somas are carefully. slyly regular requests sleep blithely atop the thinly express deposits. stealthily express packages cajole daringly express requests. carefully special requests after t|
+151|6|1484|71.68|riously final requests sleep according to the regular deposits? slyly ironic ideas wake furiously. quickly even theodolites use fluffily. regular, unusual courts according to the regular |
+151|3|9417|244.06| foxes along the hockey players are slyly about the blithely even packages. unusu|
+152|3|1396|164.60|d the instructions. carefully pending accounts haggle fluffily ruthless instruc|
+152|10|2653|432.12|carefully pending requests. quickly ironic requests haggle carefully special theodolites. blithely special requests aga|
+152|7|3599|77.38| quick excuses according to the pending, ironic requests snooze carefully slyly even foxes: slyly regular instru|
+152|4|1283|142.73|olites above the furiously even requests dazzle blithely against the busy, regular pains. furiously blit|
+153|4|4695|539.86|atelets. dolphins haggle blithely carefully ironic deposits? express, final accounts wake about the requests. even deposits should use quickly. regular,|
+153|1|8464|680.14|cording to the final instructions. carefully fluffy asymptotes haggle carefully |
+153|8|2916|685.52|ully express deposits boost daringly packages. furiously ironic accounts sleep slyly ironic instructions. special deposits integrate blithely. |
+153|5|6397|285.92|furiously special platelets haggle quickly even, bold pinto beans. blithely close pinto beans boost around the furiously regular packages. quickly express requests cajole.|
+154|5|3961|474.19|quickly pending requests nag express dependencies. furiously unusual requests about the regular, pending packages wake according to the ironic packages! theodolites wake about the unusual, regula|
+154|2|3231|829.29|ins along the packages use carefully requests. furiously unusual packages kindle fluffily quick|
+154|9|7315|960.19|uickly regular dolphins ought to believe among the q|
+154|6|1682|160.31|refully except the sly, even requests. careful ideas haggle after the slyly regular foxes: slyly special packages at the slyly regular deposits wake carefully theod|
+155|6|2893|222.02|. bold packages are toward the silent pinto beans. quickly fin|
+155|3|7077|413.24|lar instructions against the furiously unusual instructions breach furiously for the bold, even platelets. ironic accounts must have to are quickly across the |
+155|10|3029|413.02|the carefully ironic asymptotes. even, unusual accounts sleep furiously about the blithely regular ideas. quickly re|
+155|7|977|751.45|quickly silent deposits doubt above the unusual instructions. special r|
+156|7|4755|453.69|e of the excuses. slyly even theodolites boost about the slyly final foxes? requests after the carefully regular platelets sleep above the furiously pending d|
+156|4|7228|994.19|odolites wake quickly slyly final dinos. requests cajole slyly along the instructions. furiously regular deposits cajole slyly blithely ironic instructions. instructions wake. blithely even pinto be|
+156|1|3043|252.66|lithely express, silent decoys. bold, special requests along the carefully even accounts|
+156|8|3191|9.87|ar instructions-- quickly special deposits wake fluffily about the blithely e|
+157|8|5414|369.44|ong the carefully bold ideas boost across the regular, ironic requests. ironic fo|
+157|5|763|568.46|cial packages boost along the ideas. packages sleep slyly express packages. ironic, bold requests|
+157|2|3718|901.53|, ironic foxes. blithely even foxes wake about the carefully special req|
+157|9|3400|288.41|encies integrate carefully even accounts. regular, regular sentiments are against the slyly regular deposits-- even, even ideas use inside the carefull|
+158|9|2845|408.72|y. slyly final pinto beans believe fluffily pending, regular deposits. final, unusual ideas according to t|
+158|6|8715|452.31|instructions along the ironic, final requests are fluffily regular deposits. regular deposits cajole carefully about the silent instructions|
+158|3|4465|837.16| wake carefully in place of the furiously express deposits. slyly regular instructions engage. fluffily f|
+158|10|4251|431.90|etly special accounts boost carefully final multipliers. carefu|
+159|10|9200|356.66|ccording to the furiously final accounts. carefully fluffy foxes wake idly against the quickly final requests. evenly even pinto beans must have to are against the carefully regular de|
+159|7|3585|629.29|g platelets wake furiously slyly bold deposits? slyly regular accounts across the stealthily ironic accounts cajole along the special, ironic pearls. fluffily regular pinto|
+159|4|6085|171.40|ross the blithely special deposits are quickly carefully ironic Tiresias. quickly regular deposits was furiously. unusual accounts affix blithely about the regular deposits. asymptotes ab|
+159|1|3717|790.87|y blithe dependencies. final accounts haggle furiously. even, special asymptotes|
+160|1|2434|525.73|lithely. furiously silent theodolites after the ca|
+160|8|8324|999.93|ly final instructions. closely final deposits nag furiously alongside of the furiously dogged theodolites. blithely unusual theodolites are furi|
+160|5|6034|733.59| furiously against the final instructions. silent accounts sleep blithely after the boldly final requests. ex|
+160|2|6872|872.20|ions are carefully. carefully express foxes nag slyly before the carefully final excuses. accounts after the furiously ironic packages are furio|
+161|2|9365|790.03|scapades. packages use. slyly final accounts haggle across the quickly final th|
+161|10|8421|394.05|cial ideas. ironic instructions eat blithely slyly special packages. furiously final packages alongside of the furiously final instructions boost carefully against the quickly |
+161|8|9679|688.47|ns. blithely express requests sleep slyly foxes. blithely unusual ideas |
+161|6|679|893.72| the fluffily final requests. ironic, pending epitaphs affix slyly. qui|
+162|3|315|923.04| ideas. carefully final dugouts will have to wake quickly regular asymptotes. express grouches unwind carefully after the regula|
+162|1|2604|104.20|usly regular excuses. silent, even sheaves are according to the regular requests. packages grow blithely slyly regular accounts. ca|
+162|9|7014|236.07|cording to the stealthily fluffy theodolites. carefully unusual excuses around the regular deposits cajole slyly amo|
+162|7|4381|824.36|as across the furiously ironic notornis print blithely alongside of the final, pending deposits. fluffily express deposits slee|
+163|4|9241|955.81|cial dolphins. furiously bold foxes could have to use. never sly accounts cajole fluffily about the unusual, special pinto beans. pending, even requests around the quickly special deposits use f|
+163|2|3427|499.51|ithely bold packages integrate slyly quiet pinto beans. carefully even deposits boost slyly about the furiously fluffy packages. evenly regular dependencies wa|
+163|10|5323|920.75|sly even theodolites against the carefully bold packages wake final pinto beans. furiously pending deposits dazzle furiously. blithely exp|
+163|8|9676|573.48|ending accounts haggle blithely ironic, even packages. carefully pending packages wake carefully across the ruthlessly pending accounts! pinto beans wake. slyly final deposits boost slyly. fluffily|
+164|5|1295|341.95| bold instructions cajole slyly ironic deposits. quickly ironic foxes are carefully final, bold theodolites. ironic deposi|
+164|3|2134|84.02|ns believe. carefully express theodolites impress. carefully fina|
+164|1|3245|814.67|brave accounts cajole according to the final platelets. furiously final dolphins across the furi|
+164|9|3028|64.89|fully furiously regular requests. furiously bold orbits serve about the regular packages? carefully final deposits p|
+165|6|4424|943.82|ular requests. regular accounts cajole against the blithely ironic deposits. blithely even packages cajole. furiously final deposits cajole. thinly pending deposits hagg|
+165|4|5534|717.83| quickly regular deposits above the fluffily thin deposits haggle furiously against the quickly final depend|
+165|2|3780|730.28| furiously quickly regular foxes. pending requests engage evenly blithel|
+165|10|6114|210.84|foxes. foxes haggle. dolphins use carefully according to the fluffily regular packages. blithely special accounts according to the slyly final frets breach blithely after the care|
+166|7|6527|309.00|lly. dependencies haggle carefully at the slyly special packages. regular, final packages|
+166|5|6508|714.49|y express deposits cajole furiously above the carefully even theod|
+166|3|9364|581.52|pinto beans. pinto beans cajole furiously carefully special requests-- quickly |
+166|1|6713|631.58| sleep carefully. quickly even deposits run carefully fluffily ironic orbits. ironic deposits wake furiously. close sheaves along the special packages sleep carefully special instr|
+167|8|4933|666.70|ular deposits among the even dolphins are quickly express accounts. final, ironic theodolites cajole closely. th|
+167|6|5789|524.27| are furiously final, even dugouts. ironic, regular packages nag fu|
+167|4|4756|336.75|es are carefully along the carefully express tithes. furiously even deposits cajole slyly slyly regular deposits. bold excuses about the carefully ironic requests sleep blithely instructions|
+167|2|6748|704.97|t the silent ideas are blithely carefully even packages; blithely|
+168|9|347|394.83|hely blithely final theodolites. blithely final deposits among the quickly even ideas haggle about the blithely bold d|
+168|7|1281|771.90|, pending packages. ironic pinto beans use carefully. fluffily bold deposits|
+168|5|9089|508.37|ests are always. regular ideas sleep fluffily; special, express instructions cajole slowly. pending platelets boost furiously against the bold, even instructions. bold instructi|
+168|3|7519|963.43|requests above the quickly regular deposits use carefully aft|
+169|10|6914|619.53|uickly along the dependencies. furiously pending notornis cajole at the carefully special attainments. carefully ironic packages impress slyly care|
+169|8|6589|947.03|gside of the quickly regular asymptotes. quickly even theodolites against the theodolites promise express requests. ironic accounts wake careful|
+169|6|6731|713.35| the quickly special excuses wake blithely alongside of the carefully silent accounts. regular dolphin|
+169|4|7691|476.19|slyly alongside of the warthogs. fluffily even instructions poach under the slyly pending packages. blithely silent deposits use across the fur|
+170|1|7516|581.65| pinto beans. unusual ideas was fluffily. excuses cajole carefully final dependencies. platelets nag quickly according to the furiously ironic requests. carefully regular dependenci|
+170|9|838|667.16|orges do sleep furiously. fluffily furious requests among the final requests sleep after the slyly bold ideas? regular pinto beans might ha|
+170|7|6498|251.19| fluffily regular accounts integrate. blithely even packages cajole fluffily. furiously ironic excuses haggle by the finally final requ|
+170|5|6593|202.07|ep blithely final packages. quickly bold pains cajole carefully across the somet|
+171|2|8217|859.60|ress deposits. carefully special requests are furiously final requests. accounts cajole carefully blith|
+171|1|2311|864.96|s are along the blithely final deposits. regular asymptotes nag slyly against the requests. accounts cajole carefully carefully |
+171|10|8561|22.69|y close ideas are quickly silently regular packages. even, silent requests wake against the slyly special dependencies; regular accounts sleep doggedly furiously final pinto beans. slyly unusual pac|
+171|9|7589|935.29|s above the theodolites wake slyly along the carefully unusual dependencies. carefully express theodolites a|
+172|3|9799|184.96|ts. slyly even asymptotes nag blithely regular accounts. final platelets cajole furiously slyly bold packages. ironic accounts sleep slyly. pendi|
+172|2|8333|920.74|ronic foxes. quickly unusual accounts cajole blithely. blithely bold deposits cajole. blithely close pinto beans cajole requests. quickly express excuses around the quickly even deposits nag agai|
+172|1|3589|437.86|posits should have to boost furiously near the unusual ideas. final packages cajole blithely. carefully final deposits boost carefully. carefully special attainments boost quickly af|
+172|10|1661|687.13|y among the slyly even requests. ideas according to the slyly pending dinos print quickly slyly ironic foxes. pending, even excuses dazzle car|
+173|4|2536|353.84|ons-- final, silent dependencies sleep across the special, special excuses. furiously even accounts must have to mold after the ironic accounts. reque|
+173|3|8307|70.22|alongside of the furiously even packages. furiously final requests snooze blithely alongside of the carefull|
+173|2|6050|683.78|e after the slyly regular ideas. unusual pinto beans cajole even asymptotes-- silent, stealthy requests after the even accounts haggle blithely regular instructions. slyly ev|
+173|1|6162|877.84|es. slyly bold requests after the blithely regular dependencies cajole slyly even ideas. unusual deposits integrate about the final somas. |
+174|5|2103|681.95|sual, express requests wake furiously ruthless, final accounts. carefully ironic somas dazzle furiously. unusual asymptotes sleep-- patterns about the furiousl|
+174|4|6795|143.48|regular theodolites. special accounts integrate across the carefully ironic Tiresias. blithely even platelets detect. foxes about t|
+174|3|111|135.46| express packages-- quickly unusual courts lose carefully requests. bold accounts solve about the theodolites; pinto beans use. ironic foxes|
+174|2|8404|126.20|nding accounts mold furiously. slyly ironic foxes boost express sheaves. daringly final packages along the stealthy dependencies are blithely ironic requests. furiously pending pin|
+175|6|5515|487.68|ages sleep against the Tiresias. slyly pending packages print slyly above the evenly ironic dolphins. furiously ironic packages use f|
+175|5|7522|784.93| affix. quickly final theodolites haggle furiously after the slowly even pinto beans. furiously final packages use slyly. slyly regular reque|
+175|4|8501|706.61|int above the instructions. furiously regular requests integrate blithely according to the instructions. slyly pending foxes are asymptotes. slyly ruthless accounts wake. r|
+175|3|9456|978.56| regular packages. carefully ironic packages use. blithely ironic accounts among the pending, |
+176|7|7180|179.09|riously final requests. accounts doubt blithely regular somas. slyly even platelets are. theodolites across |
+176|6|3589|157.38|inal excuses. express deposits haggle carefully even deposits. carefully unusual requests haggle along the fluffily bold deposits. even, final requests affix. furi|
+176|5|5407|947.51|ending accounts eat carefully instructions. carefully pending packages detect slyly express accounts. foxes wake fluffily across th|
+176|4|1783|861.63|g the carefully special platelets. dogged, ironic asymptotes wake requests. regular excus|
+177|8|1239|44.75|requests use furiously regular, final requests. regular requests on the pending, ironic deposits use slyly among the excuses. carefully regular sheaves are.|
+177|7|4349|63.36|osits sleep among the fluffily unusual instructions. ironic dolphins cajole. furiously bold deposits sleep carefully. even, unusual accounts|
+177|6|9872|252.42|sual platelets. bold foxes affix furiously. pending, pending accounts lose furiously. pending platelets along the unusual, even foxes wake regular, even theo|
+177|5|4727|859.82|es are. slyly ironic packages haggle around the slyly bold deposits. bold foxes haggle blithely. f|
+178|9|4231|558.56|deposits. patterns use against the furiously unusual accounts. accounts wake carefully above the careful|
+178|8|1919|362.26| ironic dependencies. blithely regular packages detect fluffily special theodolites. regular instructions poach-- ironic deposits along the final requests |
+178|7|6836|864.93|y. ideas integrate regular pinto beans. special foxes wake above the slyly ironic asymptotes. quickly ironic ideas sleep. silent dependencies against the slyly bold packa|
+178|6|6922|475.18| regular patterns. fluffily express accounts about the furiously bold deposits cajole slyly about the furiously silent foxe|
+179|10|6956|444.38|g the furiously careful excuses haggle quickly thinly special Tiresias. furiously express foxes after the quickly regular deposits sleep ironic packages|
+179|9|1954|372.75|even dependencies print carefully. deposits boost blithely about the ironic, ironic accounts. express, regular deposits are. bli|
+179|8|2710|277.15|d the frets. pending packages doze quickly across the furiously regular deposits. pending, even deposits impress ironic ideas. quickly regular r|
+179|7|4776|8.39|sly special pinto beans. pinto beans cajole. carefully unusual ideas around the silent accounts are blithely carefully ev|
+180|1|2467|440.25| instructions affix. regular packages cajole quickly. carefully express asymptotes use furiously around the pendin|
+180|10|1108|934.59|hinly after the regular, unusual asymptotes! carefully regular theodolites sublate. regular, ironic deposits against the regular pinto beans nag ca|
+180|9|724|426.16|e, regular accounts. furiously final ideas are furiously above the bold, silent asymptotes. sly instructions are carefully quickly final sentiments. furiously ironic foxes cajole bold, exp|
+180|8|5899|864.83|hin the carefully furious pinto beans. furiously ironic pinto beans use slyly above the even instructio|
+181|2|2416|844.44|ully. theodolites throughout the blithely unusual pinto bea|
+181|2|3242|886.53| express ideas nag carefully brave accounts. slyly express deposits would affix. final, special requests against the slyl|
+181|2|215|938.29| accounts boost furiously furiously blithe theodolites. slyly bold requests unwind special, unusual requests. furious ideas boost quickly pending |
+181|2|1122|657.25|lyly fluffily pending foxes. fluffily ironic pains haggle. thinly regular requests against the deposits affix after the never ev|
+182|3|9699|535.27|ound the furiously regular foxes. pending requests dazzle along |
+182|3|960|519.36|arefully pending dependencies are always slyly unusual pin|
+182|3|6243|741.46|accounts are slyly. furiously ironic requests haggle. express, special instructions against the ironic theodolites use s|
+182|3|6146|365.00|s. blithely express theodolites sleep blithely alongside of the requests?|
+183|4|30|875.44|slyly. furiously regular instructions cajole slyly about the pending, final theodolites. blithely final deposits cajole fluffily alo|
+183|4|4482|424.86|es. depths affix fluffily. bold instructions haggle. ruthless instructions must have to boost|
+183|4|8707|884.26|posits wake. blithely pending requests nag furiously alongside of the p|
+183|4|333|678.16|ost final, final theodolites. slyly bold foxes dazzle carefully furiously regular accounts. regular, sly instructions about the furiously regular excuses nag blithely abou|
+184|5|7069|449.45|nal ideas. blithely final ideas haggle against the pinto beans. qu|
+184|5|9193|576.88|uickly quick dependencies could detect furiously. final packages p|
+184|5|6400|551.90|ss dependencies. quickly even pinto beans are. express accounts a|
+184|5|831|186.84|kages cajole carefully furiously ironic instructions. deposits use bl|
+185|6|1475|538.58|unts hinder slyly. quickly express ideas sleep carefully |
+185|6|6244|213.04|ly unusual decoys are furiously quickly regular packages. bold, ironic foxes cajole fluffily around|
+185|6|7245|426.74|sleep blithely alongside of the regular excuses. even, regular|
+185|6|8014|510.23|lithely even ideas. regular platelets wake carefully ironic, special instructions! final pearls above the fluffily quiet ideas use furiously about the |
+186|7|1095|252.84|. carefully regular pinto beans according to the blithely close asymptotes haggle carefully special requests. packages cajole up the furi|
+186|7|1945|18.75|nic foxes boost carefully careful packages: express, fluffy dolphins nag quickly ironic packages. slyly bold requests nag amon|
+186|7|8838|729.42|ing asymptotes. enticingly regular theodolites mai|
+186|7|7898|812.37|ctions sleep silently carefully bold platelets. furiously ironic dependencies boost. regular de|
+187|8|8656|238.66|tes use along the even foxes? final foxes haggle pinto beans. slyly ironic theodolites are according to the deposits. furiously pending reques|
+187|8|4945|316.64|eposits boost quickly bold requests. furiously regular ideas boost boldly. special, express dependencies are fluffily slyly reg|
+187|8|3183|362.75|t the bold platelets. fluffily express platelets cajole fluffily along the always bold requests. blith|
+187|8|7440|989.71|e slyly against the slyly regular pinto beans. requests haggle carefully around the asymptotes. regular, regular asymptotes use furiously some|
+188|9|4835|771.95|pains are fluffily about the fluffily pending asymptot|
+188|9|2620|331.70|elets nag slyly regular pinto beans. slyly even dugouts above the blithely unusual theodolites su|
+188|9|730|713.62|nag against the final accounts. blithely pending attainments lose. silent requests wake quickly. careful|
+188|9|5430|920.20|uriously. special, regular instructions sleep along the accounts. quickly even foxes across the regular theodolites hang u|
+189|10|1305|392.50|packages. regular, unusual accounts lose furiously fluffily regular platelets. requests sleep carefully dependenc|
+189|10|8777|573.22|beans cajole slyly ironic requests. requests are quickly unusual, even packages. ironic frays haggle. blithely pending requests nod slyly. express, silent requests against the slyly unusual |
+189|10|6369|946.07|ts hinder slyly regular, unusual foxes. final sentiments use above the slyly r|
+189|10|2505|593.23| the deposits. special deposits sleep-- furiously regular sauternes solve furiously across the furiously regular pack|
+190|1|535|621.53|unts must have to haggle; slyly ironic accounts affix slyly alongside of the carefully even accounts. furious deposits haggle quietly among the packages. blithely |
+190|1|5845|608.91| haggle along the carefully unusual pinto beans. quickly final accounts sleep a|
+190|1|4579|396.60|inal, final foxes. regular, even deposits wake blithely! silent, regular packages integrate according to the slyly regular deposits. ironic, ironic notornis ha|
+190|1|2861|458.00|s cajole slyly across the daring, final pinto beans. carefully quiet requests affix along the a|
+191|2|8310|521.06|the slowly regular deposits. special accounts along the quickly unusual|
+191|3|1693|464.46|y. slyly unusual waters across the special pinto beans nag blithely according to the busy deposits. carefully regular accounts are against the regular accounts; perman|
+191|4|597|126.96|ly final accounts should have to boost above the doggedly express pinto beans. blithely regular packages cajole furiously bold requests. fluf|
+191|5|9673|119.41|press deposits kindle theodolites! slyly final dependencies against the blithely final packages sleep slyly regular requests. theodolites cajole furiously quickly bold a|
+192|3|606|198.69|inal platelets integrate regular accounts. accounts wake ironic, silent accounts. slyly unusual accounts kindle carefully-|
+192|4|2656|916.16|uickly. slyly bold ideas affix special, close theodolites. ironic, pending requests use carefully. blithely regular |
+192|5|1811|359.59|ly carefully special asymptotes. furiously pending instructions haggle blithely bravely pending requests. carefully f|
+192|6|8305|861.23|s against the carefully regular foxes haggle fluffily across the pending accounts. blithely final packages sleep after the furiously ironic theodolites. quickly bold r|
+193|4|6184|335.98| quickly bold deposits cajole furiously ruthless courts. carefully|
+193|5|4762|606.19|ns sleep against the furiously regular asymptotes. carefully even asymptotes across the daringly final packages sleep fluf|
+193|6|385|571.71|ons. slyly ironic deposits wake furiously ironic, unus|
+193|7|9791|478.52|quests. carefully even requests use regular excuses. pending accounts are. furiously even pinto beans haggle furi|
+194|5|4289|662.17|ic Tiresias serve along the ironic, express accounts. quickly final requests are slyly among the carefully special requests. accounts boost.|
+194|6|377|430.21|efully instead of the special ideas. fluffily unusual asymptotes cajole blithely after the regular ideas. final accounts along the silent ex|
+194|7|5294|913.46|indle fluffily despite the carefully silent instructions. furiously regular hockey players cajole slyly unusual accounts. furiously regular realms cajole furiously according to the e|
+194|8|7890|79.40|ctions sleep. carefully unusual theodolites should wake furiously across the deposits-- furiously bold excuses boost furiously carefully slow accounts. boldly final accounts grow. regular excuse|
+195|6|9985|20.39|efully among the fluffily even accounts! requests are slyly ag|
+195|7|2947|271.39|yly regular requests cajole carefully. carefully fina|
+195|8|319|102.58|ts. ironic foxes wake carefully slyly special pinto beans. blithely silent excuses hinder blithely quietly regular accounts. quickly careful foxes maintain slyly above the slyly express fo|
+195|9|2803|992.27|xes according to the regular foxes wake furiously final theodolites. furiously regular packages sleep slyly express theodolites. slyly thin instructions sleep r|
+196|7|3843|859.90|l platelets use blithely alongside of the enticingly final deposits. fluffily final requests boost furiously ag|
+196|8|2515|966.01|final theodolites. fluffily even deposits are against the|
+196|9|4778|37.61|fully final requests cajole fluffily across the furiously ironic accounts. qui|
+196|10|1068|928.25| cajole about the blithely regular ideas. final ideas hin|
+197|8|9678|753.88|ously. slyly stealthy requests use alongside of the express, unusual packages. final deposits wake. carefully unusual theodolites cajole slyly about the regular foxes. slyly iron|
+197|9|2631|279.05|e blithely. quickly final deposits wake fluffily excuses. even, unusual deposits x-ray among the final accounts. even ideas above the blithely ironic requests sleep furiously slyly final inst|
+197|10|7598|845.51|lets according to the regular deposits wake furiously about the carefully daring theodolites. blithely express dolphins poach after th|
+197|1|8950|897.33|ideas. requests wake above the blithely unusual deposits. slyly regular |
+198|9|6878|587.41|y even accounts poach carefully about the asymptotes. deposits haggle slyly. finally unusual requests run silently regular, bold packages: instructions after the |
+198|10|6493|673.99|y express excuses use blithely among the pending accounts. stealthy ide|
+198|1|8410|166.93|kages. blithely final theodolites dazzle fluffily. accounts boost furiously. furiously unu|
+198|2|6190|697.10|beans nag fluffily about the asymptotes. slyly bold escapades haggle quickly. fluffily special requests haggle above the ironic,|
+199|10|9343|79.70|ending accounts nag across the instructions. carefully express packages over the blithely even pac|
+199|1|8199|46.52|oost slyly. ironic platelets sleep blithely about the slyly silent foxes. furiously even pl|
+199|2|2742|890.63| the special deposits? carefully final deposits about the carefully regular sauternes |
+199|3|7167|884.56|onic platelets use carefully along the slowly stealthy ideas. slyly dogged instructions are quickly above the slyly u|
+200|1|3120|776.41|ntly final packages kindle furiously blithely ironic accounts. carefully final packages according to the carefully |
+200|2|5392|242.52|y unusual ideas. ruthlessly express asymptotes cajole. regular theodolites are. carefully silent deposits poach carefully across the fluffily even theodolites. carefully express realms hag|
+200|3|9408|307.79|oxes! fluffily regular requests use against the unusual, slow ideas. ironic accounts doze b|
+200|4|331|466.07| slyly even requests. fluffily final packages boost carefully express instructions. slyly regular forges are blithely unusual, regular |
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/region.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/region.tbl
new file mode 100644
index 0000000..c5ebb63
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/region.tbl
@@ -0,0 +1,5 @@
+0|AFRICA|lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to |
+1|AMERICA|hs use ironic, even requests. s|
+2|ASIA|ges. thinly even pinto beans ca|
+3|EUROPE|ly final courts cajole furiously final excuse|
+4|MIDDLE EAST|uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/supplier.tbl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/supplier.tbl
new file mode 100644
index 0000000..d9c0e9f
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/supplier.tbl
@@ -0,0 +1,10 @@
+1|Supplier#000000001| N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ|17|27-918-335-1736|5755.94|each slyly above the careful|
+2|Supplier#000000002|89eJ5ksX3ImxJQBvxObC,|5|15-679-861-2259|4032.68| slyly bold instructions. idle dependen|
+3|Supplier#000000003|q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3|1|11-383-516-1199|4192.40|blithely silent requests after the express dependencies are sl|
+4|Supplier#000000004|Bk7ah4CK8SYQTepEmvMkkgMwg|15|25-843-787-7479|4641.08|riously even requests above the exp|
+5|Supplier#000000005|Gcdm2rJRzl5qlTVzc|11|21-151-690-3663|-283.84|. slyly regular pinto bea|
+6|Supplier#000000006|tQxuVm7s7CnK|14|24-696-997-4969|1365.79|final accounts. regular dolphins use against the furiously ironic decoys. |
+7|Supplier#000000007|s,4TicNGB4uO6PaSqNBUq|23|33-990-965-2201|6820.35|s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit|
+8|Supplier#000000008|9Sq4bBH2FQEmaFOocY45sRTxo6yuoG|17|27-498-742-3860|7627.85|al pinto beans. asymptotes haggl|
+9|Supplier#000000009|1KhUgZegwM3ua7dsYmekYBsK|10|20-403-398-8662|5302.37|s. unusual, even requests along the furiously regular pac|
+10|Supplier#000000010|Saygah3gYWMp72i PY|24|34-852-489-8585|3891.91|ing waters. regular requests ar|
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/tpch.ddl b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/tpch.ddl
new file mode 100644
index 0000000..89efe56
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/data/tpch0.001/tpch.ddl
@@ -0,0 +1,70 @@
+-- Sccsid: @(#)dss.ddl 2.1.8.1
+CREATE TABLE NATION ( N_NATIONKEY INTEGER NOT NULL,
+ N_NAME CHAR(25) NOT NULL,
+ N_REGIONKEY INTEGER NOT NULL,
+ N_COMMENT VARCHAR(152));
+
+CREATE TABLE REGION ( R_REGIONKEY INTEGER NOT NULL,
+ R_NAME CHAR(25) NOT NULL,
+ R_COMMENT VARCHAR(152));
+
+CREATE TABLE PART ( P_PARTKEY INTEGER NOT NULL,
+ P_NAME VARCHAR(55) NOT NULL,
+ P_MFGR CHAR(25) NOT NULL,
+ P_BRAND CHAR(10) NOT NULL,
+ P_TYPE VARCHAR(25) NOT NULL,
+ P_SIZE INTEGER NOT NULL,
+ P_CONTAINER CHAR(10) NOT NULL,
+ P_RETAILPRICE DECIMAL(15,2) NOT NULL,
+ P_COMMENT VARCHAR(23) NOT NULL );
+
+CREATE TABLE SUPPLIER ( S_SUPPKEY INTEGER NOT NULL,
+ S_NAME CHAR(25) NOT NULL,
+ S_ADDRESS VARCHAR(40) NOT NULL,
+ S_NATIONKEY INTEGER NOT NULL,
+ S_PHONE CHAR(15) NOT NULL,
+ S_ACCTBAL DECIMAL(15,2) NOT NULL,
+ S_COMMENT VARCHAR(101) NOT NULL);
+
+CREATE TABLE PARTSUPP ( PS_PARTKEY INTEGER NOT NULL,
+ PS_SUPPKEY INTEGER NOT NULL,
+ PS_AVAILQTY INTEGER NOT NULL,
+ PS_SUPPLYCOST DECIMAL(15,2) NOT NULL,
+ PS_COMMENT VARCHAR(199) NOT NULL );
+
+CREATE TABLE CUSTOMER ( C_CUSTKEY INTEGER NOT NULL,
+ C_NAME VARCHAR(25) NOT NULL,
+ C_ADDRESS VARCHAR(40) NOT NULL,
+ C_NATIONKEY INTEGER NOT NULL,
+ C_PHONE CHAR(15) NOT NULL,
+ C_ACCTBAL DECIMAL(15,2) NOT NULL,
+ C_MKTSEGMENT CHAR(10) NOT NULL,
+ C_COMMENT VARCHAR(117) NOT NULL);
+
+CREATE TABLE ORDERS ( O_ORDERKEY INTEGER NOT NULL,
+ O_CUSTKEY INTEGER NOT NULL,
+ O_ORDERSTATUS CHAR(1) NOT NULL,
+ O_TOTALPRICE DECIMAL(15,2) NOT NULL,
+ O_ORDERDATE DATE NOT NULL,
+ O_ORDERPRIORITY CHAR(15) NOT NULL,
+ O_CLERK CHAR(15) NOT NULL,
+ O_SHIPPRIORITY INTEGER NOT NULL,
+ O_COMMENT VARCHAR(79) NOT NULL);
+
+CREATE TABLE LINEITEM ( L_ORDERKEY INTEGER NOT NULL,
+ L_PARTKEY INTEGER NOT NULL,
+ L_SUPPKEY INTEGER NOT NULL,
+ L_LINENUMBER INTEGER NOT NULL,
+ L_QUANTITY DECIMAL(15,2) NOT NULL,
+ L_EXTENDEDPRICE DECIMAL(15,2) NOT NULL,
+ L_DISCOUNT DECIMAL(15,2) NOT NULL,
+ L_TAX DECIMAL(15,2) NOT NULL,
+ L_RETURNFLAG CHAR(1) NOT NULL,
+ L_LINESTATUS CHAR(1) NOT NULL,
+ L_SHIPDATE DATE NOT NULL,
+ L_COMMITDATE DATE NOT NULL,
+ L_RECEIPTDATE DATE NOT NULL,
+ L_SHIPINSTRUCT CHAR(25) NOT NULL,
+ L_SHIPMODE CHAR(10) NOT NULL,
+ L_COMMENT VARCHAR(44) NOT NULL);
+
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/pom.xml b/hyracks-algebricks/hyracks-algebricks-tests/pom.xml
new file mode 100644
index 0000000..587e9bc
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/pom.xml
@@ -0,0 +1,84 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-algebricks-tests</artifactId>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.7.2</version>
+ <configuration>
+ <forkMode>pertest</forkMode>
+ <argLine>-enableassertions -Djava.util.logging.config.file=src/test/resources/logging.properties</argLine>
+ <includes>
+ <include>**/*Test.java</include>
+ <include>**/*Suite.java</include>
+ </includes>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <configuration>
+ <tasks>
+ <ant antfile="build-script.xml" target="build">
+ <property name="main.class" value="edu.uci.ics.hyracks.algebricks.tests.script.IdentityStreamingScript" />
+ <property name="script.classpath" refid="maven.compile.classpath" />
+ <property name="jvm.params" value="" />
+ <property name="program.params" value="" />
+ <property name="source" value="${basedir}/src/main/scripts/run" />
+ <property name="target.dir" value="${basedir}/target/testscripts" />
+ <property name="target" value="idscript" />
+ </ant>
+ </tasks>
+ </configuration>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-algebricks-compiler</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>4.8.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-cc</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntArrayUnnester.java b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntArrayUnnester.java
new file mode 100644
index 0000000..ed98eec
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntArrayUnnester.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.tests.pushruntime;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunction;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class IntArrayUnnester implements IUnnestingFunctionFactory {
+
+ private int[] x;
+
+ public IntArrayUnnester(int[] x) {
+ this.x = x;
+ }
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IUnnestingFunction createUnnestingFunction(IDataOutputProvider provider) throws AlgebricksException {
+
+ final DataOutput out = provider.getDataOutput();
+
+ return new IUnnestingFunction() {
+
+ private int pos;
+
+ @Override
+ public void init(IFrameTupleReference tuple) throws AlgebricksException {
+ pos = 0;
+ }
+
+ @Override
+ public boolean step() throws AlgebricksException {
+ try {
+ if (pos < x.length) {
+ // Writes one byte to distinguish between null
+ // values and end of sequence.
+ out.writeInt(x[pos]);
+ ++pos;
+ return true;
+ } else {
+ return false;
+ }
+
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ };
+
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerAddEvalFactory.java b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerAddEvalFactory.java
new file mode 100644
index 0000000..57c0789
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerAddEvalFactory.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.tests.pushruntime;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class IntegerAddEvalFactory implements IEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private IEvaluatorFactory evalLeftFactory;
+ private IEvaluatorFactory evalRightFactory;
+
+ public IntegerAddEvalFactory(IEvaluatorFactory evalLeftFactory, IEvaluatorFactory evalRightFactory) {
+ this.evalLeftFactory = evalLeftFactory;
+ this.evalRightFactory = evalRightFactory;
+ }
+
+ @Override
+ public IEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new IEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
+
+ private IEvaluator evalLeft = evalLeftFactory.createEvaluator(argOut);
+ private IEvaluator evalRight = evalRightFactory.createEvaluator(argOut);
+
+ @SuppressWarnings("static-access")
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ argOut.reset();
+ evalLeft.evaluate(tuple);
+ int v1 = IntegerSerializerDeserializer.INSTANCE.getInt(argOut.getBytes(), 0);
+ argOut.reset();
+ evalRight.evaluate(tuple);
+ int v2 = IntegerSerializerDeserializer.INSTANCE.getInt(argOut.getBytes(), 0);
+ try {
+ out.writeInt(v1 + v2);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerConstantEvalFactory.java b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerConstantEvalFactory.java
new file mode 100644
index 0000000..880a4df
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerConstantEvalFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.tests.pushruntime;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class IntegerConstantEvalFactory implements IEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final int value;
+
+ public IntegerConstantEvalFactory(int value) {
+ this.value = value;
+ }
+
+ @Override
+ public String toString() {
+ return "IntConstantEvalFactory " + value;
+ }
+
+ @Override
+ public IEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new IEvaluator() {
+
+ private DataOutput out = output.getDataOutput();
+ private ArrayBackedValueStorage buf = new ArrayBackedValueStorage();
+ boolean first = true;
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ if (first) {
+ first = false;
+ try {
+ IntegerSerializerDeserializer.INSTANCE.serialize(value, buf.getDataOutput());
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ try {
+ out.write(buf.getBytes(), 0, buf.getLength());
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+ };
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerEqualsEvalFactory.java b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerEqualsEvalFactory.java
new file mode 100644
index 0000000..e15851b
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerEqualsEvalFactory.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.tests.pushruntime;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class IntegerEqualsEvalFactory implements IEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private IEvaluatorFactory evalFact1, evalFact2;
+
+ public IntegerEqualsEvalFactory(IEvaluatorFactory evalFact1, IEvaluatorFactory evalFact2) {
+ this.evalFact1 = evalFact1;
+ this.evalFact2 = evalFact2;
+ }
+
+ @Override
+ public IEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new IEvaluator() {
+ private DataOutput dataout = output.getDataOutput();
+ private ArrayBackedValueStorage out1 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage out2 = new ArrayBackedValueStorage();
+ private IEvaluator eval1 = evalFact1.createEvaluator(out1);
+ private IEvaluator eval2 = evalFact2.createEvaluator(out2);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ out1.reset();
+ eval1.evaluate(tuple);
+ out2.reset();
+ eval2.evaluate(tuple);
+ int v1 = IntegerSerializerDeserializer.getInt(out1.getBytes(), 0);
+ int v2 = IntegerSerializerDeserializer.getInt(out2.getBytes(), 0);
+ boolean r = v1 == v2;
+ try {
+ dataout.writeBoolean(r);
+ } catch (IOException ioe) {
+ throw new AlgebricksException(ioe);
+ }
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerGreaterThanEvalFactory.java b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerGreaterThanEvalFactory.java
new file mode 100644
index 0000000..5c5d6d4
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/IntegerGreaterThanEvalFactory.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.tests.pushruntime;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IDataOutputProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class IntegerGreaterThanEvalFactory implements IEvaluatorFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private IEvaluatorFactory evalFact1, evalFact2;
+
+ public IntegerGreaterThanEvalFactory(IEvaluatorFactory evalFact1, IEvaluatorFactory evalFact2) {
+ this.evalFact1 = evalFact1;
+ this.evalFact2 = evalFact2;
+ }
+
+ @Override
+ public IEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+ return new IEvaluator() {
+ private DataOutput dataout = output.getDataOutput();
+ private ArrayBackedValueStorage out1 = new ArrayBackedValueStorage();
+ private ArrayBackedValueStorage out2 = new ArrayBackedValueStorage();
+ private IEvaluator eval1 = evalFact1.createEvaluator(out1);
+ private IEvaluator eval2 = evalFact2.createEvaluator(out2);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+ out1.reset();
+ eval1.evaluate(tuple);
+ out2.reset();
+ eval2.evaluate(tuple);
+ int v1 = IntegerSerializerDeserializer.getInt(out1.getBytes(), 0);
+ int v2 = IntegerSerializerDeserializer.getInt(out2.getBytes(), 0);
+ boolean r = v1 > v2;
+ try {
+ dataout.writeBoolean(r);
+ } catch (IOException ioe) {
+ throw new AlgebricksException(ioe);
+ }
+ }
+ };
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/script/IdentityStreamingScript.java b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/script/IdentityStreamingScript.java
new file mode 100644
index 0000000..c914c7c
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/java/edu/uci/ics/hyracks/algebricks/tests/script/IdentityStreamingScript.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.tests.script;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+public class IdentityStreamingScript {
+
+ public static void main(String args[]) throws IOException {
+ BufferedReader r = new BufferedReader(new InputStreamReader(System.in));
+ String s;
+ while ((s = r.readLine()) != null) {
+ System.out.println(s);
+ }
+ r.close();
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/scripts/run.cmd b/hyracks-algebricks/hyracks-algebricks-tests/src/main/scripts/run.cmd
new file mode 100644
index 0000000..b8eb4a0
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/scripts/run.cmd
@@ -0,0 +1,63 @@
+@ECHO OFF
+SETLOCAL
+
+:: Licensed to the Apache Software Foundation (ASF) under one or more
+:: contributor license agreements. See the NOTICE file distributed with
+:: this work for additional information regarding copyright ownership.
+:: The ASF licenses this file to You under the Apache License, Version 2.0
+:: (the "License"); you may not use this file except in compliance with
+:: the License. You may obtain a copy of the License at
+::
+:: http://www.apache.org/licenses/LICENSE-2.0
+::
+:: Unless required by applicable law or agreed to in writing, software
+:: distributed under the License is distributed on an "AS IS" BASIS,
+:: WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+:: See the License for the specific language governing permissions and
+:: limitations under the License.
+
+:: JAVA classpath
+:: Use the local variable CLASSPATH to add custom entries (e.g. JDBC drivers) to
+:: the classpath. Separate multiple paths with ":". Enclose the value
+:: in double quotes. Adding additional files or locations on separate
+:: lines makes things clearer.
+:: Note: If under running under cygwin use "/cygdrive/c/..." for "C:/..."
+:: Example:
+::
+:: Set the CLASSPATH to a jar file and a directory. Note that
+:: "classes dir" is a directory of class files with a space in the name.
+::
+:: CLASSPATH="usr/local/Product1/lib/product.jar"
+:: CLASSPATH="${CLASSPATH}:../MyProject/classes dir"
+::
+SET CLASSPATH="@classpath@"
+
+:: JVM parameters
+:: If you want to modify the default parameters (e.g. maximum heap size -Xmx)
+:: for the Java virtual machine set the local variable JVM_PARAMETERS below
+:: Example:
+:: JVM_PARAMETERS=-Xms100M -Xmx200M
+::
+:: Below are the JVM parameters needed to do remote debugging using Intellij
+:: IDEA. Uncomment and then do: JVM_PARAMETERS="$IDEA_REMOTE_DEBUG_PARAMS"
+:: IDEA_REMOTE_DEBUG_PARAMS="-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"
+::
+:: JVM_PARAMETERS=
+
+:: ---------------------------------------------------------------------------
+:: Default configuration. Do not modify below this line.
+:: ---------------------------------------------------------------------------
+:: Application specific parameters
+
+SET MAIN_CLASS=@main.class@
+SET JVM_PARAMS=@jvm.params@
+SET PROGRAM_PARAMS=@program.params@
+
+:: Try to find java virtual machine
+IF NOT DEFINED JAVA (
+ IF NOT DEFINED JAVA_HOME SET JAVA="java.exe"
+ IF DEFINED JAVA_HOME SET JAVA="%JAVA_HOME%\bin\java.exe"
+)
+
+:: Run program
+%JAVA% %JVM_PARAMS% %JVM_PARAMETERS% -classpath %CLASSPATH% %MAIN_CLASS% %PROGRAM_PARAMS% %*
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/main/scripts/run.sh b/hyracks-algebricks/hyracks-algebricks-tests/src/main/scripts/run.sh
new file mode 100644
index 0000000..a998626
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/main/scripts/run.sh
@@ -0,0 +1,81 @@
+#!/bin/sh
+# JAVA classpath
+# Use the local variable CLASSPATH to add custom entries (e.g. JDBC drivers) to
+# the classpath. Separate multiple paths with ":". Enclose the value
+# in double quotes. Adding additional files or locations on separate
+# lines makes things clearer.
+# Note: If under running under cygwin use "/cygdrive/c/..." for "C:/..."
+# Example:
+#
+# Set the CLASSPATH to a jar file and a directory. Note that
+# "classes dir" is a directory of class files with a space in the name.
+#
+# CLASSPATH="usr/local/Product1/lib/product.jar"
+# CLASSPATH="${CLASSPATH}:../MyProject/classes dir"
+#
+CLASSPATH="@classpath@"
+
+# JVM parameters
+# If you want to modify the default parameters (e.g. maximum heap size -Xmx)
+# for the Java virtual machine set the local variable JVM_PARAMETERS below
+# Example:
+# JVM_PARAMETERS=-Xms100M -Xmx200M
+#
+# Below are the JVM parameters needed to do remote debugging using Intellij
+# IDEA. Uncomment and then do: JVM_PARAMETERS="$IDEA_REMOTE_DEBUG_PARAMS"
+# IDEA_REMOTE_DEBUG_PARAMS="-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"
+#
+# JVM_PARAMETERS=
+
+#run with shared memory setup
+#if [ -n "${RUN_SHARED_MEM}"]; then
+# JVM_PARAMETERS="${JVM_PARAMETERS} -Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_shmem,server=n,address=javadebug,suspend=y"
+#fi
+
+# ---------------------------------------------------------------------------
+# Default configuration. Do not modify below this line.
+# ---------------------------------------------------------------------------
+# Application specific parameters
+
+MAIN_CLASS="@main.class@"
+JVM_PARAMS="@jvm.params@"
+PROGRAM_PARAMS="@program.params@"
+
+# Cygwin support. $cygwin _must_ be set to either true or false.
+case "`uname`" in
+ CYGWIN*) cygwin=true ;;
+ *) cygwin=false ;;
+esac
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin; then
+ [ -n "$JAVA_HOME" ] &&
+ JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+ [ -n "$CLASSPATH" ] &&
+ CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# Try to find java virtual machine
+if [ -z "${JAVA}" ]; then
+ if [ -z "${JAVA_HOME}" ]; then
+ JAVA=java
+ else
+ JAVA=${JAVA_HOME}/bin/java
+ fi
+fi
+
+# Try to find directory where this script is located
+COMMAND="${PWD}/$0"
+if [ ! -f "${COMMAND}" ]; then
+ COMMAND="$0"
+fi
+BASEDIR=`expr "${COMMAND}" : '\(.*\)/\.*'`
+
+# For Cygwin, switch paths to Windows format before running java
+if $cygwin; then
+# JAVA=`cygpath --path --windows "$JAVA"`
+ CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
+fi
+
+# Run program
+${JAVA} ${JVM_PARAMS} ${JVM_PARAMETERS} -classpath "${CLASSPATH}" ${MAIN_CLASS} ${PROGRAM_PARAMS} $*
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
new file mode 100644
index 0000000..6bd055d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -0,0 +1,893 @@
+package edu.uci.ics.hyracks.algebricks.tests.pushruntime;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.aggregators.TupleCountAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.aggregators.TupleCountRunningAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IRunningAggregateFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IUnnestingFunctionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.evaluators.ColumnAccessEvalFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data.BinaryBooleanInspectorImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data.BinaryIntegerInspectorImpl;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data.IntegerPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.jobgen.data.UTF8StringPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.AggregateRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.NestedPlansAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.group.MicroPreClusteredGroupRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.meta.SubplanRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.sort.InMemorySortRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.AssignRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.NestedTupleSourceRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.PrinterRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.RunningAggregateRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.SinkWriterRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StreamLimitRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StreamProjectRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StreamSelectRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.StringStreamingRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.operators.std.UnnestRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.writers.PrinterBasedWriterFactory;
+import edu.uci.ics.hyracks.algebricks.tests.util.AlgebricksHyracksIntegrationUtil;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+
+public class PushRuntimeTest {
+
+ private static final String SEPARATOR = System.getProperty("file.separator");
+ private static final String PATH_ACTUAL = "rttest";
+ private static final String PATH_BASE = "src" + SEPARATOR + "test" + SEPARATOR + "resources";
+ private static final String PATH_EXPECTED = PATH_BASE + SEPARATOR + "results";
+
+ private static final String[] DEFAULT_NODES = new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID };
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ File outdir = new File(PATH_ACTUAL);
+ outdir.mkdirs();
+ AlgebricksHyracksIntegrationUtil.init();
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ AlgebricksHyracksIntegrationUtil.deinit();
+ File outdir = new File(PATH_ACTUAL);
+ File[] files = outdir.listFiles();
+ if (files == null || files.length == 0) {
+ outdir.delete();
+ }
+ }
+
+ @Test
+ public void etsAssignPrint() throws Exception {
+ JobSpecification spec = new JobSpecification();
+ IntegerConstantEvalFactory const1 = new IntegerConstantEvalFactory(400);
+ IntegerConstantEvalFactory const2 = new IntegerConstantEvalFactory(3);
+
+ EmptyTupleSourceRuntimeFactory ets = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor etsDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+ AssignRuntimeFactory assign = new AssignRuntimeFactory(new int[] { 0, 1 }, new IEvaluatorFactory[] { const1,
+ const2 }, new int[] { 0, 1 });
+ RecordDescriptor assignDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ PrinterRuntimeFactory printer = new PrinterRuntimeFactory(new int[] { 0, 1 }, new IPrinterFactory[] {
+ IntegerPrinterFactory.INSTANCE, IntegerPrinterFactory.INSTANCE }, assignDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 0,
+ new IPushRuntimeFactory[] { ets, assign, printer },
+ new RecordDescriptor[] { etsDesc, assignDesc, null });
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp, DEFAULT_NODES);
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+ }
+
+ @Test
+ public void etsAssignWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+ IntegerConstantEvalFactory const1 = new IntegerConstantEvalFactory(400);
+ IntegerConstantEvalFactory const2 = new IntegerConstantEvalFactory(3);
+
+ EmptyTupleSourceRuntimeFactory ets = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor etsDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+ AssignRuntimeFactory assign = new AssignRuntimeFactory(new int[] { 0, 1 }, new IEvaluatorFactory[] { const1,
+ const2 }, new int[] { 0, 1 });
+ RecordDescriptor assignDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "etsAssignWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0, 1 }, new IPrinterFactory[] {
+ IntegerPrinterFactory.INSTANCE, IntegerPrinterFactory.INSTANCE }, outFile,
+ PrinterBasedWriterFactory.INSTANCE, assignDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 0,
+ new IPushRuntimeFactory[] { ets, assign, writer }, new RecordDescriptor[] { etsDesc, assignDesc, null });
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp, DEFAULT_NODES);
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("400; 3", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void scanSelectWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // the scanner
+ FileSplit[] intFileSplits = new FileSplit[1];
+ intFileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
+ "data/simple/int-part1.tbl")));
+ IFileSplitProvider intSplitProvider = new ConstantFileSplitProvider(intFileSplits);
+ RecordDescriptor intScannerDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+ IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE };
+ FileScanOperatorDescriptor intScanner = new FileScanOperatorDescriptor(spec, intSplitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), intScannerDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, intScanner, DEFAULT_NODES);
+
+ // the algebricks op.
+ IEvaluatorFactory cond = new IntegerGreaterThanEvalFactory(new IntegerConstantEvalFactory(2),
+ new ColumnAccessEvalFactory(0));
+ StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 0 },
+ BinaryBooleanInspectorImpl.INSTANCE);
+ RecordDescriptor selectDesc = intScannerDesc;
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "scanSelectWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ selectDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { select, writer }, new RecordDescriptor[] { selectDesc, null });
+
+ PartitionConstraintHelper.addPartitionCountConstraint(spec, algebricksOp, 1);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), intScanner, 0, algebricksOp, 0);
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("0", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void etsAssignProjectWrite() throws Exception {
+
+ JobSpecification spec = new JobSpecification();
+ IntegerConstantEvalFactory const1 = new IntegerConstantEvalFactory(400);
+ IntegerConstantEvalFactory const2 = new IntegerConstantEvalFactory(3);
+
+ EmptyTupleSourceRuntimeFactory ets = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor etsDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+ AssignRuntimeFactory assign = new AssignRuntimeFactory(new int[] { 0, 1 }, new IEvaluatorFactory[] { const1,
+ const2 }, new int[] { 0, 1 });
+ RecordDescriptor assignDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+ StreamProjectRuntimeFactory project = new StreamProjectRuntimeFactory(new int[] { 1 });
+ RecordDescriptor projectDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "etsAssignProjectWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ projectDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 0,
+ new IPushRuntimeFactory[] { ets, assign, project, writer }, new RecordDescriptor[] { etsDesc,
+ assignDesc, projectDesc, null });
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp, DEFAULT_NODES);
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("3", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void scanLimitWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // the scanner
+ FileSplit[] fileSplits = new FileSplit[1];
+ fileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
+ "data/tpch0.001/customer.tbl")));
+ IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
+
+ RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+ IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE };
+ FileScanOperatorDescriptor scanner = new FileScanOperatorDescriptor(spec, splitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanner, DEFAULT_NODES);
+
+ // the algebricks op.
+ StreamLimitRuntimeFactory limit = new StreamLimitRuntimeFactory(new IntegerConstantEvalFactory(2), null,
+ new int[] { 0 }, BinaryIntegerInspectorImpl.INSTANCE);
+ RecordDescriptor limitDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "scanLimitWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ limitDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { limit, writer }, new RecordDescriptor[] { limitDesc, null });
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, algebricksOp, 0);
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("12", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void etsUnnestWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ EmptyTupleSourceRuntimeFactory ets = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor etsDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+ IUnnestingFunctionFactory aggregFactory = new IntArrayUnnester(new int[] { 100, 200, 300 });
+ UnnestRuntimeFactory unnest = new UnnestRuntimeFactory(0, aggregFactory, new int[] { 0 });
+ RecordDescriptor unnestDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "etsUnnestWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ unnestDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 0,
+ new IPushRuntimeFactory[] { ets, unnest, writer }, new RecordDescriptor[] { etsDesc, unnestDesc, null });
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("100200300", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void scanAggregateWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // the scanner
+ FileSplit[] fileSplits = new FileSplit[1];
+ fileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
+ "data/tpch0.001/customer-part1.tbl")));
+ IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
+ RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+ IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE };
+ FileScanOperatorDescriptor scanner = new FileScanOperatorDescriptor(spec, splitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanner,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the algebricks op.
+ AggregateRuntimeFactory agg = new AggregateRuntimeFactory(
+ new IAggregateFunctionFactory[] { new TupleCountAggregateFunctionFactory() });
+ RecordDescriptor aggDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "scanAggregateWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ aggDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { agg, writer }, new RecordDescriptor[] { aggDesc, null });
+
+ PartitionConstraintHelper.addPartitionCountConstraint(spec, algebricksOp, 1);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, algebricksOp, 0);
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("75", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void scanSortGbySelectWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // the scanner
+ FileSplit[] fileSplits = new FileSplit[1];
+ fileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
+ "data/tpch0.001/customer.tbl")));
+ IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
+ RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+ IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE };
+ FileScanOperatorDescriptor scanner = new FileScanOperatorDescriptor(spec, splitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanner,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the sort (by nation id)
+ RecordDescriptor sortDesc = scannerDesc;
+ InMemorySortOperatorDescriptor sort = new InMemorySortOperatorDescriptor(spec, new int[] { 3 },
+ new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, sortDesc);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sort,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the group-by
+ NestedTupleSourceRuntimeFactory nts = new NestedTupleSourceRuntimeFactory();
+ RecordDescriptor ntsDesc = sortDesc;
+ AggregateRuntimeFactory agg = new AggregateRuntimeFactory(
+ new IAggregateFunctionFactory[] { new TupleCountAggregateFunctionFactory() });
+ RecordDescriptor aggDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+ AlgebricksPipeline pipeline = new AlgebricksPipeline(new IPushRuntimeFactory[] { nts, agg },
+ new RecordDescriptor[] { ntsDesc, aggDesc });
+ NestedPlansAccumulatingAggregatorFactory npaaf = new NestedPlansAccumulatingAggregatorFactory(
+ new AlgebricksPipeline[] { pipeline }, new int[] { 3 }, new int[] {});
+ RecordDescriptor gbyDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+ PreclusteredGroupOperatorDescriptor gby = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 3 },
+ new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, npaaf, gbyDesc);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, gby,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the algebricks op.
+ IEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
+ new ColumnAccessEvalFactory(0)); // Canadian customers
+ StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 1 },
+ BinaryBooleanInspectorImpl.INSTANCE);
+ RecordDescriptor selectDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "scanSortGbySelectWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ selectDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { select, writer }, new RecordDescriptor[] { selectDesc, null });
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sort, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, gby, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), gby, 0, algebricksOp, 0);
+ spec.addRoot(algebricksOp);
+
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("9", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void scanHashGbySelectWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // the scanner
+ FileSplit[] fileSplits = new FileSplit[1];
+ fileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
+ "data/tpch0.001/customer.tbl")));
+ IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
+ RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+ IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE };
+ FileScanOperatorDescriptor scanner = new FileScanOperatorDescriptor(spec, splitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanner,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the group-by
+ RecordDescriptor gbyDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+ ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(new int[] { 3 },
+ new IBinaryHashFunctionFactory[] { IntegerBinaryHashFunctionFactory.INSTANCE });
+ IAggregateFunctionFactory[] aggFuns = new IAggregateFunctionFactory[] { new TupleCountAggregateFunctionFactory() };
+ IAccumulatingAggregatorFactory aggFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(aggFuns,
+ new int[] { 3 }, new int[] {});
+ HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(spec, new int[] { 3 }, tpcf,
+ new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, aggFactory, gbyDesc, 1024);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, gby,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the algebricks op.
+ IEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
+ new ColumnAccessEvalFactory(0)); // Canadian customers
+ StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 1 },
+ BinaryBooleanInspectorImpl.INSTANCE);
+ RecordDescriptor selectDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "scanHashGbySelectWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ selectDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { select, writer }, new RecordDescriptor[] { selectDesc, null });
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, gby, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), gby, 0, algebricksOp, 0);
+ spec.addRoot(algebricksOp);
+
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("9", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void etsUnnestRunningaggregateWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ EmptyTupleSourceRuntimeFactory ets = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor etsDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+ IUnnestingFunctionFactory aggregFactory = new IntArrayUnnester(new int[] { 100, 200, 300 });
+ UnnestRuntimeFactory unnest = new UnnestRuntimeFactory(0, aggregFactory, new int[] { 0 });
+ RecordDescriptor unnestDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ RunningAggregateRuntimeFactory ragg = new RunningAggregateRuntimeFactory(new int[] { 1 },
+ new IRunningAggregateFunctionFactory[] { new TupleCountRunningAggregateFunctionFactory() }, new int[] {
+ 0, 1 });
+ RecordDescriptor raggDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "etsUnnestRunningaggregateWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 1 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ raggDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 0,
+ new IPushRuntimeFactory[] { ets, unnest, ragg, writer }, new RecordDescriptor[] { etsDesc, unnestDesc,
+ raggDesc, null });
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("123", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void etsAssignScriptWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+ IntegerConstantEvalFactory const1 = new IntegerConstantEvalFactory(400);
+ IntegerConstantEvalFactory const2 = new IntegerConstantEvalFactory(3);
+
+ EmptyTupleSourceRuntimeFactory ets = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor etsDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+ AssignRuntimeFactory assign = new AssignRuntimeFactory(new int[] { 0, 1 }, new IEvaluatorFactory[] { const1,
+ const2 }, new int[] { 0, 1 });
+ RecordDescriptor assignDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ IValueParserFactory[] valueParsers = { IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE };
+
+ String osname = System.getProperty("os.name");
+ String command;
+ if (osname.equals("Linux")) {
+ command = "bash target/testscripts/idscript";
+ } else if (osname.startsWith("Windows")) {
+ command = "target\\testscripts\\idscript.cmd";
+ } else {
+ // don't know how to test
+ return;
+ }
+
+ StringStreamingRuntimeFactory script = new StringStreamingRuntimeFactory(command, new IPrinterFactory[] {
+ IntegerPrinterFactory.INSTANCE, IntegerPrinterFactory.INSTANCE }, ' ',
+ new DelimitedDataTupleParserFactory(valueParsers, ' '));
+ RecordDescriptor scriptDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "etsAssignScriptWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0, 1 }, new IPrinterFactory[] {
+ IntegerPrinterFactory.INSTANCE, IntegerPrinterFactory.INSTANCE }, outFile,
+ PrinterBasedWriterFactory.INSTANCE, scriptDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 0,
+ new IPushRuntimeFactory[] { ets, assign, script, writer }, new RecordDescriptor[] { etsDesc,
+ assignDesc, scriptDesc, null });
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("400; 3", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void scanSplitWrite() throws Exception {
+ final int outputArity = 2;
+
+ JobSpecification spec = new JobSpecification();
+
+ String inputFileName = "data/tpch0.001/customer.tbl";
+ File inputFile = new File(inputFileName);
+ File[] outputFile = new File[outputArity];
+ for (int i = 0; i < outputArity; i++) {
+ outputFile[i] = File.createTempFile("splitop", null);
+ }
+
+ FileSplit[] inputSplits = new FileSplit[] { new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID,
+ new FileReference(inputFile)) };
+
+ DelimitedDataTupleParserFactory stringParser = new DelimitedDataTupleParserFactory(
+ new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE }, '\u0000');
+ RecordDescriptor stringRec = new RecordDescriptor(
+ new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE, });
+
+ FileScanOperatorDescriptor scanOp = new FileScanOperatorDescriptor(spec, new ConstantFileSplitProvider(
+ inputSplits), stringParser, stringRec);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ SplitOperatorDescriptor splitOp = new SplitOperatorDescriptor(spec, stringRec, outputArity);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, splitOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ IOperatorDescriptor outputOp[] = new IOperatorDescriptor[outputFile.length];
+ for (int i = 0; i < outputArity; i++) {
+ outputOp[i] = new LineFileWriteOperatorDescriptor(spec, new FileSplit[] { new FileSplit(
+ AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(outputFile[i])) });
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, outputOp[i],
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+ }
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanOp, 0, splitOp, 0);
+ for (int i = 0; i < outputArity; i++) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), splitOp, i, outputOp[i], 0);
+ }
+
+ for (int i = 0; i < outputArity; i++) {
+ spec.addRoot(outputOp[i]);
+ }
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ for (int i = 0; i < outputArity; i++) {
+ compareFiles(inputFileName, outputFile[i].getAbsolutePath());
+ }
+ }
+
+ @Test
+ public void scanMicroSortWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // the scanner
+ FileSplit[] fileSplits = new FileSplit[1];
+ fileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
+ "data/tpch0.001/nation.tbl")));
+ IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
+ RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+ IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE };
+ FileScanOperatorDescriptor scanner = new FileScanOperatorDescriptor(spec, splitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanner,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the algebricks op.
+ InMemorySortRuntimeFactory sort = new InMemorySortRuntimeFactory(new int[] { 1 }, null,
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, null);
+ RecordDescriptor sortDesc = scannerDesc;
+
+ String fileName = "scanMicroSortWrite.out";
+ String filePath = PATH_ACTUAL + SEPARATOR + fileName;
+ String resultFilePath = PATH_EXPECTED + SEPARATOR + fileName;
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0, 1, 2, 3 }, new IPrinterFactory[] {
+ IntegerPrinterFactory.INSTANCE, UTF8StringPrinterFactory.INSTANCE, IntegerPrinterFactory.INSTANCE,
+ UTF8StringPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE, sortDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { sort, writer }, new RecordDescriptor[] { sortDesc, null });
+
+ PartitionConstraintHelper.addPartitionCountConstraint(spec, algebricksOp, 1);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, algebricksOp, 0);
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ compareFiles(filePath, resultFilePath);
+ outFile.delete();
+ }
+
+ @Test
+ public void etsAssignSubplanProjectWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+ IntegerConstantEvalFactory const1 = new IntegerConstantEvalFactory(400);
+ IntegerConstantEvalFactory const2 = new IntegerConstantEvalFactory(3);
+
+ EmptyTupleSourceRuntimeFactory ets = new EmptyTupleSourceRuntimeFactory();
+ RecordDescriptor etsDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
+
+ AssignRuntimeFactory assign1 = new AssignRuntimeFactory(new int[] { 0 }, new IEvaluatorFactory[] { const1 },
+ new int[] { 0 });
+ RecordDescriptor assign1Desc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ NestedTupleSourceRuntimeFactory nts = new NestedTupleSourceRuntimeFactory();
+
+ AssignRuntimeFactory assign2 = new AssignRuntimeFactory(new int[] { 1 },
+ new IEvaluatorFactory[] { new IntegerAddEvalFactory(new ColumnAccessEvalFactory(0), const2) },
+ new int[] { 0, 1 });
+ RecordDescriptor assign2Desc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ StreamProjectRuntimeFactory project1 = new StreamProjectRuntimeFactory(new int[] { 1 });
+ RecordDescriptor project1Desc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ AlgebricksPipeline pipeline = new AlgebricksPipeline(new IPushRuntimeFactory[] { nts, assign2, project1 },
+ new RecordDescriptor[] { assign1Desc, assign2Desc, project1Desc });
+
+ SubplanRuntimeFactory subplan = new SubplanRuntimeFactory(pipeline,
+ new INullWriterFactory[] { NoopNullWriterFactory.INSTANCE }, assign1Desc, null);
+
+ RecordDescriptor subplanDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ StreamProjectRuntimeFactory project2 = new StreamProjectRuntimeFactory(new int[] { 1 });
+ RecordDescriptor project2Desc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "etsAssignSubplanProjectWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ project2Desc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 0,
+ new IPushRuntimeFactory[] { ets, assign1, subplan, project2, writer }, new RecordDescriptor[] {
+ etsDesc, assign1Desc, subplanDesc, project2Desc, null });
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp, DEFAULT_NODES);
+
+ spec.addRoot(algebricksOp);
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("403", buf.toString());
+ outFile.delete();
+ }
+
+ @Test
+ public void scanMicroSortGbySelectWrite() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // the scanner
+ FileSplit[] fileSplits = new FileSplit[1];
+ fileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
+ "data/tpch0.001/customer.tbl")));
+ IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
+ RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+ IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE };
+ FileScanOperatorDescriptor scanner = new FileScanOperatorDescriptor(spec, splitProvider,
+ new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanner,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ // the sort (by nation id)
+ RecordDescriptor sortDesc = scannerDesc;
+ InMemorySortRuntimeFactory sort = new InMemorySortRuntimeFactory(new int[] { 3 }, null,
+ new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, null);
+
+ // the group-by
+ NestedTupleSourceRuntimeFactory nts = new NestedTupleSourceRuntimeFactory();
+ RecordDescriptor ntsDesc = sortDesc;
+ AggregateRuntimeFactory agg = new AggregateRuntimeFactory(
+ new IAggregateFunctionFactory[] { new TupleCountAggregateFunctionFactory() });
+ RecordDescriptor aggDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+ AlgebricksPipeline pipeline = new AlgebricksPipeline(new IPushRuntimeFactory[] { nts, agg },
+ new RecordDescriptor[] { ntsDesc, aggDesc });
+ NestedPlansAccumulatingAggregatorFactory npaaf = new NestedPlansAccumulatingAggregatorFactory(
+ new AlgebricksPipeline[] { pipeline }, new int[] { 3 }, new int[] {});
+ RecordDescriptor gbyDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+ MicroPreClusteredGroupRuntimeFactory gby = new MicroPreClusteredGroupRuntimeFactory(new int[] { 3 },
+ new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, npaaf, sortDesc, gbyDesc,
+ null);
+
+ // the algebricks op.
+ IEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
+ new ColumnAccessEvalFactory(0)); // Canadian customers
+ StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 1 },
+ BinaryBooleanInspectorImpl.INSTANCE);
+ RecordDescriptor selectDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ String filePath = PATH_ACTUAL + SEPARATOR + "scanSortGbySelectWrite.out";
+ File outFile = new File(filePath);
+ SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
+ new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
+ selectDesc);
+
+ AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { sort, gby, select, writer }, new RecordDescriptor[] { sortDesc, gbyDesc,
+ selectDesc, null });
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
+ new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, algebricksOp, 0);
+ spec.addRoot(algebricksOp);
+
+ AlgebricksHyracksIntegrationUtil.runJob(spec);
+ StringBuilder buf = new StringBuilder();
+ readFileToString(outFile, buf);
+ Assert.assertEquals("9", buf.toString());
+ outFile.delete();
+ }
+
+ private static void readFileToString(File file, StringBuilder buf) throws Exception {
+ BufferedReader result = new BufferedReader(new FileReader(file));
+ boolean first = true;
+ while (true) {
+ String s = result.readLine();
+ if (s == null) {
+ break;
+ } else {
+ if (!first) {
+ first = false;
+ buf.append('\n');
+ }
+ buf.append(s);
+ }
+ }
+ result.close();
+ }
+
+ public void compareFiles(String fileNameA, String fileNameB) throws IOException {
+ BufferedReader fileA = new BufferedReader(new FileReader(fileNameA));
+ BufferedReader fileB = new BufferedReader(new FileReader(fileNameB));
+
+ String lineA, lineB;
+ while ((lineA = fileA.readLine()) != null) {
+ lineB = fileB.readLine();
+ Assert.assertEquals(lineA, lineB);
+ }
+ Assert.assertNull(fileB.readLine());
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/tools/WriteValueTest.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/tools/WriteValueTest.java
new file mode 100644
index 0000000..73dc16d
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/tools/WriteValueTest.java
@@ -0,0 +1,97 @@
+package edu.uci.ics.hyracks.algebricks.tests.tools;
+
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.algebricks.core.utils.WriteValueTools;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ByteArrayAccessibleOutputStream;
+
+public class WriteValueTest {
+
+ private ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
+
+ @Test
+ public void writeIntegers() throws Exception {
+ writeIntTest(6);
+ writeIntTest(1234);
+ writeIntTest(-1234);
+ writeIntTest(Integer.MAX_VALUE);
+ writeIntTest(Integer.MAX_VALUE - 1);
+ writeIntTest(Integer.MIN_VALUE);
+ writeIntTest(Integer.MIN_VALUE + 1);
+ }
+
+ @Test
+ public void writeLongs() throws Exception {
+ writeLongTest(Integer.MAX_VALUE);
+ writeLongTest(Integer.MAX_VALUE - 1);
+ writeLongTest(Integer.MIN_VALUE);
+ writeLongTest(Integer.MIN_VALUE + 1);
+ writeLongTest(0L);
+ writeLongTest(1234567890L);
+ writeLongTest(-1234567890L);
+ }
+
+ @Test
+ public void writeUTF8Strings() throws Exception {
+ ByteArrayAccessibleOutputStream interm = new ByteArrayAccessibleOutputStream();
+ DataOutput dout = new DataOutputStream(interm);
+ writeUTF8Test("abcdefABCDEF", dout, interm);
+ writeUTF8Test("šťžľčěďňřůĺ", dout, interm);
+ writeUTF8Test("Ă㪺Ţţ", dout, interm);
+ }
+
+ private void writeIntTest(int i) throws Exception {
+ baaos.reset();
+ WriteValueTools.writeInt(i, baaos);
+ byte[] goal = Integer.toString(i).getBytes();
+ if (baaos.size() != goal.length) {
+ throw new Exception("Expecting to write " + i + " in " + goal.length + " bytes, but found " + baaos.size()
+ + " bytes.");
+ }
+ for (int k = 0; k < goal.length; k++) {
+ if (goal[k] != baaos.getByteArray()[k]) {
+ throw new Exception("Expecting to write " + i + " as " + goal + ", but found " + baaos.getByteArray()
+ + " instead.");
+ }
+ }
+ }
+
+ private void writeLongTest(long x) throws Exception {
+ baaos.reset();
+ WriteValueTools.writeLong(x, baaos);
+ byte[] goal = Long.toString(x).getBytes();
+ if (baaos.size() != goal.length) {
+ throw new Exception("Expecting to write " + x + " in " + goal.length + " bytes, but found " + baaos.size()
+ + " bytes.");
+ }
+ for (int k = 0; k < goal.length; k++) {
+ if (goal[k] != baaos.getByteArray()[k]) {
+ throw new Exception("Expecting to write " + x + " as " + goal + ", but found " + baaos.getByteArray()
+ + " instead.");
+ }
+ }
+ }
+
+ private void writeUTF8Test(String str, DataOutput dout, ByteArrayAccessibleOutputStream interm) throws Exception {
+ interm.reset();
+ dout.writeUTF(str);
+ baaos.reset();
+ WriteValueTools.writeUTF8String(interm.getByteArray(), 0, interm.size(), baaos);
+ byte[] b = str.getBytes();
+ if (baaos.size() != b.length + 2) {
+ throw new Exception("Expecting to write " + b + " in " + b.length + " bytes, but found " + baaos.size()
+ + " bytes.");
+ }
+ if (baaos.getByteArray()[0] != '\"' || baaos.getByteArray()[baaos.size() - 1] != '\"') {
+ throw new Exception("Missing quotes.");
+ }
+ for (int k = 0; k < b.length; k++) {
+ if (b[k] != baaos.getByteArray()[k + 1]) {
+ throw new Exception("Expecting to write " + b + ", but found " + baaos.getByteArray() + " instead.");
+ }
+ }
+ }
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
new file mode 100644
index 0000000..d6c2165
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 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.
+ */
+package edu.uci.ics.hyracks.algebricks.tests.util;
+
+import java.util.EnumSet;
+
+import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
+import edu.uci.ics.hyracks.api.client.HyracksLocalConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class AlgebricksHyracksIntegrationUtil {
+
+ public static final String NC1_ID = "nc1";
+ public static final String NC2_ID = "nc2";
+
+ public static final int DEFAULT_HYRACKS_CC_PORT = 1099;
+
+ public static final int TEST_HYRACKS_CC_PORT = 4322;
+
+ private static ClusterControllerService cc;
+ private static NodeControllerService nc1;
+ private static NodeControllerService nc2;
+ private static IHyracksClientConnection hcc;
+
+ public static void init() throws Exception {
+ CCConfig ccConfig = new CCConfig();
+ ccConfig.port = TEST_HYRACKS_CC_PORT;
+ // ccConfig.useJOL = true;
+ cc = new ClusterControllerService(ccConfig);
+ cc.start();
+
+ NCConfig ncConfig1 = new NCConfig();
+ ncConfig1.ccHost = "localhost";
+ ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
+ ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.nodeId = NC1_ID;
+ nc1 = new NodeControllerService(ncConfig1);
+ nc1.start();
+
+ NCConfig ncConfig2 = new NCConfig();
+ ncConfig2.ccHost = "localhost";
+ ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
+ ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.nodeId = NC2_ID;
+ nc2 = new NodeControllerService(ncConfig2);
+ nc2.start();
+
+ hcc = new HyracksLocalConnection(cc);
+ hcc.createApplication(AlgebricksConfig.HYRACKS_APP_NAME, null);
+ }
+
+ public static void deinit() throws Exception {
+ nc2.stop();
+ nc1.stop();
+ cc.stop();
+ }
+
+ public static void runJob(JobSpecification spec) throws Exception {
+ JobId jobId = hcc.createJob(AlgebricksConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ AlgebricksConfig.ALGEBRICKS_LOGGER.info(spec.toJSON().toString());
+ cc.start(jobId);
+ AlgebricksConfig.ALGEBRICKS_LOGGER.info(jobId.toString());
+ cc.waitForCompletion(jobId);
+ }
+
+}
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/resources/results/scanMicroSortWrite.out b/hyracks-algebricks/hyracks-algebricks-tests/src/test/resources/results/scanMicroSortWrite.out
new file mode 100644
index 0000000..1c0fd6a
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/resources/results/scanMicroSortWrite.out
@@ -0,0 +1,25 @@
+0; "ALGERIA"; 0; " haggle. carefully final deposits detect slyly agai"
+1; "ARGENTINA"; 1; "al foxes promise slyly according to the regular accounts. bold requests alon"
+2; "BRAZIL"; 1; "y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special "
+3; "CANADA"; 1; "eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold"
+18; "CHINA"; 2; "c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos"
+4; "EGYPT"; 4; "y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d"
+5; "ETHIOPIA"; 0; "ven packages wake quickly. regu"
+6; "FRANCE"; 3; "refully final requests. regular, ironi"
+7; "GERMANY"; 3; "l platelets. regular accounts x-ray: unusual, regular acco"
+8; "INDIA"; 2; "ss excuses cajole slyly across the packages. deposits print aroun"
+9; "INDONESIA"; 2; " slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull"
+10; "IRAN"; 4; "efully alongside of the slyly final dependencies. "
+11; "IRAQ"; 4; "nic deposits boost atop the quickly final requests? quickly regula"
+12; "JAPAN"; 2; "ously. final, express gifts cajole a"
+13; "JORDAN"; 4; "ic deposits are blithely about the carefully regular pa"
+14; "KENYA"; 0; " pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t"
+15; "MOROCCO"; 0; "rns. blithely bold courts among the closely regular packages use furiously bold platelets?"
+16; "MOZAMBIQUE"; 0; "s. ironic, unusual asymptotes wake blithely r"
+17; "PERU"; 1; "platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun"
+19; "ROMANIA"; 3; "ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account"
+22; "RUSSIA"; 3; " requests against the platelets use never according to the quickly regular pint"
+20; "SAUDI ARABIA"; 4; "ts. silent requests haggle. closely express packages sleep across the blithely"
+23; "UNITED KINGDOM"; 3; "eans boost carefully special requests. accounts are. carefull"
+24; "UNITED STATES"; 1; "y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be"
+21; "VIETNAM"; 2; "hely enticingly express accounts. even, final "
diff --git a/hyracks-algebricks/pom.xml b/hyracks-algebricks/pom.xml
new file mode 100644
index 0000000..01987f5
--- /dev/null
+++ b/hyracks-algebricks/pom.xml
@@ -0,0 +1,19 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-algebricks</artifactId>
+ <packaging>pom</packaging>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <modules>
+ <module>hyracks-algebricks-compiler</module>
+ <module>hyracks-algebricks-core</module>
+ <module>hyracks-algebricks-rewriter</module>
+ <module>hyracks-algebricks-tests</module>
+ <module>hyracks-algebricks-examples</module>
+ </modules>
+</project>