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/26 09:48:48 UTC

git commit: TAJO-712: Fix some bugs after database is supported. (hyunsik)

Repository: tajo
Updated Branches:
  refs/heads/master 9ecd9f6e0 -> a2a2c3ca9


TAJO-712: Fix some bugs after database is supported. (hyunsik)


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

Branch: refs/heads/master
Commit: a2a2c3ca97b058d96c127ae46527dff8fef73904
Parents: 9ecd9f6
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Mar 26 17:44:11 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Mar 26 17:44:11 2014 +0900

----------------------------------------------------------------------
 .../org/apache/tajo/catalog/CatalogUtil.java    | 125 ++++++++++++++++++-
 .../org/apache/tajo/catalog/DDLBuilder.java     |   6 +-
 .../apache/tajo/catalog/store/DerbyStore.java   |   2 +-
 .../src/main/resources/schemas/derby/tables.sql |   2 +-
 .../src/main/resources/schemas/mysql/tables.sql |   4 +-
 .../org/apache/tajo/catalog/TestCatalog.java    |  32 +++++
 .../apache/tajo/cli/ConnectDatabaseCommand.java |  27 ++--
 .../java/org/apache/tajo/client/TajoDump.java   |  11 +-
 .../java/org/apache/tajo/util/StringUtils.java  |  24 ++++
 .../org/apache/tajo/engine/parser/SQLParser.g4  |   2 +-
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |   5 +-
 .../org/apache/tajo/client/TestDDLBuilder.java  |  65 +++++++++-
 .../apache/tajo/engine/query/TestCTASQuery.java |  12 ++
 .../TestCTASQuery/CtasWithManagedTable.sql      |  12 ++
 .../testJoinOnMultipleDatabases.sql             |   6 +-
 .../results/testBuildDDLForBaseTable.result     |   5 -
 .../results/testBuildDDLForExternalTable.result |   5 -
 .../testBuildDDLForBaseTable.result             |   5 +
 .../testBuildDDLForExternalTable.result         |   5 +
 .../testBuildDDLQuotedTableName1.result         |   5 +
 .../testBuildDDLQuotedTableName2.result         |   5 +
 21 files changed, 325 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/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 5ad2eca..92df86a 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
