You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2014/06/26 22:45:07 UTC

[07/16] git commit: DRILL-972: Add newline if no delimiter is specified; avoid rescanning (contains DRILL-950).

DRILL-972: Add newline if no delimiter is specified; avoid rescanning (contains DRILL-950).


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

Branch: refs/heads/master
Commit: d84190f22c57e78f2e236afa5f5b03b179b031d9
Parents: dbaec9d
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Tue Jun 17 14:21:38 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Jun 25 17:54:48 2014 -0700

----------------------------------------------------------------------
 .../exec/store/easy/text/TextFormatPlugin.java  |  2 +-
 .../exec/store/text/DrillTextRecordReader.java  | 14 +++-
 .../drill/exec/store/text/TestTextColumn.java   | 73 ++++++++++++++++++++
 .../resources/bootstrap-storage-plugins.json    |  4 ++
 .../test/resources/store/text/data/letters.csv  |  3 +
 .../test/resources/store/text/data/letters.txt  |  3 +
 pom.xml                                         |  1 +
 7 files changed, 96 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d84190f2/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
index 3935008..81a23b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
@@ -100,7 +100,7 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
   public static class TextFormatConfig implements FormatPluginConfig {
 
     public List<String> extensions;
-    public String delimiter;
+    public String delimiter = "\n";
 
     public List<String> getExtensions() {
       return extensions;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d84190f2/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java
index b5b5b3c..5bf571d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/text/DrillTextRecordReader.java
@@ -126,9 +126,13 @@ public class DrillTextRecordReader implements RecordReader {
             break;
           }
           start = end;
-          end = find(value, delimiter, start + 1);
-          if (end == -1) {
+          if (delimiter == '\n') {
             end = value.getLength();
+          } else {
+            end = find(value, delimiter, start + 1);
+            if (end == -1) {
+              end = value.getLength();
+            }
           }
           if (numCols > 0 && i++ < columnIds.get(p)) {
             if (!vector.getMutator().addSafe(recordCount, value.getBytes(), start + 1, 0)) {
@@ -166,8 +170,12 @@ public class DrillTextRecordReader implements RecordReader {
     int len = text.getLength();
     int p = start;
     byte[] bytes = text.getBytes();
+    boolean inQuotes = false;
     while (p < len) {
-      if (bytes[p] == what) {
+      if ('\"' == bytes[p]) {
+        inQuotes = !inQuotes;
+      }
+      if (!inQuotes && bytes[p] == what) {
         return p;
       }
       p++;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d84190f2/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
index fb729d5..395ec02 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
@@ -18,8 +18,19 @@
 package org.apache.drill.exec.store.text;
 
 import org.apache.drill.BaseTestQuery;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
 public class TestTextColumn extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTextColumn.class);
 
@@ -27,4 +38,66 @@ public class TestTextColumn extends BaseTestQuery{
   public void testCsvColumnSelection() throws Exception{
     test("select columns[0] as region_id, columns[1] as country from dfs.`[WORKING_PATH]/src/test/resources/store/text/data/regions.csv`");
   }
+
+  @Test
+  public void testDefaultDelimiterColumnSelection() throws Exception {
+    List<QueryResultBatch> batches = testSqlWithResults("SELECT columns[0] as entire_row " +
+      "from dfs.`[WORKING_PATH]/src/test/resources/store/text/data/letters.txt`");
+
+    List<List<String>> expectedOutput = Arrays.asList(
+      Arrays.asList("\"a, b,\",\"c\",\"d,, \\n e\""),
+      Arrays.asList("\"d, e,\",\"f\",\"g,, \\n h\""),
+      Arrays.asList("\"g, h,\",\"i\",\"j,, \\n k\""));
+
+    List<List<String>> actualOutput = getOutput(batches);
+    validateOutput(expectedOutput, actualOutput);
+  }
+
+  @Test
+  public void testCsvColumnSelectionCommasInsideQuotes() throws Exception {
+    List<QueryResultBatch> batches = testSqlWithResults("SELECT columns[0] as col1, columns[1] as col2, columns[2] as col3," +
+      "columns[3] as col4 from dfs.`[WORKING_PATH]/src/test/resources/store/text/data/letters.csv`");
+
+    List<List<String>> expectedOutput = Arrays.asList(
+      Arrays.asList("\"a, b,\"", "\"c\"", "\"d,, \\n e\"","\"f\\\"g\""),
+      Arrays.asList("\"d, e,\"", "\"f\"", "\"g,, \\n h\"","\"i\\\"j\""),
+      Arrays.asList("\"g, h,\"", "\"i\"", "\"j,, \\n k\"","\"l\\\"m\""));
+
+    List<List<String>> actualOutput = getOutput(batches);
+    validateOutput(expectedOutput, actualOutput);
+  }
+
+  private List<List<String>> getOutput(List<QueryResultBatch> batches) throws SchemaChangeException {
+    List<List<String>> output = new ArrayList<>();
+    RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
+    int last = 0;
+    for(QueryResultBatch batch : batches) {
+      int rows = batch.getHeader().getRowCount();
+      if(batch.getData() != null) {
+        loader.load(batch.getHeader().getDef(), batch.getData());
+        for (int i = 0; i < rows; ++i) {
+          output.add(new ArrayList<String>());
+          for (VectorWrapper<?> vw: loader) {
+            ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
+            output.get(last).add(accessor.getObject(i).toString());
+          }
+          ++last;
+        }
+      }
+      loader.clear();
+      batch.release();
+    }
+    return output;
+  }
+
+  private void validateOutput(List<List<String>> expected, List<List<String>> actual) {
+    assertEquals(expected.size(), actual.size());
+    for (int i = 0 ; i < expected.size(); ++i) {
+      assertEquals(expected.get(i).size(), actual.get(i).size());
+      for (int j = 0; j < expected.get(i).size(); ++j) {
+        assertEquals(expected.get(i).get(j), actual.get(i).get(j));
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d84190f2/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json b/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
index 9503482..534b120 100644
--- a/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
+++ b/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
@@ -35,6 +35,10 @@
             },
             "parquet" : {
               type: "parquet"
+            },
+            "txt" : {
+              type : "text",
+              extensions: [ "txt" ]
             }
           }
     },

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d84190f2/exec/java-exec/src/test/resources/store/text/data/letters.csv
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/store/text/data/letters.csv b/exec/java-exec/src/test/resources/store/text/data/letters.csv
new file mode 100644
index 0000000..4d724e8
--- /dev/null
+++ b/exec/java-exec/src/test/resources/store/text/data/letters.csv
@@ -0,0 +1,3 @@
+"a, b,","c","d,, \n e","f\"g"
+"d, e,","f","g,, \n h","i\"j"
+"g, h,","i","j,, \n k","l\"m"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d84190f2/exec/java-exec/src/test/resources/store/text/data/letters.txt
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/store/text/data/letters.txt b/exec/java-exec/src/test/resources/store/text/data/letters.txt
new file mode 100644
index 0000000..14b9cb6
--- /dev/null
+++ b/exec/java-exec/src/test/resources/store/text/data/letters.txt
@@ -0,0 +1,3 @@
+"a, b,","c","d,, \n e"
+"d, e,","f","g,, \n h"
+"g, h,","i","j,, \n k"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d84190f2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7dbdef0..129cc22 100644
--- a/pom.xml
+++ b/pom.xml
@@ -128,6 +128,7 @@
             <exclude>**/*.sql</exclude>
             <exclude>**/git.properties</exclude>
             <exclude>**/*.csv</exclude>
+            <exclude>**/*.txt</exclude>
             <exclude>**/drill-*.conf</exclude>
             <exclude>**/.buildpath</exclude>
             <exclude>**/*.proto</exclude>