johnyangk closed pull request #124: [NEMO-185] BeamSQL TPC-H Integration Test
URL: https://github.com/apache/incubator-nemo/pull/124
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/client/src/main/java/org/apache/nemo/client/JobLauncher.java 
b/client/src/main/java/org/apache/nemo/client/JobLauncher.java
index 0ae7dfdd4..91a6329f3 100644
--- a/client/src/main/java/org/apache/nemo/client/JobLauncher.java
+++ b/client/src/main/java/org/apache/nemo/client/JobLauncher.java
@@ -151,6 +151,7 @@ public static void main(final String[] args) throws 
Exception {
       throw new RuntimeException(e);
     } finally {
       // Close everything that's left
+
       driverRPCServer.shutdown();
       driverLauncher.close();
       final Optional<Throwable> possibleError = 
driverLauncher.getStatus().getError();
diff --git 
a/common/src/main/java/org/apache/nemo/common/test/ExampleTestUtil.java 
b/common/src/main/java/org/apache/nemo/common/test/ExampleTestUtil.java
index 157e9bf2e..ea1932d89 100644
--- a/common/src/main/java/org/apache/nemo/common/test/ExampleTestUtil.java
+++ b/common/src/main/java/org/apache/nemo/common/test/ExampleTestUtil.java
@@ -29,7 +29,8 @@
  * Test Utils for Examples.
  */
 public final class ExampleTestUtil {
-  private static final Double ERROR = 1e-8;
+  private static final Double ERROR_ALS = 1e-8;
+  private static final Double ERROR_SQL = 1e-2;
   /**
    * Private constructor.
    */
@@ -41,46 +42,25 @@ private ExampleTestUtil() {
    *
    * @param resourcePath root folder for both resources.
    * @param outputFileName output file name.
-   * @param testResourceFileName the test result file name.
+   * @param expectedFileName the test result file name.
    * @throws RuntimeException if the output is invalid.
    */
   public static void ensureOutputValidity(final String resourcePath,
                                           final String outputFileName,
-                                          final String testResourceFileName) 
throws IOException {
-
-    final String testOutput;
-    try (final Stream<Path> fileStream = Files.list(Paths.get(resourcePath))) {
-      testOutput = fileStream
-          .filter(Files::isRegularFile)
-          .filter(path -> 
path.getFileName().toString().startsWith(outputFileName))
-          .flatMap(path -> {
-            try {
-              return Files.lines(path);
-            } catch (final IOException e) {
-              throw new RuntimeException(e);
-            }
-          })
-          .sorted()
-          .reduce("", (p, q) -> (p + "\n" + q));
-    }
-
-    final String resourceOutput;
-
-    try (final Stream<String> lineStream = Files.lines(Paths.get(resourcePath 
+ testResourceFileName))) {
-      resourceOutput = lineStream
-          .sorted()
-          .reduce("", (p, q) -> (p + "\n" + q));
-    }
-
-    if (!testOutput.equals(resourceOutput)) {
+                                          final String expectedFileName) 
throws IOException {
+    final String testOutput =
+      getSortedLineStream(resourcePath, outputFileName, false).reduce("", (p, 
q) -> (p + "\n" + q));
+    final String expectedOutput =
+      getSortedLineStream(resourcePath, expectedFileName, true).reduce("", (p, 
q) -> (p + "\n" + q));
+    if (!testOutput.equals(expectedOutput)) {
       final String outputMsg =
-          "Test output mismatch while comparing [" + outputFileName + "] from 
[" + testResourceFileName + "] under "
-              + resourcePath + ":\n"
-              + "=============" + outputFileName + "=================="
-              + testOutput
-              + "\n=============" + testResourceFileName + "=================="
-              + resourceOutput
-              + "\n===============================";
+        "Test output mismatch while comparing [" + outputFileName + "] from [" 
+ expectedFileName + "] under "
+          + resourcePath + ":\n"
+          + "=============" + outputFileName + "=================="
+          + testOutput
+          + "\n=============" + expectedFileName + "=================="
+          + expectedOutput
+          + "\n===============================";
       throw new RuntimeException(outputMsg);
     }
   }
@@ -92,53 +72,72 @@ public static void ensureOutputValidity(final String 
resourcePath,
    *
    * @param resourcePath path to resources.
    * @param outputFileName name of output file.
-   * @param testResourceFileName name of the file to compare the outputs to.
+   * @param expectedFileName name of the file to compare the outputs to.
    * @throws RuntimeException if the output is invalid.
    * @throws IOException exception.
    */
   public static void ensureALSOutputValidity(final String resourcePath,
                                              final String outputFileName,
-                                             final String 
testResourceFileName) throws IOException {
+                                             final String expectedFileName) 
throws IOException {
+    final List<List<Double>> testOutput = getSortedLineStream(resourcePath, 
outputFileName, false)
+      .filter(line -> !line.trim().equals(""))
+      .map(line -> Arrays.asList(line.split("\\s*,\\s*"))
+        .stream().map(s -> Double.valueOf(s)).collect(Collectors.toList()))
+      .collect(Collectors.toList());
 
-    final List<List<Double>> testOutput;
-    try (final Stream<Path> fileStream = Files.list(Paths.get(resourcePath))) {
-      testOutput = fileStream
-          .filter(Files::isRegularFile)
-          .filter(path -> 
path.getFileName().toString().startsWith(outputFileName))
-          .flatMap(path -> {
-            try {
-              return Files.lines(path);
-            } catch (final IOException e) {
-              throw new RuntimeException(e);
-            }
-          })
-          .sorted()
-          .filter(line -> !line.trim().equals(""))
-          .map(line -> Arrays.asList(line.split("\\s*,\\s*"))
-              .stream().map(s -> 
Double.valueOf(s)).collect(Collectors.toList()))
-          .collect(Collectors.toList());
+    final List<List<Double>> expectedOutput = 
getSortedLineStream(resourcePath, expectedFileName, true)
+      .filter(line -> !line.trim().equals(""))
+      .map(line -> Arrays.asList(line.split("\\s*,\\s*"))
+        .stream().map(s -> Double.valueOf(s)).collect(Collectors.toList()))
+      .collect(Collectors.toList());
+
+    if (testOutput.size() != expectedOutput.size()) {
+      throw new RuntimeException(testOutput.size() + " is not " + 
expectedOutput.size());
     }
 
-    final List<List<Double>> resourceOutput;
-    try (final Stream<String> lineStream = Files.lines(Paths.get(resourcePath 
+ testResourceFileName))) {
-      resourceOutput = lineStream
-          .sorted()
-          .filter(line -> !line.trim().equals(""))
-          .map(line -> Arrays.asList(line.split("\\s*,\\s*"))
-              .stream().map(s -> 
Double.valueOf(s)).collect(Collectors.toList()))
-          .collect(Collectors.toList());
+    for (int i = 0; i < testOutput.size(); i++) {
+      for (int j = 0; j < testOutput.get(i).size(); j++) {
+        final Double testElement = testOutput.get(i).get(j);
+        final Double expectedElement = expectedOutput.get(i).get(j);
+        if (Math.abs(testElement - expectedElement) / expectedElement > 
ERROR_ALS) {
+          throw new RuntimeException("output mismatch");
+        }
+      }
     }
+  }
 
-    if (testOutput.size() != resourceOutput.size()) {
-      throw new RuntimeException("output mismatch");
+  public static void ensureSQLOutputValidity(final String resourcePath,
+                                             final String outputFileName,
+                                             final String expectedFileName) 
throws IOException {
+    final List<List<String>> testOutput = getSortedLineStream(resourcePath, 
outputFileName, false)
+      .map(line -> Arrays.asList(line.split("\\|")))
+      .collect(Collectors.toList());
+
+    final List<List<String>> expectedOutput = 
getSortedLineStream(resourcePath, expectedFileName, true)
+      .map(line -> Arrays.asList(line.split("\\|")))
+      .collect(Collectors.toList());
+
+    if (testOutput.size() != expectedOutput.size()) {
+      throw new RuntimeException(testOutput.size() + " is not " + 
expectedOutput.size());
     }
 
     for (int i = 0; i < testOutput.size(); i++) {
       for (int j = 0; j < testOutput.get(i).size(); j++) {
-        final Double testElement = testOutput.get(i).get(j);
-        final Double resourceElement = resourceOutput.get(i).get(j);
-        if (Math.abs(testElement - resourceElement) / resourceElement > ERROR) 
{
-          throw new RuntimeException("output mismatch");
+        final String testElement = testOutput.get(i).get(j);
+        final String expectedElement = expectedOutput.get(i).get(j);
+
+        try {
+          // This element is double: Account for floating errors
+          final double testElementDouble = Double.valueOf(testElement);
+          final double expectedElementDouble = Double.valueOf(expectedElement);
+          if (Math.abs(testElementDouble - expectedElementDouble) / 
expectedElementDouble > ERROR_SQL) {
+            throw new RuntimeException(testElement + " is not " + 
expectedElement);
+          }
+        } catch (NumberFormatException e) {
+          // This element is not double: Simply compare the strings
+          if (!testElement.equals(expectedElement)) {
+            throw new RuntimeException(testElement + " is not " + 
expectedElement);
+          }
         }
       }
     }
@@ -155,12 +154,40 @@ public static void deleteOutputFile(final String 
directory,
                                       final String outputFileName) throws 
IOException {
     try (final Stream<Path> fileStream = Files.list(Paths.get(directory))) {
       final Set<Path> outputFilePaths = fileStream
-          .filter(Files::isRegularFile)
-          .filter(path -> 
path.getFileName().toString().startsWith(outputFileName))
-          .collect(Collectors.toSet());
+        .filter(Files::isRegularFile)
+        .filter(path -> 
path.getFileName().toString().startsWith(outputFileName))
+        .collect(Collectors.toSet());
       for (final Path outputFilePath : outputFilePaths) {
         Files.delete(outputFilePath);
       }
     }
   }
+
+  private static Stream<String> getSortedLineStream(final String directory,
+                                                    final String fileName,
+                                                    final boolean 
isExactMatch) {
+    try {
+      final Stream<Path> files;
+      if (isExactMatch) {
+        files = Files.list(Paths.get(directory))
+          .filter(Files::isRegularFile)
+          .filter(path -> path.getFileName().toString().equals(fileName));
+      } else {
+        files = Files.list(Paths.get(directory))
+          .filter(Files::isRegularFile)
+          .filter(path -> path.getFileName().toString().startsWith(fileName));
+      }
+
+      return files.flatMap(file -> {
+        try {
+          return Files.lines(file);
+        } catch (final IOException e) {
+          throw new RuntimeException(e);
+        }
+      }).sorted();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+  }
 }
diff --git 
a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/BeamKeyExtractor.java
 
b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/BeamKeyExtractor.java
index b64cb6041..900337201 100644
--- 
a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/BeamKeyExtractor.java
+++ 
b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/BeamKeyExtractor.java
@@ -16,9 +16,12 @@
 package org.apache.nemo.compiler.frontend.beam;
 
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.Row;
 import org.apache.nemo.common.KeyExtractor;
 import org.apache.beam.sdk.values.KV;
 
+import java.util.Arrays;
+
 /**
  * Extracts the key from a KV element.
  * For non-KV elements, the elements themselves become the key.
@@ -31,7 +34,14 @@ public Object extractKey(final Object element) {
     if (value instanceof KV) {
       // Handle null keys, since Beam allows KV with null keys.
       final Object key = ((KV) value).getKey();
-      return key == null ? 0 : key;
+      if (key == null) {
+        return 0;
+      } else if (key instanceof Row) {
+        // TODO #223: Use Row.hashCode in BeamKeyExtractor
+        return Arrays.hashCode(((Row) key).getValues().toArray());
+      } else {
+        return key;
+      }
     } else {
       return element;
     }
diff --git 
a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java
 
b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java
index a4e5d1b11..b5b6bf96b 100644
--- 
a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java
+++ 
b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java
@@ -303,6 +303,7 @@ private static void combineTranslator(final 
TranslationContext ctx,
     final boolean handlesBeamRow = Stream
       .concat(transformVertex.getNode().getInputs().values().stream(),
         transformVertex.getNode().getOutputs().values().stream())
+      .filter(pValue -> getCoder(pValue, ctx.root) instanceof KvCoder)
       .map(pValue -> (KvCoder) getCoder(pValue, ctx.root)) // Input and output 
of combine should be KV
       .map(kvCoder -> kvCoder.getValueCoder().getEncodedTypeDescriptor()) // 
We're interested in the 'Value' of KV
       .anyMatch(valueTypeDescriptor -> 
TypeDescriptor.of(Row.class).equals(valueTypeDescriptor));
@@ -563,8 +564,7 @@ private void addEdgeTo(final IRVertex dst, final PValue 
input) {
         windowCoder = ((PCollection) 
input).getWindowingStrategy().getWindowFn().windowCoder();
       } else if (input instanceof PCollectionView) {
         coder = getCoderForView((PCollectionView) input, root);
-        windowCoder = ((PCollectionView) input).getPCollection()
-          .getWindowingStrategy().getWindowFn().windowCoder();
+        windowCoder = ((PCollectionView) 
input).getPCollection().getWindowingStrategy().getWindowFn().windowCoder();
       } else {
         throw new RuntimeException(String.format("While adding an edge from 
%s, to %s, coder for PValue %s cannot "
           + "be determined", src, dst, input));
diff --git 
a/examples/beam/src/main/java/org/apache/nemo/examples/beam/GenericSourceSink.java
 
b/examples/beam/src/main/java/org/apache/nemo/examples/beam/GenericSourceSink.java
index 6a09f3bf3..d4242245d 100644
--- 
a/examples/beam/src/main/java/org/apache/nemo/examples/beam/GenericSourceSink.java
+++ 
b/examples/beam/src/main/java/org/apache/nemo/examples/beam/GenericSourceSink.java
@@ -39,7 +39,7 @@
  * Helper class for handling source/sink in a generic way.
  * Assumes String-type PCollections.
  */
-final class GenericSourceSink {
+public final class GenericSourceSink {
   /**
    * Default Constructor.
    */
diff --git 
a/examples/beam/src/main/java/org/apache/nemo/examples/beam/tpch/Schemas.java 
b/examples/beam/src/main/java/org/apache/nemo/examples/beam/tpch/Schemas.java
new file mode 100644
index 000000000..2467c1b77
--- /dev/null
+++ 
b/examples/beam/src/main/java/org/apache/nemo/examples/beam/tpch/Schemas.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nemo.examples.beam.tpch;
+
+import org.apache.beam.sdk.schemas.Schema;
+
+/**
+ * TPC-H Schemas.
+ * (Copied and adapted from https://github.com/apache/beam/pull/6240)
+ */
+final class Schemas {
+  /**
+   * Private.
+   */
+  private Schemas() {
+  }
+
+  static final Schema ORDER_SCHEMA =
+    Schema.builder()
+      .addInt32Field("o_orderkey")
+      .addInt32Field("o_custkey")
+      .addStringField("o_orderstatus")
+      .addFloatField("o_totalprice")
+      .addStringField("o_orderdate")
+      .addStringField("o_orderpriority")
+      .addStringField("o_clerk")
+      .addInt32Field("o_shippriority")
+      .addStringField("o_comment")
+      .build();
+
+  static final Schema CUSTOMER_SCHEMA =
+    Schema.builder()
+      .addInt32Field("c_custkey")
+      .addStringField("c_name")
+      .addStringField("c_address")
+      .addInt32Field("c_nationkey")
+      .addStringField("c_phone")
+      .addFloatField("c_acctbal")
+      .addStringField("c_mktsegment")
+      .addStringField("c_comment")
+      .build();
+
+  static final Schema LINEITEM_SCHEMA =
+    Schema.builder()
+      .addInt32Field("l_orderkey")
+      .addInt32Field("l_partkey")
+      .addInt32Field("l_suppkey")
+      .addInt32Field("l_linenumber")
+      .addFloatField("l_quantity")
+      .addFloatField("l_extendedprice")
+      .addFloatField("l_discount")
+      .addFloatField("l_tax")
+      .addStringField("l_returnflag")
+      .addStringField("l_linestatus")
+      .addStringField("l_shipdate")
+      .addStringField("l_commitdate")
+      .addStringField("l_receiptdate")
+      .addStringField("l_shipinstruct")
+      .addStringField("l_shipmode")
+      .addStringField("l_comment")
+      .build();
+
+  static final Schema PARTSUPP_SCHEMA =
+    Schema.builder()
+      .addInt32Field("ps_partkey")
+      .addInt32Field("ps_suppkey")
+      .addInt32Field("ps_availqty")
+      .addFloatField("ps_supplycost")
+      .addStringField("ps_comment")
+      .build();
+
+  static final Schema REGION_SCHEMA =
+    Schema.builder()
+      .addInt32Field("r_regionkey")
+      .addStringField("r_name")
+      .addStringField("r_comment")
+      .build();
+
+  static final Schema SUPPLIER_SCHEMA =
+    Schema.builder()
+      .addInt32Field("s_suppkey")
+      .addStringField("s_name")
+      .addStringField("s_address")
+      .addInt32Field("s_nationkey")
+      .addStringField("s_phone")
+      .addFloatField("s_acctbal")
+      .addStringField("s_comment")
+      .build();
+
+  static final Schema PART_SCHEMA =
+    Schema.builder()
+      .addInt32Field("p_partkey")
+      .addStringField("p_name")
+      .addStringField("p_mfgr")
+      .addStringField("p_brand")
+      .addStringField("p_type")
+      .addInt32Field("p_size")
+      .addStringField("p_container")
+      .addFloatField("p_retailprice")
+      .addStringField("p_comment")
+      .build();
+
+  static final Schema NATION_SCHEMA =
+    Schema.builder()
+      .addInt32Field("n_nationkey")
+      .addStringField("n_name")
+      .addInt32Field("n_regionkey")
+      .addStringField("n_comment")
+      .build();
+}
diff --git 
a/examples/beam/src/main/java/org/apache/nemo/examples/beam/tpch/TpchQueryRunner.java
 
b/examples/beam/src/main/java/org/apache/nemo/examples/beam/tpch/TpchQueryRunner.java
new file mode 100644
index 000000000..4f2dbdb4c
--- /dev/null
+++ 
b/examples/beam/src/main/java/org/apache/nemo/examples/beam/tpch/TpchQueryRunner.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nemo.examples.beam.tpch;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.sql.SqlTransform;
+import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.*;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.nemo.compiler.frontend.beam.NemoPipelineOptions;
+import org.apache.nemo.compiler.frontend.beam.NemoPipelineRunner;
+import org.apache.nemo.examples.beam.GenericSourceSink;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.*;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * TPC-H query runner.
+ * (Copied and adapted from https://github.com/apache/beam/pull/6240)
+ */
+public final class TpchQueryRunner {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TpchQueryRunner.class.getName());
+
+  /**
+   * Private Constructor.
+   */
+  private TpchQueryRunner() {
+  }
+
+  private static PCollectionTuple loadTable(final Pipeline pipeline,
+                                            final CSVFormat csvFormat,
+                                            final String inputDirectory,
+                                            final String query) {
+    final ImmutableMap<String, Schema> hSchemas = ImmutableMap.<String, 
Schema>builder()
+      .put("lineitem", Schemas.LINEITEM_SCHEMA)
+      .put("customer", Schemas.CUSTOMER_SCHEMA)
+      .put("orders", Schemas.ORDER_SCHEMA)
+      .put("supplier", Schemas.SUPPLIER_SCHEMA)
+      .put("nation", Schemas.NATION_SCHEMA)
+      .put("region", Schemas.REGION_SCHEMA)
+      .put("part", Schemas.PART_SCHEMA)
+      .put("partsupp", Schemas.PARTSUPP_SCHEMA)
+      .build();
+
+    PCollectionTuple tables = PCollectionTuple.empty(pipeline);
+    for (final Map.Entry<String, Schema> tableSchema : hSchemas.entrySet()) {
+      final String tableName = tableSchema.getKey();
+      if (query.contains(tableName)) {
+        final String filePattern = inputDirectory + tableSchema.getKey() + 
".*";
+        final PCollection<Row> table = GenericSourceSink.read(pipeline, 
filePattern)
+          .apply("StringToRow", new 
TextTableProvider.CsvToRow(tableSchema.getValue(), csvFormat))
+          .setCoder(tableSchema.getValue().getRowCoder())
+          .setName(tableSchema.getKey());
+        tables = tables.and(new TupleTag<>(tableSchema.getKey()), table);
+        LOG.info("Will load table {} from {}", tableName, filePattern);
+      }
+    }
+    return tables;
+  }
+
+  /**
+   * @param args arguments.
+   */
+  public static void main(final String[] args) {
+    final String queryFilePath = args[0];
+    final String inputDirectory = args[1];
+    final String outputFilePath = args[2];
+
+    LOG.info("{} / {} / {}", queryFilePath, inputDirectory, outputFilePath);
+
+    final PipelineOptions options = 
PipelineOptionsFactory.create().as(NemoPipelineOptions.class);
+    options.setRunner(NemoPipelineRunner.class);
+    options.setJobName("TPC-H");
+    final Pipeline p = Pipeline.create(options);
+
+    final String queryString = getQueryString(queryFilePath);
+
+    // Create tables.
+    final CSVFormat csvFormat = CSVFormat.MYSQL
+      .withDelimiter('|')
+      .withNullString("")
+      .withTrailingDelimiter();
+    final PCollectionTuple tables = loadTable(p, csvFormat, inputDirectory, 
queryString);
+
+    // Run the TPC-H query.
+    final PCollection<Row> result = 
tables.apply(SqlTransform.query(queryString));
+
+    // Write the results.
+    final PCollection<String> resultToWrite = result
+      .apply(MapElements.into(TypeDescriptors.strings()).via(input -> {
+        final List<String> stringRow =
+          input.getValues().stream().map(val -> 
val.toString()).collect(Collectors.toList());
+        return String.join("|", stringRow);
+      }));
+    GenericSourceSink.write(resultToWrite, outputFilePath);
+
+    // Run the pipeline.
+    p.run();
+  }
+
+  private static String getQueryString(final String queryFilePath) {
+    final List<String> lines = new ArrayList<>();
+    try (final Stream<String> stream  = Files.lines(Paths.get(queryFilePath))) 
{
+      stream.forEach(lines::add);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    final StringBuilder sb = new StringBuilder();
+    lines.forEach(line -> {
+      sb.append(" ");
+      sb.append(line);
+    });
+    final String concate = sb.toString();
+
+    final String cleanOne = concate.replaceAll("\n", " ");
+    final String cleanTwo = cleanOne.replaceAll("\t", " ");
+
+    LOG.info("Will execute SQL file {} with query: {}", queryFilePath, 
cleanTwo);
+
+    return cleanTwo;
+  }
+}
diff --git 
a/examples/beam/src/test/java/org/apache/nemo/examples/beam/ModifiedTPCHITCase.java
 
b/examples/beam/src/test/java/org/apache/nemo/examples/beam/ModifiedTPCHITCase.java
new file mode 100644
index 000000000..1e7342c38
--- /dev/null
+++ 
b/examples/beam/src/test/java/org/apache/nemo/examples/beam/ModifiedTPCHITCase.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright (C) 2018 Seoul National University
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *         http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nemo.examples.beam;
+
+import org.apache.nemo.client.JobLauncher;
+import org.apache.nemo.common.test.ArgBuilder;
+import org.apache.nemo.common.test.ExampleTestUtil;
+import org.apache.nemo.examples.beam.policy.DefaultPolicyParallelismFive;
+import org.apache.nemo.examples.beam.tpch.TpchQueryRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.powermock.modules.junit4.PowerMockRunnerDelegate;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Test modified TPC-H queries.
+ * (1) Queries: 'WHERE' clauses with constant values have been removed to 
ensure the final query result is non-null.
+ * (2) Tables: Some rows are modified such that the results of joins are 
non-null.
+ */
+@RunWith(PowerMockRunner.class)
+@PowerMockRunnerDelegate(Parameterized.class)
+@PrepareForTest(JobLauncher.class)
+public final class ModifiedTPCHITCase {
+  private static final int TIMEOUT = 180000;
+  private static ArgBuilder builder;
+  private static final String fileBasePath = System.getProperty("user.dir") + 
"/../resources/";
+  private static final String tpchTestPath = fileBasePath + "tpch/";
+  private static final String executorResourceFileName = fileBasePath + 
"beam_test_executor_resources.json";
+  private static final String tableDirectoryPath = fileBasePath + 
"tpch/tables/";
+
+  @Parameterized.Parameters(name = "Query{0}")
+  public static Collection<Integer> queries() {
+    // return Arrays.asList(12);
+    return Arrays.asList(3, 4, 6, 10, 12, 13, 14);
+  }
+
+  private final int queryNum;
+
+  public ModifiedTPCHITCase(final int queryNum) {
+    this.queryNum = queryNum;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    builder = new ArgBuilder()
+        .addResourceJson(executorResourceFileName);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      ExampleTestUtil.ensureSQLOutputValidity(tpchTestPath, 
getOutputFileName(), getExpectedOutputFileName());
+    } finally {
+      ExampleTestUtil.deleteOutputFile(tpchTestPath, getOutputFileName());
+    }
+  }
+
+  @Test (timeout = TIMEOUT)
+  public void test() throws Exception {
+    JobLauncher.main(builder
+      .addUserMain(TpchQueryRunner.class.getCanonicalName())
+      .addUserArgs(getQueryFilePath(), tableDirectoryPath, tpchTestPath + 
getOutputFileName())
+      .addJobId(ModifiedTPCHITCase.class.getSimpleName() + "-Query" + queryNum)
+      
.addOptimizationPolicy(DefaultPolicyParallelismFive.class.getCanonicalName())
+      .build());
+  }
+
+  private String getQueryFilePath() {
+    return String.format("%smodified_queries/tpch_query%d.sql", tpchTestPath, 
queryNum);
+  }
+
+  private String getOutputFileName() {
+    return String.format("test_output_%d_", queryNum);
+  }
+
+  private String getExpectedOutputFileName() {
+    return String.format("expected_output_%d", queryNum);
+  }
+}
diff --git a/examples/resources/tpch/expected_output_10 
b/examples/resources/tpch/expected_output_10
new file mode 100644
index 000000000..6512229a9
--- /dev/null
+++ b/examples/resources/tpch/expected_output_10
@@ -0,0 +1,17 @@
+9|Customer#000000009|979245.8|8324.07|INDIA|xKiAFTjUsCuxfeleNqefumTrjS|18-338-906-3675|r
 theodolites according to the requests wake thinly excuses: pending requests 
haggle furiousl
+2|Customer#000000002|543566.44|121.65|JORDAN|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|23-768-687-3665|l
 accounts. blithely ironic theodolites integrate boldly: caref
+8|Customer#000000008|289146.22|6819.74|PERU|I0B10bB0AymmC, 
0PrRYBCP1yGJ8xcBPmWhl5|27-147-574-9335|among the slyly regular theodolites 
kindle blithely courts. carefully even theodolites haggle slyly along the ide
+5|Customer#000000005|276679.3|794.47|CANADA|KvpyuHCplrB84WgAiGV6sYpZq7Tj|13-750-942-6364|n
 accounts will have to unwind. foxes cajole accor
+91|Customer#000000091|251360.94|4643.14|INDIA|S8OMYFrpHwoNHaGBeuS6E 
6zhHGZiprw1b7 q|18-239-400-3677|onic accounts. fluffily silent pinto beans 
boost blithely according to the fluffily exp
+64|Customer#000000064|239504.3|-646.64|CANADA|MbCeGY20kaKK3oalJD,OT|13-558-731-7204|structions
 after the quietly ironic theodolites cajole be
+12|Customer#000000012|230456.73|3396.49|JORDAN|9PWKuhzT4Zr1Q|23-791-276-1263| 
to the carefully final braids. blithely regular requests nag. ironic 
theodolites boost quickly along
+32|Customer#000000032|228617.98|3471.53|MOROCCO|jD2xZzi 
UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|25-430-914-2194|cial ideas. final, furious 
requests across the e
+1|Customer#000000001|161556.8|711.56|MOROCCO|IVhzIApeRb 
ot,c,E|25-989-741-2988|to the even, regular platelets. regular, ironic epitaphs 
nag e
+94|Customer#000000094|130670.08|5500.11|INDONESIA|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|19-953-499-8833|latelets
 across the bold, final requests sleep according to the fluffily bold accounts. 
unusual deposits amon
+54|Customer#000000054|127601.41|868.9|EGYPT|,k4vf 
5vECGWFy,hosTE,|14-776-370-4745|sual, silent accounts. furiously express 
accounts cajole special deposits. final, final accounts use furi
+78|Customer#000000078|92766.78|7136.97|INDONESIA|HBOta,ZNqpg3U2cSL0kbrftkPwzX|19-960-700-9191|ests.
 blithely bold pinto beans h
+86|Customer#000000086|82959.78|3306.32|ALGERIA|US6EGGHXbTTXPL9SBsxQJsuvy|10-677-951-2353|quests.
 pending dugouts are carefully aroun
+3|Customer#000000003|71798.72|7498.12|ARGENTINA|MG9kdTD2WBHm|11-719-748-3364| 
deposits eat slyly ironic, even instructions. express foxes detect slyly. 
blithely even accounts abov
+76|Customer#000000076|65778.53|5745.33|ALGERIA|m3sbCvjMOHyaOofH,e 
UkGPtqc4|10-349-718-3044|pecial deposits. ironic ideas boost blithely according 
to the closely ironic theodolites! furiously final deposits n
+63|Customer#000000063|61669.57|9331.13|VIETNAM|IXRSpVWWZraKII|31-952-552-9584|ithely
 even accounts detect slyly above the fluffily ir
+4|Customer#000000004|49720.01|2866.83|EGYPT|XxVSJsLAGtn|14-128-190-5944| 
requests. final, regular ideas sleep final accou
diff --git a/examples/resources/tpch/expected_output_12 
b/examples/resources/tpch/expected_output_12
new file mode 100644
index 000000000..769578b97
--- /dev/null
+++ b/examples/resources/tpch/expected_output_12
@@ -0,0 +1,5 @@
+AIR|0|1
+FOB|1|0
+REG AIR|0|2
+SHIP|0|3
+TRUCK|2|1
diff --git a/examples/resources/tpch/expected_output_13 
b/examples/resources/tpch/expected_output_13
new file mode 100644
index 000000000..e2c824d6f
--- /dev/null
+++ b/examples/resources/tpch/expected_output_13
@@ -0,0 +1,6 @@
+1|81
+2|11
+3|4
+5|2
+6|1
+4|1
diff --git a/examples/resources/tpch/expected_output_14 
b/examples/resources/tpch/expected_output_14
new file mode 100644
index 000000000..f3935d037
--- /dev/null
+++ b/examples/resources/tpch/expected_output_14
@@ -0,0 +1 @@
+22.963415272
diff --git a/examples/resources/tpch/expected_output_3 
b/examples/resources/tpch/expected_output_3
new file mode 100644
index 000000000..f2b8f9875
--- /dev/null
+++ b/examples/resources/tpch/expected_output_3
@@ -0,0 +1,10 @@
+39|346414.38|1996-09-20|0
+68|344627.5|1998-04-18|0
+69|273498.6|1994-06-04|0
+3|251360.94|1993-10-14|0
+35|239504.3|1995-10-23|0
+7|230456.73|1996-01-10|0
+71|228617.98|1998-01-24|0
+1|215941.45|1996-01-02|0
+33|178583.2|1993-10-27|0
+37|161556.8|1992-06-03|0
diff --git a/examples/resources/tpch/expected_output_4 
b/examples/resources/tpch/expected_output_4
new file mode 100644
index 000000000..592b4be16
--- /dev/null
+++ b/examples/resources/tpch/expected_output_4
@@ -0,0 +1,5 @@
+1-URGENT|4
+2-HIGH|3
+3-MEDIUM|6
+4-NOT SPECIFIED|4
+5-LOW|6
diff --git a/examples/resources/tpch/expected_output_6 
b/examples/resources/tpch/expected_output_6
new file mode 100644
index 000000000..82eabc925
--- /dev/null
+++ b/examples/resources/tpch/expected_output_6
@@ -0,0 +1 @@
+228583.94
diff --git a/examples/resources/tpch/modified_queries/tpch_query10.sql 
b/examples/resources/tpch/modified_queries/tpch_query10.sql
new file mode 100644
index 000000000..5a5357b31
--- /dev/null
+++ b/examples/resources/tpch/modified_queries/tpch_query10.sql
@@ -0,0 +1,29 @@
+select
+       c_custkey,
+       c_name,
+       sum(l_extendedprice * (1 - l_discount)) as revenue,
+       c_acctbal,
+       n_name,
+       c_address,
+       c_phone,
+       c_comment
+from
+       customer,
+       orders,
+       lineitem,
+       nation
+where
+       c_custkey = o_custkey
+       and l_orderkey = o_orderkey
+       and c_nationkey = n_nationkey
+group by
+       c_custkey,
+       c_name,
+       c_acctbal,
+       c_phone,
+       n_name,
+       c_address,
+       c_comment
+order by
+       revenue desc
+limit 20
diff --git a/examples/resources/tpch/modified_queries/tpch_query12.sql 
b/examples/resources/tpch/modified_queries/tpch_query12.sql
new file mode 100644
index 000000000..3baffc6c6
--- /dev/null
+++ b/examples/resources/tpch/modified_queries/tpch_query12.sql
@@ -0,0 +1,26 @@
+select
+       l_shipmode,
+       sum(case
+               when o_orderpriority = '1-URGENT'
+                       or o_orderpriority = '2-HIGH'
+                       then 1
+               else 0
+       end) as high_line_count,
+       sum(case
+               when o_orderpriority <> '1-URGENT'
+                       and o_orderpriority <> '2-HIGH'
+                       then 1
+               else 0
+       end) as low_line_count
+from
+       orders,
+       lineitem
+where
+       o_orderkey = l_orderkey
+       and l_commitdate < l_receiptdate
+       and l_shipdate < l_commitdate
+group by
+       l_shipmode
+order by
+       l_shipmode
+LIMIT 1000
diff --git a/examples/resources/tpch/modified_queries/tpch_query13.sql 
b/examples/resources/tpch/modified_queries/tpch_query13.sql
new file mode 100644
index 000000000..4041f3c9b
--- /dev/null
+++ b/examples/resources/tpch/modified_queries/tpch_query13.sql
@@ -0,0 +1,21 @@
+select
+       c_count,
+       count(*) as custdist
+from
+       (
+               select
+                       c_custkey,
+                       count(o_orderkey) as c_count
+               from
+                       customer left outer join orders on
+                               c_custkey = o_custkey
+                               and o_comment not like '%unusual%accounts%'
+               group by
+                       c_custkey
+       ) c_orders
+group by
+       c_count
+order by
+       custdist desc,
+       c_count desc
+LIMIT 1000
diff --git a/examples/resources/tpch/modified_queries/tpch_query14.sql 
b/examples/resources/tpch/modified_queries/tpch_query14.sql
new file mode 100644
index 000000000..318ca9c5d
--- /dev/null
+++ b/examples/resources/tpch/modified_queries/tpch_query14.sql
@@ -0,0 +1,11 @@
+select
+       100.00 * sum(case
+               when p_type like 'PROMO%'
+                       then l_extendedprice * (1 - l_discount)
+               else 0
+       end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue
+from
+       lineitem,
+       part
+where
+       l_partkey = p_partkey
diff --git a/examples/resources/tpch/modified_queries/tpch_query3.sql 
b/examples/resources/tpch/modified_queries/tpch_query3.sql
new file mode 100644
index 000000000..ae611ebd2
--- /dev/null
+++ b/examples/resources/tpch/modified_queries/tpch_query3.sql
@@ -0,0 +1,20 @@
+select
+       l_orderkey,
+       sum(l_extendedprice * (1 - l_discount)) as revenue,
+       o_orderdate,
+       o_shippriority
+from
+       customer,
+       orders,
+       lineitem
+where
+       c_custkey = o_custkey
+       and l_orderkey = o_orderkey
+group by
+       l_orderkey,
+       o_orderdate,
+       o_shippriority
+order by
+       revenue desc,
+       o_orderdate
+limit 10
diff --git a/examples/resources/tpch/modified_queries/tpch_query4.sql 
b/examples/resources/tpch/modified_queries/tpch_query4.sql
new file mode 100644
index 000000000..27b0d82ca
--- /dev/null
+++ b/examples/resources/tpch/modified_queries/tpch_query4.sql
@@ -0,0 +1,20 @@
+select
+       o_orderpriority,
+       count(*) as order_count
+from
+       orders as o
+where
+       exists (
+               select
+                       *
+               from
+                       lineitem
+               where
+                       l_orderkey = o.o_orderkey
+                       and l_commitdate < l_receiptdate
+       )
+group by
+       o_orderpriority
+order by
+       o_orderpriority
+LIMIT 1000
diff --git a/examples/resources/tpch/modified_queries/tpch_query6.sql 
b/examples/resources/tpch/modified_queries/tpch_query6.sql
new file mode 100644
index 000000000..14ee2e84d
--- /dev/null
+++ b/examples/resources/tpch/modified_queries/tpch_query6.sql
@@ -0,0 +1,4 @@
+select
+       sum(l_extendedprice * l_discount) as revenue
+from
+       lineitem
diff --git a/examples/resources/tpch/tables/customer.tbl 
b/examples/resources/tpch/tables/customer.tbl
new file mode 100644
index 000000000..94beb82ae
--- /dev/null
+++ b/examples/resources/tpch/tables/customer.tbl
@@ -0,0 +1,100 @@
+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 |
diff --git a/examples/resources/tpch/tables/lineitem.tbl 
b/examples/resources/tpch/tables/lineitem.tbl
new file mode 100644
index 000000000..5b4714ff6
--- /dev/null
+++ b/examples/resources/tpch/tables/lineitem.tbl
@@ -0,0 +1,100 @@
+1|1|768951|1|17|33203.72|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER
 IN PERSON|TRUCK|egular courts above the|
+1|6|73|2|36|69788.52|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|63|36|3|8|16381.28|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|213150|463151|4|28|29767.92|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites.
 fluffily even de|
+1|2402664|152671|5|24|37596.96|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB|
 pending foxes. slyly re|
+1|15|63|6|32|48267.84|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER 
IN PERSON|MAIL|arefully slyly ex|
+2|10616973|116994|1|38|71798.72|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|429697|179698|1|45|73200.15|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside
 of the furiously brave acco|
+3|1903543|653547|2|49|75776.05|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE
 BACK RETURN|RAIL| unusual accounts. eve|
+3|12844823|344848|3|27|47713.86|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER
 IN PERSON|SHIP|nal foxes wake. |
+3|2937961|187964|4|2|3997.64|0.01|0.06|A|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y.
 fluffily pending d|
+3|18309408|59463|5|28|39661.72|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE
 BACK RETURN|FOB|ages nag slyly pending|
+3|6|9|6|26|30522.70|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|8803469|553494|1|30|41160.60|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|10856929|856930|1|15|28280.70|0.02|0.04|R|F|1994-10-31|1994-08-31|1994-11-20|NONE|AIR|ts
 wake furiously |
+5|12392679|392680|2|26|46047.56|0.07|0.08|R|F|1994-10-16|1994-09-25|1994-10-19|NONE|FOB|sts
 use slyly quickly special instruc|
+5|3753018|3022|3|50|53541.50|0.08|0.03|A|F|1994-08-08|1994-10-13|1994-08-26|DELIVER
 IN PERSON|AIR|eodolites. fluffily unusual|
+6|13963546|213560|1|37|59527.45|0.08|0.03|A|F|1992-04-27|1992-05-15|1992-05-02|TAKE
 BACK RETURN|TRUCK|p furiously special foxes|
+7|18205184|955239|1|12|13059.24|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|14524275|774290|2|9|11686.95|0.08|0.08|N|O|1996-02-01|1996-03-02|1996-02-19|TAKE
 BACK RETURN|SHIP|es. instructions|
+7|9477974|977993|3|46|89769.00|0.10|0.07|N|O|1996-01-15|1996-03-27|1996-02-03|COLLECT
 COD|MAIL| unusual reques|
+7|16307205|307206|4|28|33918.92|0.03|0.04|N|O|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|.
 slyly special requests haggl|
+7|15189381|939427|5|38|55845.94|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|7925015|175023|6|35|36386.70|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|15723703|223734|7|5|8629.60|0.04|0.02|N|O|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely
 regula|
+32|8|77|1|28|37014.32|0.05|0.08|N|O|1995-10-23|1995-08-27|1995-10-26|TAKE BACK 
RETURN|TRUCK|sleep quickly. req|
+32|19792017|42037|2|32|35456.96|0.02|0.00|N|O|1995-08-14|1995-10-07|1995-08-27|COLLECT
 COD|AIR|lithely regular deposits. fluffily |
+32|4416034|666039|3|2|1899.62|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|274207|774208|4|4|4724.76|0.09|0.03|N|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG
 AIR|e slyly final pac|
+32|8581018|831027|5|44|48337.96|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|1161468|411470|6|6|9176.46|0.04|0.03|N|O|1995-07-21|1995-09-23|1995-07-25|COLLECT
 COD|RAIL| gifts cajole carefully.|
+33|6133519|883538|1|31|48118.51|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|6051869|551882|2|32|58257.92|0.02|0.05|A|F|1993-12-09|1994-01-04|1993-12-28|COLLECT
 COD|MAIL|gular theodolites|
+33|13746855|996869|3|5|9505.85|0.05|0.03|A|F|1993-12-09|1993-12-25|1993-12-23|TAKE
 BACK RETURN|AIR|. stealthily bold exc|
+33|3391749|391750|4|41|75463.78|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE
 BACK RETURN|MAIL|unusual packages doubt caref|
+34|8836137|86146|1|13|13944.97|0.00|0.07|N|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG
 AIR|nic accounts. deposits are alon|
+34|8941332|191341|2|22|30203.58|0.08|0.06|N|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely
 slyly p|
+34|16954305|454338|3|6|8150.76|0.02|0.06|N|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar
 foxes sleep |
+35|44993|294994|1|24|46511.76|0.02|0.00|N|O|1996-02-21|1996-01-03|1996-03-18|TAKE
 BACK RETURN|FOB|, regular tithe|
+35|16193973|443990|2|34|70249.78|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|12089518|839555|3|7|10548.37|0.06|0.04|N|O|1996-01-19|1995-12-22|1996-01-29|NONE|MAIL|
 the carefully regular |
+35|8517403|767412|4|25|35499.50|0.06|0.05|N|O|1995-11-26|1995-12-25|1995-12-21|DELIVER
 IN PERSON|SHIP| quickly unti|
+35|11991616|491639|5|34|58038.68|0.08|0.06|N|O|1995-11-08|1996-01-15|1995-11-26|COLLECT
 COD|MAIL|. silent, unusual deposits boost|
+35|3076173|326177|6|28|32172.56|0.03|0.02|N|O|1996-02-01|1995-12-24|1996-02-28|COLLECT
 COD|RAIL|ly alongside of |
+36|11976645|976646|1|42|72284.10|0.09|0.00|N|O|1996-02-03|1996-01-21|1996-02-23|COLLECT
 COD|SHIP| careful courts. special |
+37|2262908|512911|1|40|74831.60|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|12678128|178153|2|39|43114.11|0.05|0.02|A|F|1992-07-02|1992-08-18|1992-07-28|TAKE
 BACK RETURN|RAIL|the final requests. ca|
+37|1290295|540297|3|43|55264.89|0.05|0.08|A|F|1992-07-10|1992-07-06|1992-08-02|DELIVER
 IN PERSON|TRUCK|iously ste|
+38|17583881|83916|1|44|86416.44|0.04|0.02|N|O|1996-09-29|1996-11-17|1996-09-30|COLLECT
 COD|MAIL|s. blithely unusual theodolites am|
+39|231967|981968|1|44|83553.80|0.09|0.06|N|O|1996-11-14|1996-12-15|1996-12-12|COLLECT
 COD|RAIL|eodolites. careful|
+39|18658106|408161|2|26|27642.42|0.08|0.04|N|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages
 across the slyly silent|
+39|67|53|3|46|50309.28|0.06|0.08|N|O|1996-09-26|1996-12-19|1996-10-26|DELIVER 
IN PERSON|AIR|he carefully e|
+39|20|30|4|32|62396.48|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|54|95|5|43|77984.37|0.01|0.01|N|O|1996-10-17|1996-11-14|1996-10-26|COLLECT 
COD|MAIL|yly regular i|
+39|94|68|6|40|66410.00|0.06|0.05|N|O|1996-12-08|1996-10-22|1997-01-01|COLLECT 
COD|AIR|quickly ironic fox|
+64|85|59|1|21|24835.86|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|59|46|1|26|37702.34|0.03|0.03|A|F|1995-04-20|1995-04-25|1995-05-13|NONE|TRUCK|pending
 deposits nag even packages. ca|
+65|73|88|2|22|33837.98|0.00|0.05|N|O|1995-07-17|1995-06-04|1995-07-19|COLLECT 
COD|FOB| ideas. special, r|
+65|13|38|3|21|37185.33|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|13|76|1|31|53448.34|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|36|34|2|41|77242.77|0.04|0.07|A|F|1994-02-21|1994-03-01|1994-03-18|COLLECT 
COD|AIR| regular de|
+67|5|9|1|4|6273.60|0.09|0.04|N|O|1997-04-17|1997-01-31|1997-04-20|NONE|SHIP| 
cajole thinly expres|
+67|40|5|2|12|13909.68|0.09|0.05|N|O|1997-01-27|1997-02-21|1997-02-22|NONE|REG 
AIR| even packages cajole|
+67|955|60|3|5|10070.45|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|58|79|4|44|61992.48|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|74|87|5|23|28406.61|0.05|0.07|N|O|1997-04-19|1997-02-14|1997-05-06|DELIVER 
IN PERSON|REG AIR|ly regular deposit|
+67|46|8|6|29|42793.85|0.02|0.05|N|O|1997-01-25|1997-01-27|1997-01-27|DELIVER 
IN PERSON|FOB|ultipliers |
+68|1|956702|1|3|5123.01|0.05|0.02|N|O|1998-07-04|1998-06-05|1998-07-21|NONE|RAIL|fully
 special instructions cajole. furious|
+68|910|267962|2|46|88643.84|0.02|0.05|N|O|1998-06-26|1998-06-07|1998-07-05|NONE|MAIL|
 requests are unusual, regular pinto |
+68|16|7|3|46|88032.04|0.04|0.05|N|O|1998-08-13|1998-07-08|1998-08-29|NONE|RAIL|egular
 dependencies affix ironically along |
+68|37|2|4|20|34185.20|0.07|0.01|N|O|1998-06-27|1998-05-23|1998-07-02|NONE|REG 
AIR| excuses integrate fluffily |
+68|34|5|5|27|46151.64|0.03|0.06|N|O|1998-06-19|1998-06-25|1998-06-29|DELIVER 
IN PERSON|SHIP|ccounts. deposits use. furiously|
+68|80|5|6|30|31067.10|0.05|0.06|N|O|1998-08-11|1998-07-11|1998-08-14|NONE|RAIL|oxes
 are slyly blithely fin|
+68|46|1|7|41|68467.95|0.09|0.08|N|O|1998-06-24|1998-06-27|1998-07-06|NONE|SHIP|eposits
 nag special ideas. furiousl|
+69|20|7|1|48|88332.00|0.01|0.07|A|F|1994-08-17|1994-08-11|1994-09-08|NONE|TRUCK|regular
 epitaphs. carefully even ideas hag|
+69|05|917926|2|32|58316.16|0.08|0.06|A|F|1994-08-24|1994-08-17|1994-08-31|NONE|REG
 AIR|s sleep carefully bold, |
+69|47|476687|3|17|28440.32|0.09|0.00|A|F|1994-07-02|1994-07-07|1994-07-03|TAKE 
BACK RETURN|AIR|final, pending instr|
+69|76|250183|4|3|3677.97|0.09|0.04|R|F|1994-06-06|1994-07-27|1994-06-15|NONE|MAIL|
 blithely final d|
+69|9|7|5|42|79611.84|0.07|0.04|R|F|1994-07-31|1994-07-26|1994-08-28|DELIVER IN 
PERSON|REG AIR|tect regular, speci|
+69|1850383|100385|6|23|30665.67|0.05|0.00|A|F|1994-10-03|1994-08-06|1994-10-24|NONE|SHIP|nding
 accounts ca|
+70|6412782|912795|1|8|13555.68|0.03|0.08|R|F|1994-01-12|1994-02-27|1994-01-14|TAKE
 BACK RETURN|FOB|ggle. carefully pending dependenc|
+70|19615517|115556|2|13|18609.89|0.06|0.06|A|F|1994-03-03|1994-02-13|1994-03-26|COLLECT
 COD|AIR|lyly special packag|
+70|17980876|730928|3|1|1955.98|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|4573316|73325|4|11|15279.99|0.01|0.05|A|F|1994-03-17|1994-03-17|1994-03-27|NONE|MAIL|alongside
 of the deposits. fur|
+70|3713070|213077|5|37|40066.93|0.09|0.04|R|F|1994-02-13|1994-03-16|1994-02-21|COLLECT
 COD|MAIL|n accounts are. q|
+70|5|3|6|19|28122.66|0.06|0.03|A|F|1994-01-26|1994-02-17|1994-02-06|TAKE BACK 
RETURN|SHIP| packages wake pending accounts.|
+71|61|193052|1|25|28593.75|0.09|0.07|N|O|1998-04-10|1998-04-22|1998-04-11|COLLECT
 COD|FOB|ckly. slyly|
+71|65|341526|2|3|4794.54|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|3|193199|3|45|50940.45|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|9|914455|4|33|46495.68|0.00|0.01|N|O|1998-04-12|1998-03-20|1998-04-15|NONE|FOB|
 serve quickly fluffily bold deposi|
+71|10|5|5|39|56897.88|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|19|63461|6|34|50471.30|0.04|0.01|N|O|1998-03-05|1998-04-22|1998-03-30|DELIVER
 IN PERSON|TRUCK|s cajole. |
+96|1|5|1|23|36569.31|0.10|0.06|A|F|1994-07-19|1994-06-29|1994-07-25|DELIVER IN 
PERSON|TRUCK|ep-- carefully reg|
+96|1|53|2|30|60459.00|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|8|197710|1|13|22686.30|0.00|0.02|R|F|1993-04-01|1993-04-04|1993-04-08|NONE|TRUCK|ayers
 cajole against the furiously|
+97|4956731|206736|2|37|66137.13|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|77|51|3|19|37325.69|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|40|21|1|28|42514.92|0.06|0.07|A|F|1994-12-24|1994-10-25|1995-01-16|COLLECT 
COD|REG AIR| pending, regular accounts s|
+98|10|72|2|1|1338.72|0.00|0.00|A|F|1994-12-01|1994-12-12|1994-12-15|DELIVER IN 
PERSON|TRUCK|. unusual instructions against|
+98|4470561|470562|3|14|21438.76|0.05|0.02|A|F|1994-12-30|1994-11-22|1995-01-27|COLLECT
 COD|AIR| cajole furiously. blithely ironic ideas |
diff --git a/examples/resources/tpch/tables/nation.tbl 
b/examples/resources/tpch/tables/nation.tbl
new file mode 100644
index 000000000..ed3fd5b8c
--- /dev/null
+++ b/examples/resources/tpch/tables/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/examples/resources/tpch/tables/orders.tbl 
b/examples/resources/tpch/tables/orders.tbl
new file mode 100644
index 000000000..3ae3c7396
--- /dev/null
+++ b/examples/resources/tpch/tables/orders.tbl
@@ -0,0 +1,100 @@
+1|9|O|224560.83|1996-01-02|5-LOW|Clerk#000095055|0|nstructions sleep furiously 
among |
+2|3|O|75388.65|1996-12-01|1-URGENT|Clerk#000087916|0| foxes. pending accounts 
at the pending, silent asymptot|
+3|91|F|255287.36|1993-10-14|5-LOW|Clerk#000095426|0|sly final accounts boost. 
carefully regular ideas cajole carefully. depos|
+4|2|O|43119.84|1995-10-11|5-LOW|Clerk#000012340|0|sits. slyly regular warthogs 
cajole. regular, regular theodolites acro|
+5|9|F|125809.76|1994-07-30|5-LOW|Clerk#000092480|0|quickly. bold deposits 
sleep slyly. packages use slyly|
+6|2|F|56408.20|1992-02-21|4-NOT SPECIFIED|Clerk#000005798|0|ggle. special, 
final requests are against the furiously specia|
+7|12|O|240358.24|1996-01-10|2-HIGH|Clerk#000046961|0|ly special requests |
+32|94|O|136666.23|1995-07-16|2-HIGH|Clerk#000061561|0|ise blithely bold, 
regular requests. quickly unusual dep|
+33|8|F|183460.23|1993-10-27|3-MEDIUM|Clerk#000040860|0|uriously. furiously 
final request|
+34|4|O|52842.63|1998-07-21|3-MEDIUM|Clerk#000022278|0|ly final packages. 
fluffily final deposits wake blithely ideas. spe|
+35|64|O|250679.99|1995-10-23|4-NOT SPECIFIED|Clerk#000025837|0|zzle. carefully 
enticing deposits nag furio|
+36|76|O|65778.53|1995-11-03|1-URGENT|Clerk#000035709|0| quick packages are 
blithely. slyly silent accounts wake qu|
+37|1|F|168618.98|1992-06-03|3-MEDIUM|Clerk#000045557|0|kly regular pinto 
beans. carefully unusual waters cajole never|
+38|86|O|84618.97|1996-08-21|4-NOT SPECIFIED|Clerk#000060366|0|haggle blithely. 
furiously express ideas haggle blithely furiously regular re|
+39|9|O|362571.59|1996-09-20|3-MEDIUM|Clerk#000065858|0|ole express, ironic 
requests: ir|
+64|9|F|24065.94|1994-07-16|3-MEDIUM|Clerk#000066030|0|wake fluffily. sometimes 
ironic pinto beans about the dolphin|
+65|2|P|109405.61|1995-03-18|1-URGENT|Clerk#000063199|0|ular requests are 
blithely pending orbits-- even requests against the deposit|
+66|54|F|137067.96|1994-01-20|5-LOW|Clerk#000074229|0|y pending requests 
integrate|
+67|5|O|163045.34|1996-12-19|4-NOT SPECIFIED|Clerk#000054641|0|symptotes haggle 
slyly around the furiously iron|
+68|2|O|363053.07|1998-04-18|3-MEDIUM|Clerk#000043937|0| pinto beans sleep 
carefully. blithely ironic deposits haggle furiously acro|
+69|9|F|285934.45|1994-06-04|4-NOT SPECIFIED|Clerk#000032989|0| depths atop the 
slyly thin deposits detect among the furiously silent accou|
+70|8|F|115767.18|1993-12-18|5-LOW|Clerk#000032167|0| carefully ironic request|
+71|32|O|238400.84|1998-01-24|4-NOT SPECIFIED|Clerk#000027015|0| express 
deposits along the blithely regul|
+96|78|F|98332.78|1994-04-17|2-HIGH|Clerk#000039452|0|oost furiously. pinto|
+97|5|F|129736.29|1993-01-29|3-MEDIUM|Clerk#000054683|0|hang blithely along the 
regular accounts. furiously even ideas after the|
+98|63|F|84447.83|1994-09-25|1-URGENT|Clerk#000044779|0|c asymptotes. quickly 
regular packages should have to nag re|
+99|1|F|121822.18|1994-03-13|4-NOT SPECIFIED|Clerk#000097225|0|e carefully 
ironic packages. pending|
+100|292|O|264796.29|1998-02-28|4-NOT SPECIFIED|Clerk#000057604|0|heodolites 
detect slyly alongside of the ent|
+101|31|O|116964.71|1996-03-17|3-MEDIUM|Clerk#000041814|0|ding accounts above 
the slyly final asymptote|
+102|32|O|163169.70|1997-05-09|2-HIGH|Clerk#000059518|0| slyly according to the 
asymptotes. carefully final packages integrate furious|
+103|44|O|132518.31|1996-06-20|4-NOT SPECIFIED|Clerk#000008963|0|ges. carefully 
unusual instructions haggle quickly regular f|
+128|57|F|62348.97|1992-06-15|1-URGENT|Clerk#000038412|0|ns integrate fluffily. 
ironic asymptotes after the regular excuses nag around |
+129|28|F|296851.35|1992-11-19|5-LOW|Clerk#000085870|0|ing tithes. carefully 
pending deposits boost about the silently express |
+130|34|F|225040.44|1992-05-08|2-HIGH|Clerk#000003521|0|le slyly unusual, 
regular packages? express deposits det|
+131|65|F|136291.80|1994-06-08|3-MEDIUM|Clerk#000062483|0|after the fluffily 
special foxes integrate s|
+132|41|F|171775.44|1993-06-11|3-MEDIUM|Clerk#000048705|0|sits are daringly 
accounts. carefully regular foxes sleep slyly about the|
+133|90|O|128099.91|1997-11-29|1-URGENT|Clerk#000073724|0|usly final asymptotes 
|
+134|5|F|185430.27|1992-05-01|4-NOT SPECIFIED|Clerk#000071095|0|lar theodolites 
boos|
+135|72|O|268619.37|1995-10-21|4-NOT SPECIFIED|Clerk#000080338|0|l platelets 
use according t|
+160|41|O|140479.65|1996-12-19|4-NOT SPECIFIED|Clerk#000034131|0|thely special 
sauternes wake slyly of t|
+161|93|F|36161.17|1994-08-31|2-HIGH|Clerk#000032126|0|carefully! special 
instructions sin|
+162|43|O|3580.77|1995-05-08|3-MEDIUM|Clerk#000037785|0|nts hinder fluffily 
ironic instructions. express, express excuses |
+163|14|O|184581.82|1997-09-05|3-MEDIUM|Clerk#000037831|0|y final packages. 
final foxes since the quickly even|
+164|15|F|344904.32|1992-10-21|5-LOW|Clerk#000020863|0|cajole ironic courts. 
slyly final ideas are slyly. blithely final Tiresias sub|
+165|15|F|237922.96|1993-01-30|4-NOT SPECIFIED|Clerk#000029181|0|across the 
blithely regular accounts. bold|
+166|41|O|139982.28|1995-09-12|2-HIGH|Clerk#000043909|0|lets. ironic, bold 
asymptotes kindle|
+167|160|F|66094.95|1993-01-04|4-NOT SPECIFIED|Clerk#000073068|0|s nag 
furiously bold excuses. fluffily iron|
+192|53|O|184246.01|1997-11-25|5-LOW|Clerk#000048278|0|y unusual platelets 
among the final instructions integrate rut|
+193|03|F|78960.99|1993-08-08|1-URGENT|Clerk#000002454|0|the furiously final 
pin|
+194|42|F|170839.85|1992-04-05|3-MEDIUM|Clerk#000035124|0|egular requests 
haggle slyly regular, regular pinto beans. asymptote|
+195|065|F|186323.04|1993-12-28|3-MEDIUM|Clerk#000021593|0|old forges are 
furiously sheaves. slyly fi|
+196|14|F|40786.44|1993-03-17|2-HIGH|Clerk#000098715|0|beans boost at the 
foxes. silent foxes|
+197|89|P|153342.22|1995-04-07|2-HIGH|Clerk#000096831|0|solve quickly about the 
even braids. carefully express deposits affix care|
+198|617|O|165318.92|1998-01-02|4-NOT SPECIFIED|Clerk#000033085|0|its. 
carefully ironic requests sleep. furiously express fox|
+199|15|O|107400.78|1996-03-07|2-HIGH|Clerk#000048877|0|g theodolites. special 
packag|
+224|5|F|254004.68|1994-06-18|4-NOT SPECIFIED|Clerk#000064141|0|r the quickly 
thin courts. carefully|
+225|54|P|213071.34|1995-05-25|1-URGENT|Clerk#000017629|0|s. blithely ironic 
accounts wake quickly fluffily special acc|
+226|534|F|272536.56|1993-03-10|2-HIGH|Clerk#000075599|0|s are carefully at the 
blithely ironic acc|
+227|4|O|54123.73|1995-11-10|5-LOW|Clerk#000091823|0| express instructions. 
slyly regul|
+228|58|F|5076.19|1993-02-25|1-URGENT|Clerk#000056159|0|es was slyly among the 
regular foxes. blithely regular dependenci|
+229|742|F|207725.96|1993-12-29|1-URGENT|Clerk#000062769|0|he fluffily even 
instructions. furiously i|
+230|413|F|154763.49|1993-10-27|1-URGENT|Clerk#000051993|0|odolites. carefully 
quick requ|
+231|81|F|222521.76|1994-09-29|2-HIGH|Clerk#000044594|0| packages haggle slyly 
after the carefully ironic instruct|
+256|31|F|139554.18|1993-10-19|4-NOT SPECIFIED|Clerk#000083333|0|he fluffily 
final ideas might are final accounts. carefully f|
+257|245|O|11707.92|1998-03-28|3-MEDIUM|Clerk#000067914|0|ts against the sly 
warhorses cajole slyly accounts|
+258|13|F|264198.36|1993-12-29|1-URGENT|Clerk#000016622|0|dencies. blithely 
quick packages cajole. ruthlessly final accounts|
+259|68|F|109707.34|1993-09-29|4-NOT SPECIFIED|Clerk#000060094|0|ages doubt 
blithely against the final foxes. carefully express deposits dazzle|
+260|698|O|278829.02|1996-12-10|3-MEDIUM|Clerk#000095956|0|lently regular pinto 
beans sleep after the slyly e|
+261|91|F|342835.32|1993-06-29|3-MEDIUM|Clerk#000030925|0|ully fluffily brave 
instructions. furiousl|
+262|41|O|156996.97|1995-11-25|4-NOT SPECIFIED|Clerk#000055032|0|l packages. 
blithely final pinto beans use carefu|
+263|815|F|111379.10|1994-05-17|2-HIGH|Clerk#000008777|0| pending instructions. 
blithely un|
+288|2|O|220364.17|1997-02-21|1-URGENT|Clerk#000010830|0|uriously final 
requests. even, final ideas det|
+289|952|O|241439.49|1997-02-10|3-MEDIUM|Clerk#000010280|0|sily. slyly special 
excuse|
+290|33|F|76512.31|1994-01-01|4-NOT SPECIFIED|Clerk#000073499|0|efully dogged 
deposits. furiou|
+291|970|F|94081.55|1994-03-13|1-URGENT|Clerk#000092249|0|dolites. carefully 
regular pinto beans cajol|
+292|44|F|52007.94|1992-01-13|2-HIGH|Clerk#000019246|0|g pinto beans will have 
to sleep f|
+293|4|F|65700.56|1992-10-02|2-HIGH|Clerk#000062828|0|re bold, ironic deposits. 
platelets c|
+294|96|F|62890.83|1993-07-16|3-MEDIUM|Clerk#000049814|0|kly according to the 
frays. final dolphins affix quickly |
+295|18|F|122727.08|1994-09-29|2-HIGH|Clerk#000015432|0| unusual pinto beans 
play. regular ideas haggle|
+320|23|O|60089.83|1997-11-21|2-HIGH|Clerk#000057205|0|ar foxes nag blithely|
+321|81|F|87042.13|1993-03-21|3-MEDIUM|Clerk#000028888|0|equests run. blithely 
final dependencies after the deposits wake caref|
+322|490|F|184303.59|1992-03-19|1-URGENT|Clerk#000015726|0|fully across the 
slyly bold packages. packages against the quickly regular i|
+323|82|F|132963.26|1994-03-26|1-URGENT|Clerk#000095867|0|arefully pending 
foxes sleep blithely. slyly express accoun|
+324|500|F|37597.36|1992-03-20|1-URGENT|Clerk#000035187|0| about the ironic, 
regular deposits run blithely against the excuses|
+325|58|F|101593.76|1993-10-17|5-LOW|Clerk#000084343|0|ly sometimes pending pa|
+326|88|O|329579.91|1995-06-04|2-HIGH|Clerk#000046505|0| requests. furiously 
ironic asymptotes mold carefully alongside of the blit|
+327|788|P|33297.18|1995-04-17|5-LOW|Clerk#000099137|0|ng the slyly final 
courts. slyly even escapades eat |
+352|576|F|19651.49|1994-03-08|2-HIGH|Clerk#000093149|0|ke slyly bold pinto 
beans. blithely regular accounts against the spe|
+353|5|F|249754.40|1993-12-31|5-LOW|Clerk#000044812|0| quiet ideas sleep. even 
instructions cajole slyly. silently spe|
+354|80|O|237094.37|1996-03-14|2-HIGH|Clerk#000051012|0|ly regular ideas wake 
across the slyly silent ideas. final deposits eat b|
+355|36|F|131255.65|1994-06-14|5-LOW|Clerk#000053109|0|s. sometimes regular 
requests cajole. regular, pending accounts a|
+356|850|F|194317.32|1994-06-30|4-NOT SPECIFIED|Clerk#000094368|0|as wake along 
the bold accounts. even, |
+357|63|O|135357.57|1996-10-09|2-HIGH|Clerk#000030068|0|e blithely about the 
express, final accounts. quickl|
+358|8|F|354125.33|1993-09-20|2-HIGH|Clerk#000039192|0|l, silent instructions 
are slyly. silently even de|
+359|40|F|226692.28|1994-12-19|3-MEDIUM|Clerk#000093374|0|n dolphins. special 
courts above the carefully ironic requests use|
+384|85|F|140827.96|1992-03-03|5-LOW|Clerk#000020573|0|, even accounts use 
furiously packages. slyly ironic pla|
+385|21|O|78611.41|1996-03-22|5-LOW|Clerk#000059908|0|hless accounts unwind 
bold pain|
+386|9|F|105060.90|1995-01-25|2-HIGH|Clerk#000064794|0| haggle quickly. 
stealthily bold asymptotes haggle among the furiously even re|
+387|3|O|188747.81|1997-01-26|4-NOT SPECIFIED|Clerk#000076798|0| are carefully 
among the quickly even deposits. furiously silent req|
+388|52|F|217673.35|1992-12-16|4-NOT SPECIFIED|Clerk#000035575|0|ar foxes above 
the furiously ironic deposits nag slyly final reque|
diff --git a/examples/resources/tpch/tables/part.tbl 
b/examples/resources/tpch/tables/part.tbl
new file mode 100644
index 000000000..1e03f7abb
--- /dev/null
+++ b/examples/resources/tpch/tables/part.tbl
@@ -0,0 +1,100 @@
+1|goldenrod lavender spring chocolate lace|Manufacturer#1|Brand#13|PROMO 
BURNISHED COPPER|7|JUMBO PKG|901.00|ly. slyly ironi|
+2|blush thistle blue yellow saddle|Manufacturer#1|Brand#13|LARGE BRUSHED 
BRASS|1|LG CASE|902.00|lar accounts amo|
+3|spring green yellow purple cornsilk|Manufacturer#4|Brand#42|STANDARD 
POLISHED BRASS|21|WRAP CASE|903.00|egular deposits hag|
+4|cornflower chocolate smoke green pink|Manufacturer#3|Brand#34|SMALL PLATED 
BRASS|14|MED DRUM|904.00|p furiously r|
+5|forest brown coral puff cream|Manufacturer#3|Brand#32|STANDARD POLISHED 
TIN|15|SM PKG|905.00| wake carefully |
+6|bisque cornflower lawn forest magenta|Manufacturer#2|Brand#24|PROMO PLATED 
STEEL|4|MED BAG|906.00|sual a|
+7|moccasin green thistle khaki floral|Manufacturer#1|Brand#11|SMALL PLATED 
COPPER|45|SM BAG|907.00|lyly. ex|
+8|misty lace thistle snow royal|Manufacturer#4|Brand#44|PROMO BURNISHED 
TIN|41|LG DRUM|908.00|eposi|
+9|thistle dim navajo dark gainsboro|Manufacturer#4|Brand#43|SMALL BURNISHED 
STEEL|12|WRAP CASE|909.00|ironic foxe|
+10|linen pink saddle puff powder|Manufacturer#5|Brand#54|LARGE BURNISHED 
STEEL|44|LG CAN|910.01|ithely final deposit|
+11|spring maroon seashell almond orchid|Manufacturer#2|Brand#25|STANDARD 
BURNISHED NICKEL|43|WRAP BOX|911.01|ng gr|
+12|cornflower wheat orange maroon ghost|Manufacturer#3|Brand#33|MEDIUM 
ANODIZED STEEL|25|JUMBO CASE|912.01| quickly|
+13|ghost olive orange rosy thistle|Manufacturer#5|Brand#55|MEDIUM BURNISHED 
NICKEL|1|JUMBO PACK|913.01|osits.|
+14|khaki seashell rose cornsilk navajo|Manufacturer#1|Brand#13|SMALL POLISHED 
STEEL|28|JUMBO BOX|914.01|kages c|
+15|blanched honeydew sky turquoise medium|Manufacturer#1|Brand#15|LARGE 
ANODIZED BRASS|45|LG CASE|915.01|usual ac|
+16|deep sky turquoise drab peach|Manufacturer#3|Brand#32|PROMO PLATED 
TIN|2|MED PACK|916.01|unts a|
+17|indian navy coral pink deep|Manufacturer#4|Brand#43|ECONOMY BRUSHED 
STEEL|16|LG BOX|917.01| regular accounts|
+18|turquoise indian lemon lavender misty|Manufacturer#1|Brand#11|SMALL 
BURNISHED STEEL|42|JUMBO PACK|918.01|s cajole slyly a|
+19|chocolate navy tan deep brown|Manufacturer#2|Brand#23|SMALL ANODIZED 
NICKEL|33|WRAP BOX|919.01| pending acc|
+20|ivory navy honeydew sandy midnight|Manufacturer#1|Brand#12|LARGE POLISHED 
NICKEL|48|MED BAG|920.02|are across the asympt|
+21|lemon floral azure frosted lime|Manufacturer#3|Brand#33|SMALL BURNISHED 
TIN|31|MED BAG|921.02|ss packages. pendin|
+22|medium forest blue ghost black|Manufacturer#4|Brand#43|PROMO POLISHED 
BRASS|19|LG DRUM|922.02| even p|
+23|coral lavender seashell rosy burlywood|Manufacturer#3|Brand#35|MEDIUM 
BURNISHED TIN|42|JUMBO JAR|923.02|nic, fina|
+24|seashell coral metallic midnight floral|Manufacturer#5|Brand#52|MEDIUM 
PLATED STEEL|20|MED CASE|924.02| final the|
+25|aquamarine steel firebrick light turquoise|Manufacturer#5|Brand#55|STANDARD 
BRUSHED COPPER|3|JUMBO BAG|925.02|requests wake|
+26|beige frosted moccasin chocolate snow|Manufacturer#3|Brand#32|SMALL BRUSHED 
STEEL|32|SM CASE|926.02| instructions i|
+27|saddle puff beige linen yellow|Manufacturer#1|Brand#14|LARGE ANODIZED 
TIN|20|MED PKG|927.02|s wake. ir|
+28|navajo yellow drab white misty|Manufacturer#4|Brand#44|SMALL PLATED 
COPPER|19|JUMBO PKG|928.02|x-ray pending, iron|
+29|lemon sky grey salmon orchid|Manufacturer#3|Brand#33|PROMO PLATED 
COPPER|7|LG DRUM|929.02| carefully fluffi|
+30|cream misty steel spring medium|Manufacturer#4|Brand#42|PROMO ANODIZED 
TIN|17|LG BOX|930.03|carefully bus|
+31|slate seashell steel medium moccasin|Manufacturer#5|Brand#53|STANDARD 
BRUSHED TIN|10|LG BAG|931.03|uriously s|
+32|sandy wheat coral spring burnished|Manufacturer#4|Brand#42|ECONOMY PLATED 
BRASS|31|LG CASE|932.03|urts. carefully fin|
+33|spring bisque salmon slate pink|Manufacturer#2|Brand#22|ECONOMY PLATED 
NICKEL|16|LG PKG|933.03|ly eve|
+34|khaki steel rose ghost salmon|Manufacturer#1|Brand#13|LARGE BRUSHED 
STEEL|8|JUMBO BOX|934.03|riously ironic|
+35|green blush tomato burlywood seashell|Manufacturer#4|Brand#43|MEDIUM 
ANODIZED BRASS|14|JUMBO PACK|935.03|e carefully furi|
+36|chiffon tan forest moccasin dark|Manufacturer#2|Brand#25|SMALL BURNISHED 
COPPER|3|JUMBO CAN|936.03|olites o|
+37|royal coral orange burnished navajo|Manufacturer#4|Brand#45|LARGE POLISHED 
TIN|48|JUMBO BOX|937.03|silent |
+38|seashell papaya white mint brown|Manufacturer#4|Brand#43|ECONOMY ANODIZED 
BRASS|11|SM JAR|938.03|structions inte|
+39|rose medium floral salmon powder|Manufacturer#5|Brand#53|SMALL POLISHED 
TIN|43|JUMBO JAR|939.03|se slowly above the fl|
+40|lemon midnight metallic sienna steel|Manufacturer#2|Brand#25|ECONOMY 
BURNISHED COPPER|27|SM CASE|940.04|! blithely specia|
+41|burlywood goldenrod pink peru sienna|Manufacturer#2|Brand#23|ECONOMY 
ANODIZED TIN|7|WRAP JAR|941.04|uriously. furiously cl|
+42|midnight turquoise lawn beige thistle|Manufacturer#5|Brand#52|MEDIUM 
BURNISHED TIN|45|LG BOX|942.04|the slow|
+43|medium lace midnight royal chartreuse|Manufacturer#4|Brand#44|PROMO 
POLISHED STEEL|5|WRAP CASE|943.04|e slyly along the ir|
+44|saddle cream wheat lemon burnished|Manufacturer#4|Brand#45|MEDIUM PLATED 
TIN|48|SM PACK|944.04|pinto beans. carefully|
+45|lawn peru ghost khaki maroon|Manufacturer#4|Brand#43|SMALL BRUSHED 
NICKEL|9|WRAP BAG|945.04|nts bo|
+46|honeydew turquoise aquamarine spring tan|Manufacturer#1|Brand#11|STANDARD 
POLISHED TIN|45|WRAP CASE|946.04|the blithely unusual |
+47|honeydew red azure magenta brown|Manufacturer#4|Brand#45|LARGE BURNISHED 
BRASS|14|JUMBO PACK|947.04| even plate|
+48|slate thistle cornsilk pale forest|Manufacturer#5|Brand#53|STANDARD BRUSHED 
STEEL|27|JUMBO CASE|948.04|ng to the depo|
+49|light firebrick cyan puff blue|Manufacturer#2|Brand#24|SMALL BURNISHED 
TIN|31|MED DRUM|949.04|ar pack|
+50|linen blanched tomato slate medium|Manufacturer#3|Brand#33|LARGE ANODIZED 
TIN|25|WRAP PKG|950.05|kages m|
+51|lime frosted indian dodger linen|Manufacturer#4|Brand#45|ECONOMY BURNISHED 
NICKEL|34|JUMBO PACK|951.05|n foxes|
+52|lemon midnight lace sky deep|Manufacturer#3|Brand#35|STANDARD BURNISHED 
TIN|25|WRAP CASE|952.05| final deposits. fu|
+53|bisque rose cornsilk seashell purple|Manufacturer#2|Brand#23|ECONOMY 
BURNISHED NICKEL|32|MED BAG|953.05|mptot|
+54|blanched mint yellow papaya cyan|Manufacturer#2|Brand#21|LARGE BURNISHED 
COPPER|19|WRAP CASE|954.05|e blithely|
+55|sky cream deep tomato rosy|Manufacturer#2|Brand#23|ECONOMY BRUSHED 
COPPER|9|MED BAG|955.05|ly final pac|
+56|antique beige brown deep dodger|Manufacturer#1|Brand#12|MEDIUM PLATED 
STEEL|20|WRAP DRUM|956.05|ts. blithel|
+57|purple blue light sienna deep|Manufacturer#3|Brand#32|MEDIUM BURNISHED 
BRASS|49|MED PKG|957.05|lly abov|
+58|linen hot cornsilk drab bisque|Manufacturer#5|Brand#53|STANDARD POLISHED 
TIN|44|LG PACK|958.05| fluffily blithely reg|
+59|misty brown medium mint salmon|Manufacturer#5|Brand#53|MEDIUM POLISHED 
TIN|2|LG BAG|959.05|regular exc|
+60|snow spring sandy olive tomato|Manufacturer#1|Brand#11|LARGE POLISHED 
COPPER|27|JUMBO CASE|960.06| integ|
+61|light tan linen tomato peach|Manufacturer#5|Brand#54|SMALL BURNISHED 
NICKEL|18|WRAP DRUM|961.06|es. blithely en|
+62|tan cornsilk spring grey chocolate|Manufacturer#3|Brand#35|STANDARD BRUSHED 
BRASS|39|JUMBO BOX|962.06|ckly across the carefu|
+63|burnished puff coral light papaya|Manufacturer#3|Brand#32|STANDARD 
BURNISHED NICKEL|10|JUMBO CAN|963.06| quickly |
+64|aquamarine coral lemon ivory gainsboro|Manufacturer#2|Brand#21|MEDIUM 
ANODIZED BRASS|1|JUMBO CAN|964.06|efully regular pi|
+65|slate drab medium puff gainsboro|Manufacturer#5|Brand#53|MEDIUM BRUSHED 
COPPER|3|MED CAN|965.06|posits after the quic|
+66|cornflower pale almond lemon linen|Manufacturer#3|Brand#35|PROMO ANODIZED 
NICKEL|46|SM CASE|966.06|haggle blithely iro|
+67|slate salmon rose spring seashell|Manufacturer#2|Brand#21|SMALL BRUSHED 
TIN|31|WRAP DRUM|967.06| regular, p|
+68|bisque ivory mint purple almond|Manufacturer#1|Brand#11|PROMO ANODIZED 
STEEL|10|WRAP BOX|968.06|eposits shall h|
+69|lace burnished rosy antique metallic|Manufacturer#5|Brand#52|MEDIUM 
POLISHED BRASS|2|SM BOX|969.06|ely final depo|
+70|violet seashell firebrick dark navajo|Manufacturer#1|Brand#11|STANDARD 
BRUSHED STEEL|42|LG PACK|970.07|inal gifts. sl|
+71|violet firebrick cream peru white|Manufacturer#3|Brand#33|STANDARD PLATED 
BRASS|26|WRAP DRUM|971.07| packages alongside|
+72|hot spring yellow azure dodger|Manufacturer#2|Brand#23|STANDARD ANODIZED 
TIN|25|JUMBO PACK|972.07|efully final the|
+73|cream moccasin royal dim chiffon|Manufacturer#2|Brand#21|SMALL BRUSHED 
COPPER|35|WRAP DRUM|973.07|ts haggl|
+74|frosted grey aquamarine thistle papaya|Manufacturer#5|Brand#55|ECONOMY 
ANODIZED BRASS|25|JUMBO CASE|974.07|ent foxes|
+75|aquamarine maroon wheat salmon metallic|Manufacturer#3|Brand#35|SMALL 
BURNISHED NICKEL|39|SM JAR|975.07|s sleep furiou|
+76|rosy light lime puff sandy|Manufacturer#3|Brand#34|MEDIUM BRUSHED 
COPPER|9|SM PKG|976.07|n accounts sleep qu|
+77|mint bisque chiffon snow firebrick|Manufacturer#5|Brand#52|STANDARD BRUSHED 
COPPER|13|MED PKG|977.07|uests.|
+78|blush forest slate seashell puff|Manufacturer#1|Brand#14|ECONOMY POLISHED 
STEEL|24|LG JAR|978.07|icing deposits wake|
+79|gainsboro pink grey tan almond|Manufacturer#4|Brand#45|PROMO ANODIZED 
BRASS|22|JUMBO BAG|979.07| foxes are slyly regu|
+80|tomato chartreuse coral turquoise linen|Manufacturer#4|Brand#44|PROMO 
PLATED BRASS|28|MED CAN|980.08|unusual dependencies i|
+81|misty sandy cornsilk dodger blush|Manufacturer#5|Brand#53|ECONOMY BRUSHED 
TIN|21|MED BAG|981.08|ove the furiou|
+82|khaki tomato purple almond tan|Manufacturer#1|Brand#15|ECONOMY POLISHED 
TIN|12|WRAP BOX|982.08|ial requests haggle |
+83|blush green dim lawn peru|Manufacturer#1|Brand#12|PROMO BURNISHED 
NICKEL|47|SM CAN|983.08|ly regul|
+84|salmon floral cream rose dark|Manufacturer#4|Brand#45|SMALL ANODIZED 
NICKEL|26|JUMBO PACK|984.08|ideas nag|
+85|dim deep aquamarine smoke pale|Manufacturer#5|Brand#55|PROMO ANODIZED 
NICKEL|16|LG BAG|985.08| silent|
+86|green blanched firebrick dim cream|Manufacturer#4|Brand#44|STANDARD PLATED 
TIN|37|LG CASE|986.08| daring sheaves |
+87|purple lace seashell antique orange|Manufacturer#4|Brand#41|LARGE PLATED 
STEEL|41|WRAP PACK|987.08|yly final|
+88|lime orange bisque chartreuse lemon|Manufacturer#4|Brand#44|PROMO PLATED 
COPPER|16|SM CASE|988.08|e regular packages. |
+89|ghost lace lemon sienna saddle|Manufacturer#5|Brand#53|STANDARD BURNISHED 
STEEL|7|MED JAR|989.08|y final pinto |
+90|hot rosy violet plum pale|Manufacturer#5|Brand#51|ECONOMY POLISHED 
STEEL|49|JUMBO CAN|990.09|caref|
+91|misty bisque lavender spring turquoise|Manufacturer#2|Brand#21|STANDARD 
BRUSHED TIN|32|JUMBO PKG|991.09|counts dete|
+92|blush magenta ghost tomato rose|Manufacturer#2|Brand#22|STANDARD ANODIZED 
TIN|35|JUMBO PKG|992.09|he ironic accounts. sp|
+93|pale yellow cornsilk dodger moccasin|Manufacturer#2|Brand#24|LARGE ANODIZED 
TIN|2|WRAP DRUM|993.09| platel|
+94|blanched pink frosted mint snow|Manufacturer#3|Brand#35|STANDARD POLISHED 
BRASS|32|SM BOX|994.09|s accounts cajo|
+95|dodger beige wheat orchid navy|Manufacturer#3|Brand#33|LARGE BRUSHED 
TIN|36|WRAP DRUM|995.09| final pinto beans |
+96|chocolate light firebrick rose indian|Manufacturer#5|Brand#53|STANDARD 
BRUSHED STEEL|32|SM CASE|996.09|ng to the bli|
+97|coral dodger beige black chartreuse|Manufacturer#3|Brand#33|MEDIUM POLISHED 
BRASS|49|WRAP CAN|997.09|ss excuses sleep am|
+98|frosted peru chiffon yellow aquamarine|Manufacturer#5|Brand#54|STANDARD 
ANODIZED BRASS|22|MED JAR|998.09|e the q|
+99|mint grey purple sienna metallic|Manufacturer#2|Brand#21|SMALL BURNISHED 
STEEL|11|JUMBO PKG|999.09|press|
+100|cyan orchid indian cornflower saddle|Manufacturer#3|Brand#33|ECONOMY 
ANODIZED TIN|4|LG BAG|1000.10|of the steal|
diff --git a/examples/resources/tpch/tables/partsupp.tbl 
b/examples/resources/tpch/tables/partsupp.tbl
new file mode 100644
index 000000000..45b61bbac
--- /dev/null
+++ b/examples/resources/tpch/tables/partsupp.tbl
@@ -0,0 +1,100 @@
+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|3|8076|993.49|ven ideas. quickly even packages print. pending multipliers 
must have to are fluff|
+1|5|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|66|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|25|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|55|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|75|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|25|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|50|3917|645.40|of the blithely regular theodolites. final theodolites haggle 
blithely carefully unusual ideas. blithely even f|
+3|75|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|2|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|5|2694|51.37|g, regular deposits: quick instructions run across the 
carefully ironic theodolites-- final dependencies haggle into the dependencies. 
f|
+4|7|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|25|9653|50.52|y stealthy deposits. furiously final pinto beans wake furiou|
+5|5|1329|219.83|iously regular deposits wake deposits. pending pinto beans 
promise ironic dependencies. even, regular pinto beans integrate|
+5|75|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|25|1627|424.25| quick packages. ironic deposits print. furiously silent 
platelets across the carefully final requests are slyly along the furiously 
even instructi|
+6|50|3336|642.13|final instructions. courts wake packages. blithely unusual 
realms along the multipliers nag |
+6|75|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|2|2770|149.66|hould have to nag after the blithely final asymptotes. 
fluffily spe|
+7|50|3377|68.77|usly against the daring asymptotes. slyly regular platelets 
sleep quickly blithely regular deposits. boldly regular deposits wake blithely 
ironic accounts|
+7|75|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|2|396|957.34|r accounts. furiously pending dolphins use even, regular 
platelets. final|
+8|9|9845|220.62|s against the fluffily special packages snooze slyly slyly 
regular p|
+8|75|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|25|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|50|9583|381.31|d foxes. final, even braids sleep slyly slyly regular ideas. 
unusual ideas above|
+9|75|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|11|2952|996.12| bold foxes wake quickly even, final asymptotes. blithely 
even depe|
+10|25|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|50|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|75|841|374.02|refully above the ironic packages. quickly regular packages 
haggle foxes. blithely ironic deposits a|
+11|12|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|25|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|50|3708|818.74|inal accounts nag quickly slyly special frays; bold, final 
theodolites play slyly after the furiously pending packages. f|
+11|75|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|13|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|25|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|50|824|337.06|es are unusual deposits. fluffily even deposits across the 
blithely final theodolites doubt across the unusual accounts. regular, |
+12|75|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|14|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|25|7268|862.70|s sleep slyly packages. final theodolites to the express 
packages haggle quic|
+13|50|864|38.64|s after the slyly pending instructions haggle even, express 
requests. permanently regular pinto beans are. slyly pending req|
+13|77|9736|327.18|tect after the express instructions. furiously silent ideas 
sleep blithely special ideas. attainments sleep furiously. carefully bold 
requests ab|
+14|15|5278|650.07|e quickly among the furiously ironic accounts. special, 
final sheaves against the|
+14|250015|5334|889.50|ss dependencies are furiously silent excuses. blithely 
ironic pinto beans affix quickly according to the slyly ironic asymptotes. 
final packag|
+14|500015|3676|893.39|sits are according to the fluffily silent asymptotes. 
final ideas are slyly above the regular instructions. furiousl|
+14|750015|4947|310.13| final deposits boost slyly regular packages; carefully 
pending theodolites |
+15|16|7047|835.70|blithely quick requests sleep carefully fluffily regular 
pinto beans. ironic pinto beans around the slyly regular foxe|
+15|250016|3336|784.55|slyly. fluffily bold accounts cajole furiously. 
furiously regular dependencies wak|
+15|500016|3316|265.89|e express instructions. ironic requests haggle fluffily 
along the carefully even packages. furiously final acco|
+15|750016|5255|458.67|refully bold instructions among the silent grouches must 
boost against the express deposits:|
+16|17|5282|709.16|lithely ironic theodolites should have to are furiously-- |
+16|250017|9412|887.53|ly special accounts wake. fluffily bold ideas believe 
blith|
+16|500017|854|781.91| unusual excuses. requests after the carefully regular 
pinto |
+16|750017|1491|918.51|unts cajole furiously across the fluffily pending 
instructions. slyly special accounts could have to boost b|
+17|18|8555|995.35|are furiously final accounts. carefully unusual accounts 
snooze across the requests. carefully special dolphins|
+17|250018|7737|648.75|e blithely express accounts. foxes kindle slyly unusual 
dinos. quickly special f|
+17|500018|3123|555.04|ly bold accounts. regular packages use silently. quickly 
unusual sentiments around the quickly ironic theodolites haggle furiously 
pending requests. care|
+17|750018|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|19|1125|664.17|. ironic, regular accounts across the furiously express |
+18|250019|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|500019|3133|568.61|riously bold accounts. packages boost daringly. blithely 
regular requests cajole. regular foxes wake carefully final accounts. blithely 
unusual excuses det|
+18|750019|6475|386.29|. furiously regular accounts cajole slyly across the 
pending|
+19|20|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|250020|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|500020|8800|635.66|sual requests sleep carefully. deposits cajole carefully 
over the regular, regular requests. quickly unusual asymptotes use some|
+19|750020|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|21|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|250021|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|500021|5905|546.66|ing deposits use furiously. ironically final pinto bea|
+20|750021|4271|115.89|xcuses wake at the deposits. regular pinto beans nag 
slyly fluffi|
+21|22|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|250022|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|500022|7153|664.50|blithely enticing instructions use alongside of the 
carefully thin deposits. blithely bold requests are fluffily|
+21|750022|367|584.86|ong the even theodolites. pending, pending accounts 
sleep-- courts boost quickly at the accounts. quickly fin|
+22|23|4410|786.18|even accounts. final excuses try to sleep regular, even 
packages. carefully express dolphins cajole; furiously special pinto bea|
+22|2|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|5|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|7|1434|597.21|ix across the blithely express packages. carefully regular 
pinto beans boost across the special, pending d|
+23|24|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|25|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|50|9898|233.94|l, express packages wake permanently. quickly even deposits 
sleep quickly slyly silent id|
+23|75|7035|51.75|xcuses; decoys wake after the pending packages. final 
instructions are furi|
+24|2|5180|905.41|heodolites above the ironic requests poach fluffily carefully 
unusual pinto beans. even packages acc|
+24|25|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|50|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|75|5318|62.15| careful requests cajole blithely realms. special asymptotes 
sleep. pinto beans sleep carefully furiously ironic packages. furiously |
+25|26|9029|832.74|fully fluffily regular frets. sometimes even requests after 
the requests wake slyly at the quickly ruthless requests. a|
+25|29|9062|928.96|he foxes. final, final accounts sleep. boldly ironic excuses 
thrash quick|
+25|50|9946|694.35|ld, ironic requests. furiously special packages cajole 
furiously enticing instructions.|
+25|75|7340|746.59|dly final packages haggle blithely according to the pending 
packages. slyly regula|
diff --git a/examples/resources/tpch/tables/region.tbl 
b/examples/resources/tpch/tables/region.tbl
new file mode 100644
index 000000000..c5ebb63b6
--- /dev/null
+++ b/examples/resources/tpch/tables/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/examples/resources/tpch/tables/supplier.tbl 
b/examples/resources/tpch/tables/supplier.tbl
new file mode 100644
index 000000000..f9cce832f
--- /dev/null
+++ b/examples/resources/tpch/tables/supplier.tbl
@@ -0,0 +1,100 @@
+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|
+11|Supplier#000000011|JfwTs,LZrV, M,9C|18|28-613-996-1505|3393.08|y ironic 
packages. slyly ironic accounts affix furiously; ironically unusual excuses 
across the flu|
+12|Supplier#000000012|aLIW  q0HYd|8|18-179-925-7181|1432.69|al packages nag 
alongside of the bold instructions. express, daring accounts|
+13|Supplier#000000013|HK71HQyWoqRWOX8GI 
FpgAifW,2PoH|3|13-727-620-7813|9107.22|requests engage regularly instructions. 
furiously special requests ar|
+14|Supplier#000000014|EXsnO5pTNj4iZRm|15|25-656-247-5058|9189.82|l accounts 
boost. fluffily bold warhorses wake|
+15|Supplier#000000015|olXVbNBfVzRqgokr1T,Ie|8|18-453-357-6394|308.56| across 
the furiously regular platelets wake even deposits. quickly express she|
+16|Supplier#000000016|YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh|22|32-822-502-4215|2972.26|ously
 express ideas haggle quickly dugouts? fu|
+17|Supplier#000000017|c2d,ESHRSkK3WYnxpgw6aOqN0q|19|29-601-884-9219|1687.81|eep
 against the furiously bold ideas. fluffily bold packa|
+18|Supplier#000000018|PGGVE5PWAMwKDZw |16|26-729-551-1115|7040.82|accounts 
snooze slyly furiously bold |
+19|Supplier#000000019|edZT3es,nBFD8lBXTGeTl|24|34-278-310-2731|6150.38|refully 
final foxes across the dogged theodolites sleep slyly abou|
+20|Supplier#000000020|iybAE,RmTymrZVYaFZva2SH,j|3|13-715-945-6730|530.82|n, 
ironic ideas would nag blithely about the slyly regular accounts. silent, expr|
+21|Supplier#000000021|81CavellcrJ0PQ3CPBID0Z0JwyJm0ka5igEs|2|12-253-590-5816|9365.80|d.
 instructions integrate sometimes slyly pending instructions. accounts nag 
among the |
+22|Supplier#000000022|okiiQFk 8lm6EVX6Q0,bEcO|4|14-144-830-2814|-966.20| 
ironically among the deposits. closely expre|
+23|Supplier#000000023|ssetugTcXc096qlD7 
2TL5crEEeS3zk|9|19-559-422-5776|5926.41|ges could have to are ironic deposits. 
regular, even request|
+24|Supplier#000000024|C4nPvLrVmKPPabFCj|0|10-620-939-2254|9170.71|usly pending 
deposits. slyly final accounts run |
+25|Supplier#000000025|RCQKONXMFnrodzz6w7fObFVV6CUm2q|22|32-431-945-3541|9198.31|ely
 regular deposits. carefully regular sauternes engage furiously above the 
regular accounts. idly |
+26|Supplier#000000026|iV,MHzAx6Z939uzFNkq09M0a1 
MBfH7|21|31-758-894-4436|21.18| ideas poach carefully after the blithely bold 
asymptotes. furiously pending theodoli|
+27|Supplier#000000027|lC4CjKwNHUr6L4xIpzOBK4NlHkFTg|18|28-708-999-2028|1887.62|s
 according to the quickly regular hockey playe|
+28|Supplier#000000028|GBhvoRh,7YIN V|0|10-538-384-8460|-891.99|ld requests 
across the pinto beans are carefully against the quickly final courts. accounts 
sleep |
+29|Supplier#000000029|658tEqXLPvRd6xpFdqC2|1|11-555-705-5922|-811.62|y express 
ideas play furiously. even accounts sleep fluffily across the accounts. careful|
+30|Supplier#000000030|84NmC1rmQfO0fj3zkobLT|16|26-940-594-4852|8080.14|ias. 
carefully silent accounts cajole blithely. pending, special accounts cajole 
quickly above the f|
+31|Supplier#000000031|fRJimA7zchyApqRLHcQeocVpP|16|26-515-530-4159|5916.91|into
 beans wake after the special packages. slyly fluffy requests cajole furio|
+32|Supplier#000000032|yvoD3TtZSx1skQNCK8agk5bZlZLug|23|33-484-637-7873|3556.47|usly
 even depths. quickly ironic theodolites s|
+33|Supplier#000000033|gfeKpYw3400L0SDywXA6Ya1Qmq1w6YB9f3R|7|17-138-897-9374|8564.12|n
 sauternes along the regular asymptotes are regularly along the |
+34|Supplier#000000034|mYRe3KvA2O4lL4HhxDKkkrPUDPMKRCSp,Xpa|10|20-519-982-2343|237.31|eposits.
 slyly final deposits toward the slyly regular dependencies sleep among the 
excu|
+35|Supplier#000000035|QymmGXxjVVQ5OuABCXVVsu,4eF 
gU0Qc6|21|31-720-790-5245|4381.41| ironic deposits! final, bold platelets 
haggle quickly quickly pendin|
+36|Supplier#000000036|mzSpBBJvbjdx3UKTW3bLFewRD78D91lAC879|13|23-273-493-3679|2371.51|ular
 theodolites must haggle regular, bold accounts. slyly final pinto beans bo|
+37|Supplier#000000037|cqjyB5h1nV|0|10-470-144-1330|3017.47|iously final 
instructions. quickly special accounts hang fluffily above the accounts. 
deposits|
+38|Supplier#000000038|xEcx45vD0FXHT7c9mvWFY|4|14-361-296-6426|2512.41|ins. 
fluffily special accounts haggle slyly af|
+39|Supplier#000000039|ZM, 
nSYpEPWr1yAFHaC91qjFcijjeU5eH|8|18-851-856-5633|6115.65|le slyly requests. 
special packages shall are blithely. slyly unusual packages sleep |
+40|Supplier#000000040|zyIeWzbbpkTV37vm1nmSGBxSgd2Kp|22|32-231-247-6991|-290.06|
 final patterns. accounts haggle idly pas|
+41|Supplier#000000041|G 1FKHR435 wMKFmyt|18|28-739-447-2525|6942.67|odolites 
boost across the furiously regular fo|
+42|Supplier#000000042|1Y5lwEgpe3j2vbUBYj3SwLhK62JlwEMtDC|22|32-698-298-6317|6565.11|
 fluffily even requests cajole blithely fu|
+43|Supplier#000000043|Z5mLuAoTUEeKY5v22VnnA4D87Ao6jF2LvMYnlX8h|12|22-421-568-4862|7773.41|unts.
 unusual, final asymptotes |
+44|Supplier#000000044|kERxlLDnlIZJdN66zAPHklyL|7|17-713-930-5667|9759.38|x. 
carefully quiet account|
+45|Supplier#000000045|LcKnsa8XGtIO0WYSB7hkOrH 
rnzRg1|9|19-189-635-8862|2944.23|iously according to the ironic, silent 
accounts.|
+46|Supplier#000000046|e0URUXfDOYMdKe16Z5h5StMRbzGmTs,D2cjap|24|34-748-308-3215|3580.35|gular,
 regular ideas across th|
+47|Supplier#000000047|3XM1x,Pcxqw,HK4XNlgbnZMbLhBHLA|14|24-810-354-4471|2958.09|sly
 ironic deposits sleep carefully along t|
+48|Supplier#000000048|jg0U 
FNPMQDuyuKvTnLXXaLf3Wl6OtONA6mQlWJ|14|24-722-551-9498|5630.62|xpress 
instructions affix. fluffily even requests boos|
+49|Supplier#000000049|Nvq 6macF4GtJvz|24|34-211-567-6800|9915.24|the finally 
bold dependencies. dependencies after the fluffily final foxes boost fluffi|
+50|Supplier#000000050|rGobqSMMYz0ErrPhCGS|9|19-561-560-7437|4515.87|warhorses. 
ironic, regular accounts detect slyly after the quickly f|
+51|Supplier#000000051|rDkBXb01POIKjOwrij62uM8O4|9|19-475-537-1368|7241.40|ges 
nag at the blithely busy instructions. fluffy packages wake quickly. even, 
ironic ideas boost b|
+52|Supplier#000000052|WCk 
XCHYzBA1dvJDSol4ZJQQcQN,|19|29-974-934-4713|287.16|dolites are slyly against 
the furiously regular packages. ironic, final deposits cajole quickly|
+53|Supplier#000000053|i9v3 
EsYCfLKFU6PIt8iihBOHBB37yR7b3GD7Rt|7|17-886-101-6083|6177.35|onic, special 
deposits wake furio|
+54|Supplier#000000054|J1s,Wxb5pg|12|22-966-435-7200|2733.69|blithely pending 
dolphins. quickly regular theodolites haggle slyly|
+55|Supplier#000000055|OqdYSiOQeG4eGi636Tj|24|34-876-912-6007|7162.15|kages. 
blithely even foxes cajole special, final accounts. blithely even dependencies 
r|
+56|Supplier#000000056|fUVtlUVal GiHBOuYoUQ 
XQ9NfNLQR3Gl|16|26-471-195-5486|-632.16| sleep special deposits. unusual 
requests wake blithely slyly regular ideas. |
+57|Supplier#000000057|bEWqUVRR f0mb2o18Y|17|27-681-514-6892|-831.07|detect 
according to the furiously br|
+58|Supplier#000000058|01dEADIZoCULZXg|16|26-659-969-5586|92.44|refully final 
foxes are. even, express courts according to the b|
+59|Supplier#000000059|N8lKbYjMnVlEHmTPRmBgtLiX8rrJx|17|27-249-395-9123|586.16|ffily
 along the even decoys. final instructions abov|
+60|Supplier#000000060|cb08ntDTARo47WmnBcYXu|8|18-550-360-2464|4515.80|thely 
express ideas use blithely |
+61|Supplier#000000061|Oz0M1qBR9I|8|18-396-489-9719|6096.58|s the slyly regular 
ideas shall |
+62|Supplier#000000062|bSmlFYUKBeRsqJxwC9 
zS6xpFdEf5jNTb|19|29-603-653-2494|9202.57|ts. furiously ironic pinto beans are 
permanently after the bold ideas. regular, express f|
+63|Supplier#000000063|NlV0OQyIoPvPkw5AYuWGomX,hgqm1|5|15-781-401-3047|5742.03|ar
 deposits. blithely bold accounts against the slyly final pinto beans sleep 
about the exp|
+64|Supplier#000000064|w80JjnIP 
lGoLdUjRutbv81gGlqqpW4PQBeOtSYU|24|34-278-790-7004|1309.70|uickly regular 
requests use. carefully i|
+65|Supplier#000000065|BsAnHUmSFArppKrM|22|32-444-835-2434|-963.79|l ideas wake 
carefully around the regular packages. furiously ruthless pinto bea|
+66|Supplier#000000066|qYdruFJQJYYiKvnNVmYfCVydVB8bcW,AW,U6SOV3|23|33-300-836-9529|2455.98|ar
 requests. express orbits de|
+67|Supplier#000000067|7YrEKJncHFk5D W7ZaqfAXV|4|14-563-538-1657|3576.55|ray 
slyly final foxes. furio|
+68|Supplier#000000068|Ue6N50wH2CwE4PPgTGLmat,ibGYYlDoOb3xQwtgb|21|31-267-327-4328|5119.38|inal
 requests. ruthlessly ironic packages cajole|
+69|Supplier#000000069|T2Dl9,f97e333eRuMi2z |4|14-491-707-8310|8466.50|! 
carefully ironic instructions nag quickly pending requests. fluffily even 
deposits sleep a|
+70|Supplier#000000070|INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4|6|16-821-608-1166|9508.37|ests
 sleep quickly express ideas. ironic ideas haggle about the final T|
+71|Supplier#000000071|YFo8an7P6wi Q|1|11-743-919-7272|8179.68| final accounts. 
bold, final escapades must have to cajole about the special platelets. fu|
+72|Supplier#000000072|mKpAJojtawk2alqV4 ZEbJ3PH3wfYqy 
AM8rGq1|18|28-113-898-6643|7014.50| theodolites sublate furiously about the 
regularly e|
+73|Supplier#000000073|HBZA1NHvrswQCxTTjg 
5XrfSOGgMRKNCe2ovE|16|26-758-310-7496|3793.13|. never pending asymptotes a|
+74|Supplier#000000074|uM3yV5NOc6b5wNdpxF69CW 
8QvDxqvKubRJtA|20|30-166-486-1559|4170.51|carefully along the quickly regular 
sentiments. ironic accounts sleep. regular deposits are blith|
+75|Supplier#000000075|7f3gN4rP1livII|18|28-716-704-8686|-224.84|eans. even, 
silent packages c|
+76|Supplier#000000076|JBhSBa3cLYvNgHUYtUHmtECCD|14|24-228-763-7840|2971.10|, 
even instructions. furiously unusual deposits wake slyly about the ev|
+77|Supplier#000000077|wVtcr0uH3CyrSiWMLsqnB09Syo,UuZxPMeBghlY|7|17-281-345-4863|4186.95|the
 slyly final asymptotes. blithely pending theodoli|
+78|Supplier#000000078|9y3OZ2CV 
hGrsrQxzB7V3zTtygHVHlG3SD6yrz|5|15-670-998-6860|1044.10|, regular packages wake 
quickly bold requests. carefully unusual requests about the unusual request|
+79|Supplier#000000079|p0u3tztSXUD2J8vFfLNFNKsrRRv7qyUtTBTA|14|24-402-227-9600|1191.94|nto
 beans integrate slyly across the fluffily pending multipliers. carefully ste|
+80|Supplier#000000080|cJ2MHSEJ13rIL2Wj3D5i6hRo30,ZiNUXhqn|21|31-646-289-1906|-40.45|ackages.
 blithely bold requests wake quickly. carefully regular foxes are slyly 
instructions. caref|
+81|Supplier#000000081|SLlacbhgpKmVa,gF3saYv12e0|12|22-535-310-6971|166.32|oost 
carefully quickly regular packages. carefully final excuses sleep blithely 
slyly pendi|
+82|Supplier#000000082|WyTKA7ZpF15t1aCNlT3|18|28-177-572-9691|-724.31|xpress 
multipliers wake furiously even foxes. furiously iro|
+83|Supplier#000000083|WRJUkzCn050seVz57oAfrbCuw|14|24-529-559-2461|1467.77|ly 
pending courts would cajole enticingly even deposits. slyly express|
+84|Supplier#000000084|DcYjWMiZGQqEKOJi4wAmIV08ikx|24|34-869-118-7803|4780.93|even
 depths. regular foxes use slyly. theod|
+85|Supplier#000000085|Ckls9RtlzKSF|7|17-167-806-8199|7174.74|egular packages. 
bold pinto beans wake fur|
+86|Supplier#000000086|J1fgg5QaqnN|19|29-903-665-7065|1883.37|cajole furiously 
special, final requests: furiously spec|
+87|Supplier#000000087|WCw7URDj8zoZ7tqC3cpm7|24|34-860-229-1674|4746.66|all are 
quickly after the ironic platelets. pending dolphins are. final the|
+88|Supplier#000000088|yOshY8wwzMgS|11|21-191-938-9469|3086.13|furiously 
special excuses aff|
+89|Supplier#000000089|fhtzZcSorhud1|9|19-259-876-1014|1638.02|en instructions 
across the slyly ironic requests engage|
+90|Supplier#000000090|bPE6Uhz1f2m3gwSGMrnRt,g,3gq37r5kxgphqss1|6|16-380-123-9217|6201.77|hely
 fluffily regular theodoli|
+91|Supplier#000000091|YV45D7TkfdQanOOZ7q9QxkyGUapU1oOWU6q3|3|13-604-986-9056|6255.87|nstructions
 use carefully according to the special packages: quickly silent th|
+92|Supplier#000000092|n48Wy4QI3lml8T217rk|2|12-701-432-8346|2470.84| even 
theodolites wake against the blithely fluffy packages|
+93|Supplier#000000093|wd1djjKXT,4zBm|16|26-528-528-1157|368.76|yly final 
accounts could are carefully. fluffily ironic instruct|
+94|Supplier#000000094|lK,pLPjAMVGJOXN80zPZuNQjpChliE|4|14-728-888-8882|2766.80|usly.
 furiously slow theodolites could haggle carefully fina|
+95|Supplier#000000095|p2 2hBfH5TD|19|29-142-539-1403|2588.83|sual instructions 
cajole slyly with the final ac|
+96|Supplier#000000096|Lbxp3WIipye 
o2wZme1i9iJx,xTt1Mp|12|22-888-654-7193|3437.24|efully unusual excuses wake 
fluffily regular theodo|
+97|Supplier#000000097|MrCQha2G6ndX1fp6CA|4|14-618-678-1789|6642.22|s, regular 
accounts. furiously bold ideas u|
+98|Supplier#000000098|ogHn8dpXB5Q|21|31-914-775-1978|5873.07|esias use slyly 
under the ironic foxes. re|
+99|Supplier#000000099|4SxkTHG28nZrtT0,MnVF9H|18|28-272-909-1617|3671.34|nic 
dependencies dazzle flu|
+100|Supplier#000000100|rIlN li8zvW22l2slbcx 
ECP4fL|21|31-351-324-5062|3191.70|es. regular instructions sleep carefully. 
slyly ironic packages across the foxes boost |


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to