@@ -26,12 +26,15 @@ import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
 import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.util.StringUtils;
 
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import static org.apache.tajo.common.TajoDataTypes.Type;
@@ -41,13 +44,100 @@ public class CatalogUtil {
   public final static String IDENTIFIER_DELIMITER_REGEXP = "\\.";
 
   /**
-   * Normalize an identifier
+   * Normalize an identifier. Normalization means a translation from a identifier to be a refined identifier name.
+   *
+   * Identifier can be composed of multiple parts as follows:
+   * <pre>
+   *   database_name.table_name.column_name
+   * </pre>
+   *
+   * Each regular identifier part can be composed alphabet ([a-z][A-Z]), number([0-9]), and underscore([_]).
+   * Also, the first letter must be an alphabet character.
+   *
+   * <code>normalizeIdentifier</code> normalizes each part of an identifier.
+   *
+   * In detail, for each part, it performs as follows:
+   * <ul>
+   *   <li>changing a part without double quotation to be lower case letters</li>
+   *   <li>eliminating double quotation marks from identifier</li>
+   * </ul>
    *
    * @param identifier The identifier to be normalized
    * @return The normalized identifier
    */
   public static String normalizeIdentifier(String identifier) {
-    return isDelimited(identifier) ? stripQuote(identifier).toLowerCase() : identifier.toLowerCase();
+    String [] splitted = identifier.split(IDENTIFIER_DELIMITER_REGEXP);
+
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+    for (String part : splitted) {
+      if (first) {
+        first = false;
+      } else {
+        sb.append(IDENTIFIER_DELIMITER);
+      }
+      sb.append(normalizeIdentifierPart(part));
+    }
+    return sb.toString();
+  }
+
+  public static String normalizeIdentifierPart(String part) {
+    return isDelimited(part) ? stripQuote(part) : part.toLowerCase();
+  }
+
+  /**
+   * Denormalize an identifier. Denormalize means a translation from a stored identifier
+   * to be a printable identifier name.
+   *
+   * In detail, for each part, it performs as follows:
+   * <ul>
+   *   <li>changing a part including upper case character or non-ascii character to be lower case letters</li>
+   *   <li>eliminating double quotation marks from identifier</li>
+   * </ul>
+   *
+   * @param identifier The identifier to be normalized
+   * @return The denormalized identifier
+   */
+  public static String denormalizeIdentifier(String identifier) {
+    String [] splitted = identifier.split(IDENTIFIER_DELIMITER_REGEXP);
+
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+    for (String part : splitted) {
+      if (first) {
+        first = false;
+      } else {
+        sb.append(IDENTIFIER_DELIMITER);
+      }
+      sb.append(denormalizePart(part));
+    }
+    return sb.toString();
+  }
+
+  public static String denormalizePart(String identifier) {
+    if (isShouldBeQuoted(identifier)) {
+      return StringUtils.doubleQuote(identifier);
+    } else {
+      return identifier;
+    }
+  }
+
+  public static boolean isShouldBeQuoted(String columnName) {
+    for (char character : columnName.toCharArray()) {
+      if (Character.isUpperCase(character)) {
+        return true;
+      }
+
+      if (!StringUtils.isPartOfAnsiSQLIdentifier(character)) {
+        return true;
+      }
+
+      if (RESERVED_KEYWORDS_SET.contains(columnName.toUpperCase())) {
+        return true;
+      }
+    }
+
+    return false;
   }
 
   public static String stripQuote(String str) {
@@ -274,7 +364,7 @@ public class CatalogUtil {
   }
 
   public static String columnToDDLString(Column column) {
-    StringBuilder sb = new StringBuilder(column.getSimpleName());
+    StringBuilder sb = new StringBuilder(denormalizeIdentifier(column.getSimpleName()));
     sb.append(" ").append(column.getDataType().getType());
     if (column.getDataType().hasLength()) {
       sb.append(" (").append(column.getDataType().getLength()).append(")");
@@ -320,4 +410,33 @@ public class CatalogUtil {
       closeQuietly(stmt);
     }
   }
+
+  public static final Set<String> RESERVED_KEYWORDS_SET = new HashSet<String>();
+
+  static final String [] RESERVED_KEYWORDS = {
+      "AS", "ALL", "AND", "ANY", "ASYMMETRIC", "ASC",
+      "BOTH",
+      "CASE", "CAST", "CREATE", "CROSS",
+      "DESC", "DISTINCT",
+      "END", "ELSE", "EXCEPT",
+      "FALSE", "FULL", "FROM",
+      "GROUP",
+      "HAVING",
+      "ILIKE", "IN", "INNER", "INTERSECT", "INTO", "IS",
+      "JOIN",
+      "LEADING", "LEFT", "LIKE", "LIMIT",
+      "NATURAL", "NOT", "NULL",
+      "ON", "OUTER", "OR", "ORDER",
+      "RIGHT",
+      "SELECT", "SOME", "SYMMETRIC",
+      "TABLE", "THEN", "TRAILING", "TRUE",
+      "UNION", "UNIQUE", "USING",
+      "WHEN", "WHERE", "WITH"
+  };
+
+  static {
+    for (String keyword : RESERVED_KEYWORDS) {
+      RESERVED_KEYWORDS_SET.add(keyword);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
index fd2fe8f..d9cbcee 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
@@ -33,7 +33,7 @@ public class DDLBuilder {
       .append(" Storage: ").append(desc.getMeta().getStoreType().name());
     sb.append("\n-- Path: ").append(desc.getPath());
     sb.append("\n--\n");
-    sb.append("CREATE EXTERNAL TABLE ").append(desc.getName());
+    sb.append("CREATE EXTERNAL TABLE ").append(CatalogUtil.denormalizeIdentifier(desc.getName()));
     buildSchema(sb, desc.getSchema());
     buildUsingClause(sb, desc.getMeta());
     buildWithClause(sb, desc.getMeta());
@@ -56,7 +56,7 @@ public class DDLBuilder {
         .append(" Storage: ").append(desc.getMeta().getStoreType().name());
     sb.append("\n-- Path: ").append(desc.getPath());
     sb.append("\n--\n");
-    sb.append("CREATE TABLE ").append(desc.getName());
+    sb.append("CREATE TABLE ").append(CatalogUtil.denormalizeIdentifier(desc.getName()));
     buildSchema(sb, desc.getSchema());
     buildUsingClause(sb, desc.getMeta());
     buildWithClause(sb, desc.getMeta());
@@ -80,7 +80,7 @@ public class DDLBuilder {
         sb.append(", ");
       }
 
-      sb.append(column.getSimpleName()).append(" ");
+      sb.append(CatalogUtil.denormalizeIdentifier(column.getSimpleName())).append(" ");
       TajoDataTypes.DataType dataType = column.getDataType();
       sb.append(dataType.getType().name());
       if (column.getDataType().hasLength() && column.getDataType().getLength() > 0) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
index 974ded9..d6f9fc3 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
@@ -135,7 +135,7 @@ public class DerbyStore extends AbstractDBStore {
 
 
 
-        sql = "CREATE UNIQUE INDEX idx_tables_name on TABLES (TABLE_NAME)";
+        sql = "CREATE UNIQUE INDEX idx_tables_name on TABLES (DB_ID, TABLE_NAME)";
         if (LOG.isDebugEnabled()) {
           LOG.debug(sql);
         }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/tables.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/tables.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/tables.sql
index 2fe0e7d..3e15c5b 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/tables.sql
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/tables.sql
@@ -6,5 +6,5 @@ CREATE TABLE TABLES (
   PATH VARCHAR(4096),
   STORE_TYPE CHAR(16),
   CONSTRAINT TABLES_PK PRIMARY KEY (TID),
-  CONSTRAINT C_TABLE_ID_UNIQ UNIQUE (TABLE_NAME)
+  CONSTRAINT C_TABLE_ID_UNIQ UNIQUE (DB_ID, TABLE_NAME)
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/tables.sql
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/tables.sql b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/tables.sql
index 98c0c94..e7297f7 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/tables.sql
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/tables.sql
@@ -1,11 +1,11 @@
 CREATE TABLE TABLES (
   TID INT NOT NULL AUTO_INCREMENT PRIMARY KEY,
   DB_ID INT NOT NULL,
-  TABLE_NAME VARCHAR(128) NOT NULL UNIQUE,
+  TABLE_NAME VARCHAR(128) NOT NULL,
   TABLE_TYPE VARCHAR(128) NOT NULL,
   PATH VARCHAR(4096),
   STORE_TYPE CHAR(16),
   FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID),
   INDEX IDX_DB_ID (DB_ID),
-  UNIQUE INDEX IDX_TABLE_ID (TABLE_NAME)
+  UNIQUE INDEX IDX_TABLE_ID (DB_ID, TABLE_NAME)
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
index cfae15d..36c6b6b 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
@@ -323,6 +323,38 @@ public class TestCatalog {
         CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, tableName), schema, meta,
         new Path(CommonTestingUtil.getTestDir(), "indexed"));
   }
+
+  @Test
+  public void testCreateSameTables() throws IOException {
+    assertTrue(catalog.createDatabase("tmpdb3", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    assertTrue(catalog.existDatabase("tmpdb3"));
+    assertTrue(catalog.createDatabase("tmpdb4", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    assertTrue(catalog.existDatabase("tmpdb4"));
+
+    TableDesc table1 = createMockupTable("tmpdb3", "table1");
+    assertTrue(catalog.createTable(table1));
+    TableDesc table2 = createMockupTable("tmpdb3", "table2");
+    assertTrue(catalog.createTable(table2));
+    assertTrue(catalog.existsTable("tmpdb3", "table1"));
+    assertTrue(catalog.existsTable("tmpdb3", "table2"));
+
+    TableDesc table3 = createMockupTable("tmpdb4", "table1");
+    assertTrue(catalog.createTable(table3));
+    TableDesc table4 = createMockupTable("tmpdb4", "table2");
+    assertTrue(catalog.createTable(table4));
+    assertTrue(catalog.existsTable("tmpdb4", "table1"));
+    assertTrue(catalog.existsTable("tmpdb4", "table2"));
+
+    assertTrue(catalog.dropTable("tmpdb3.table1"));
+    assertTrue(catalog.dropTable("tmpdb3.table2"));
+    assertTrue(catalog.dropTable("tmpdb4.table1"));
+    assertTrue(catalog.dropTable("tmpdb4.table2"));
+
+    assertFalse(catalog.existsTable("tmpdb3.table1"));
+    assertFalse(catalog.existsTable("tmpdb3.table2"));
+    assertFalse(catalog.existsTable("tmpdb4.table1"));
+    assertFalse(catalog.existsTable("tmpdb4.table2"));
+  }
 	
 	@Test
 	public void testAddAndDelIndex() throws Exception {

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java
index 02f195a..78774e5 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java
@@ -37,17 +37,22 @@ public class ConnectDatabaseCommand extends TajoShellCommand {
       context.getOutput().write(String.format("You are now connected to database \"%s\" as user \"%s\".%n",
           client.getCurrentDatabase(), client.getUserInfo().getUserName()));
     } else if (cmd.length == 2) {
-      try {
-        if (client.selectDatabase(cmd[1])) {
-          context.setCurrentDatabase(client.getCurrentDatabase());
-          context.getOutput().write(String.format("You are now connected to database \"%s\" as user \"%s\".%n",
-              context.getCurrentDatabase(), client.getUserInfo().getUserName()));
-        }
-      } catch (ServiceException se) {
-        if (se.getMessage() != null) {
-          context.getOutput().write(se.getMessage());
-        } else {
-          context.getOutput().write(String.format("cannot connect the database \"%s\"", cmd[1]));
+
+      if (!client.existDatabase(cmd[1])) {
+        context.getOutput().write("No Database Found\n");
+      } else {
+        try {
+          if (client.selectDatabase(cmd[1])) {
+            context.setCurrentDatabase(client.getCurrentDatabase());
+            context.getOutput().write(String.format("You are now connected to database \"%s\" as user \"%s\".%n",
+                context.getCurrentDatabase(), client.getUserInfo().getUserName()));
+          }
+        } catch (ServiceException se) {
+          if (se.getMessage() != null) {
+            context.getOutput().write(se.getMessage());
+          } else {
+            context.getOutput().write(String.format("cannot connect the database \"%s\"", cmd[1]));
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
index 9aaef8e..224de45 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
@@ -26,6 +26,7 @@ import org.apache.tajo.catalog.DDLBuilder;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.util.Pair;
+import org.apache.tajo.util.TUtil;
 
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -33,6 +34,8 @@ import java.sql.SQLException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
+import java.util.Collections;
+import java.util.List;
 
 public class TajoDump {
   private static final org.apache.commons.cli.Options options;
@@ -143,8 +146,12 @@ public class TajoDump {
     writer.write(String.format("CREATE DATABASE IF NOT EXISTS %s;", databaseName));
     writer.write("\n");
 
-    for (String tableName : client.getTableList(databaseName)) {
-      TableDesc table = client.getTableDesc(CatalogUtil.buildFQName(databaseName, tableName));
+    // returned list is immutable.
+    List<String> tableNames = TUtil.newList(client.getTableList(databaseName));
+    Collections.sort(tableNames);
+    for (String tableName : tableNames) {
+      TableDesc table =
+          client.getTableDesc(CatalogUtil.denormalizeIdentifier(CatalogUtil.buildFQName(databaseName,tableName)));
       if (table.isExternal()) {
         writer.write(DDLBuilder.buildDDLForExternalTable(table));
       } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
index ad60a50..ad607a1 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
@@ -62,4 +62,28 @@ public class StringUtils {
   public static String doubleQuote(String str) {
     return "\"" + str + "\"";
   }
+
+  public static boolean isPartOfAnsiSQLIdentifier(char character) {
+    return
+        isLowerCaseAlphabet(character) ||
+        isUpperCaseAlphabet(character) ||
+        isDigit(character)             ||
+        isUndersscore(character);
+  }
+
+  public static boolean isUndersscore(char character) {
+    return character == '_';
+  }
+
+  public static boolean isLowerCaseAlphabet(char character) {
+    return 'a' <= character && character <= 'z';
+  }
+
+  public static boolean isUpperCaseAlphabet(char character) {
+    return 'A' <= character && character <= 'Z';
+  }
+
+  public static boolean isDigit(char character) {
+    return '0' <= character && character <= '9';
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/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 ce78c98..c0edf09 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
@@ -1086,7 +1086,7 @@ set_qualifier
   ;
 
 column_reference
-  : (tb_name=identifier DOT)? name=identifier
+  : ((db_name = identifier DOT)? (tb_name=identifier DOT))? name=identifier
   ;
 
 as_clause

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 784e737..30d4c2e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -29,6 +29,7 @@ import org.apache.tajo.algebra.*;
 import org.apache.tajo.algebra.Aggregation.GroupType;
 import org.apache.tajo.algebra.LiteralValue.LiteralType;
 import org.apache.tajo.catalog.CatalogConstants;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.engine.parser.SQLParser.*;
 
 import java.util.*;
@@ -801,7 +802,9 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
   @Override
   public ColumnReferenceExpr visitColumn_reference(SQLParser.Column_referenceContext ctx) {
     ColumnReferenceExpr column = new ColumnReferenceExpr(ctx.name.getText());
-    if (ctx.tb_name != null) {
+    if (checkIfExist(ctx.db_name)) {
+      column.setQualifier(CatalogUtil.buildFQName(ctx.db_name.getText(), ctx.tb_name.getText()));
+    } else if (ctx.tb_name != null) {
       column.setQualifier(ctx.tb_name.getText());
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestDDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestDDLBuilder.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestDDLBuilder.java
index 1ae4753..73f7d5e 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestDDLBuilder.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestDDLBuilder.java
@@ -28,6 +28,8 @@ import org.apache.tajo.util.FileUtil;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 
 public class TestDDLBuilder {
@@ -60,14 +62,73 @@ public class TestDDLBuilder {
     TableDesc desc = new TableDesc("db1.table1", schema1, meta1, new Path("/table1"));
     desc.setPartitionMethod(partitionMethod1);
     desc.setExternal(true);
-    assertEquals(FileUtil.readTextFileFromResource("results/testBuildDDLForExternalTable.result"),
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLForExternalTable.result"),
         DDLBuilder.buildDDLForExternalTable(desc));
   }
 
   @Test
+  public void testBuildDDLQuotedTableName() throws Exception {
+    Schema schema2 = new Schema();
+    schema2.addColumn("name", TajoDataTypes.Type.BLOB);
+    schema2.addColumn("addr", TajoDataTypes.Type.TEXT);
+    schema2.addColumn("FirstName", TajoDataTypes.Type.TEXT);
+    schema2.addColumn("LastName", TajoDataTypes.Type.TEXT);
+    schema2.addColumn("with", TajoDataTypes.Type.TEXT);
+
+    Schema expressionSchema2 = new Schema();
+    expressionSchema2.addColumn("BirthYear", TajoDataTypes.Type.INT4);
+
+    PartitionMethodDesc partitionMethod2 = new PartitionMethodDesc(
+        "db1",
+        "table1",
+        CatalogProtos.PartitionType.COLUMN,
+        "key,key2",
+        expressionSchema2);
+
+    TableDesc desc = new TableDesc("db1.TABLE2", schema2, meta1, new Path("/table1"));
+    desc.setPartitionMethod(partitionMethod2);
+    desc.setExternal(true);
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName1.result"),
+        DDLBuilder.buildDDLForExternalTable(desc));
+
+    desc = new TableDesc("db1.TABLE1", schema2, meta1, new Path("/table1"));
+    desc.setPartitionMethod(partitionMethod2);
+    desc.setExternal(false);
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName2.result"),
+        DDLBuilder.buildDDLForBaseTable(desc));
+  }
+
+  @Test
   public void testBuildDDLForBaseTable() throws Exception {
     TableDesc desc = new TableDesc("db1.table2", schema1, meta1, new Path("/table1"));
-    assertEquals(FileUtil.readTextFileFromResource("results/testBuildDDLForBaseTable.result"),
+    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLForBaseTable.result"),
         DDLBuilder.buildDDLForBaseTable(desc));
   }
+
+  @Test
+  public void testBuildColumn() throws Exception {
+    String [] tobeUnquoted = {
+        "column_name",
+        "columnname",
+        "column_1",
+    };
+
+    for (String columnName : tobeUnquoted) {
+      assertFalse(CatalogUtil.isShouldBeQuoted(columnName));
+    }
+
+    String [] quoted = {
+        "Column_Name",
+        "COLUMN_NAME",
+        "컬럼",
+        "$column_name",
+        "Column_Name1",
+        "with",
+        "when"
+    };
+
+    for (String columnName : quoted) {
+      assertTrue(CatalogUtil.isShouldBeQuoted(columnName));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/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 82c4be9..e6840f6 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
@@ -209,4 +209,16 @@ public class TestCTASQuery extends QueryTestCaseBase {
     assertNotNull(options);
     assertEquals(StringEscapeUtils.escapeJava("\u0001"), options.get(CatalogConstants.CSVFILE_DELIMITER));
   }
+
+  @Test
+  public final void testCtasWithManagedTable() throws Exception {
+    ResultSet res = executeFile("CtasWithManagedTable.sql");
+    res.close();
+
+    assertFalse(client.existTable("MANAGED_TABLE1"));
+    assertTrue(client.existTable("\"MANAGED_TABLE1\""));
+    TableDesc desc =  client.getTableDesc("\"MANAGED_TABLE1\"");
+    assertNotNull(desc);
+    assertEquals("MANAGED_TABLE1", desc.getPath().getName());
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/queries/TestCTASQuery/CtasWithManagedTable.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestCTASQuery/CtasWithManagedTable.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCTASQuery/CtasWithManagedTable.sql
new file mode 100644
index 0000000..1dd5e90
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestCTASQuery/CtasWithManagedTable.sql
@@ -0,0 +1,12 @@
+create table "MANAGED_TABLE1" (col1 float, col2 float) using rcfile as
+select
+  sum(l_orderkey) as total1,
+  avg(l_partkey) as total2
+from
+  lineitem
+group by
+  l_quantity
+order by
+  l_quantity
+limit
+  3;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testJoinOnMultipleDatabases.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testJoinOnMultipleDatabases.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testJoinOnMultipleDatabases.sql
index 4598dd8..eb5a2c5 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testJoinOnMultipleDatabases.sql
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testJoinOnMultipleDatabases.sql
@@ -2,10 +2,10 @@ select
 	s_acctbal,
 	s_name,
 	n_name,
-	p_partkey,
+	JOINS.part_.p_partkey,
 	p_mfgr,
-	s_address,
-	s_phone,
+	JOINS.supplier_.s_address,
+	JOINS.supplier_.s_phone,
 	s_comment
 from
 	JOINS.part_,

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForBaseTable.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForBaseTable.result b/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForBaseTable.result
deleted file mode 100644
index bd59e2d..0000000
--- a/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForBaseTable.result
+++ /dev/null
@@ -1,5 +0,0 @@
---
--- Name: db1.table2; Type: TABLE; Storage: CSV
--- Path: /table1
---
-CREATE TABLE db1.table2 (name BLOB, addr TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForExternalTable.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForExternalTable.result b/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForExternalTable.result
deleted file mode 100644
index 78c64ff..0000000
--- a/tajo-core/tajo-core-backend/src/test/resources/results/testBuildDDLForExternalTable.result
+++ /dev/null
@@ -1,5 +0,0 @@
---
--- Name: db1.table1; Type: TABLE; Storage: CSV
--- Path: /table1
---
-CREATE EXTERNAL TABLE db1.table1 (name BLOB, addr TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|') PARTITION BY COLUMN(key INT4, key2 TEXT) LOCATION '/table1';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result
new file mode 100644
index 0000000..bd59e2d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result
@@ -0,0 +1,5 @@
+--
+-- Name: db1.table2; Type: TABLE; Storage: CSV
+-- Path: /table1
+--
+CREATE TABLE db1.table2 (name BLOB, addr TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForExternalTable.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForExternalTable.result b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForExternalTable.result
new file mode 100644
index 0000000..78c64ff
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLForExternalTable.result
@@ -0,0 +1,5 @@
+--
+-- Name: db1.table1; Type: TABLE; Storage: CSV
+-- Path: /table1
+--
+CREATE EXTERNAL TABLE db1.table1 (name BLOB, addr TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|') PARTITION BY COLUMN(key INT4, key2 TEXT) LOCATION '/table1';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName1.result b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName1.result
new file mode 100644
index 0000000..ee9845e
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName1.result
@@ -0,0 +1,5 @@
+--
+-- Name: db1.TABLE2; Type: TABLE; Storage: CSV
+-- Path: /table1
+--
+CREATE EXTERNAL TABLE db1."TABLE2" (name BLOB, addr TEXT, "FirstName" TEXT, "LastName" TEXT, "with" TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|') PARTITION BY COLUMN("BirthYear" INT4) LOCATION '/table1';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/a2a2c3ca/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result
new file mode 100644
index 0000000..40e4ff1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result
@@ -0,0 +1,5 @@
+--
+-- Name: db1.TABLE1; Type: TABLE; Storage: CSV
+-- Path: /table1
+--
+CREATE TABLE db1."TABLE1" (name BLOB, addr TEXT, "FirstName" TEXT, "LastName" TEXT, "with" TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|') PARTITION BY COLUMN("BirthYear" INT4);
\ No newline at end of file