You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2016/07/20 12:11:42 UTC

tajo git commit: TAJO-2166: Disallow csv file format in DDL Languages.

Repository: tajo
Updated Branches:
  refs/heads/master 6a52ba937 -> fdd2ca208


TAJO-2166: Disallow csv file format in DDL Languages.

Closes #1037

Signed-off-by: Jihoon Son <ji...@apache.org>


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

Branch: refs/heads/master
Commit: fdd2ca2081a18950b831ce7db956d49b4af71917
Parents: 6a52ba9
Author: Lee Dongjin <do...@apache.org>
Authored: Wed Jul 20 21:10:25 2016 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Wed Jul 20 21:11:21 2016 +0900

----------------------------------------------------------------------
 CHANGES                                                        | 2 ++
 .../src/test/java/org/apache/tajo/client/TestTajoClient.java   | 2 +-
 .../org/apache/tajo/engine/planner/TestLogicalPlanner.java     | 4 ++--
 .../java/org/apache/tajo/engine/query/TestCreateTable.java     | 6 ++++++
 .../java/org/apache/tajo/engine/query/TestInsertQuery.java     | 4 ++--
 .../queries/TestSQLAnalyzer/positive/create_table_10.sql       | 2 +-
 .../src/main/java/org/apache/tajo/parser/sql/SQLAnalyzer.java  | 3 +++
 7 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/fdd2ca20/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index f16f2b8..9428f13 100644
--- a/CHANGES
+++ b/CHANGES
@@ -298,6 +298,8 @@ Release 0.12.0 - unreleased
 
   TASKS
 
+    TAJO-2166: Disallow csv file format in DDL Languages. (Lee Dongjin via jihoon)
+
     TAJO-2176: Refining function documents. (Jongyoung)
 
     TAJO-2154: Refactor Datum to use new Type implementation. (hyunsik)

http://git-wip-us.apache.org/repos/asf/tajo/blob/fdd2ca20/tajo-core-tests/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core-tests/src/test/java/org/apache/tajo/client/TestTajoClient.java
index 828f60c..4ca74af 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -366,7 +366,7 @@ public class TestTajoClient {
     assertFalse(client.existTable(tableName));
     String sql =
         "create external table " + tableName + " (deptname text, score int4) "
-            + "using csv location '" + tablePath + "'";
+            + "using text location '" + tablePath + "'";
     client.executeQueryAndGetResult(sql);
     assertTrue(client.existTable(tableName));
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/fdd2ca20/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 142b2c3..8dce782 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -953,7 +953,7 @@ public class TestLogicalPlanner {
   }
 
   static final String CREATE_TABLE [] = {
-    "create external table table1 (name text, age int, earn bigint, score real) using csv with ('csv.delimiter'='|') location '/tmp/data'"
+    "create external table table1 (name text, age int, earn bigint, score real) using text with ('text.delimiter'='|') location '/tmp/data'"
   };
 
   @Test
@@ -978,7 +978,7 @@ public class TestLogicalPlanner {
     assertTrue("TEXT".equalsIgnoreCase(createTable.getStorageType()));
     assertEquals("file://tmp/data", createTable.getUri().toString());
     assertTrue(createTable.hasOptions());
-    assertEquals("|", createTable.getOptions().get("csv.delimiter"));
+    assertEquals("\\u007c", createTable.getOptions().get("text.delimiter"));
   }
 
   private static final List<Set<Column>> testGenerateCuboidsResult

http://git-wip-us.apache.org/repos/asf/tajo/blob/fdd2ca20/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
index cdef490..e099a2a 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
@@ -24,6 +24,7 @@ import org.apache.tajo.QueryTestCaseBase;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.exception.SQLSyntaxError;
 import org.apache.tajo.schema.IdentifierUtil;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.KeyValueSet;
@@ -97,6 +98,11 @@ public class TestCreateTable extends QueryTestCaseBase {
     assertDatabaseNotExists("d2");
   }
 
+  @Test(expected = SQLSyntaxError.class)
+  public final void testCreateTableWithCsv() throws Exception {
+    executeString("CREATE TABLE D1.table1 (age int) using csv;").close();
+  }
+
   private final void assertPathOfCreatedTable(final String databaseName,
                                               final String originalTableName,
                                               final String newTableName,

http://git-wip-us.apache.org/repos/asf/tajo/blob/fdd2ca20/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
index 6f2a546..dd00718 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
@@ -191,7 +191,7 @@ public class TestInsertQuery extends QueryTestCaseBase {
   public final void testInsertIntoPartitionedTable() throws Exception {
     String tableName = IdentifierUtil.normalizeIdentifier("testInsertIntoPartitionedTable");
     executeString("create table " + tableName + " (n_name TEXT, n_regionkey INT4)" +
-        "USING csv PARTITION by column(n_nationkey INT4)" ).close();
+        "USING text PARTITION by column(n_nationkey INT4)" ).close();
 
     try {
       executeString("insert into " + tableName + " select n_name, n_regionkey, n_nationkey from default.nation").close();
@@ -754,7 +754,7 @@ public class TestInsertQuery extends QueryTestCaseBase {
   public final void testInsertOverwritePathWithNonFromQuery() throws Exception {
     ResultSet res = executeString("insert overwrite into location " +
         "'/tajo-data/testInsertOverwritePathWithNonFromQuery' " +
-        "USING csv WITH ('text.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
+        "USING text WITH ('text.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
         "select 1::INT4, 2.1::FLOAT4, 'test'");
 
     res.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/fdd2ca20/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_table_10.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_table_10.sql b/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_table_10.sql
index 43314fa..b2b0f17 100644
--- a/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_table_10.sql
+++ b/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_table_10.sql
@@ -1 +1 @@
-create external table table1 (name text, age int, earn bigint, score float) using csv location '/tmp/data'
\ No newline at end of file
+create external table table1 (name text, age int, earn bigint, score float) using text location '/tmp/data'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/fdd2ca20/tajo-core/src/main/java/org/apache/tajo/parser/sql/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/parser/sql/SQLAnalyzer.java b/tajo-core/src/main/java/org/apache/tajo/parser/sql/SQLAnalyzer.java
index fdb4c0b..12ea9b4 100644
--- a/tajo-core/src/main/java/org/apache/tajo/parser/sql/SQLAnalyzer.java
+++ b/tajo-core/src/main/java/org/apache/tajo/parser/sql/SQLAnalyzer.java
@@ -1325,6 +1325,9 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
 
       if (checkIfExist(ctx.USING())) {
         String fileType = ctx.storage_type.getText();
+        if (fileType.equals("csv")) {
+          throw new TajoRuntimeException(new SQLSyntaxError("Using csv storage type is disallowed. Please use the text storage type instead."));
+        }
         createTable.setStorageType(fileType);
       }