You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2014/03/25 04:20:13 UTC

[1/3] TAJO-644: Support quoted identifiers. (hyunsik)

Repository: tajo
Updated Branches:
  refs/heads/master 3ba262412 -> e12e038b5


http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
index 7fdfb44..120113f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
@@ -78,10 +78,10 @@ public class TestNLJoinExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", Type.INT4);
-    schema.addColumn("empId", Type.INT4);
-    schema.addColumn("memId", Type.INT4);
-    schema.addColumn("deptName", Type.TEXT);
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("memid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -102,8 +102,8 @@ public class TestNLJoinExec {
     catalog.createTable(employee);
     
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", Type.INT4);
-    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
     peopleSchema.addColumn("name", Type.TEXT);
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index c7aa33d..f32cd1e 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -105,11 +105,11 @@ public class TestPhysicalPlanner {
 
     Schema employeeSchema = new Schema();
     employeeSchema.addColumn("name", Type.TEXT);
-    employeeSchema.addColumn("empId", Type.INT4);
-    employeeSchema.addColumn("deptName", Type.TEXT);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
 
     Schema scoreSchema = new Schema();
-    scoreSchema.addColumn("deptName", Type.TEXT);
+    scoreSchema.addColumn("deptname", Type.TEXT);
     scoreSchema.addColumn("class", Type.TEXT);
     scoreSchema.addColumn("score", Type.INT4);
     scoreSchema.addColumn("nullable", Type.TEXT);
@@ -526,7 +526,7 @@ public class TestPhysicalPlanner {
     LogicalPlan plan = planner.createPlan(session, context);
 
     int numPartitions = 3;
-    Column key1 = new Column("default.score.deptName", Type.TEXT);
+    Column key1 = new Column("default.score.deptname", Type.TEXT);
     Column key2 = new Column("default.score.class", Type.TEXT);
     DataChannel dataChannel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
         ShuffleType.HASH_SHUFFLE, numPartitions);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
index 65b5dba..c60e05c 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
@@ -82,9 +82,9 @@ public class TestProgressExternalSortExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", TajoDataTypes.Type.INT4);
-    schema.addColumn("empId", TajoDataTypes.Type.INT4);
-    schema.addColumn("deptName", TajoDataTypes.Type.TEXT);
+    schema.addColumn("managerid", TajoDataTypes.Type.INT4);
+    schema.addColumn("empid", TajoDataTypes.Type.INT4);
+    schema.addColumn("deptname", TajoDataTypes.Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.RAW);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -171,8 +171,8 @@ public class TestProgressExternalSortExec {
     exec.init();
     TupleComparator comparator = new TupleComparator(proj.getSchema(),
         new SortSpec[]{
-            new SortSpec(new Column("managerId", TajoDataTypes.Type.INT4)),
-            new SortSpec(new Column("empId", TajoDataTypes.Type.INT4))
+            new SortSpec(new Column("managerid", TajoDataTypes.Type.INT4)),
+            new SortSpec(new Column("empid", TajoDataTypes.Type.INT4))
         });
 
     float initProgress = 0.0f;

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
index a33a0e4..87262e8 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
@@ -70,9 +70,9 @@ public class TestSortExec {
     sm = StorageManagerFactory.getStorageManager(conf, workDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", Type.INT4);
-    schema.addColumn("empId", Type.INT4);
-    schema.addColumn("deptName", Type.TEXT);
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
 
     employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
index 79db68c..e61f509 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
@@ -26,6 +26,7 @@ import org.apache.tajo.QueryTestCaseBase;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -56,12 +57,13 @@ public class TestCTASQuery extends QueryTestCaseBase {
     ResultSet res = executeQuery();
     res.close();
 
+    String tableName = CatalogUtil.normalizeIdentifier("testCtasWithoutTableDefinition");
     CatalogService catalog = testBase.getTestingCluster().getMaster().getCatalog();
-    String tableName = buildFQName(DEFAULT_DATABASE_NAME, "testCtasWithoutTableDefinition");
-    TableDesc desc = catalog.getTableDesc(tableName);
-    assertTrue(catalog.existsTable(tableName));
+    String qualifiedTableName = buildFQName(DEFAULT_DATABASE_NAME, tableName);
+    TableDesc desc = catalog.getTableDesc(qualifiedTableName);
+    assertTrue(catalog.existsTable(qualifiedTableName));
 
-    assertTrue(desc.getSchema().contains("default.testCtasWithoutTableDefinition.col1"));
+    assertTrue(desc.getSchema().contains("default.testctaswithouttabledefinition.col1"));
     PartitionMethodDesc partitionDesc = desc.getPartitionMethod();
     assertEquals(partitionDesc.getPartitionType(), CatalogProtos.PartitionType.COLUMN);
     assertEquals("key", partitionDesc.getExpressionSchema().getColumns().get(0).getSimpleName());
@@ -99,10 +101,12 @@ public class TestCTASQuery extends QueryTestCaseBase {
     ResultSet res = executeQuery();
     res.close();
 
+    String tableName = CatalogUtil.normalizeIdentifier("testCtasWithColumnedPartition");
+
     TajoTestingCluster cluster = testBase.getTestingCluster();
     CatalogService catalog = cluster.getMaster().getCatalog();
-    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, "testCtasWithColumnedPartition");
-    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, "testCtasWithColumnedPartition"));
+    TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
+    assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
     PartitionMethodDesc partitionDesc = desc.getPartitionMethod();
     assertEquals(partitionDesc.getPartitionType(), CatalogProtos.PartitionType.COLUMN);
     assertEquals("key", partitionDesc.getExpressionSchema().getColumns().get(0).getSimpleName());

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java
index c1c2591..453c174 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateDatabase.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.query;
 
 import org.apache.tajo.IntegrationTest;
 import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -30,12 +31,14 @@ public class TestCreateDatabase extends QueryTestCaseBase {
 
   @Test
   public final void testCreateAndDropDatabase() throws Exception {
+    String databaseName = CatalogUtil.normalizeIdentifier("testCreateAndDropDatabase");
+
     ResultSet res = null;
     try {
       res = executeString("CREATE DATABASE testCreateAndDropDatabase;");
-      assertDatabaseExists("testCreateAndDropDatabase");
+      assertDatabaseExists(databaseName);
       executeString("DROP DATABASE testCreateAndDropDatabase;");
-      assertDatabaseNotExists("testCreateAndDropDatabase");
+      assertDatabaseNotExists(databaseName);
     } finally {
       cleanupQuery(res);
     }
@@ -43,7 +46,7 @@ public class TestCreateDatabase extends QueryTestCaseBase {
 
   @Test
   public final void testCreateIfNotExists() throws Exception {
-    String databaseName = "testCreateIfNotExists";
+    String databaseName = CatalogUtil.normalizeIdentifier("testCreateIfNotExists");
 
     assertDatabaseNotExists(databaseName);
     executeString("CREATE DATABASE " + databaseName + ";").close();
@@ -58,7 +61,7 @@ public class TestCreateDatabase extends QueryTestCaseBase {
 
   @Test
   public final void testDropIfExists() throws Exception {
-    String databaseName = "testDropIfExists";
+    String databaseName = CatalogUtil.normalizeIdentifier("testDropIfExists");
     assertDatabaseNotExists(databaseName);
     executeString("CREATE DATABASE " + databaseName + ";").close();
     assertDatabaseExists(databaseName);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
index f2882dd..a5b57b2 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
@@ -23,6 +23,7 @@ import org.apache.tajo.QueryTestCaseBase;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.sql.ResultSet;
 import java.util.List;
 
 @Category(IntegrationTest.class)
@@ -108,6 +109,75 @@ public class TestCreateTable extends QueryTestCaseBase {
   }
 
   @Test
+  public final void testDelimitedIdentifierWithNonAsciiCharacters() throws Exception {
+
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      ResultSet res = null;
+      try {
+        List<String> tableNames = executeDDL("quoted_identifier_non_ascii_ddl.sql", "table1", "\"테이블1\"");
+        assertTableExists(tableNames.get(0));
+
+        // SELECT "아이디", "텍스트", "숫자" FROM "테이블1";
+        res = executeFile("quoted_identifier_non_ascii_1.sql");
+        assertResultSet(res, "quoted_identifier_non_ascii_1.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      // SELECT "아이디" as "진짜아이디", "텍스트" as text, "숫자" FROM "테이블1" as "테이블 별명"
+      try {
+        res = executeFile("quoted_identifier_non_ascii_2.sql");
+        assertResultSet(res, "quoted_identifier_non_ascii_2.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      // SELECT "아이디" "진짜아이디", char_length("텍스트") as "길이", "숫자" * 2 FROM "테이블1" "테이블 별명"
+      try {
+        res = executeFile("quoted_identifier_non_ascii_3.sql");
+        assertResultSet(res, "quoted_identifier_non_ascii_3.result");
+      } finally {
+        cleanupQuery(res);
+      }
+    }
+  }
+
+  @Test
+  public final void testDelimitedIdentifierWithMixedCharacters() throws Exception {
+    if (!testingCluster.isHCatalogStoreRunning()) {
+      ResultSet res = null;
+
+      try {
+        List<String> tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table1", "\"TABLE1\"");
+        assertTableExists(tableNames.get(0));
+
+        tableNames = executeDDL("quoted_identifier_mixed_chars_ddl_1.sql", "table2", "\"tablE1\"");
+        assertTableExists(tableNames.get(0));
+
+        // SELECT "aGe", "tExt", "Number" FROM "TABLE1";
+        res = executeFile("quoted_identifier_mixed_chars_1.sql");
+        assertResultSet(res, "quoted_identifier_mixed_chars_1.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      try {
+        res = executeFile("quoted_identifier_mixed_chars_2.sql");
+        assertResultSet(res, "quoted_identifier_mixed_chars_2.result");
+      } finally {
+        cleanupQuery(res);
+      }
+
+      try {
+        res = executeFile("quoted_identifier_mixed_chars_3.sql");
+        assertResultSet(res, "quoted_identifier_mixed_chars_3.result");
+      } finally {
+        cleanupQuery(res);
+      }
+    }
+  }
+
+  @Test
   public final void testNonreservedKeywordTableNames() throws Exception {
     List<String> createdNames = null;
     createdNames = executeDDL("table1_ddl.sql", "table1", "filter");

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
index 250be47..e058943 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
@@ -28,6 +28,7 @@ import org.apache.tajo.IntegrationTest;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.TpchTestBase;
 import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -53,7 +54,7 @@ public class TestInsertQuery {
 
   @Test
   public final void testInsertOverwrite() throws Exception {
-    String tableName ="InsertOverwrite";
+    String tableName = CatalogUtil.normalizeIdentifier("InsertOverwrite");
     ResultSet res = tpch.execute("create table " + tableName +" (col1 int4, col2 int4, col3 float8)");
     res.close();
     TajoTestingCluster cluster = tpch.getTestingCluster();
@@ -72,7 +73,7 @@ public class TestInsertQuery {
 
   @Test
   public final void testInsertOverwriteSmallerColumns() throws Exception {
-    String tableName = "insertoverwritesmallercolumns";
+    String tableName = CatalogUtil.normalizeIdentifier("insertoverwritesmallercolumns");
     ResultSet res = tpch.execute("create table " + tableName + " (col1 int4, col2 int4, col3 float8)");
     res.close();
     TajoTestingCluster cluster = tpch.getTestingCluster();
@@ -91,7 +92,7 @@ public class TestInsertQuery {
 
   @Test
   public final void testInsertOverwriteWithTargetColumns() throws Exception {
-    String tableName = "InsertOverwriteWithTargetColumns";
+    String tableName = CatalogUtil.normalizeIdentifier("InsertOverwriteWithTargetColumns");
     ResultSet res = tpch.execute("create table " + tableName + " (col1 int4, col2 int4, col3 float8)");
     res.close();
     TajoTestingCluster cluster = tpch.getTestingCluster();
@@ -99,7 +100,8 @@ public class TestInsertQuery {
     assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
     TableDesc originalDesc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
 
-    res = tpch.execute("insert overwrite into " + tableName + " (col1, col3) select l_orderkey, l_quantity from lineitem");
+    res = tpch.execute(
+        "insert overwrite into " + tableName + " (col1, col3) select l_orderkey, l_quantity from lineitem");
     res.close();
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
     if (!cluster.isHCatalogStoreRunning()) {
@@ -145,7 +147,7 @@ public class TestInsertQuery {
 
   @Test
   public final void testInsertOverwriteWithAsterisk() throws Exception {
-    String tableName = "testinsertoverwritewithasterisk";
+    String tableName = CatalogUtil.normalizeIdentifier("testinsertoverwritewithasterisk");
     ResultSet res = tpch.execute("create table " + tableName + " as select * from lineitem");
     res.close();
     TajoTestingCluster cluster = tpch.getTestingCluster();
@@ -162,7 +164,7 @@ public class TestInsertQuery {
 
   @Test
   public final void testInsertOverwriteIntoSelect() throws Exception {
-    String tableName = "insertoverwriteintoselect";
+    String tableName = CatalogUtil.normalizeIdentifier("insertoverwriteintoselect");
     ResultSet res = tpch.execute(
         "create table " + tableName + " as select l_orderkey from lineitem");
     assertFalse(res.next());
@@ -191,7 +193,7 @@ public class TestInsertQuery {
 
   @Test
   public final void testInsertOverwriteCapitalTableName() throws Exception {
-    String tableName = "testInsertOverwriteCapitalTableName";
+    String tableName = CatalogUtil.normalizeIdentifier("testInsertOverwriteCapitalTableName");
     ResultSet res = tpch.execute("create table " + tableName + " as select * from lineitem");
     res.close();
     TajoTestingCluster cluster = tpch.getTestingCluster();
@@ -218,7 +220,7 @@ public class TestInsertQuery {
 
   @Test
   public final void testInsertOverwriteWithCompression() throws Exception {
-    String tableName = "testInsertOverwriteWithCompression";
+    String tableName = CatalogUtil.normalizeIdentifier("testInsertOverwriteWithCompression");
     ResultSet res = tpch.execute("create table " + tableName + " (col1 int4, col2 int4, col3 float8) USING csv WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec')");
     res.close();
     TajoTestingCluster cluster = tpch.getTestingCluster();

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index a53fff7..d11cad9 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -29,6 +29,7 @@ import org.apache.tajo.QueryTestCaseBase;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
 import org.junit.Test;
 
@@ -48,7 +49,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testCreateColumnPartitionedTable() throws Exception {
-    String tableName ="testCreateColumnPartitionedTable";
+    String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTable");
     ResultSet res = executeString(
         "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
     res.close();
@@ -64,7 +65,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testCreateColumnPartitionedTableWithSelectedColumns() throws Exception {
-    String tableName ="testCreateColumnPartitionedTableWithSelectedColumns";
+    String tableName = CatalogUtil.normalizeIdentifier("testCreateColumnPartitionedTableWithSelectedColumns");
     ResultSet res = executeString(
         "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
@@ -80,7 +81,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testColumnPartitionedTableByOneColumn() throws Exception {
-    String tableName ="testColumnPartitionedTableByOneColumn";
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByOneColumn");
     ResultSet res = executeString(
         "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
@@ -125,7 +126,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testQueryCasesOnColumnPartitionedTable() throws Exception {
-    String tableName ="testQueryCasesOnColumnPartitionedTable";
+    String tableName = CatalogUtil.normalizeIdentifier("testQueryCasesOnColumnPartitionedTable");
     ResultSet res = executeString(
         "create table " + tableName + " (col1 int4, col2 int4, null_col int4) partition by column(key float8) ");
     res.close();
@@ -155,9 +156,9 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testColumnPartitionedTableByThreeColumns() throws Exception {
-    String tableName ="testColumnPartitionedTableByThreeColumns";
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByThreeColumns");
     ResultSet res = testBase.execute(
-        "create table " + tableName +" (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
+        "create table " + tableName + " (col4 text) partition by column(col1 int4, col2 int4, col3 float8) ");
     res.close();
     TajoTestingCluster cluster = testBase.getTestingCluster();
     CatalogService catalog = cluster.getMaster().getCatalog();
@@ -219,7 +220,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testColumnPartitionedTableByOneColumnsWithCompression() throws Exception {
-    String tableName = "testColumnPartitionedTableByOneColumnsWithCompression";
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByOneColumnsWithCompression");
     ResultSet res = executeString(
         "create table " + tableName + " (col2 int4, col3 float8) USING csv " +
             "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
@@ -255,7 +256,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testColumnPartitionedTableByTwoColumnsWithCompression() throws Exception {
-    String tableName = "testColumnPartitionedTableByTwoColumnsWithCompression";
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByTwoColumnsWithCompression");
     ResultSet res = executeString("create table " + tableName + " (col3 float8, col4 text) USING csv " +
         "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
         "PARTITION by column(col1 int4, col2 int4)");
@@ -299,7 +300,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testColumnPartitionedTableByThreeColumnsWithCompression() throws Exception {
-    String tableName = "testColumnPartitionedTableByThreeColumnsWithCompression";
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableByThreeColumnsWithCompression");
     ResultSet res = executeString(
         "create table " + tableName + " (col4 text) USING csv " +
             "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +
@@ -382,7 +383,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   @Test
   public final void testColumnPartitionedTableNoMatchedPartition() throws Exception {
-    String tableName = "testColumnPartitionedTableNoMatchedPartition";
+    String tableName = CatalogUtil.normalizeIdentifier("testColumnPartitionedTableNoMatchedPartition");
     ResultSet res = executeString(
         "create table " + tableName + " (col4 text) USING csv " +
             "WITH ('csvfile.delimiter'='|','compression.codec'='org.apache.hadoop.io.compress.DeflateCodec') " +

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
index 1279245..9d0352c 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
@@ -224,7 +224,7 @@ public class TestTajoJdbc extends QueryTestCaseBase {
     ResultSet rs = null;
 
     try {
-      String tableName = "lineitem";
+      String tableName = CatalogUtil.normalizeIdentifier("lineitem");
       rs = dbmd.getColumns(null, null, tableName, null);
 
       ResultSetMetaData rsmd = rs.getMetaData();

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
index ddf8ecb..fe8c070 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
@@ -21,6 +21,7 @@ package org.apache.tajo.master.querymaster;
 import org.apache.tajo.IntegrationTest;
 import org.apache.tajo.QueryTestCaseBase;
 import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.worker.TajoWorker;
 import org.junit.Test;
@@ -96,7 +97,7 @@ public class TestQueryUnitStatusUpdate extends QueryTestCaseBase {
   }
 
   private void createColumnPartitionedTable() throws Exception {
-    String tableName ="ColumnPartitionedTable";
+    String tableName = CatalogUtil.normalizeIdentifier("ColumnPartitionedTable");
     ResultSet res = executeString(
         "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
     res.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
index 0b3a22c..de4560e 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
@@ -90,7 +90,7 @@ public class TestRangeRetrieverHandler {
     optimizer = new LogicalOptimizer(conf);
 
     schema = new Schema();
-    schema.addColumn("empId", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
     schema.addColumn("age", Type.INT4);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/dataset/TestCreateTable/table2/table2.tbl
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/dataset/TestCreateTable/table2/table2.tbl b/tajo-core/tajo-core-backend/src/test/resources/dataset/TestCreateTable/table2/table2.tbl
new file mode 100644
index 0000000..953ee3d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/dataset/TestCreateTable/table2/table2.tbl
@@ -0,0 +1,3 @@
+4|jkl|7
+5|opq|8
+6|stu|9
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/org/apache/tajo/jdbc/TestTajoResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/org/apache/tajo/jdbc/TestTajoResultSet.java b/tajo-core/tajo-core-backend/src/test/resources/org/apache/tajo/jdbc/TestTajoResultSet.java
deleted file mode 100644
index 92a8a6a..0000000
--- a/tajo-core/tajo-core-backend/src/test/resources/org/apache/tajo/jdbc/TestTajoResultSet.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * 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.tajo.jdbc;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-
-import java.util.Arrays;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class TestTajoResultSet {
-  @Test
-  public final void testFileNameComparator() {
-
-    Path[] paths = new Path [] {
-        new Path("hdfs://xtajox.com:9010/tmp/tajo-hadoop/staging/q_1391511584109_0001/RESULT/part-02-000104"),
-        new Path("hdfs://xtajox.com:9010/tmp/tajo-hadoop/staging/q_1391511584109_0001/RESULT/part-02-000000"),
-        new Path("hdfs://xtajox.com:9010/tmp/tajo-hadoop/staging/q_1391511584109_0001/RESULT/part-02-000105"),
-        new Path("hdfs://xtajox.com:9010/tmp/tajo-hadoop/staging/q_1391511584109_0001/RESULT/part-02-000001")
-    };
-
-    FileStatus [] fileStatuses = new FileStatus[paths.length];
-
-    for (int i = 0; i < paths.length; i++) {
-      fileStatuses[i] = mock(FileStatus.class);
-      when(fileStatuses[i].getPath()).thenReturn(paths[i]);
-    }
-
-    TajoResultSet.FileNameComparator comparator = new TajoResultSet.FileNameComparator();
-    Arrays.sort(fileStatuses, comparator);
-
-    FileStatus prev = null;
-    for (int i = 0; i < fileStatuses.length; i++) {
-      if (prev == null) {
-        prev = fileStatuses[i];
-      } else {
-        assertTrue(comparator.compare(prev, fileStatuses[i]) <= 0);
-      }
-
-      assertTrue(prev.getPath().getName().compareTo(fileStatuses[i].getPath().getName()) <= 0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql
new file mode 100644
index 0000000..2084456
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_1.sql
@@ -0,0 +1 @@
+SELECT "aGe", "teXt", "Number" FROM "TABLE1";
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql
new file mode 100644
index 0000000..4b05657
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_2.sql
@@ -0,0 +1 @@
+SELECT "tb1"."aGe" as "AgE", "tb1"."teXt" as "TEXT", "Number" as "NUMBER" FROM "TABLE1" "tb1";
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql
new file mode 100644
index 0000000..fbb00a0
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_3.sql
@@ -0,0 +1,22 @@
+SELECT
+  *
+FROM (
+  SELECT
+    "tb1"."aGe",
+    "tb1"."teXt",
+    "Number"
+  FROM
+    "TABLE1" "tb1"
+
+  UNION
+
+  SELECT
+    "aGe",
+    "teXt",
+    "Number"
+  FROM
+    "tablE1"
+) T1
+ORDER BY
+  "aGe";
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_1.sql
new file mode 100644
index 0000000..f93e1e6
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_1.sql
@@ -0,0 +1 @@
+CREATE EXTERNAL TABLE ${0} ("aGe" int, "teXt" text, "Number" int) using csv location ${table.path};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_2.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_2.sql
new file mode 100644
index 0000000..f93e1e6
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_mixed_chars_ddl_2.sql
@@ -0,0 +1 @@
+CREATE EXTERNAL TABLE ${0} ("aGe" int, "teXt" text, "Number" int) using csv location ${table.path};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_1.sql
new file mode 100644
index 0000000..39a7bfa
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_1.sql
@@ -0,0 +1 @@
+SELECT "아이디", "텍스트", "숫자" FROM "테이블1";
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_2.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_2.sql
new file mode 100644
index 0000000..f51364d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_2.sql
@@ -0,0 +1 @@
+SELECT "아이디" as "진짜아이디", "텍스트" as text, "숫자" FROM "테이블1" as "테이블 별명"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_3.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_3.sql
new file mode 100644
index 0000000..7e8b64c
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_3.sql
@@ -0,0 +1 @@
+SELECT "아이디" "진짜아이디", char_length("텍스트") as "길이", "숫자" * 2 FROM "테이블1" "테이블 별명"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_ddl.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_ddl.sql
new file mode 100644
index 0000000..c1f073d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCreateTable/quoted_identifier_non_ascii_ddl.sql
@@ -0,0 +1 @@
+CREATE EXTERNAL TABLE ${0} ("아이디" int, "텍스트" text, "숫자" int) using csv location ${table.path};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_1.result
new file mode 100644
index 0000000..e31bc89
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_1.result
@@ -0,0 +1,5 @@
+aGe,teXt,Number
+-------------------------------
+1,abc,2
+2,def,5
+3,ghi,8
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_2.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_2.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_2.result
new file mode 100644
index 0000000..d0c2732
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_2.result
@@ -0,0 +1,5 @@
+AgE,TEXT,NUMBER
+-------------------------------
+1,abc,2
+2,def,5
+3,ghi,8
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_3.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_3.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_3.result
new file mode 100644
index 0000000..b04e5c1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_mixed_chars_3.result
@@ -0,0 +1,8 @@
+aGe,teXt,Number
+-------------------------------
+1,abc,2
+2,def,5
+3,ghi,8
+4,jkl,7
+5,opq,8
+6,stu,9
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_1.result
new file mode 100644
index 0000000..4656661
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_1.result
@@ -0,0 +1,5 @@
+아이디,텍스트,숫자
+-------------------------------
+1,abc,2
+2,def,5
+3,ghi,8
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_2.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_2.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_2.result
new file mode 100644
index 0000000..7917cb4
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_2.result
@@ -0,0 +1,5 @@
+진짜아이디,text,숫자
+-------------------------------
+1,abc,2
+2,def,5
+3,ghi,8
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_3.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_3.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_3.result
new file mode 100644
index 0000000..3b51f2c
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestCreateTable/quoted_identifier_non_ascii_3.result
@@ -0,0 +1,5 @@
+진짜아이디,길이,?multiply
+-------------------------------
+1,3,4
+2,3,10
+3,3,16
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables1.result
index 88df9a8..2d89f42 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables1.result
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables1.result
@@ -1,4 +1,4 @@
-table_cat,table_schem,table_name,table_type,remarks
+TABLE_CAT,TABLE_SCHEM,TABLE_NAME,TABLE_TYPE,REMARKS
 -------------------------------
 jdbc_test1,public,table1,TABLE,
 jdbc_test1,public,table2,TABLE,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables2.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables2.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables2.result
index 06afed2..042cb52 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables2.result
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestTajoJdbc/getTables2.result
@@ -1,4 +1,4 @@
-table_cat,table_schem,table_name,table_type,remarks
+TABLE_CAT,TABLE_SCHEM,TABLE_NAME,TABLE_TYPE,REMARKS
 -------------------------------
 jdbc_test2,public,table3,TABLE,
 jdbc_test2,public,table4,TABLE,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoDatabaseMetaData.java b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoDatabaseMetaData.java
index f70b762..ac122c8 100644
--- a/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoDatabaseMetaData.java
+++ b/tajo-jdbc/src/main/java/org/apache/tajo/jdbc/TajoDatabaseMetaData.java
@@ -20,6 +20,7 @@ package org.apache.tajo.jdbc;
 import com.google.protobuf.ServiceException;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.ResultSetUtil;
@@ -370,7 +371,7 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
           } else if (c == '_') {
             result.append('.');
           } else {
-            result.append(Character.toLowerCase(c));
+            result.append(c);
           }
         }
       }
@@ -380,18 +381,19 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
   }
 
   @Override
-  public ResultSet getTables(@Nullable String catalog, String schemaPattern, String tableNamePattern, String [] types)
-      throws SQLException {
+  public ResultSet getTables(@Nullable String catalog, @Nullable String schemaPattern,
+                             @Nullable String tableNamePattern, @Nullable String [] types) throws SQLException {
     try {
       final List<MetaDataTuple> resultTables = new ArrayList<MetaDataTuple>();
       final String resultCatalog;
       if (catalog == null) {
-        resultCatalog = "default";
+        resultCatalog = TajoConstants.DEFAULT_DATABASE_NAME;
       } else {
-        resultCatalog = catalog;
+        resultCatalog = CatalogUtil.normalizeIdentifier(catalog);
       }
 
-      String regtableNamePattern = convertPattern(tableNamePattern);
+      String regtableNamePattern =
+          convertPattern(tableNamePattern == null ? null : CatalogUtil.normalizeIdentifier(tableNamePattern));
       try {
         TajoClient tajoClient = conn.getTajoClient();
         List<String> tableNames = tajoClient.getTableList(resultCatalog);
@@ -501,7 +503,8 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
   }
 
   @Override
-  public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
+  public ResultSet getColumns(@Nullable String catalog, @Nullable String schemaPattern,
+                              @Nullable String tableNamePattern, @Nullable String columnNamePattern)
       throws SQLException {
     List<MetaDataTuple> columns = new ArrayList<MetaDataTuple>();
     try {
@@ -509,8 +512,10 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
         catalog = TajoConstants.DEFAULT_DATABASE_NAME;
       }
 
-      String regtableNamePattern = convertPattern(tableNamePattern);
-      String regcolumnNamePattern = convertPattern(columnNamePattern);
+      String regtableNamePattern =
+          convertPattern(tableNamePattern == null ? null : CatalogUtil.normalizeIdentifier(tableNamePattern));
+      String regcolumnNamePattern =
+          convertPattern(columnNamePattern == null ? null : CatalogUtil.normalizeIdentifier(columnNamePattern));
 
       List<String> tables = conn.getTajoClient().getTableList(catalog);
       for (String table: tables) {
@@ -763,7 +768,7 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
 
   @Override
   public boolean nullPlusNonNullIsNull() throws SQLException {
-    return false;
+    return true;
   }
 
   @Override
@@ -833,7 +838,7 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
 
   @Override
   public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
-    return false;
+    return true;
   }
 
   @Override
@@ -944,7 +949,7 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
 
   @Override
   public boolean supportsExpressionsInOrderBy() throws SQLException {
-    return false;
+    return true;
   }
 
   @Override
@@ -954,7 +959,7 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
 
   @Override
   public boolean supportsFullOuterJoins() throws SQLException {
-    return false;
+    return true;
   }
 
   @Override
@@ -1004,7 +1009,7 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
 
   @Override
   public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
-    return false;
+    return true;
   }
 
   @Override
@@ -1059,7 +1064,7 @@ public class TajoDatabaseMetaData implements DatabaseMetaData {
 
   @Override
   public boolean supportsOuterJoins() throws SQLException {
-    return false;
+    return true;
   }
 
   @Override


[2/3] git commit: TAJO-644: Support quoted identifiers. (hyunsik)

Posted by hy...@apache.org.
TAJO-644: Support quoted identifiers. (hyunsik)


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

Branch: refs/heads/master
Commit: 37fef799b161dbe2f9cea1284c4818b92f7fddbc
Parents: 3ba2624
Author: Hyunsik Choi <hy...@apache.org>
Authored: Sun Mar 23 23:29:31 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Sun Mar 23 23:29:31 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 +-
 .../tajo/algebra/ColumnReferenceExpr.java       |  8 +--
 .../tajo/algebra/QualifiedAsteriskExpr.java     |  2 +-
 .../tajo/catalog/AbstractCatalogClient.java     |  2 +-
 .../org/apache/tajo/catalog/CatalogUtil.java    | 23 ++++++-
 .../java/org/apache/tajo/catalog/Column.java    |  4 +-
 .../java/org/apache/tajo/catalog/IndexDesc.java |  4 +-
 .../java/org/apache/tajo/catalog/Schema.java    | 14 ++--
 .../java/org/apache/tajo/catalog/TableDesc.java |  7 +-
 .../tajo/catalog/store/HCatalogStore.java       |  2 +-
 .../org/apache/tajo/catalog/CatalogServer.java  | 64 +++++++++---------
 .../tajo/catalog/store/AbstractDBStore.java     | 16 ++---
 .../org/apache/tajo/catalog/store/MemStore.java |  4 +-
 .../java/org/apache/tajo/client/TajoClient.java | 31 +++++----
 .../apache/tajo/engine/parser/HiveQLLexer.g4    |  4 +-
 .../org/apache/tajo/engine/parser/SQLLexer.g4   |  9 ++-
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  7 +-
 .../apache/tajo/engine/planner/LogicalPlan.java |  8 +--
 .../tajo/engine/planner/NamedExprsManager.java  |  6 +-
 .../org/apache/tajo/engine/planner/Target.java  |  2 +-
 .../tajo/engine/planner/logical/ScanNode.java   |  2 +-
 .../planner/logical/TableSubQueryNode.java      |  3 +-
 .../apache/tajo/engine/query/QueryContext.java  |  3 +-
 .../org/apache/tajo/master/GlobalEngine.java    |  4 +-
 .../tajo/master/TajoMasterClientService.java    |  2 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |  4 +-
 .../org/apache/tajo/client/TestTajoClient.java  | 42 ++++++------
 .../apache/tajo/engine/eval/ExprTestBase.java   |  7 +-
 .../apache/tajo/engine/eval/TestPredicates.java |  5 +-
 .../tajo/engine/eval/TestSQLExpression.java     | 12 ++++
 .../engine/function/TestDateTimeFunctions.java  | 29 ++++++--
 .../engine/planner/TestLogicalOptimizer.java    |  8 +--
 .../tajo/engine/planner/TestLogicalPlanner.java |  2 +-
 .../tajo/engine/planner/TestPlannerUtil.java    |  8 +--
 .../planner/physical/TestBNLJoinExec.java       | 12 ++--
 .../planner/physical/TestBSTIndexExec.java      | 10 +--
 .../planner/physical/TestExternalSortExec.java  | 10 +--
 .../planner/physical/TestHashAntiJoinExec.java  | 12 ++--
 .../planner/physical/TestHashJoinExec.java      | 12 ++--
 .../planner/physical/TestHashSemiJoinExec.java  | 12 ++--
 .../planner/physical/TestMergeJoinExec.java     | 12 ++--
 .../engine/planner/physical/TestNLJoinExec.java | 12 ++--
 .../planner/physical/TestPhysicalPlanner.java   |  8 +--
 .../physical/TestProgressExternalSortExec.java  | 10 +--
 .../engine/planner/physical/TestSortExec.java   |  6 +-
 .../apache/tajo/engine/query/TestCTASQuery.java | 16 +++--
 .../tajo/engine/query/TestCreateDatabase.java   | 11 +--
 .../tajo/engine/query/TestCreateTable.java      | 70 ++++++++++++++++++++
 .../tajo/engine/query/TestInsertQuery.java      | 18 ++---
 .../tajo/engine/query/TestTablePartitions.java  | 21 +++---
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java |  2 +-
 .../querymaster/TestQueryUnitStatusUpdate.java  |  3 +-
 .../tajo/worker/TestRangeRetrieverHandler.java  |  2 +-
 .../dataset/TestCreateTable/table2/table2.tbl   |  3 +
 .../org/apache/tajo/jdbc/TestTajoResultSet.java | 63 ------------------
 .../quoted_identifier_mixed_chars_1.sql         |  1 +
 .../quoted_identifier_mixed_chars_2.sql         |  1 +
 .../quoted_identifier_mixed_chars_3.sql         | 22 ++++++
 .../quoted_identifier_mixed_chars_ddl_1.sql     |  1 +
 .../quoted_identifier_mixed_chars_ddl_2.sql     |  1 +
 .../quoted_identifier_non_ascii_1.sql           |  1 +
 .../quoted_identifier_non_ascii_2.sql           |  1 +
 .../quoted_identifier_non_ascii_3.sql           |  1 +
 .../quoted_identifier_non_ascii_ddl.sql         |  1 +
 .../quoted_identifier_mixed_chars_1.result      |  5 ++
 .../quoted_identifier_mixed_chars_2.result      |  5 ++
 .../quoted_identifier_mixed_chars_3.result      |  8 +++
 .../quoted_identifier_non_ascii_1.result        |  5 ++
 .../quoted_identifier_non_ascii_2.result        |  5 ++
 .../quoted_identifier_non_ascii_3.result        |  5 ++
 .../results/TestTajoJdbc/getTables1.result      |  2 +-
 .../results/TestTajoJdbc/getTables2.result      |  2 +-
 .../apache/tajo/jdbc/TajoDatabaseMetaData.java  | 35 +++++-----
 73 files changed, 463 insertions(+), 316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7b379b3..a935225 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -132,7 +132,9 @@ Release 0.8.0 - unreleased
 
   IMPROVEMENTS
 
-    TAJO-553: Add a method to the TajoClient to get finished query lists. 
+    TAJO-644: Support quoted identifiers. (hyunsik)
+
+    TAJO-553: Add a method to the TajoClient to get finished query lists.
     (Ilhyun Suh via jihoon)
 
     TAJO-670: Change daemon's hostname to canonical hostname 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnReferenceExpr.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnReferenceExpr.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnReferenceExpr.java
index e1369df..58f7b81 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnReferenceExpr.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnReferenceExpr.java
@@ -32,8 +32,8 @@ public class ColumnReferenceExpr extends Expr {
 
   public ColumnReferenceExpr(String qualifier, String columnName) {
     super(OpType.Column);
-    this.qualifier = qualifier.toLowerCase();
-    this.name = columnName.toLowerCase();
+    this.qualifier = qualifier;
+    this.name = columnName;
   }
 
   public boolean hasQualifier() {
@@ -41,7 +41,7 @@ public class ColumnReferenceExpr extends Expr {
   }
 
   public void setQualifier(String qualifier) {
-    this.qualifier = qualifier.toLowerCase();
+    this.qualifier = qualifier;
   }
 
   public String getQualifier() {
@@ -53,7 +53,7 @@ public class ColumnReferenceExpr extends Expr {
   }
 
   public void setName(String qualifiedName) {
-    String [] parts = qualifiedName.toLowerCase().split("\\.");
+    String [] parts = qualifiedName.split("\\.");
 
     if (parts.length > 1) {
       qualifier = qualifiedName.substring(0, qualifiedName.lastIndexOf("."));

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java
index ab8e677..d26deb9 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/QualifiedAsteriskExpr.java
@@ -39,7 +39,7 @@ public class QualifiedAsteriskExpr extends Expr {
   }
 
   public void setQualifier(String qualifier) {
-    this.qualifier = qualifier.toLowerCase();
+    this.qualifier = qualifier;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
index 6ee788c..6fb385e 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
@@ -408,7 +408,7 @@ public abstract class AbstractCatalogClient implements CatalogService {
 
           GetIndexByColumnRequest.Builder builder = GetIndexByColumnRequest.newBuilder();
           builder.setTableIdentifier(CatalogUtil.buildTableIdentifier(databaseName, tableName));
-          builder.setColumnName(CatalogUtil.normalizeIdentifier(columnName));
+          builder.setColumnName(columnName);
 
           CatalogProtocolService.BlockingInterface stub = getStub(client);
           return stub.existIndexByColumn(null, builder.build()).getValue();

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
index 05a18bd..2ddbb17 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@ -47,7 +47,26 @@ public class CatalogUtil {
    * @return The normalized identifier
    */
   public static String normalizeIdentifier(String identifier) {
-    return identifier.toLowerCase();
+    return isDelimited(identifier) ? stripQuote(identifier).toLowerCase() : identifier.toLowerCase();
+  }
+
+  public static String stripQuote(String str) {
+    return str.substring(1, str.length() - 1);
+  }
+
+  public static boolean isDelimited(String identifier) {
+    boolean openQuote = identifier.charAt(0) == '"';
+    boolean closeQuote = identifier.charAt(identifier.length() - 1) == '"';
+
+    if (openQuote ^ closeQuote || identifier.length() < 2) {
+      throw new IllegalArgumentException("Invalid Identifier: " + identifier);
+    }
+
+    if (openQuote && closeQuote && identifier.length() == 2) {
+      throw new IllegalArgumentException("zero-length delimited identifier: " + identifier);
+    }
+
+    return openQuote && closeQuote && identifier.length() > 2;
   }
 
   public static boolean isFQColumnName(String tableName) {
@@ -60,7 +79,7 @@ public class CatalogUtil {
   }
 
   public static String [] splitFQTableName(String qualifiedName) {
-    String [] splitted = CatalogUtil.splitTableName(CatalogUtil.normalizeIdentifier(qualifiedName));
+    String [] splitted = CatalogUtil.splitTableName(qualifiedName);
     if (splitted.length == 1) {
       throw new IllegalArgumentException("createTable() requires a qualified table name, but it is \""
           + qualifiedName + "\".");

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
index b75cd1c..2e29536 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
@@ -40,7 +40,7 @@ public class Column implements ProtoObject<ColumnProto>, GsonObject {
    * @param dataType Data Type with length
    */
 	public Column(String name, DataType dataType) {
-    this.name = CatalogUtil.normalizeIdentifier(name);
+    this.name = name;
 		this.dataType = dataType;
 	}
 
@@ -64,7 +64,7 @@ public class Column implements ProtoObject<ColumnProto>, GsonObject {
   }
 
 	public Column(ColumnProto proto) {
-    name = CatalogUtil.normalizeIdentifier(proto.getName());
+    name = proto.getName();
     dataType = proto.getDataType();
 	}
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java
index ef65b36..d77408a 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/IndexDesc.java
@@ -45,9 +45,9 @@ public class IndexDesc implements ProtoObject<IndexDescProto>, Cloneable {
   public IndexDesc(String idxName, String databaseName, String tableName, Column column,
                    IndexMethod type,  boolean isUnique, boolean isClustered, boolean isAscending) {
     this();
-    this.indexName = idxName.toLowerCase();
+    this.indexName = idxName;
     this.databaseName = databaseName;
-    this.tableName = tableName.toLowerCase();
+    this.tableName = tableName;
     this.column = column;
     this.indexMethod = type;
     this.isUnique = isUnique;

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
index 0c812f1..b2dde3d 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
@@ -147,7 +147,7 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
    * @return The Column matched to a given qualified name
    */
   private Column getColumnByQName(String qualifiedName) {
-		Integer cid = fieldsByQualifiedName.get(qualifiedName.toLowerCase());
+		Integer cid = fieldsByQualifiedName.get(qualifiedName);
 		return cid != null ? fields.get(cid) : null;
 	}
 
@@ -161,7 +161,7 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
    * @return The Column matched to a given name.
    */
 	private Column getColumnByName(String columnName) {
-    String normalized = columnName.toLowerCase();
+    String normalized = columnName;
 	  List<Integer> list = fieldsByName.get(normalized);
 
     if (list == null || list.size() == 0) {
@@ -213,7 +213,7 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
 
   public int getColumnIdByName(String colName) {
     for (Column col : fields) {
-      if (col.getSimpleName().equals(colName.toLowerCase())) {
+      if (col.getSimpleName().equals(colName)) {
         String qualifiedName = col.getQualifiedName();
         return fieldsByQualifiedName.get(qualifiedName);
       }
@@ -226,10 +226,10 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
 	}
 
   public boolean contains(String name) {
-    if (fieldsByQualifiedName.containsKey(name.toLowerCase())) {
+    if (fieldsByQualifiedName.containsKey(name)) {
       return true;
     }
-    if (fieldsByName.containsKey(name.toLowerCase())) {
+    if (fieldsByName.containsKey(name)) {
       if (fieldsByName.size() > 1) {
         throw new RuntimeException("Ambiguous Column name");
       }
@@ -258,7 +258,7 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
   }
 	
 	public boolean containsByQualifiedName(String qualifiedName) {
-		return fieldsByQualifiedName.containsKey(qualifiedName.toLowerCase());
+		return fieldsByQualifiedName.containsKey(qualifiedName);
 	}
 
   public boolean containsByName(String colName) {
@@ -281,7 +281,7 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
   }
 
   public synchronized Schema addColumn(String name, DataType dataType) {
-		String normalized = name.toLowerCase();
+		String normalized = name;
 		if(fieldsByQualifiedName.containsKey(normalized)) {
 		  LOG.error("Already exists column " + normalized);
 			throw new AlreadyExistsFieldException(normalized);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
index 30102eb..4251116 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
@@ -56,7 +56,7 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
   public TableDesc(String tableName, Schema schema, TableMeta meta,
                    Path path, boolean external) {
     this();
-    this.tableName = tableName.toLowerCase();
+    this.tableName = tableName;
     this.schema = schema;
     this.meta = meta;
     this.uri = path;
@@ -64,7 +64,7 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
   }
 
 	public TableDesc(String tableName, Schema schema, TableMeta meta, Path path) {
-		this(CatalogUtil.normalizeIdentifier(tableName), schema, meta, path, true);
+		this(tableName, schema, meta, path, true);
 	}
 	
 	public TableDesc(String tableName, Schema schema, StoreType type, Options options, Path path) {
@@ -83,8 +83,7 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
 	}
 	
 	public void setName(String tableId) {
-	  // tajo deems all identifiers as lowcase characters
-		this.tableName = tableId.toLowerCase();
+		this.tableName = tableId;
 	}
 	
   public String getName() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
index 50726a6..6ef255f 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
@@ -377,7 +377,7 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
     HCatalogStoreClientPool.HCatalogStoreClient client = null;
 
     TableDesc tableDesc = new TableDesc(tableDescProto);
-    String [] splitted = CatalogUtil.splitFQTableName(CatalogUtil.normalizeIdentifier(tableDesc.getName()));
+    String [] splitted = CatalogUtil.splitFQTableName(tableDesc.getName());
     String databaseName = splitted[0];
     String tableName = splitted[1];
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
index d4ee769..a171fb4 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
@@ -209,7 +209,7 @@ public class CatalogServer extends AbstractService {
 
     @Override
     public BoolProto createTablespace(RpcController controller, CreateTablespaceRequest request) throws ServiceException {
-      final String tablespaceName = CatalogUtil.normalizeIdentifier(request.getTablespaceName());
+      final String tablespaceName = request.getTablespaceName();
       final String uri = request.getTablespaceUri();
 
       wlock.lock();
@@ -232,7 +232,7 @@ public class CatalogServer extends AbstractService {
 
     @Override
     public BoolProto dropTablespace(RpcController controller, StringProto request) throws ServiceException {
-      String tablespaceName = CatalogUtil.normalizeIdentifier(request.getValue());
+      String tablespaceName = request.getValue();
 
       wlock.lock();
       try {
@@ -253,7 +253,7 @@ public class CatalogServer extends AbstractService {
 
     @Override
     public BoolProto existTablespace(RpcController controller, StringProto request) throws ServiceException {
-      String tablespaceName = CatalogUtil.normalizeIdentifier(request.getValue());
+      String tablespaceName = request.getValue();
 
       rlock.lock();
       try {
@@ -285,8 +285,8 @@ public class CatalogServer extends AbstractService {
 
     @Override
     public BoolProto createDatabase(RpcController controller, CreateDatabaseRequest request) throws ServiceException {
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String tablespaceName = CatalogUtil.normalizeIdentifier(request.getTablespaceName());
+      String databaseName = request.getDatabaseName();
+      String tablespaceName = request.getTablespaceName();
 
       wlock.lock();
       try {
@@ -307,7 +307,7 @@ public class CatalogServer extends AbstractService {
 
     @Override
     public BoolProto dropDatabase(RpcController controller, StringProto request) throws ServiceException {
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getValue());
+      String databaseName = request.getValue();
 
       wlock.lock();
       try {
@@ -328,7 +328,7 @@ public class CatalogServer extends AbstractService {
 
     @Override
     public BoolProto existDatabase(RpcController controller, StringProto request) throws ServiceException {
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getValue());
+      String databaseName = request.getValue();
 
       rlock.lock();
       try {
@@ -361,8 +361,8 @@ public class CatalogServer extends AbstractService {
     @Override
     public TableDescProto getTableDesc(RpcController controller,
                                        TableIdentifierProto request) throws ServiceException {
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(request.getTableName());
+      String databaseName = request.getDatabaseName();
+      String tableName = request.getTableName();
 
       rlock.lock();
       try {
@@ -375,7 +375,7 @@ public class CatalogServer extends AbstractService {
           if (contain) {
             return store.getTable(databaseName, tableName);
           } else {
-            throw new NoSuchTableException(databaseName);
+            throw new NoSuchTableException(tableName);
           }
         } else {
           throw new NoSuchDatabaseException(databaseName);
@@ -392,7 +392,7 @@ public class CatalogServer extends AbstractService {
     public StringListProto getAllTableNames(RpcController controller, StringProto request)
         throws ServiceException {
 
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getValue());
+      String databaseName = request.getValue();
 
       rlock.lock();
       try {
@@ -425,7 +425,7 @@ public class CatalogServer extends AbstractService {
     public BoolProto createTable(RpcController controller, TableDescProto request)throws ServiceException {
 
       String [] splitted =
-          CatalogUtil.splitFQTableName(CatalogUtil.normalizeIdentifier(request.getTableName()));
+          CatalogUtil.splitFQTableName(request.getTableName());
 
       String databaseName = splitted[0];
       String tableName = splitted[1];
@@ -459,8 +459,8 @@ public class CatalogServer extends AbstractService {
     @Override
     public BoolProto dropTable(RpcController controller, TableIdentifierProto request) throws ServiceException {
 
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(request.getTableName());
+      String databaseName = request.getDatabaseName();
+      String tableName = request.getTableName();
 
       wlock.lock();
       try {
@@ -490,8 +490,8 @@ public class CatalogServer extends AbstractService {
     @Override
     public BoolProto existsTable(RpcController controller, TableIdentifierProto request)
         throws ServiceException {
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(request.getTableName());
+      String databaseName = request.getDatabaseName();
+      String tableName = request.getTableName();
 
       rlock.lock();
       try {
@@ -520,8 +520,8 @@ public class CatalogServer extends AbstractService {
     public PartitionMethodProto getPartitionMethodByTableName(RpcController controller,
                                                               TableIdentifierProto request)
         throws ServiceException {
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(request.getTableName());
+      String databaseName = request.getDatabaseName();
+      String tableName = request.getTableName();
 
       rlock.lock();
       try {
@@ -554,8 +554,8 @@ public class CatalogServer extends AbstractService {
     @Override
     public BoolProto existPartitionMethod(RpcController controller, TableIdentifierProto request)
         throws ServiceException {
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(request.getTableName());
+      String databaseName = request.getDatabaseName();
+      String tableName = request.getTableName();
 
       rlock.lock();
       try {
@@ -646,8 +646,8 @@ public class CatalogServer extends AbstractService {
     @Override
     public BoolProto existIndexByName(RpcController controller, IndexNameProto request) throws ServiceException {
 
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String indexName = CatalogUtil.normalizeIdentifier(request.getIndexName());
+      String databaseName = request.getDatabaseName();
+      String indexName = request.getIndexName();
 
       rlock.lock();
       try {
@@ -665,9 +665,9 @@ public class CatalogServer extends AbstractService {
         throws ServiceException {
 
       TableIdentifierProto identifier = request.getTableIdentifier();
-      String databaseName = CatalogUtil.normalizeIdentifier(identifier.getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(identifier.getTableName());
-      String columnName = CatalogUtil.normalizeIdentifier(request.getColumnName());
+      String databaseName = identifier.getDatabaseName();
+      String tableName = identifier.getTableName();
+      String columnName = request.getColumnName();
 
       rlock.lock();
       try {
@@ -685,8 +685,8 @@ public class CatalogServer extends AbstractService {
     public IndexDescProto getIndexByName(RpcController controller, IndexNameProto request)
         throws ServiceException {
 
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String indexName = CatalogUtil.normalizeIdentifier(request.getIndexName());
+      String databaseName = request.getDatabaseName();
+      String indexName = request.getIndexName();
 
       rlock.lock();
       try {
@@ -707,9 +707,9 @@ public class CatalogServer extends AbstractService {
         throws ServiceException {
 
       TableIdentifierProto identifier = request.getTableIdentifier();
-      String databaseName = CatalogUtil.normalizeIdentifier(identifier.getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(identifier.getTableName());
-      String columnName = CatalogUtil.normalizeIdentifier(request.getColumnName());
+      String databaseName = identifier.getDatabaseName();
+      String tableName = identifier.getTableName();
+      String columnName = request.getColumnName();
 
       rlock.lock();
       try {
@@ -729,8 +729,8 @@ public class CatalogServer extends AbstractService {
     public BoolProto dropIndex(RpcController controller, IndexNameProto request)
         throws ServiceException {
 
-      String databaseName = CatalogUtil.normalizeIdentifier(request.getDatabaseName());
-      String indexName = CatalogUtil.normalizeIdentifier(request.getIndexName());
+      String databaseName = request.getDatabaseName();
+      String indexName = request.getIndexName();
 
       wlock.lock();
       try {

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
index 98c289e..d10f545 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
@@ -572,7 +572,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       conn = getConnection();
       conn.setAutoCommit(false);
 
-      String [] splitted = CatalogUtil.splitTableName(CatalogUtil.normalizeIdentifier(table.getTableName()));
+      String [] splitted = CatalogUtil.splitTableName(table.getTableName());
       if (splitted.length == 1) {
         throw new IllegalArgumentException("createTable() requires a qualified table name, but it is \""
             + table.getTableName() + "\".");
@@ -1106,8 +1106,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         LOG.debug(ADD_PARTITION_SQL);
       }
 
-      String databaseName = CatalogUtil.normalizeIdentifier(partitionsProto.getTableIdentifier().getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(partitionsProto.getTableIdentifier().getTableName());
+      String databaseName = partitionsProto.getTableIdentifier().getDatabaseName();
+      String tableName = partitionsProto.getTableIdentifier().getTableName();
 
       int databaseId = getDatabaseId(databaseName);
       int tableId = getTableId(databaseId, databaseName, tableName);
@@ -1157,8 +1157,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         LOG.debug(sql);
       }
 
-      String databaseName = CatalogUtil.normalizeIdentifier(proto.getTableIdentifier().getDatabaseName());
-      String tableName = CatalogUtil.normalizeIdentifier(proto.getTableIdentifier().getTableName());
+      String databaseName = proto.getTableIdentifier().getDatabaseName();
+      String tableName = proto.getTableIdentifier().getTableName();
 
       int databaseId = getDatabaseId(databaseName);
       int tableId = getTableId(databaseId, databaseName, tableName);
@@ -1353,9 +1353,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     Connection conn = null;
     PreparedStatement pstmt = null;
 
-    String databaseName = CatalogUtil.normalizeIdentifier(proto.getTableIdentifier().getDatabaseName());
-    String tableName = CatalogUtil.normalizeIdentifier(proto.getTableIdentifier().getTableName());
-    String columnName = CatalogUtil.extractSimpleName(CatalogUtil.normalizeIdentifier(proto.getColumn().getName()));
+    String databaseName = proto.getTableIdentifier().getDatabaseName();
+    String tableName = proto.getTableIdentifier().getTableName();
+    String columnName = CatalogUtil.extractSimpleName(proto.getColumn().getName());
 
     try {
       int databaseId = getDatabaseId(databaseName);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
index 9fc9d2a..ef98ee2 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
@@ -121,7 +121,7 @@ public class MemStore implements CatalogStore {
 
   @Override
   public void createTable(CatalogProtos.TableDescProto request) throws CatalogException {
-    String [] splitted = CatalogUtil.splitTableName(CatalogUtil.normalizeIdentifier(request.getTableName()));
+    String [] splitted = CatalogUtil.splitTableName(request.getTableName());
     if (splitted.length == 1) {
       throw new IllegalArgumentException("createTable() requires a qualified table name, but it is \""
           + request.getTableName() + "\".");
@@ -131,7 +131,7 @@ public class MemStore implements CatalogStore {
 
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, databaseName);
 
-    String tbName = CatalogUtil.normalizeIdentifier(tableName);
+    String tbName = tableName;
     if (database.containsKey(tbName)) {
       throw new AlreadyExistsTableException(tbName);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java
index bf77f03..9d7c4ee 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java
@@ -77,7 +77,7 @@ public class TajoClient implements Closeable {
     this(conf, NetUtils.createSocketAddr(conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS)), null);
   }
 
-  public TajoClient(TajoConf conf, String baseDatabase) throws IOException {
+  public TajoClient(TajoConf conf, @Nullable String baseDatabase) throws IOException {
     this(conf, NetUtils.createSocketAddr(conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS)), baseDatabase);
   }
 
@@ -89,7 +89,7 @@ public class TajoClient implements Closeable {
     //Don't share connection pool per client
     connPool = RpcConnectionPool.newPool(conf, getClass().getSimpleName(), workerNum);
     userInfo = UserGroupInformation.getCurrentUser();
-    this.baseDatabase = baseDatabase;
+    this.baseDatabase = baseDatabase != null ? CatalogUtil.normalizeIdentifier(baseDatabase) : null;
   }
 
   public boolean isConnected() {
@@ -197,7 +197,8 @@ public class TajoClient implements Closeable {
         checkSessionAndGet(client);
 
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
-        return tajoMasterService.selectDatabase(null, convertSessionedString(databaseName)).getValue();
+        return tajoMasterService.selectDatabase(null,
+            convertSessionedString(CatalogUtil.normalizeIdentifier(databaseName))).getValue();
       }
     }.withRetries();
   }
@@ -501,7 +502,8 @@ public class TajoClient implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
-        return tajoMasterService.createDatabase(null, convertSessionedString(databaseName)).getValue();
+        return tajoMasterService.createDatabase(null,
+            convertSessionedString(CatalogUtil.normalizeIdentifier(databaseName))).getValue();
       }
     }.withRetries();
   }
@@ -511,7 +513,8 @@ public class TajoClient implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
-        return tajoMasterService.existDatabase(null, convertSessionedString(databaseName)).getValue();
+        return tajoMasterService.existDatabase(null,
+            convertSessionedString(CatalogUtil.normalizeIdentifier(databaseName))).getValue();
       }
     }.withRetries();
   }
@@ -521,7 +524,8 @@ public class TajoClient implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
-        return tajoMasterService.dropDatabase(null, convertSessionedString(databaseName)).getValue();
+        return tajoMasterService.dropDatabase(null,
+            convertSessionedString(CatalogUtil.normalizeIdentifier(databaseName))).getValue();
       }
     }.withRetries();
   }
@@ -550,7 +554,8 @@ public class TajoClient implements Closeable {
       public Boolean call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
-        return tajoMasterService.existTable(null, convertSessionedString(name)).getValue();
+        return tajoMasterService.existTable(null,
+            convertSessionedString(CatalogUtil.normalizeIdentifier(name))).getValue();
       }
     }.withRetries();
   }
@@ -566,7 +571,7 @@ public class TajoClient implements Closeable {
 
         CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
         builder.setSessionId(sessionId);
-        builder.setName(name);
+        builder.setName(CatalogUtil.normalizeIdentifier(name));
         builder.setSchema(schema.getProto());
         builder.setMeta(meta.getProto());
         builder.setPath(path.toUri().toString());
@@ -600,7 +605,7 @@ public class TajoClient implements Closeable {
 
         DropTableRequest.Builder builder = DropTableRequest.newBuilder();
         builder.setSessionId(sessionId);
-        builder.setName(tableName);
+        builder.setName(CatalogUtil.normalizeIdentifier(tableName));
         builder.setPurge(purge);
         return tajoMasterService.dropTable(null, builder.build()).getValue();
       }
@@ -666,12 +671,14 @@ public class TajoClient implements Closeable {
       public List<String> call(NettyClientBase client) throws ServiceException {
         checkSessionAndGet(client);
 
+        final String normalizedDBName = databaseName == null ? null : CatalogUtil.normalizeIdentifier(databaseName);
+
         TajoMasterClientProtocolService.BlockingInterface tajoMasterService = client.getStub();
 
         GetTableListRequest.Builder builder = GetTableListRequest.newBuilder();
         builder.setSessionId(sessionId);
-        if (databaseName != null) {
-          builder.setDatabaseName(databaseName);
+        if (normalizedDBName != null) {
+          builder.setDatabaseName(normalizedDBName);
         }
         GetTableListResponse res = tajoMasterService.getTableList(null, builder.build());
         return res.getTablesList();
@@ -689,7 +696,7 @@ public class TajoClient implements Closeable {
 
         GetTableDescRequest.Builder builder = GetTableDescRequest.newBuilder();
         builder.setSessionId(sessionId);
-        builder.setTableName(tableName);
+        builder.setTableName(CatalogUtil.normalizeIdentifier(tableName));
         TableResponse res = tajoMasterService.getTableDesc(null, builder.build());
         if (res.getResultCode() == ResultCode.OK) {
           return CatalogUtil.newTableDesc(res.getTableDesc());

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
index 8eba9a4..f7b76ef 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/HiveQLLexer.g4
@@ -370,9 +370,7 @@ Number
     ;
     
 Identifier
-    :
-    (Letter | Digit) (Letter | Digit | '_')*
-    | '`' RegexComponent+ '`'
+    : (Letter | Digit) (Letter | Digit | '_')* { setText(getText().toLowerCase()); }
     ;
 
 CharSetName

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
index 8966fd3..6eccd12 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
@@ -405,13 +405,12 @@ LineComment
 ===============================================================================
 */
 
-Identifier
-  : Regular_Identifier
+Regular_Identifier
+  : ('a'..'z'|'A'..'Z'|'_') ('a'..'z'|'A'..'Z'|Digit|'_')* { setText(getText().toLowerCase()); }
   ;
 
-fragment
-Regular_Identifier
-  : ('a'..'z'|'A'..'Z'|'_') ('a'..'z'|'A'..'Z'|Digit|'_')*
+Quoted_Identifier
+  : DOUBLE_QUOTE ( ESC_SEQ | ~('\\'|'"') )* DOUBLE_QUOTE { setText(getText().substring(1, getText().length()-1)); }
   ;
 
 /*

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
index 4dada45..ce78c98 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
@@ -198,8 +198,9 @@ drop_table_statement
 */
 
 identifier
-  : Identifier
+  : Regular_Identifier
   | nonreserved_keywords
+  | Quoted_Identifier
   ;
 
 nonreserved_keywords
@@ -1055,7 +1056,7 @@ table_or_query_name
   ;
 
 table_name
-  : identifier  ( DOT  identifier (  DOT identifier )? )?
+  : identifier (DOT identifier ( DOT identifier)? )?
   ;
 
 query_specification
@@ -1076,7 +1077,7 @@ derived_column
   ;
 
 qualified_asterisk
-  : (tb_name=Identifier DOT)? MULTIPLY
+  : (tb_name=identifier DOT)? MULTIPLY
   ;
 
 set_qualifier

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
index c3bca66..63dd428 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
@@ -144,7 +144,7 @@ public class LogicalPlan {
     } else { // if a generated column name
       generatedName = attachSeqIdToGeneratedColumnName(getGeneratedPrefixFromExpr(expr));
     }
-    return generatedName.toLowerCase();
+    return generatedName;
   }
 
   /**
@@ -578,15 +578,15 @@ public class LogicalPlan {
     }
 
     public boolean existsRelation(String name) {
-      return nameToRelationMap.containsKey(CatalogUtil.normalizeIdentifier(name));
+      return nameToRelationMap.containsKey(name);
     }
 
     public RelationNode getRelation(String name) {
-      return nameToRelationMap.get(CatalogUtil.normalizeIdentifier(name));
+      return nameToRelationMap.get(name);
     }
 
     public void addRelation(RelationNode relation) {
-      nameToRelationMap.put(CatalogUtil.normalizeIdentifier(relation.getCanonicalName()), relation);
+      nameToRelationMap.put(relation.getCanonicalName(), relation);
     }
 
     public Collection<RelationNode> getRelations() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java
index 33f6fc6..90d829f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/NamedExprsManager.java
@@ -89,7 +89,7 @@ public class NamedExprsManager {
   }
 
   private static String normalizeName(String name) {
-    return name.toLowerCase();
+    return name;
   }
 
   /**
@@ -121,7 +121,7 @@ public class NamedExprsManager {
   }
 
   public NamedExpr getNamedExpr(String name) {
-    String normalized = name.toLowerCase();
+    String normalized = name;
     return new NamedExpr(getExpr(name), normalized);
   }
 
@@ -227,7 +227,7 @@ public class NamedExprsManager {
    * @param evalNode EvalNode to be added.
    */
   public void markAsEvaluated(String referenceName, EvalNode evalNode) throws PlanningException {
-    String normalized = referenceName.toLowerCase();
+    String normalized = referenceName;
 
     int refId = nameToIdMap.get(normalized);
     evaluationStateMap.put(refId, true);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
index d83a394..6e4580d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
@@ -43,7 +43,7 @@ public class Target implements Cloneable, GsonObject {
   public Target(final EvalNode eval, final String alias) {
     this.expr = eval;
     // force lower case
-    String normalized = alias.toLowerCase();
+    String normalized = alias;
 
     // If an expr is a column reference and its alias is equivalent to column name, ignore a given alias.
     if (eval instanceof FieldEval && eval.getName().equals(normalized)) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
index 8e4911f..0f346b5 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
@@ -54,7 +54,7 @@ public class ScanNode extends RelationNode implements Projectable, Cloneable {
   
 	public void init(TableDesc desc, String alias) {
     this.tableDesc = desc;
-    this.alias = CatalogUtil.normalizeIdentifier(alias);
+    this.alias = alias;
 
     if (!CatalogUtil.isFQTableName(this.tableDesc.getName())) {
       throw new IllegalArgumentException("the name in TableDesc must be qualified, but it is \"" +

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
index 2652030..b37a229 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
@@ -20,7 +20,6 @@ package org.apache.tajo.engine.planner.logical;
 
 import com.google.common.base.Objects;
 import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.engine.planner.PlanString;
 import org.apache.tajo.engine.planner.PlannerUtil;
@@ -37,7 +36,7 @@ public class TableSubQueryNode extends RelationNode implements Projectable {
   }
 
   public void init(String tableName, LogicalNode subQuery) {
-    this.tableName = CatalogUtil.normalizeIdentifier(tableName);
+    this.tableName = tableName;
     if (subQuery != null) {
       this.subQuery = subQuery;
       setOutSchema(SchemaUtil.clone(this.subQuery.getOutSchema()));

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryContext.java
index 5076b87..44c4ddc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryContext.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryContext.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.engine.query;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Options;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.conf.TajoConf;
@@ -102,7 +101,7 @@ public class QueryContext extends Options {
    * @param tableName The target table name
    */
   public void setOutputTable(String tableName) {
-    put(OUTPUT_TABLE_NAME, CatalogUtil.normalizeIdentifier(tableName));
+    put(OUTPUT_TABLE_NAME, tableName);
   }
 
   public String getOutputTable() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
index d0e8dc4..fe3caeb 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -287,7 +287,7 @@ public class GlobalEngine extends AbstractService {
     if(createTable.isExternal()){
       Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
     } else {
-      Path tablePath = new Path(sm.getWarehouseDir(), createTable.getTableName().toLowerCase());
+      Path tablePath = new Path(sm.getWarehouseDir(), createTable.getTableName());
       createTable.setPath(tablePath);
     }
 
@@ -382,7 +382,7 @@ public class GlobalEngine extends AbstractService {
     }
 
     if (catalog.createDatabase(databaseName, tablespaceName)) {
-      String normalized = CatalogUtil.normalizeIdentifier(databaseName);
+      String normalized = databaseName;
       Path databaseDir = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR), normalized);
       FileSystem fs = databaseDir.getFileSystem(context.getConf());
       fs.mkdirs(databaseDir);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index a504e7b..acbf839 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -234,7 +234,7 @@ public class TajoMasterClientService extends AbstractService {
     public BoolProto selectDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
       try {
         String sessionId = request.getSessionId().getId();
-        String databaseName = CatalogUtil.normalizeIdentifier(request.getValue());
+        String databaseName = request.getValue();
 
         if (context.getCatalog().existDatabase(databaseName)) {
           context.getSessionManager().getSession(sessionId).selectDatabase(databaseName);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java
index a5c5d78..e8c70ac 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -175,7 +175,7 @@ public class QueryTestCaseBase {
     // if the current database is "default", shouldn't drop it.
     if (!currentDatabase.equals(TajoConstants.DEFAULT_DATABASE_NAME)) {
       for (String tableName : catalog.getAllTableNames(currentDatabase)) {
-        client.updateQuery("DROP TABLE IF EXISTS " +tableName + " PURGE");
+        client.updateQuery("DROP TABLE IF EXISTS " + tableName + " PURGE");
       }
 
       client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME);
@@ -185,7 +185,7 @@ public class QueryTestCaseBase {
   }
 
   public QueryTestCaseBase() {
-    this.currentDatabase = getClass().getSimpleName();
+    this.currentDatabase = CatalogUtil.normalizeIdentifier(getClass().getSimpleName());
     init();
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
index 5949d1b..c8b11a8 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -110,13 +110,14 @@ public class TestTajoClient {
     int currentNum = client.getAllDatabaseNames().size();
     assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase());
 
-    assertTrue(client.createDatabase("testcurrentdatabase"));
+    String databaseName = CatalogUtil.normalizeIdentifier("testcurrentdatabase");
+    assertTrue(client.createDatabase(databaseName));
     assertEquals(currentNum + 1, client.getAllDatabaseNames().size());
     assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase());
-    assertTrue(client.selectDatabase("testcurrentdatabase"));
-    assertEquals("testcurrentdatabase", client.getCurrentDatabase());
-    assertTrue(client.selectDatabase("default"));
-    assertTrue(client.dropDatabase("testcurrentdatabase"));
+    assertTrue(client.selectDatabase(databaseName));
+    assertEquals(databaseName, client.getCurrentDatabase());
+    assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME));
+    assertTrue(client.dropDatabase(databaseName));
 
     assertEquals(currentNum, client.getAllDatabaseNames().size());
   }
@@ -139,19 +140,20 @@ public class TestTajoClient {
   @Test
   public final void testDropCurrentDatabase() throws IOException, ServiceException, InterruptedException {
     int currentNum = client.getAllDatabaseNames().size();
-    assertTrue(client.createDatabase("testdropcurrentdatabase"));
-    assertTrue(client.selectDatabase("testdropcurrentdatabase"));
-    assertEquals("testdropcurrentdatabase", client.getCurrentDatabase());
+    String databaseName = CatalogUtil.normalizeIdentifier("testdropcurrentdatabase");
+    assertTrue(client.createDatabase(databaseName));
+    assertTrue(client.selectDatabase(databaseName));
+    assertEquals(databaseName, client.getCurrentDatabase());
 
     try {
-      client.dropDatabase("testdropcurrentdatabase");
+      client.dropDatabase(databaseName);
       assertFalse(true);
     } catch (Throwable t) {
       assertFalse(false);
     }
 
-    assertTrue(client.selectDatabase("default"));
-    assertTrue(client.dropDatabase("testdropcurrentdatabase"));
+    assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME));
+    assertTrue(client.dropDatabase(databaseName));
     assertEquals(currentNum, client.getAllDatabaseNames().size());
   }
 
@@ -204,7 +206,7 @@ public class TestTajoClient {
 
   @Test
   public final void testUpdateQuery() throws IOException, ServiceException {
-    final String tableName = "testUpdateQuery";
+    final String tableName = CatalogUtil.normalizeIdentifier("testUpdateQuery");
     Path tablePath = writeTmpTable(tableName);
 
     assertFalse(client.existTable(tableName));
@@ -257,7 +259,7 @@ public class TestTajoClient {
   @Test
   public final void testCreateAndDropExternalTableByExecuteQuery() throws IOException, ServiceException {
     TajoConf conf = cluster.getConfiguration();
-    final String tableName = "testCreateAndDropExternalTableByExecuteQuery";
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropExternalTableByExecuteQuery");
 
     Path tablePath = writeTmpTable(tableName);
     assertFalse(client.existTable(tableName));
@@ -277,7 +279,7 @@ public class TestTajoClient {
   @Test
   public final void testCreateAndPurgeExternalTableByExecuteQuery() throws IOException, ServiceException {
     TajoConf conf = cluster.getConfiguration();
-    final String tableName = "testCreateAndPurgeExternalTableByExecuteQuery";
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndPurgeExternalTableByExecuteQuery");
 
     Path tablePath = writeTmpTable(tableName);
     assertFalse(client.existTable(tableName));
@@ -297,7 +299,7 @@ public class TestTajoClient {
   @Test
   public final void testCreateAndDropTableByExecuteQuery() throws IOException, ServiceException, SQLException {
     TajoConf conf = cluster.getConfiguration();
-    final String tableName = "testCreateAndDropTableByExecuteQuery";
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropTableByExecuteQuery");
 
     assertFalse(client.existTable(tableName));
 
@@ -318,7 +320,7 @@ public class TestTajoClient {
   @Test
   public final void testCreateAndPurgeTableByExecuteQuery() throws IOException, ServiceException, SQLException {
     TajoConf conf = cluster.getConfiguration();
-    final String tableName = "testCreateAndPurgeTableByExecuteQuery";
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndPurgeTableByExecuteQuery");
 
     assertFalse(client.existTable(tableName));
 
@@ -338,7 +340,7 @@ public class TestTajoClient {
 
   @Test
   public final void testDDLByExecuteQuery() throws IOException, ServiceException {
-    final String tableName = "testDDLByExecuteQuery";
+    final String tableName = CatalogUtil.normalizeIdentifier("testDDLByExecuteQuery");
     Path tablePath = writeTmpTable(tableName);
 
     assertFalse(client.existTable(tableName));
@@ -371,7 +373,7 @@ public class TestTajoClient {
 
   @Test
   public final void testGetTableDesc() throws IOException, ServiceException, SQLException {
-    final String tableName1 = "table3";
+    final String tableName1 = CatalogUtil.normalizeIdentifier("table3");
     Path tablePath = writeTmpTable(tableName1);
     LOG.error("Full path:" + tablePath.toUri().getRawPath());
     FileSystem fs = tablePath.getFileSystem(conf);
@@ -546,7 +548,7 @@ public class TestTajoClient {
   public final void testCreateAndDropTablePartitionedColumnByExecuteQuery() throws IOException,
       ServiceException, SQLException {
     TajoConf conf = cluster.getConfiguration();
-    final String tableName = "testCreateAndDropTablePartitionedColumnByExecuteQuery";
+    final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropTablePartitionedColumnByExecuteQuery");
 
     assertFalse(client.existTable(tableName));
 
@@ -590,7 +592,7 @@ public class TestTajoClient {
   @Test
   public final void testGetFinishedQueryList() throws IOException,
       ServiceException, SQLException {
-    final String tableName = "testGetFinishedQueryList";
+    final String tableName = CatalogUtil.normalizeIdentifier("testGetFinishedQueryList");
     String sql = "create table " + tableName + " (deptname text, score int4)";
 
     client.updateQuery(sql);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index a89f2bb..d19b4d6 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -141,14 +141,17 @@ public class ExprTestBase {
 
   public void testEval(Schema schema, String tableName, String csvTuple, String query, String [] expected)
       throws IOException {
-    testEval(schema, tableName, csvTuple, query, expected, ',', true);
+    testEval(schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple, query,
+        expected, ',', true);
   }
 
   public void testEval(Schema schema, String tableName, String csvTuple, String query, String [] expected,
                        char delimiter, boolean condition) throws IOException {
     LazyTuple lazyTuple;
     VTuple vtuple  = null;
-    String qualifiedTableName = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, tableName);
+    String qualifiedTableName =
+        CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME,
+            tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null);
     Schema inputSchema = null;
     if (schema != null) {
       inputSchema = SchemaUtil.clone(schema);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
index 39c7492..b10f2df 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
@@ -356,7 +356,8 @@ public class TestPredicates extends ExprTestBase {
 
   @Test
   public void testCreateTableWithUnsupportedStoreType() throws IOException {
-    testSimpleEval("create table table1 (name text, age int) using RAW;", new String[] {"Unsupported store type :RAW"}, false);
+    testSimpleEval("create table table1 (name text, age int) using RAW;",
+        new String[] {"Wrong query statement or query plan: create table table1 (name text, age int) using RAW"},
+        false);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
index a7b0a9e..22a178f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
@@ -29,6 +29,18 @@ import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 
 public class TestSQLExpression extends ExprTestBase {
 
+  @Test
+  public void testQuotedIdentifiers() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("컬럼1", TEXT);
+    schema.addColumn("컬럼2", TEXT);
+    testEval(schema, "테이블1", "123,234", "select \"컬럼1\"::float, cast (\"컬럼2\" as float4) as a from \"테이블1\"",
+        new String[]{"123.0", "234.0"});
+    testEval(schema,
+        "테이블1", "123,234", "select char_length(\"컬럼1\"), \"컬럼2\"::float4 as \"별명1\" from \"테이블1\"",
+        new String[]{"3", "234.0"});
+  }
+
   @Test(expected = NoSuchFunctionException.class)
   public void testNoSuchFunction() throws IOException {
     testSimpleEval("select test123('abc') col1 ", new String[]{"abc"});

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
index b882e84..3f1decb 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
@@ -57,15 +57,24 @@ public class TestDateTimeFunctions extends ExprTestBase {
   public void testExtract() throws IOException {
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TIMESTAMP);
-    testEval(schema2, "table1", "1970-01-17 10:09:37", "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", new String[]{"1970.0", "1.0", "17.0"});
+    testEval(schema2, "table1",
+        "1970-01-17 10:09:37",
+        "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
 
     Schema schema3 = new Schema();
     schema3.addColumn("col1", TIME);
-    testEval(schema3, "table1", "10:09:37.5", "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;", new String[]{"10.0", "9.0", "37.5"});
+    testEval(schema3, "table1",
+        "10:09:37.5",
+        "select extract(hour from col1), extract(minute from col1), extract(second from col1) from table1;",
+        new String[]{"10.0", "9.0", "37.5"});
 
     Schema schema4 = new Schema();
     schema4.addColumn("col1", DATE);
-    testEval(schema4, "table1", "1970-01-17", "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;", new String[]{"1970.0", "1.0", "17.0"});
+    testEval(schema4, "table1",
+        "1970-01-17",
+        "select extract(year from col1), extract(month from col1), extract(day from col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
 
     testSimpleEval("select extract(century from TIMESTAMP '1970-01-17 10:09:37');", new String[]{"19.0"});
 
@@ -144,15 +153,23 @@ public class TestDateTimeFunctions extends ExprTestBase {
   public void testDatePart() throws IOException {
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TIMESTAMP);
-    testEval(schema2, "table1", "1970-01-17 10:09:37", "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", new String[]{"1970.0", "1.0", "17.0"});
+    testEval(schema2, "table1",
+        "1970-01-17 10:09:37",
+        "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
 
     Schema schema3 = new Schema();
     schema3.addColumn("col1", TIME);
-    testEval(schema3, "table1", "10:09:37.5", "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;", new String[]{"10.0", "9.0", "37.5"});
+    testEval(schema3, "table1", "10:09:37.5",
+        "select date_part('hour', col1), date_part('minute', col1), date_part('second', col1) from table1;",
+        new String[]{"10.0", "9.0", "37.5"});
 
     Schema schema4 = new Schema();
     schema4.addColumn("col1", DATE);
-    testEval(schema4, "table1", "1970-01-17", "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;", new String[]{"1970.0", "1.0", "17.0"});
+    testEval(schema4, "table1",
+        "1970-01-17",
+        "select date_part('year', col1), date_part('month', col1), date_part('day', col1) from table1;",
+        new String[]{"1970.0", "1.0", "17.0"});
 
     testSimpleEval("select date_part('century', TIMESTAMP '1970-01-17 10:09:37');", new String[]{"19.0"});
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
index c4ef32b..5acd512 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
@@ -62,15 +62,15 @@ public class TestLogicalOptimizer {
     
     Schema schema = new Schema();
     schema.addColumn("name", Type.TEXT);
-    schema.addColumn("empId", Type.INT4);
-    schema.addColumn("deptName", Type.TEXT);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
 
     Schema schema2 = new Schema();
-    schema2.addColumn("deptName", Type.TEXT);
+    schema2.addColumn("deptname", Type.TEXT);
     schema2.addColumn("manager", Type.TEXT);
 
     Schema schema3 = new Schema();
-    schema3.addColumn("deptName", Type.TEXT);
+    schema3.addColumn("deptname", Type.TEXT);
     schema3.addColumn("score", Type.INT4);
     schema3.addColumn("phone", Type.INT4);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 65ade68..6dda611 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -262,7 +262,7 @@ public class TestLogicalPlanner {
   static {
     expectedJoinSchema = new Schema();
     expectedJoinSchema.addColumn("name", Type.TEXT);
-    expectedJoinSchema.addColumn("deptName", Type.TEXT);
+    expectedJoinSchema.addColumn("deptname", Type.TEXT);
     expectedJoinSchema.addColumn("score", Type.INT4);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
index a44f5f7..746f6fb 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
@@ -63,15 +63,15 @@ public class TestPlannerUtil {
 
     Schema schema = new Schema();
     schema.addColumn("name", Type.TEXT);
-    schema.addColumn("empId", CatalogUtil.newSimpleDataType(Type.INT4));
-    schema.addColumn("deptName", Type.TEXT);
+    schema.addColumn("empid", CatalogUtil.newSimpleDataType(Type.INT4));
+    schema.addColumn("deptname", Type.TEXT);
 
     Schema schema2 = new Schema();
-    schema2.addColumn("deptName", Type.TEXT);
+    schema2.addColumn("deptname", Type.TEXT);
     schema2.addColumn("manager", Type.TEXT);
 
     Schema schema3 = new Schema();
-    schema3.addColumn("deptName", Type.TEXT);
+    schema3.addColumn("deptname", Type.TEXT);
     schema3.addColumn("score", CatalogUtil.newSimpleDataType(Type.INT4));
 
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
index 7092693..c79796b 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
@@ -78,10 +78,10 @@ public class TestBNLJoinExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", Type.INT4);
-    schema.addColumn("empId", Type.INT4);
-    schema.addColumn("memId", Type.INT4);
-    schema.addColumn("deptName", Type.TEXT);
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("memid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -100,8 +100,8 @@ public class TestBNLJoinExec {
     catalog.createTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", Type.INT4);
-    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
     peopleSchema.addColumn("name", Type.TEXT);
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
index bce9d8e..b74527a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
@@ -95,14 +95,14 @@ public class TestBSTIndexExec {
     idxPath = new Path(workDir, "test.idx");
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", Type.INT4);
-    schema.addColumn("empId", Type.INT4);
-    schema.addColumn("deptName", Type.TEXT);
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
 
     this.idxSchema = new Schema();
-    idxSchema.addColumn("managerId", Type.INT4);
+    idxSchema.addColumn("managerid", Type.INT4);
     SortSpec[] sortKeys = new SortSpec[1];
-    sortKeys[0] = new SortSpec(idxSchema.getColumn("managerId"), true, false);
+    sortKeys[0] = new SortSpec(idxSchema.getColumn("managerid"), true, false);
     this.comp = new TupleComparator(idxSchema, sortKeys);
 
     this.writer = new BSTIndex(conf).getIndexWriter(idxPath,

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
index a82eeb9..ff3befe 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
@@ -75,9 +75,9 @@ public class TestExternalSortExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema schema = new Schema();
-    schema.addColumn("managerId", Type.INT4);
-    schema.addColumn("empId", Type.INT4);
-    schema.addColumn("deptName", Type.TEXT);
+    schema.addColumn("managerid", Type.INT4);
+    schema.addColumn("empid", Type.INT4);
+    schema.addColumn("deptname", Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -150,8 +150,8 @@ public class TestExternalSortExec {
     long start = System.currentTimeMillis();
     TupleComparator comparator = new TupleComparator(proj.getSchema(),
         new SortSpec[]{
-            new SortSpec(new Column("managerId", Type.INT4)),
-            new SortSpec(new Column("empId", Type.INT4))
+            new SortSpec(new Column("managerid", Type.INT4)),
+            new SortSpec(new Column("empid", Type.INT4))
         });
 
     while ((tuple = exec.next()) != null) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
index 9d0a86a..1dbbcf0 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
@@ -74,10 +74,10 @@ public class TestHashAntiJoinExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
-    employeeSchema.addColumn("managerId", Type.INT4);
-    employeeSchema.addColumn("empId", Type.INT4);
-    employeeSchema.addColumn("memId", Type.INT4);
-    employeeSchema.addColumn("deptName", Type.TEXT);
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -101,8 +101,8 @@ public class TestHashAntiJoinExec {
     catalog.createTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", Type.INT4);
-    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
     peopleSchema.addColumn("name", Type.TEXT);
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
index da7fce0..66222da 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
@@ -77,10 +77,10 @@ public class TestHashJoinExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
-    employeeSchema.addColumn("managerId", Type.INT4);
-    employeeSchema.addColumn("empId", Type.INT4);
-    employeeSchema.addColumn("memId", Type.INT4);
-    employeeSchema.addColumn("deptName", Type.TEXT);
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -101,8 +101,8 @@ public class TestHashJoinExec {
     catalog.createTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", Type.INT4);
-    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
     peopleSchema.addColumn("name", Type.TEXT);
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
index 75a27db..4e5de98 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
@@ -74,10 +74,10 @@ public class TestHashSemiJoinExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
-    employeeSchema.addColumn("managerId", Type.INT4);
-    employeeSchema.addColumn("empId", Type.INT4);
-    employeeSchema.addColumn("memId", Type.INT4);
-    employeeSchema.addColumn("deptName", Type.TEXT);
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -101,8 +101,8 @@ public class TestHashSemiJoinExec {
     catalog.createTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", Type.INT4);
-    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
     peopleSchema.addColumn("name", Type.TEXT);
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);

http://git-wip-us.apache.org/repos/asf/tajo/blob/37fef799/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
index 2ed057c..0e4fd9a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
@@ -77,10 +77,10 @@ public class TestMergeJoinExec {
     sm = StorageManagerFactory.getStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
-    employeeSchema.addColumn("managerId", Type.INT4);
-    employeeSchema.addColumn("empId", Type.INT4);
-    employeeSchema.addColumn("memId", Type.INT4);
-    employeeSchema.addColumn("deptName", Type.TEXT);
+    employeeSchema.addColumn("managerid", Type.INT4);
+    employeeSchema.addColumn("empid", Type.INT4);
+    employeeSchema.addColumn("memid", Type.INT4);
+    employeeSchema.addColumn("deptname", Type.TEXT);
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
@@ -107,8 +107,8 @@ public class TestMergeJoinExec {
     catalog.createTable(employee);
 
     Schema peopleSchema = new Schema();
-    peopleSchema.addColumn("empId", Type.INT4);
-    peopleSchema.addColumn("fk_memId", Type.INT4);
+    peopleSchema.addColumn("empid", Type.INT4);
+    peopleSchema.addColumn("fk_memid", Type.INT4);
     peopleSchema.addColumn("name", Type.TEXT);
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);


[3/3] git commit: TAJO-682: RangePartitionAlgorithm should be improved to handle empty texts. (Alvin Henrick via hyunsik)

Posted by hy...@apache.org.
TAJO-682: RangePartitionAlgorithm should be improved to handle empty texts. (Alvin Henrick via hyunsik)


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

Branch: refs/heads/master
Commit: e12e038b5d50db015e6b9cdf8de478591b0712be
Parents: 37fef79
Author: Hyunsik Choi <hy...@apache.org>
Authored: Tue Mar 25 12:19:34 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Tue Mar 25 12:19:34 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  3 +
 .../engine/planner/RangePartitionAlgorithm.java |  7 +-
 .../engine/planner/UniformRangePartition.java   |  5 +-
 .../planner/TestUniformRangePartition.java      | 74 ++++++++++++++++++++
 4 files changed, 85 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/e12e038b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a935225..7d9862e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -826,6 +826,9 @@ Release 0.2.0 - released
 
   BUG FIXES
 
+    TAJO-682: RangePartitionAlgorithm should be improved to handle empty
+    texts. (hyunsik)
+
     TAJO-511: Sometimes, a query progress becomes higher than 100%. (jihoon)
 
     TAJO-281: 'mvn package -Pdist' generates duplicate Tajo jar files.

http://git-wip-us.apache.org/repos/asf/tajo/blob/e12e038b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java
index 500001f..c4b1ae1 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/RangePartitionAlgorithm.java
@@ -22,6 +22,7 @@ import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.SortSpec;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleRange;
 
@@ -113,10 +114,12 @@ public abstract class RangePartitionAlgorithm {
         }
         break;
       case TEXT:
+        final char textStart =  start instanceof NullDatum ? '0' : start.asChars().charAt(0);
+        final char textEnd = end instanceof NullDatum ? '0' : end.asChars().charAt(0);
         if (isAscending) {
-          columnCard = new BigDecimal(end.asChars().charAt(0) - start.asChars().charAt(0));
+          columnCard = new BigDecimal(textEnd - textStart);
         } else {
-          columnCard = new BigDecimal(start.asChars().charAt(0) - end.asChars().charAt(0));
+          columnCard = new BigDecimal(textStart - textEnd);
         }
         break;
       case DATE:

http://git-wip-us.apache.org/repos/asf/tajo/blob/e12e038b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java
index 948b19e..a7df3e8 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/UniformRangePartition.java
@@ -24,6 +24,7 @@ import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.SortSpec;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.engine.exception.RangeOverflowException;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleRange;
@@ -194,7 +195,7 @@ public class UniformRangePartition extends RangePartitionAlgorithm {
       }
       case TEXT: {
         if (sortSpecs[colId].isAscending()) {
-          candidate = inc.add(new BigDecimal((int)(last.asChars().charAt(0))));
+          candidate = inc.add(new BigDecimal((int)(last instanceof NullDatum ? '0' : last.asChars().charAt(0))));
           return new BigDecimal(range.getEnd().get(colId).asChars().charAt(0)).compareTo(candidate) < 0;
         } else {
           candidate = new BigDecimal((int)(last.asChars().charAt(0))).subtract(inc);
@@ -381,7 +382,7 @@ public class UniformRangePartition extends RangePartitionAlgorithm {
                 + incs[i].longValue())) + ""));
           } else {
             end.put(i, DatumFactory.createText(
-                ((char) (last.get(i).asChars().charAt(0) + incs[i].longValue())) + ""));
+                ((char) ((last.get(i) instanceof NullDatum ? '0': last.get(i).asChars().charAt(0)) + incs[i].longValue())) + ""));
           }
           break;
         case DATE:

http://git-wip-us.apache.org/repos/asf/tajo/blob/e12e038b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java
index 3d5cdf2..dc5108a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestUniformRangePartition.java
@@ -304,4 +304,78 @@ public class TestUniformRangePartition {
 
     assertEquals(expected, ranges[0]);
   }
+
+  @Test
+  public void testPartitionForOnePartNumWithOneOfTheValueNull() {
+    Schema schema = new Schema()
+            .addColumn("l_returnflag", Type.TEXT)
+            .addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createNullDatum());
+    s.put(1, DatumFactory.createText("F"));
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createText("R"));
+    e.put(1, DatumFactory.createNullDatum());
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner =
+            new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(1);
+
+    assertEquals(expected, ranges[0]);
+  }
+
+  @Test
+  public void testPartitionForOnePartNumWithBothValueNull() {
+    Schema schema = new Schema()
+            .addColumn("l_returnflag", Type.TEXT)
+            .addColumn("l_linestatus", Type.TEXT);
+
+    SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+    Tuple s = new VTuple(2);
+    s.put(0, DatumFactory.createNullDatum());
+    s.put(1, DatumFactory.createNullDatum());
+    Tuple e = new VTuple(2);
+    e.put(0, DatumFactory.createNullDatum());
+    e.put(1, DatumFactory.createNullDatum());
+    TupleRange expected = new TupleRange(sortSpecs, s, e);
+    RangePartitionAlgorithm partitioner =
+            new UniformRangePartition(expected, sortSpecs, true);
+    TupleRange [] ranges = partitioner.partition(1);
+
+    assertEquals(expected, ranges[0]);
+  }
+
+    @Test
+    public void testPartitionWithNull() {
+        Schema schema = new Schema();
+        schema.addColumn("l_returnflag", Type.TEXT);
+        schema.addColumn("l_linestatus", Type.TEXT);
+
+        SortSpec [] sortSpecs = PlannerUtil.schemaToSortSpecs(schema);
+
+        Tuple s = new VTuple(2);
+        s.put(0, DatumFactory.createNullDatum());
+        s.put(1, DatumFactory.createText("F"));
+        Tuple e = new VTuple(2);
+        e.put(0, DatumFactory.createNullDatum());
+        e.put(1, DatumFactory.createText("O"));
+        TupleRange expected = new TupleRange(sortSpecs, s, e);
+        RangePartitionAlgorithm partitioner
+                = new UniformRangePartition(expected, sortSpecs, true);
+        TupleRange [] ranges = partitioner.partition(10);
+
+
+        TupleRange prev = null;
+        for (TupleRange r : ranges) {
+            if (prev == null) {
+                prev = r;
+            } else {
+                assertTrue(prev.compareTo(r) > 0);
+            }
+        }
+    }
 }