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

[3/3] tajo git commit: TAJO-1832: Well support for self-describing data formats.

TAJO-1832: Well support for self-describing data formats.

Closes #756


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

Branch: refs/heads/master
Commit: 5a15586139c5fed29b66cbfc03b54e0bed09f5b7
Parents: 4a96288
Author: Jihoon Son <ji...@apache.org>
Authored: Wed Sep 23 10:12:15 2015 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Wed Sep 23 10:12:15 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/algebra/CreateTable.java    |  14 +
 .../java/org/apache/tajo/catalog/TableDesc.java |  36 +-
 .../src/main/proto/CatalogProtos.proto          |   2 +-
 .../dictionary/AbstractTableDescriptor.java     |   7 +-
 .../tajo/catalog/store/AbstractDBStore.java     |  46 +-
 .../src/main/resources/schemas/derby/derby.xml  |   4 +-
 .../main/resources/schemas/mariadb/mariadb.xml  |   4 +-
 .../src/main/resources/schemas/mysql/mysql.xml  |   4 +-
 .../main/resources/schemas/oracle/oracle.xml    |   4 +-
 .../resources/schemas/postgresql/postgresql.xml |   4 +-
 .../tajo/client/CatalogAdminClientImpl.java     |   9 +-
 tajo-client/src/main/proto/ClientProtos.proto   |   2 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |   6 +-
 .../tajo/engine/planner/TestLogicalPlanner.java | 123 ++++-
 .../tajo/engine/query/TestCreateTable.java      |  31 ++
 .../engine/query/TestQueryOnSelfDescTable.java  | 184 +++++++
 .../tajo/engine/query/TestUnionQuery.java       |   2 +-
 .../TestCreateTable/json_table/table1.json      |   4 +
 .../TestQueryOnSelfDescTable/github/github.json |   4 +
 .../TestQueryOnSelfDescTable/sample1/table.json |   3 +
 .../sample2/sample2.json                        |   1 +
 .../tweets/sample1.json                         |   4 +
 .../TestCreateTable/testSelfDescTable1.sql      |   1 +
 .../TestQueryOnSelfDescTable/testCrossJoin.sql  |   6 +
 .../TestQueryOnSelfDescTable/testGroupby.sql    |   7 +
 .../TestQueryOnSelfDescTable/testGroupby2.sql   |   7 +
 .../TestQueryOnSelfDescTable/testGroupby3.sql   |   7 +
 ...JoinOfSelfDescTablesWithQualifiedColumns.sql |   6 +
 .../testJoinWithSchemaFullTable.sql             |   8 +
 .../testJoinWithSchemaFullTable2.sql            |   8 +
 .../testJoinWithSchemaFullTable4.sql            |   8 +
 .../TestQueryOnSelfDescTable/testSelect.sql     |   8 +
 .../TestQueryOnSelfDescTable/testSelect2.sql    |   9 +
 .../TestQueryOnSelfDescTable/testSort.sql       |   7 +
 .../positive/create_self_desc_table1.sql        |   1 +
 .../positive/create_self_desc_table2.sql        |   1 +
 .../testCrossJoin.result                        |  22 +
 .../TestQueryOnSelfDescTable/testGroupby.result |   5 +
 .../testGroupby2.result                         |   3 +
 .../testGroupby3.result                         |   5 +
 .../testInSubquery.1.result                     |   4 +
 ...nOfSelfDescTablesWithQualifiedColumns.result |   3 +
 .../testJoinWithSchemaFullTable.result          |   4 +
 .../testJoinWithSchemaFullTable2.result         |   3 +
 .../testJoinWithSchemaFullTable4.result         |   5 +
 .../TestQueryOnSelfDescTable/testSelect.result  |   3 +
 .../TestQueryOnSelfDescTable/testSelect2.result |   3 +
 .../TestQueryOnSelfDescTable/testSort.result    |   6 +
 .../TestSQLAnalyzer/case_sensitivity1.result    |   1 +
 .../TestSQLAnalyzer/create_table_like_1.result  |   1 +
 .../create_table_maptype_1.result               |   1 +
 .../create_table_maptype_2.result               |   1 +
 .../create_table_maptype_3.result               |   1 +
 .../create_table_nested_1.result                |   1 +
 .../create_table_nested_2.result                |   1 +
 .../create_table_partition_by_column.result     |   1 +
 .../create_table_partition_by_hash_1.result     |   1 +
 .../create_table_partition_by_hash_2.result     |   1 +
 .../create_table_partition_by_list.result       |   1 +
 .../create_table_partition_by_range.result      |   1 +
 .../TestUnionQuery/testUnionAndFilter.result    |   6 +-
 .../tajo/master/TajoMasterClientService.java    |   8 +-
 .../tajo/master/exec/CreateTableExecutor.java   |   7 +-
 .../org/apache/tajo/parser/sql/SQLAnalyzer.java |  18 +-
 tajo-docs/src/main/sphinx/sql_language/ddl.rst  |  12 +-
 .../sphinx/table_management/file_formats.rst    |   3 +-
 .../src/main/sphinx/table_management/json.rst   | 100 ++++
 .../src/main/sphinx/table_management/text.rst   |   8 +-
 .../org/apache/tajo/plan/ExprAnnotator.java     |  15 +-
 .../org/apache/tajo/plan/LogicalPlanner.java    |  10 +-
 .../org/apache/tajo/plan/TypeDeterminant.java   |   4 +-
 .../tajo/plan/algebra/BaseAlgebraVisitor.java   |   1 +
 .../tajo/plan/logical/CreateTableNode.java      |   9 +
 .../tajo/plan/logical/StoreTableNode.java       |  12 +-
 .../tajo/plan/nameresolver/NameResolver.java    |  99 +++-
 .../plan/nameresolver/NameResolvingMode.java    |   8 +-
 .../plan/nameresolver/ResolverByLegacy.java     |  24 +-
 .../tajo/plan/nameresolver/ResolverByRels.java  |  14 +-
 .../nameresolver/ResolverByRelsAndSubExprs.java |  19 +-
 .../nameresolver/ResolverBySubExprsAndRels.java |  17 +-
 .../BaseLogicalPlanPreprocessPhaseProvider.java |   3 +-
 .../tajo/plan/rewrite/BaseSchemaBuildPhase.java |  28 +-
 .../rewrite/LogicalPlanPreprocessPhase.java     |   2 +-
 .../plan/rewrite/SelfDescSchemaBuildPhase.java  | 489 +++++++++++++++++++
 .../plan/serder/LogicalNodeDeserializer.java    |   6 +-
 .../tajo/plan/serder/LogicalNodeSerializer.java |   5 +-
 .../org/apache/tajo/plan/util/ExprFinder.java   |  12 +-
 .../org/apache/tajo/plan/util/PlannerUtil.java  |   9 +
 .../plan/verifier/PreLogicalPlanVerifier.java   |  19 +
 .../tajo/plan/visitor/SimpleAlgebraVisitor.java |   3 +
 tajo-plan/src/main/proto/Plan.proto             |   2 +-
 .../org/apache/tajo/parser/sql/SQLParser.g4     |   7 +-
 93 files changed, 1470 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index fb0c8a6..63af5dd 100644
--- a/CHANGES
+++ b/CHANGES
@@ -6,6 +6,8 @@ Release 0.11.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-1832: Well support for self-describing data formats. (jihoon)
+
     TAJO-1730: JDBC Tablespace support. (hyunsik)
 
     TAJO-1812: Timezone support in JSON file format. (hyunsik)

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
----------------------------------------------------------------------
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
index 8e081ff..6131b8d 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
@@ -54,6 +54,8 @@ public class CreateTable extends Expr {
   private boolean ifNotExists;
   @Expose @SerializedName("LikeParentTable")
   private String likeParentTable;
+  @Expose @SerializedName("HasSelfDescSchema")
+  private boolean selfDescSchema = false;
 
   public CreateTable(final String tableName, boolean ifNotExists) {
     super(OpType.CreateTable);
@@ -174,6 +176,18 @@ public class CreateTable extends Expr {
     return likeParentTable;
   }
 
+  public void setHasSelfDescSchema() {
+    selfDescSchema = true;
+  }
+
+  public void unsetHasSelfDescSchema() {
+    selfDescSchema = false;
+  }
+
+  public boolean hasSelfDescSchema() {
+    return selfDescSchema;
+  }
+
 
   @Override
   public int hashCode() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/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 add77f2..ccf084e 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
@@ -18,10 +18,12 @@
 
 package org.apache.tajo.catalog;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
@@ -35,22 +37,22 @@ import java.net.URI;
 
 public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Cloneable {
 	@Expose protected String tableName;                        // required
-  @Expose protected Schema schema;
+  @Expose protected Schema schema;                           // optional for self-describing tables
   @Expose protected TableMeta meta;                          // required
   /** uri is set if external flag is TRUE. */
-  @Expose protected URI uri;                                // optional
+  @Expose protected URI uri;                                 // required
   @Expose	protected TableStats stats;                        // optional
   /** the description of table partition */
   @Expose protected PartitionMethodDesc partitionMethodDesc; // optional
   /** True if it is an external table. False if it is a managed table. */
   @Expose protected Boolean external;                        // optional
 
+  @VisibleForTesting
 	public TableDesc() {
 	}
 
-  public TableDesc(String tableName, Schema schema, TableMeta meta,
-                   URI uri, boolean external) {
-    this();
+  public TableDesc(String tableName, @Nullable Schema schema, TableMeta meta,
+                   @Nullable URI uri, boolean external) {
     this.tableName = tableName;
     this.schema = schema;
     this.meta = meta;
@@ -58,16 +60,17 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
     this.external = external;
   }
 
-	public TableDesc(String tableName, Schema schema, TableMeta meta, URI path) {
+	public TableDesc(String tableName, @Nullable Schema schema, TableMeta meta, @Nullable URI path) {
 		this(tableName, schema, meta, path, true);
 	}
 	
-	public TableDesc(String tableName, Schema schema, String storeType, KeyValueSet options, URI path) {
+	public TableDesc(String tableName, @Nullable Schema schema, String storeType, KeyValueSet options,
+                   @Nullable URI path) {
 	  this(tableName, schema, new TableMeta(storeType, options), path);
 	}
 	
 	public TableDesc(TableDescProto proto) {
-	  this(proto.getTableName(), new Schema(proto.getSchema()),
+	  this(proto.getTableName(), proto.hasSchema() ? new Schema(proto.getSchema()) : null,
         new TableMeta(proto.getMeta()), proto.hasPath() ? URI.create(proto.getPath()) : null, proto.getIsExternal());
     if(proto.hasStats()) {
       this.stats = new TableStats(proto.getStats());
@@ -84,10 +87,6 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
   public String getName() {
     return this.tableName;
   }
-
-  public boolean hasUri() {
-    return this.uri != null;
-  }
 	
 	public void setUri(URI uri) {
 		this.uri = uri;
@@ -108,6 +107,14 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
   public void setSchema(Schema schem) {
     this.schema = schem;
   }
+
+  public boolean hasSchema() {
+    return schema != null;
+  }
+
+  public boolean hasEmptySchema() {
+    return schema.size() == 0;
+  }
 	
   public Schema getSchema() {
     return schema;
@@ -170,7 +177,8 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
       eq = eq && TUtil.checkEquals(uri, other.uri);
       eq = eq && TUtil.checkEquals(partitionMethodDesc, other.partitionMethodDesc);
       eq = eq && TUtil.checkEquals(external, other.external);
-      return eq && TUtil.checkEquals(stats, other.stats);
+      eq = eq && TUtil.checkEquals(stats, other.stats);
+      return eq;
     }
     
     return false;   
@@ -179,7 +187,7 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
 	public Object clone() throws CloneNotSupportedException {
 	  TableDesc desc = (TableDesc) super.clone();
 	  desc.tableName = tableName;
-    desc.schema = (Schema) schema.clone();
+    desc.schema = schema != null ? (Schema) schema.clone() : null;
     desc.meta = (TableMeta) meta.clone();
     desc.uri = uri;
     desc.stats = stats != null ? (TableStats) stats.clone() : null;

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index cec528a..0b7a1b4 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -98,7 +98,7 @@ message TableDescProto {
   required string table_name = 2;
   optional string path = 3;
   required TableProto meta = 4;
-  required SchemaProto schema = 5;
+  optional SchemaProto schema = 5;
   optional TableStatsProto stats = 6;
   optional PartitionMethodProto partition = 7;
   optional bool isExternal = 8 [default = false];

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java
index f2dc3cf..f9b7c62 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java
@@ -19,12 +19,7 @@
 package org.apache.tajo.catalog.dictionary;
 
 import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
 
 abstract class AbstractTableDescriptor implements TableDescriptor {

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/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 c4d1828..d46ab3c 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
@@ -835,7 +835,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       conn.setAutoCommit(false);
 
       String sql = "INSERT INTO TABLES (DB_ID, " + COL_TABLES_NAME +
-          ", TABLE_TYPE, PATH, STORE_TYPE) VALUES(?, ?, ?, ?, ?) ";
+          ", TABLE_TYPE, PATH, STORE_TYPE, HAS_SELF_DESCRIBE_SCHEMA) VALUES(?, ?, ?, ?, ?, ?) ";
 
       if (LOG.isDebugEnabled()) {
         LOG.debug(sql);
@@ -851,6 +851,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       }
       pstmt.setString(4, table.getPath());
       pstmt.setString(5, table.getMeta().getStoreType());
+      pstmt.setBoolean(6, table.getSchema() == null);
       pstmt.executeUpdate();
       pstmt.close();
 
@@ -1672,7 +1673,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       // Geting Table Description
       //////////////////////////////////////////
       String sql =
-          "SELECT TID, " + COL_TABLES_NAME + ", TABLE_TYPE, PATH, STORE_TYPE FROM TABLES " +
+          "SELECT TID, " + COL_TABLES_NAME + ", TABLE_TYPE, PATH, STORE_TYPE, HAS_SELF_DESCRIBE_SCHEMA FROM TABLES " +
               "WHERE DB_ID = ? AND " + COL_TABLES_NAME + "=?";
 
       if (LOG.isDebugEnabled()) {
@@ -1698,34 +1699,37 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
       tableBuilder.setPath(res.getString(4).trim());
       storeType = res.getString(5).trim();
+      boolean hasSelfDescSchema = res.getBoolean(6);
 
       res.close();
       pstmt.close();
 
-      //////////////////////////////////////////
-      // Geting Column Descriptions
-      //////////////////////////////////////////
-      CatalogProtos.SchemaProto.Builder schemaBuilder = CatalogProtos.SchemaProto.newBuilder();
-      sql = "SELECT COLUMN_NAME, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH from " + TB_COLUMNS +
-          " WHERE " + COL_TABLES_PK + " = ? ORDER BY ORDINAL_POSITION ASC";
+      if (!hasSelfDescSchema) {
+        //////////////////////////////////////////
+        // Geting Column Descriptions
+        //////////////////////////////////////////
+        CatalogProtos.SchemaProto.Builder schemaBuilder = CatalogProtos.SchemaProto.newBuilder();
+        sql = "SELECT COLUMN_NAME, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH from " + TB_COLUMNS +
+            " WHERE " + COL_TABLES_PK + " = ? ORDER BY ORDINAL_POSITION ASC";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(sql);
+        }
 
-      pstmt = conn.prepareStatement(sql);
-      pstmt.setInt(1, tableId);
-      res = pstmt.executeQuery();
+        pstmt = conn.prepareStatement(sql);
+        pstmt.setInt(1, tableId);
+        res = pstmt.executeQuery();
 
-      while (res.next()) {
-        schemaBuilder.addFields(resultToColumnProto(res));
-      }
+        while (res.next()) {
+          schemaBuilder.addFields(resultToColumnProto(res));
+        }
 
-      tableBuilder.setSchema(
-          CatalogUtil.getQualfiedSchema(databaseName + "." + tableName, schemaBuilder.build()));
+        tableBuilder.setSchema(
+            CatalogUtil.getQualfiedSchema(databaseName + "." + tableName, schemaBuilder.build()));
 
-      res.close();
-      pstmt.close();
+        res.close();
+        pstmt.close();
+      }
 
       //////////////////////////////////////////
       // Geting Table Properties

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
index c48c078..5280127 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
@@ -19,6 +19,7 @@
 <tns:store xmlns:tns="http://tajo.apache.org/catalogstore" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
       Catalog base version history
+      * 10 - 2015-09-22: Well support for self-describing data formats (TAJO-1832)
       * 9 - 2015-09-12: Allow external catalog store for unit testing (TAJO-1813)
       * 8 - 2015-09-02: Wrong table type problem in catalog (TAJO-1808)
       * 7 - 2015-07-30: Add a column and index for partition keys (TAJO-1346)
@@ -29,7 +30,7 @@
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-	<tns:base version="9">
+	<tns:base version="10">
 		<tns:objects>
 			<tns:Object order="0" type="table" name="META">
 				<tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -67,6 +68,7 @@
   				TABLE_TYPE VARCHAR(128) NOT NULL,
   				PATH VARCHAR(4096),
   				STORE_TYPE CHAR(16),
+  				HAS_SELF_DESCRIBE_SCHEMA BOOLEAN,
   				CONSTRAINT TABLES_PK PRIMARY KEY (TID),
   				CONSTRAINT C_TABLE_ID_UNIQ UNIQUE (DB_ID, TABLE_NAME)
 				)]]>

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
index 0bde634..0a7bfa2 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
@@ -19,6 +19,7 @@
 <tns:store xmlns:tns="http://tajo.apache.org/catalogstore" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
       Catalog base version history
+      * 10 - 2015-09-22: Well support for self-describing data formats (TAJO-1832)
       * 9 - 2015-09-12: Allow external catalog store for unit testing (TAJO-1813)
       * 8 - 2015-09-02: Wrong table type problem in catalog (TAJO-1808)
       * 7 - 2015-07-30: Add a column and index for partition keys (TAJO-1346)
@@ -29,7 +30,7 @@
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-  <tns:base version="9">
+  <tns:base version="10">
     <tns:objects>
       <tns:Object order="0" type="table" name="META">
         <tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -65,6 +66,7 @@
           TABLE_TYPE VARCHAR(128) NOT NULL,
           PATH VARCHAR(4096) BINARY,
           STORE_TYPE CHAR(16),
+          HAS_SELF_DESCRIBE_SCHEMA BOOLEAN,
           FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID),
           INDEX IDX_DB_ID (DB_ID),
           UNIQUE INDEX IDX_TABLE_ID (DB_ID, TABLE_NAME)

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
index 1251693..1113b3e 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
@@ -19,6 +19,7 @@
 <tns:store xmlns:tns="http://tajo.apache.org/catalogstore" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
     Catalog base version history
+    * 10 - 2015-09-22: Well support for self-describing data formats (TAJO-1832)
     * 9 - 2015-09-12: Allow external catalog store for unit testing (TAJO-1813)
     * 8 - 2015-09-02: Wrong table type problem in catalog (TAJO-1808)
     * 7 - 2015-07-30: Add a column and index for partition keys (TAJO-1346)
@@ -29,7 +30,7 @@
     * 2 - 2014-06-09: First versioning
     * 1-  Before 2013-03-20
   -->
-  <tns:base version="9">
+  <tns:base version="10">
     <tns:objects>
       <tns:Object order="0" type="table" name="META">
         <tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -65,6 +66,7 @@
           TABLE_TYPE VARCHAR(128) NOT NULL,
           PATH VARCHAR(4096) BINARY,
           STORE_TYPE CHAR(16),
+          HAS_SELF_DESCRIBE_SCHEMA BOOLEAN,
           FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID),
           INDEX IDX_DB_ID (DB_ID),
           UNIQUE INDEX IDX_TABLE_ID (DB_ID, TABLE_NAME)

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
index 1c09231..45d072e 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
@@ -19,6 +19,7 @@
 <tns:store xmlns:tns="http://tajo.apache.org/catalogstore" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
       Catalog base version history
+      * 10 - 2015-09-22: Well support for self-describing data formats (TAJO-1832)
       * 9 - 2015-09-12: Allow external catalog store for unit testing (TAJO-1813)
       * 8 - 2015-09-02: Wrong table type problem in catalog (TAJO-1808)
       * 7 - 2015-07-30: Add a column and index for partition keys (TAJO-1346)
@@ -29,7 +30,7 @@
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-  <tns:base version="9">
+  <tns:base version="10">
     <tns:objects>
   		<tns:Object order="0" type="table" name="meta">
   			<tns:sql><![CDATA[
@@ -95,6 +96,7 @@
 					TABLE_NAME VARCHAR2(128) NOT NULL,
 					TABLE_TYPE VARCHAR2(128) NOT NULL,
 					PATH VARCHAR2(4000),
+					HAS_SELF_DESCRIBE_SCHEMA CHAR NOT NULL,
 					STORE_TYPE CHAR(16),
 					FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID),
 					CONSTRAINT C_TABLE_ID_UNIQ UNIQUE (DB_ID, TABLE_NAME)

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
index feb6656..231dc20 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
@@ -21,6 +21,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
       Catalog base version history
+      * 10 - 2015-09-22: Well support for self-describing data formats (TAJO-1832)
       * 9 - 2015-09-12: Allow external catalog store for unit testing (TAJO-1813)
       * 8 - 2015-09-02: Wrong table type problem in catalog (TAJO-1808)
       * 7 - 2015-07-30: Add a column and index for partition keys (TAJO-1346)
@@ -32,7 +33,7 @@ xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-	<tns:base version="9">
+	<tns:base version="10">
 		<tns:objects>
 			<tns:Object name="META" type="table" order="0">
 				<tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -72,6 +73,7 @@ xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.
   				TABLE_NAME VARCHAR(128) NOT NULL,
   				TABLE_TYPE VARCHAR(128) NOT NULL,
   				PATH VARCHAR(4096),
+  				HAS_SELF_DESCRIBE_SCHEMA BOOLEAN,
   				STORE_TYPE CHAR(16),
   				FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID),
   				UNIQUE (DB_ID, TABLE_NAME)

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
index 102b39e..22e5cc7 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
@@ -135,12 +135,13 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   }
 
   @Override
-  public TableDesc createExternalTable(String tableName, Schema schema, URI path, TableMeta meta)
+  public TableDesc createExternalTable(String tableName, @Nullable Schema schema, URI path, TableMeta meta)
       throws DuplicateTableException, UnavailableTableLocationException, InsufficientPrivilegeException {
     return createExternalTable(tableName, schema, path, meta, null);
   }
 
-  public TableDesc createExternalTable(final String tableName, final Schema schema, final URI path,
+  @Override
+  public TableDesc createExternalTable(final String tableName, @Nullable final Schema schema, final URI path,
                                        final TableMeta meta, final PartitionMethodDesc partitionMethodDesc)
       throws DuplicateTableException, InsufficientPrivilegeException, UnavailableTableLocationException {
 
@@ -151,7 +152,9 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
     final ClientProtos.CreateTableRequest.Builder builder = ClientProtos.CreateTableRequest.newBuilder();
     builder.setSessionId(conn.sessionId);
     builder.setName(tableName);
-    builder.setSchema(schema.getProto());
+    if (schema != null) {
+      builder.setSchema(schema.getProto());
+    }
     builder.setMeta(meta.getProto());
     builder.setPath(path.toString());
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-client/src/main/proto/ClientProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/proto/ClientProtos.proto b/tajo-client/src/main/proto/ClientProtos.proto
index 13cb04b..5cf9639 100644
--- a/tajo-client/src/main/proto/ClientProtos.proto
+++ b/tajo-client/src/main/proto/ClientProtos.proto
@@ -194,7 +194,7 @@ message GetClusterInfoResponse {
 message CreateTableRequest {
   optional SessionIdProto sessionId = 1;
   required string name = 2;
-  required SchemaProto schema = 3;
+  optional SchemaProto schema = 3;
   required TableProto meta = 4;
   required string path = 5;
   optional PartitionMethodProto partition = 6;

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
index 9abfbd3..2fde31c 100644
--- a/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -915,7 +915,7 @@ public class QueryTestCaseBase {
     return resultPath;
   }
 
-  private Path getDataSetFile(String fileName) throws IOException {
+  protected Path getDataSetFile(String fileName) throws IOException {
     Path dataFilePath = StorageUtil.concatPath(currentDatasetPath, fileName);
     FileSystem fs = currentDatasetPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
     if (!fs.exists(dataFilePath)) {
@@ -923,10 +923,10 @@ public class QueryTestCaseBase {
         dataFilePath = StorageUtil.concatPath(namedDatasetPath, fileName);
         fs = namedDatasetPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
         if (!fs.exists(dataFilePath)) {
-          throw new IOException("Cannot find " + fileName + " at " + currentQueryPath + " and " + namedQueryPath);
+          throw new IOException("Cannot find " + fileName + " at " + currentDatasetPath);
         }
       } else {
-        throw new IOException("Cannot find " + fileName + " at " + currentQueryPath + " and " + namedQueryPath);
+        throw new IOException("Cannot find " + fileName + " at " + currentDatasetPath);
       }
     }
     return dataFilePath;

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index cfb1ed8..38d02aa 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -34,13 +34,14 @@ import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Int4Datum;
 import org.apache.tajo.datum.TextDatum;
 import org.apache.tajo.engine.function.FunctionLoader;
 import org.apache.tajo.engine.function.builtin.SumInt;
 import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.parser.sql.SQLAnalyzer;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.parser.sql.SQLAnalyzer;
 import org.apache.tajo.plan.LogicalOptimizer;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
@@ -1357,4 +1358,124 @@ public class TestLogicalPlanner {
     assertEquals(alterTableNode.getPartitionValues()[1], "01");
     assertEquals(alterTableNode.getPartitionValues()[2], "11");
   }
+
+  String[] SELF_DESC = {
+      "select id, name, dept from default.self_desc_table1", // 0
+      "select name, dept from default.self_desc_table1 where id > 10",
+  };
+
+  @Test
+  public void testSelectFromSelfDescTable() throws Exception {
+    TableDesc tableDesc = new TableDesc("default.self_desc_table1", null, CatalogUtil.newTableMeta("TEXT"),
+        CommonTestingUtil.getTestDir().toUri(), true);
+    catalog.createTable(tableDesc);
+    assertTrue(catalog.existsTable("default.self_desc_table1"));
+    tableDesc = catalog.getTableDesc("default.self_desc_table1");
+    assertTrue(tableDesc.hasEmptySchema());
+
+    QueryContext context = createQueryContext();
+    Expr expr = sqlAnalyzer.parse(SELF_DESC[0]);
+    LogicalPlan logicalPlan = planner.createPlan(context, expr);
+
+    LogicalNode node = logicalPlan.getRootNode();
+    assertEquals(NodeType.ROOT, node.getType());
+    LogicalRootNode root = (LogicalRootNode) node;
+    testJsonSerDerObject(root);
+    testCloneLogicalNode(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projectionNode = root.getChild();
+    testJsonSerDerObject(projectionNode);
+    testCloneLogicalNode(projectionNode);
+
+    // projection column test
+    Target[] targets = projectionNode.getTargets();
+    Arrays.sort(targets, new Comparator<Target>() {
+      @Override
+      public int compare(Target o1, Target o2) {
+        return o1.getCanonicalName().compareTo(o2.getCanonicalName());
+      }
+    });
+    assertEquals(3, targets.length);
+    assertEquals("default.self_desc_table1.dept", targets[0].getCanonicalName());
+    assertEquals("default.self_desc_table1.id", targets[1].getCanonicalName());
+    assertEquals("default.self_desc_table1.name", targets[2].getCanonicalName());
+
+    // scan column test
+    assertEquals(NodeType.SCAN, projectionNode.getChild().getType());
+    ScanNode scanNode = projectionNode.getChild();
+    targets = scanNode.getTargets();
+    Arrays.sort(targets, new Comparator<Target>() {
+      @Override
+      public int compare(Target o1, Target o2) {
+        return o1.getCanonicalName().compareTo(o2.getCanonicalName());
+      }
+    });
+    assertEquals(3, targets.length);
+    assertEquals("default.self_desc_table1.dept", targets[0].getCanonicalName());
+    assertEquals("default.self_desc_table1.id", targets[1].getCanonicalName());
+    assertEquals("default.self_desc_table1.name", targets[2].getCanonicalName());
+
+    catalog.dropTable("default.self_desc_table1");
+  }
+
+  @Test
+  public void testSelectWhereFromSelfDescTable() throws Exception {
+    TableDesc tableDesc = new TableDesc("default.self_desc_table1", null, CatalogUtil.newTableMeta("TEXT"),
+        CommonTestingUtil.getTestDir().toUri(), true);
+    catalog.createTable(tableDesc);
+    assertTrue(catalog.existsTable("default.self_desc_table1"));
+    tableDesc = catalog.getTableDesc("default.self_desc_table1");
+    assertTrue(tableDesc.hasEmptySchema());
+
+    QueryContext context = createQueryContext();
+    Expr expr = sqlAnalyzer.parse(SELF_DESC[1]);
+    LogicalPlan logicalPlan = planner.createPlan(context, expr);
+
+    LogicalNode node = logicalPlan.getRootNode();
+    assertEquals(NodeType.ROOT, node.getType());
+    LogicalRootNode root = (LogicalRootNode) node;
+    testJsonSerDerObject(root);
+    testCloneLogicalNode(root);
+
+    assertEquals(NodeType.PROJECTION, root.getChild().getType());
+    ProjectionNode projectionNode = root.getChild();
+    testJsonSerDerObject(projectionNode);
+    testCloneLogicalNode(projectionNode);
+
+    // projection column test
+    Target[] targets = projectionNode.getTargets();
+    Arrays.sort(targets, new Comparator<Target>() {
+      @Override
+      public int compare(Target o1, Target o2) {
+        return o1.getCanonicalName().compareTo(o2.getCanonicalName());
+      }
+    });
+    assertEquals(2, targets.length);
+    assertEquals("default.self_desc_table1.dept", targets[0].getCanonicalName());
+    assertEquals("default.self_desc_table1.name", targets[1].getCanonicalName());
+
+    assertEquals(NodeType.SELECTION, projectionNode.getChild().getType());
+    SelectionNode selectionNode = projectionNode.getChild();
+    assertEquals(new BinaryEval(EvalType.GTH, new FieldEval("default.self_desc_table1.id", CatalogUtil.newSimpleDataType(Type.TEXT)), new ConstEval(new Int4Datum(10))),
+        selectionNode.getQual());
+
+    // scan column test
+    assertEquals(NodeType.SCAN, selectionNode.getChild().getType());
+    ScanNode scanNode = selectionNode.getChild();
+    targets = scanNode.getTargets();
+    Arrays.sort(targets, new Comparator<Target>() {
+      @Override
+      public int compare(Target o1, Target o2) {
+        return o1.getCanonicalName().compareTo(o2.getCanonicalName());
+      }
+    });
+    assertEquals(4, targets.length);
+    assertEquals("?greaterthan", targets[0].getCanonicalName());
+    assertEquals("default.self_desc_table1.dept", targets[1].getCanonicalName());
+    assertEquals("default.self_desc_table1.id", targets[2].getCanonicalName());
+    assertEquals("default.self_desc_table1.name", targets[3].getCanonicalName());
+
+    catalog.dropTable("default.self_desc_table1");
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
index e33f173..f2f54ba 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
@@ -649,4 +649,35 @@ public class TestCreateTable extends QueryTestCaseBase {
     executeString("DROP TABLE D9.nested_table2");
     executeString("DROP DATABASE D9").close();
   }
+
+  @Test
+  public final void testSelfDescTable1() throws Exception {
+    executeString("create database d9;").close();
+
+    assertTableNotExists("d9.schemaless");
+    executeQuery();
+    assertTableExists("d9.schemaless");
+    TableDesc desc = getClient().getTableDesc("d9.schemaless");
+    assertTrue(desc.hasEmptySchema());
+
+    executeString("drop table d9.schemaless").close();
+    executeString("drop database d9").close();
+  }
+
+  @Test
+  public final void testSelfDescTable2() throws Exception {
+    executeString("create database d10;").close();
+
+    String className = getClass().getSimpleName();
+    Path currentDatasetPath = new Path(datasetBasePath, className);
+    Path filePath = StorageUtil.concatPath(currentDatasetPath, "table1");
+    String sql = "create external table d10.schemaless (*) using json with ('compression.codec'='none') location '" + filePath.toString() + "'";
+    executeString(sql).close();
+    assertTableExists("d10.schemaless");
+    TableDesc desc = getClient().getTableDesc("d10.schemaless");
+    assertTrue(desc.hasEmptySchema());
+
+    executeString("drop table d10.schemaless").close();
+    executeString("drop database d10").close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestQueryOnSelfDescTable.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestQueryOnSelfDescTable.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestQueryOnSelfDescTable.java
new file mode 100644
index 0000000..9ce46bc
--- /dev/null
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestQueryOnSelfDescTable.java
@@ -0,0 +1,184 @@
+/**
+ * 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.engine.query;
+
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.exception.AmbiguousColumnException;
+import org.apache.tajo.exception.TajoException;
+import org.junit.After;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.sql.SQLException;
+
+public class TestQueryOnSelfDescTable extends QueryTestCaseBase {
+
+  public TestQueryOnSelfDescTable() throws IOException, TajoException, SQLException {
+    super();
+
+    executeString(String.format("create external table if not exists self_desc_table1 (*) using json location '%s'",
+        getDataSetFile("sample1"))).close();
+
+    executeString(String.format("create external table if not exists self_desc_table2 (*) using json location '%s'",
+        getDataSetFile("sample2"))).close();
+
+    executeString(String.format("create external table if not exists self_desc_table3 (*) using json location '%s'",
+        getDataSetFile("tweets"))).close();
+
+    executeString(String.format("create external table if not exists github (*) using json location '%s'",
+        getDataSetFile("github"))).close();
+  }
+
+  @After
+  public void teardown() throws TajoException, SQLException {
+    executeString("drop table if exists self_desc_table1").close();
+    executeString("drop table if exists self_desc_table2").close();
+    executeString("drop table if exists self_desc_table3").close();
+    executeString("drop table if exists github").close();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testSelect() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testSelect2() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testGroupby() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testGroupby2() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testGroupby3() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @SimpleTest
+  public final void testSort() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testCrossJoin() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testJoinWithSchemaFullTable() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testJoinWithSchemaFullTable2() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test(expected = AmbiguousColumnException.class)
+  public final void testJoinWithSchemaFullTable3() throws Exception {
+    executeString("" +
+        "select " +
+        "  user.favourites_count::int8, " +
+        "  l_linenumber, " +
+        "  l_comment " +
+        "from " +
+        "  default.lineitem, " +
+        "  self_desc_table1, " +
+        "  self_desc_table3, " +
+        "  default.orders, " +
+        "  default.supplier " +
+        "where " +
+        "  user.favourites_count::int8 = (l_orderkey - 1) and " +
+        "  l_orderkey = o_orderkey and " +
+        "  l_linenumber = s_suppkey and " +
+        "  self_desc_table3.user.favourites_count = self_desc_table1.name.first_name");
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testJoinWithSchemaFullTable4() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test(expected = AmbiguousColumnException.class)
+  public final void testJoinOfSelfDescTables() throws Exception {
+    executeString("" +
+        "select " +
+        "  user.favourites_count::int8 " +
+        "from " +
+        "  self_desc_table1, " +
+        "  self_desc_table3 " +
+        "where " +
+        "  user.favourites_count = name.first_name");
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest
+  public final void testJoinOfSelfDescTablesWithQualifiedColumns() throws Exception {
+    runSimpleTests();
+  }
+
+  @Test(expected = AmbiguousColumnException.class)
+  public final void testJoinWithSingleQualifiedColumn() throws Exception {
+    executeString("" +
+        "select " +
+        "  self_desc_table3.user.favourites_count::int8 " +
+        "from " +
+        "  self_desc_table1, " +
+        "  self_desc_table3 " +
+        "where " +
+        "  self_desc_table3.user.favourites_count = name.first_name");
+  }
+
+  @Test
+  @Option(sort = true)
+  @SimpleTest(queries = {
+      @QuerySpec("select * from default.lineitem where l_orderkey in (select user.favourites_count::int8 + 1 from self_desc_table3)")
+  })
+  public final void testInSubquery() throws Exception {
+    runSimpleTests();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
index 5d1321a..ce22782 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestUnionQuery.java
@@ -692,7 +692,7 @@ public class TestUnionQuery extends QueryTestCaseBase {
   }
 
   @Test
-  @Option(withExplain =  true)
+  @Option(withExplain =  true, sort = true)
   @SimpleTest
   public void testUnionAndFilter() throws Exception {
     runSimpleTests();

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/dataset/TestCreateTable/json_table/table1.json
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestCreateTable/json_table/table1.json b/tajo-core-tests/src/test/resources/dataset/TestCreateTable/json_table/table1.json
new file mode 100644
index 0000000..0b597b2
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/dataset/TestCreateTable/json_table/table1.json
@@ -0,0 +1,4 @@
+{"id":"2937257753","type":"PushEvent","actor":{"id":5266949,"login":"hardrubic","gravatar_id":"","url":"https://api.github.com/users/hardrubic","avatar_url":"https://avatars.githubusercontent.com/u/5266949?"},"repo":{"id":38299397,"name":"hardrubic/rxJavaTest","url":"https://api.github.com/repos/hardrubic/rxJavaTest"},"payload":{"push_id":712081726,"size":1,"distinct_size":1,"ref":"refs/heads/master","head":"ea79d7a424f2693b70b9496726f315a5711b6fe7","before":"613f05557ad353f4bedc6df54128f8091ed1f1e9","commits":[{"sha":"ea79d7a424f2693b70b9496726f315a5711b6fe7","author":{"email":"dgzx106@163.com","name":"hardrubic"},"message":"增加rxJava例子","distinct":true,"url":"https://api.github.com/repos/hardrubic/rxJavaTest/commits/ea79d7a424f2693b70b9496726f315a5711b6fe7"}]},"public":true,"created_at":"2015-07-01T00:00:01Z"}
+{"id":"2937257758","type":"WatchEvent","actor":{"id":11455393,"login":"chrischjh","gravatar_id":"","url":"https://api.github.com/users/chrischjh","avatar_url":"https://avatars.githubusercontent.com/u/11455393?"},"repo":{"id":18218031,"name":"dead-horse/co-and-koa-talk","url":"https://api.github.com/repos/dead-horse/co-and-koa-talk"},"payload":{"action":"started"},"public":true,"created_at":"2015-07-01T00:00:01Z"}
+{"id":"2937257759","type":"CreateEvent","actor":{"id":206379,"login":"gvn","gravatar_id":"","url":"https://api.github.com/users/gvn","avatar_url":"https://avatars.githubusercontent.com/u/206379?"},"repo":{"id":24345476,"name":"gvn/webmaker-android","url":"https://api.github.com/repos/gvn/webmaker-android"},"payload":{"ref":"use-self-building","ref_type":"branch","master_branch":"master","description":"Webmaker for Firefox OS & Android","pusher_type":"user"},"public":true,"created_at":"2015-07-01T00:00:01Z"}
+{"id":"2937257761","type":"ForkEvent","actor":{"id":1088854,"login":"CAOakleyII","gravatar_id":"","url":"https://api.github.com/users/CAOakleyII","avatar_url":"https://avatars.githubusercontent.com/u/1088854?"},"repo":{"id":11909954,"name":"skycocker/chromebrew","url":"https://api.github.com/repos/skycocker/chromebrew"},"payload":{"forkee":{"id":38339291,"name":"chromebrew","full_name":"CAOakleyII/chromebrew","owner":{"login":"CAOakleyII","id":1088854,"avatar_url":"https://avatars.githubusercontent.com/u/1088854?v=3","gravatar_id":"","url":"https://api.github.com/users/CAOakleyII","html_url":"https://github.com/CAOakleyII","followers_url":"https://api.github.com/users/CAOakleyII/followers","following_url":"https://api.github.com/users/CAOakleyII/following{/other_user}","gists_url":"https://api.github.com/users/CAOakleyII/gists{/gist_id}","starred_url":"https://api.github.com/users/CAOakleyII/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/CAOakleyII/subscri
 ptions","organizations_url":"https://api.github.com/users/CAOakleyII/orgs","repos_url":"https://api.github.com/users/CAOakleyII/repos","events_url":"https://api.github.com/users/CAOakleyII/events{/privacy}","received_events_url":"https://api.github.com/users/CAOakleyII/received_events","type":"User","site_admin":false},"private":false,"html_url":"https://github.com/CAOakleyII/chromebrew","description":"Package manager for Chrome OS","fork":true,"url":"https://api.github.com/repos/CAOakleyII/chromebrew","forks_url":"https://api.github.com/repos/CAOakleyII/chromebrew/forks","keys_url":"https://api.github.com/repos/CAOakleyII/chromebrew/keys{/key_id}","collaborators_url":"https://api.github.com/repos/CAOakleyII/chromebrew/collaborators{/collaborator}","teams_url":"https://api.github.com/repos/CAOakleyII/chromebrew/teams","hooks_url":"https://api.github.com/repos/CAOakleyII/chromebrew/hooks","issue_events_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues/events{/number}","
 events_url":"https://api.github.com/repos/CAOakleyII/chromebrew/events","assignees_url":"https://api.github.com/repos/CAOakleyII/chromebrew/assignees{/user}","branches_url":"https://api.github.com/repos/CAOakleyII/chromebrew/branches{/branch}","tags_url":"https://api.github.com/repos/CAOakleyII/chromebrew/tags","blobs_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/blobs{/sha}","git_tags_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/tags{/sha}","git_refs_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/refs{/sha}","trees_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/trees{/sha}","statuses_url":"https://api.github.com/repos/CAOakleyII/chromebrew/statuses/{sha}","languages_url":"https://api.github.com/repos/CAOakleyII/chromebrew/languages","stargazers_url":"https://api.github.com/repos/CAOakleyII/chromebrew/stargazers","contributors_url":"https://api.github.com/repos/CAOakleyII/chromebrew/contributors","subscribers_url":"https://a
 pi.github.com/repos/CAOakleyII/chromebrew/subscribers","subscription_url":"https://api.github.com/repos/CAOakleyII/chromebrew/subscription","commits_url":"https://api.github.com/repos/CAOakleyII/chromebrew/commits{/sha}","git_commits_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/commits{/sha}","comments_url":"https://api.github.com/repos/CAOakleyII/chromebrew/comments{/number}","issue_comment_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues/comments{/number}","contents_url":"https://api.github.com/repos/CAOakleyII/chromebrew/contents/{+path}","compare_url":"https://api.github.com/repos/CAOakleyII/chromebrew/compare/{base}...{head}","merges_url":"https://api.github.com/repos/CAOakleyII/chromebrew/merges","archive_url":"https://api.github.com/repos/CAOakleyII/chromebrew/{archive_format}{/ref}","downloads_url":"https://api.github.com/repos/CAOakleyII/chromebrew/downloads","issues_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues{/number}","pu
 lls_url":"https://api.github.com/repos/CAOakleyII/chromebrew/pulls{/number}","milestones_url":"https://api.github.com/repos/CAOakleyII/chromebrew/milestones{/number}","notifications_url":"https://api.github.com/repos/CAOakleyII/chromebrew/notifications{?since,all,participating}","labels_url":"https://api.github.com/repos/CAOakleyII/chromebrew/labels{/name}","releases_url":"https://api.github.com/repos/CAOakleyII/chromebrew/releases{/id}","created_at":"2015-07-01T00:00:00Z","updated_at":"2015-06-28T10:11:09Z","pushed_at":"2015-06-09T07:46:57Z","git_url":"git://github.com/CAOakleyII/chromebrew.git","ssh_url":"git@github.com:CAOakleyII/chromebrew.git","clone_url":"https://github.com/CAOakleyII/chromebrew.git","svn_url":"https://github.com/CAOakleyII/chromebrew","homepage":"http://skycocker.github.io/chromebrew/","size":846,"stargazers_count":0,"watchers_count":0,"language":null,"has_issues":false,"has_downloads":true,"has_wiki":true,"has_pages":false,"forks_count":0,"mirror_url":null,"
 open_issues_count":0,"forks":0,"open_issues":0,"watchers":0,"default_branch":"master","public":true}},"public":true,"created_at":"2015-07-01T00:00:01Z"}

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/github/github.json
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/github/github.json b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/github/github.json
new file mode 100644
index 0000000..0b597b2
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/github/github.json
@@ -0,0 +1,4 @@
+{"id":"2937257753","type":"PushEvent","actor":{"id":5266949,"login":"hardrubic","gravatar_id":"","url":"https://api.github.com/users/hardrubic","avatar_url":"https://avatars.githubusercontent.com/u/5266949?"},"repo":{"id":38299397,"name":"hardrubic/rxJavaTest","url":"https://api.github.com/repos/hardrubic/rxJavaTest"},"payload":{"push_id":712081726,"size":1,"distinct_size":1,"ref":"refs/heads/master","head":"ea79d7a424f2693b70b9496726f315a5711b6fe7","before":"613f05557ad353f4bedc6df54128f8091ed1f1e9","commits":[{"sha":"ea79d7a424f2693b70b9496726f315a5711b6fe7","author":{"email":"dgzx106@163.com","name":"hardrubic"},"message":"增加rxJava例子","distinct":true,"url":"https://api.github.com/repos/hardrubic/rxJavaTest/commits/ea79d7a424f2693b70b9496726f315a5711b6fe7"}]},"public":true,"created_at":"2015-07-01T00:00:01Z"}
+{"id":"2937257758","type":"WatchEvent","actor":{"id":11455393,"login":"chrischjh","gravatar_id":"","url":"https://api.github.com/users/chrischjh","avatar_url":"https://avatars.githubusercontent.com/u/11455393?"},"repo":{"id":18218031,"name":"dead-horse/co-and-koa-talk","url":"https://api.github.com/repos/dead-horse/co-and-koa-talk"},"payload":{"action":"started"},"public":true,"created_at":"2015-07-01T00:00:01Z"}
+{"id":"2937257759","type":"CreateEvent","actor":{"id":206379,"login":"gvn","gravatar_id":"","url":"https://api.github.com/users/gvn","avatar_url":"https://avatars.githubusercontent.com/u/206379?"},"repo":{"id":24345476,"name":"gvn/webmaker-android","url":"https://api.github.com/repos/gvn/webmaker-android"},"payload":{"ref":"use-self-building","ref_type":"branch","master_branch":"master","description":"Webmaker for Firefox OS & Android","pusher_type":"user"},"public":true,"created_at":"2015-07-01T00:00:01Z"}
+{"id":"2937257761","type":"ForkEvent","actor":{"id":1088854,"login":"CAOakleyII","gravatar_id":"","url":"https://api.github.com/users/CAOakleyII","avatar_url":"https://avatars.githubusercontent.com/u/1088854?"},"repo":{"id":11909954,"name":"skycocker/chromebrew","url":"https://api.github.com/repos/skycocker/chromebrew"},"payload":{"forkee":{"id":38339291,"name":"chromebrew","full_name":"CAOakleyII/chromebrew","owner":{"login":"CAOakleyII","id":1088854,"avatar_url":"https://avatars.githubusercontent.com/u/1088854?v=3","gravatar_id":"","url":"https://api.github.com/users/CAOakleyII","html_url":"https://github.com/CAOakleyII","followers_url":"https://api.github.com/users/CAOakleyII/followers","following_url":"https://api.github.com/users/CAOakleyII/following{/other_user}","gists_url":"https://api.github.com/users/CAOakleyII/gists{/gist_id}","starred_url":"https://api.github.com/users/CAOakleyII/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/CAOakleyII/subscri
 ptions","organizations_url":"https://api.github.com/users/CAOakleyII/orgs","repos_url":"https://api.github.com/users/CAOakleyII/repos","events_url":"https://api.github.com/users/CAOakleyII/events{/privacy}","received_events_url":"https://api.github.com/users/CAOakleyII/received_events","type":"User","site_admin":false},"private":false,"html_url":"https://github.com/CAOakleyII/chromebrew","description":"Package manager for Chrome OS","fork":true,"url":"https://api.github.com/repos/CAOakleyII/chromebrew","forks_url":"https://api.github.com/repos/CAOakleyII/chromebrew/forks","keys_url":"https://api.github.com/repos/CAOakleyII/chromebrew/keys{/key_id}","collaborators_url":"https://api.github.com/repos/CAOakleyII/chromebrew/collaborators{/collaborator}","teams_url":"https://api.github.com/repos/CAOakleyII/chromebrew/teams","hooks_url":"https://api.github.com/repos/CAOakleyII/chromebrew/hooks","issue_events_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues/events{/number}","
 events_url":"https://api.github.com/repos/CAOakleyII/chromebrew/events","assignees_url":"https://api.github.com/repos/CAOakleyII/chromebrew/assignees{/user}","branches_url":"https://api.github.com/repos/CAOakleyII/chromebrew/branches{/branch}","tags_url":"https://api.github.com/repos/CAOakleyII/chromebrew/tags","blobs_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/blobs{/sha}","git_tags_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/tags{/sha}","git_refs_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/refs{/sha}","trees_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/trees{/sha}","statuses_url":"https://api.github.com/repos/CAOakleyII/chromebrew/statuses/{sha}","languages_url":"https://api.github.com/repos/CAOakleyII/chromebrew/languages","stargazers_url":"https://api.github.com/repos/CAOakleyII/chromebrew/stargazers","contributors_url":"https://api.github.com/repos/CAOakleyII/chromebrew/contributors","subscribers_url":"https://a
 pi.github.com/repos/CAOakleyII/chromebrew/subscribers","subscription_url":"https://api.github.com/repos/CAOakleyII/chromebrew/subscription","commits_url":"https://api.github.com/repos/CAOakleyII/chromebrew/commits{/sha}","git_commits_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/commits{/sha}","comments_url":"https://api.github.com/repos/CAOakleyII/chromebrew/comments{/number}","issue_comment_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues/comments{/number}","contents_url":"https://api.github.com/repos/CAOakleyII/chromebrew/contents/{+path}","compare_url":"https://api.github.com/repos/CAOakleyII/chromebrew/compare/{base}...{head}","merges_url":"https://api.github.com/repos/CAOakleyII/chromebrew/merges","archive_url":"https://api.github.com/repos/CAOakleyII/chromebrew/{archive_format}{/ref}","downloads_url":"https://api.github.com/repos/CAOakleyII/chromebrew/downloads","issues_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues{/number}","pu
 lls_url":"https://api.github.com/repos/CAOakleyII/chromebrew/pulls{/number}","milestones_url":"https://api.github.com/repos/CAOakleyII/chromebrew/milestones{/number}","notifications_url":"https://api.github.com/repos/CAOakleyII/chromebrew/notifications{?since,all,participating}","labels_url":"https://api.github.com/repos/CAOakleyII/chromebrew/labels{/name}","releases_url":"https://api.github.com/repos/CAOakleyII/chromebrew/releases{/id}","created_at":"2015-07-01T00:00:00Z","updated_at":"2015-06-28T10:11:09Z","pushed_at":"2015-06-09T07:46:57Z","git_url":"git://github.com/CAOakleyII/chromebrew.git","ssh_url":"git@github.com:CAOakleyII/chromebrew.git","clone_url":"https://github.com/CAOakleyII/chromebrew.git","svn_url":"https://github.com/CAOakleyII/chromebrew","homepage":"http://skycocker.github.io/chromebrew/","size":846,"stargazers_count":0,"watchers_count":0,"language":null,"has_issues":false,"has_downloads":true,"has_wiki":true,"has_pages":false,"forks_count":0,"mirror_url":null,"
 open_issues_count":0,"forks":0,"open_issues":0,"watchers":0,"default_branch":"master","public":true}},"public":true,"created_at":"2015-07-01T00:00:01Z"}

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample1/table.json
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample1/table.json b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample1/table.json
new file mode 100644
index 0000000..db3ad6c
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample1/table.json
@@ -0,0 +1,3 @@
+{ "title" : "Hand of the King", "name" : { "first_name": "Eddard", "last_name": "Stark"}}
+{ "title" : "Assassin", "name" : { "first_name": "Arya", "last_name": "Stark"}}
+{ "title" : "Dancing Master", "name" : { "first_name": "Syrio", "last_name": "Forel"}}

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample2/sample2.json
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample2/sample2.json b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample2/sample2.json
new file mode 100644
index 0000000..20bab62
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/sample2/sample2.json
@@ -0,0 +1 @@
+{ "glossary": { "title": "example glossary", "GlossDiv": { "title": "S", "GlossList": { "GlossEntry": { "ID": "SGML", "SortAs": "SGML", "GlossTerm": "Standard Generalized Markup Language", "Acronym": "SGML", "Abbrev": "ISO 8879:1986", "GlossDef": { "para": "A meta-markup language, used to create markup languages such as DocBook.", "GlossSeeAlso": ["GML", "XML"] }, "GlossSee": "markup" } } } } }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/tweets/sample1.json
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/tweets/sample1.json b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/tweets/sample1.json
new file mode 100644
index 0000000..78a9071
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/dataset/TestQueryOnSelfDescTable/tweets/sample1.json
@@ -0,0 +1,4 @@
+{"coordinates":null,"favorited":false,"truncated":false,"created_at":"Mon Sep 24 03:35:21 +0000 2012","id_str":"250075927172759552","entities":{"urls":[],"hashtags":[{"text":"freebandnames","indices":[20,34]}],"user_mentions":[]},"in_reply_to_user_id_str":null,"contributors":null,"text":"Aggressive Ponytail #freebandnames","metadata":{"iso_language_code":"en","result_type":"recent"},"retweet_count":1,"in_reply_to_status_id_str":null,"id":250075927172759552,"geo":null,"retweeted":false,"in_reply_to_user_id":null,"place":null,"user":{"profile_sidebar_fill_color":"DDEEF6","profile_sidebar_border_color":"C0DEED","profile_background_tile":false,"name":"Sean Cummings","profile_image_url":"http://a0.twimg.com/profile_images/2359746665/1v6zfgqo8g0d3mk7ii5s_normal.jpeg","created_at":"Mon Apr 26 06:01:55 +0000 2010","location":"LA, CA","follow_request_sent":null,"profile_link_color":"0084B4","is_translator":false,"id_str":"137238150","entities":{"url":{"urls":[{"expanded_url":null,"url":"","i
 ndices":[0,0]}]},"description":{"urls":[]}},"default_profile":true,"contributors_enabled":false,"favourites_count":0,"url":null,"profile_image_url_https":"https://si0.twimg.com/profile_images/2359746665/1v6zfgqo8g0d3mk7ii5s_normal.jpeg","utc_offset":-28800,"id":137238150,"profile_use_background_image":true,"listed_count":2,"profile_text_color":"333333","lang":"en","followers_count":70,"protected":false,"notifications":null,"profile_background_image_url_https":"https://si0.twimg.com/images/themes/theme1/bg.png","profile_background_color":"C0DEED","verified":false,"geo_enabled":true,"time_zone":"Pacific Time (US & Canada)","description":"Born 330 Live 310","default_profile_image":false,"profile_background_image_url":"http://a0.twimg.com/images/themes/theme1/bg.png","statuses_count":579,"friends_count":110,"following":null,"show_all_inline_media":false,"screen_name":"sean_cummings"},"in_reply_to_screen_name":null,"source":"<a>Twitter for Mac<\/a>","in_reply_to_status_id":null}
+{"coordinates":null,"favorited":false,"truncated":false,"created_at":"Fri Sep 21 23:40:54 +0000 2012","id_str":"249292149810667520","entities":{"urls":[],"hashtags":[{"text":"FreeBandNames","indices":[20,34]}],"user_mentions":[]},"in_reply_to_user_id_str":null,"contributors":null,"text":"Thee Namaste Nerdz. #FreeBandNames","metadata":{"iso_language_code":"pl","result_type":"recent"},"retweet_count":2,"in_reply_to_status_id_str":null,"id":249292149810667520,"geo":null,"retweeted":false,"in_reply_to_user_id":null,"place":null,"user":{"profile_sidebar_fill_color":"DDFFCC","profile_sidebar_border_color":"BDDCAD","profile_background_tile":true,"name":"Chaz Martenstein","profile_image_url":"http://a0.twimg.com/profile_images/447958234/Lichtenstein_normal.jpg","created_at":"Tue Apr 07 19:05:07 +0000 2009","location":"Durham, NC","follow_request_sent":null,"profile_link_color":"0084B4","is_translator":false,"id_str":"29516238","entities":{"url":{"urls":[{"expanded_url":null,"url":"http://bu
 llcityrecords.com/wnng/","indices":[0,32]}]},"description":{"urls":[]}},"default_profile":false,"contributors_enabled":false,"favourites_count":8,"url":"http://bullcityrecords.com/wnng/","profile_image_url_https":"https://si0.twimg.com/profile_images/447958234/Lichtenstein_normal.jpg","utc_offset":-18000,"id":29516238,"profile_use_background_image":true,"listed_count":118,"profile_text_color":"333333","lang":"en","followers_count":2052,"protected":false,"notifications":null,"profile_background_image_url_https":"https://si0.twimg.com/profile_background_images/9423277/background_tile.bmp","profile_background_color":"9AE4E8","verified":false,"geo_enabled":false,"time_zone":"Eastern Time (US & Canada)","description":"You will come to Durham, North Carolina. I will sell you some records then, here in Durham, North Carolina. Fun will happen.","default_profile_image":false,"profile_background_image_url":"http://a0.twimg.com/profile_background_images/9423277/background_tile.bmp","statuses_c
 ount":7579,"friends_count":348,"following":null,"show_all_inline_media":true,"screen_name":"bullcityrecords"},"in_reply_to_screen_name":null,"source":"web","in_reply_to_status_id":null}
+{"coordinates":null,"favorited":false,"truncated":false,"created_at":"Fri Sep 21 23:30:20 +0000 2012","id_str":"249289491129438208","entities":{"urls":[],"hashtags":[{"text":"freebandnames","indices":[29,43]}],"user_mentions":[]},"in_reply_to_user_id_str":null,"contributors":null,"text":"Mexican Heaven, Mexican Hell #freebandnames","metadata":{"iso_language_code":"en","result_type":"recent"},"retweet_count":3,"in_reply_to_status_id_str":null,"id":249289491129438208,"geo":null,"retweeted":false,"in_reply_to_user_id":null,"place":null,"user":{"profile_sidebar_fill_color":"99CC33","profile_sidebar_border_color":"829D5E","profile_background_tile":false,"name":"Thomas John Wakeman","profile_image_url":"http://a0.twimg.com/profile_images/2219333930/Froggystyle_normal.png","created_at":"Tue Sep 01 21:21:35 +0000 2009","location":"Kingston New York","follow_request_sent":null,"profile_link_color":"D02B55","is_translator":false,"id_str":"70789458","entities":{"url":{"urls":[{"expanded_url":n
 ull,"url":"","indices":[0,0]}]},"description":{"urls":[]}},"default_profile":false,"contributors_enabled":false,"favourites_count":19,"url":null,"profile_image_url_https":"https://si0.twimg.com/profile_images/2219333930/Froggystyle_normal.png","utc_offset":-18000,"id":70789458,"profile_use_background_image":true,"listed_count":1,"profile_text_color":"3E4415","lang":"en","followers_count":63,"protected":false,"notifications":null,"profile_background_image_url_https":"https://si0.twimg.com/images/themes/theme5/bg.gif","profile_background_color":"352726","verified":false,"geo_enabled":false,"time_zone":"Eastern Time (US & Canada)","description":"Science Fiction Writer, sort of. Likes Superheroes, Mole People, Alt. Timelines.","default_profile_image":false,"profile_background_image_url":"http://a0.twimg.com/images/themes/theme5/bg.gif","statuses_count":1048,"friends_count":63,"following":null,"show_all_inline_media":false,"screen_name":"MonkiesFist"},"in_reply_to_screen_name":null,"sour
 ce":"web","in_reply_to_status_id":null}
+{"coordinates":null,"favorited":false,"truncated":false,"created_at":"Fri Sep 21 22:51:18 +0000 2012","id_str":"249279667666817024","entities":{"urls":[],"hashtags":[{"text":"freebandnames","indices":[20,34]}],"user_mentions":[]},"in_reply_to_user_id_str":null,"contributors":null,"text":"The Foolish Mortals #freebandnames","metadata":{"iso_language_code":"en","result_type":"recent"},"retweet_count":4,"in_reply_to_status_id_str":null,"id":249279667666817024,"geo":null,"retweeted":false,"in_reply_to_user_id":null,"place":null,"user":{"profile_sidebar_fill_color":"BFAC83","profile_sidebar_border_color":"615A44","profile_background_tile":true,"name":"Marty Elmer","profile_image_url":"http://a0.twimg.com/profile_images/1629790393/shrinker_2000_trans_normal.png","created_at":"Mon May 04 00:05:00 +0000 2009","location":"Wisconsin, USA","follow_request_sent":null,"profile_link_color":"3B2A26","is_translator":false,"id_str":"37539828","entities":{"url":{"urls":[{"expanded_url":null,"url":"ht
 tp://www.omnitarian.me","indices":[0,24]}]},"description":{"urls":[]}},"default_profile":false,"contributors_enabled":false,"favourites_count":647,"url":"http://www.omnitarian.me","profile_image_url_https":"https://si0.twimg.com/profile_images/1629790393/shrinker_2000_trans_normal.png","utc_offset":-21600,"id":37539828,"profile_use_background_image":true,"listed_count":52,"profile_text_color":"000000","lang":"en","followers_count":608,"protected":false,"notifications":null,"profile_background_image_url_https":"https://si0.twimg.com/profile_background_images/106455659/rect6056-9.png","profile_background_color":"EEE3C4","verified":false,"geo_enabled":false,"time_zone":"Central Time (US & Canada)","description":"Cartoonist, Illustrator, and T-Shirt connoisseur","default_profile_image":false,"profile_background_image_url":"http://a0.twimg.com/profile_background_images/106455659/rect6056-9.png","statuses_count":3575,"friends_count":249,"following":null,"show_all_inline_media":true,"scree
 n_name":"Omnitarian"},"in_reply_to_screen_name":null,"source":"<a>Twitter for iPhone<\/a>","in_reply_to_status_id":null}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestCreateTable/testSelfDescTable1.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestCreateTable/testSelfDescTable1.sql b/tajo-core-tests/src/test/resources/queries/TestCreateTable/testSelfDescTable1.sql
new file mode 100644
index 0000000..6dacfbf
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestCreateTable/testSelfDescTable1.sql
@@ -0,0 +1 @@
+create table d9.schemaless (*) using json with ('compression.codec'='none') partition by column (id int8)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testCrossJoin.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testCrossJoin.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testCrossJoin.sql
new file mode 100644
index 0000000..4a49790
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testCrossJoin.sql
@@ -0,0 +1,6 @@
+select
+  user.favourites_count::int8,
+  l_linenumber,
+  l_comment
+from
+  default.lineitem, self_desc_table3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby.sql
new file mode 100644
index 0000000..9c7b33a
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby.sql
@@ -0,0 +1,7 @@
+select
+  name.first_name,
+  count(*)
+from
+  self_desc_table1
+group by
+  name.first_name
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby2.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby2.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby2.sql
new file mode 100644
index 0000000..5b302cd
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby2.sql
@@ -0,0 +1,7 @@
+select
+  coordinates,
+  avg(retweet_count::int4)
+from
+  self_desc_table3
+group by
+  coordinates
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby3.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby3.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby3.sql
new file mode 100644
index 0000000..4d752fb
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testGroupby3.sql
@@ -0,0 +1,7 @@
+select
+  user.time_zone,
+  sum(user.favourites_count::int8)
+from
+  self_desc_table3
+group by
+  user.time_zone
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinOfSelfDescTablesWithQualifiedColumns.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinOfSelfDescTablesWithQualifiedColumns.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinOfSelfDescTablesWithQualifiedColumns.sql
new file mode 100644
index 0000000..3e0feff
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinOfSelfDescTablesWithQualifiedColumns.sql
@@ -0,0 +1,6 @@
+select
+  self_desc_table3.user.favourites_count::int8
+from
+  github, self_desc_table3
+where
+  self_desc_table3.user.favourites_count = (github.actor.id::int8 - 206379)::text
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable.sql
new file mode 100644
index 0000000..2bb8ab8
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable.sql
@@ -0,0 +1,8 @@
+select
+  user.favourites_count::int8,
+  l_linenumber,
+  l_comment
+from
+  default.lineitem, self_desc_table3
+where
+  user.favourites_count::int8 = (l_orderkey - 1)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable2.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable2.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable2.sql
new file mode 100644
index 0000000..fd7eb6a
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable2.sql
@@ -0,0 +1,8 @@
+select
+  user.favourites_count::int8,
+  l_linenumber,
+  l_comment
+from
+  default.lineitem, self_desc_table3, default.orders, default.supplier
+where
+  user.favourites_count::int8 = (l_orderkey - 1) and l_orderkey = o_orderkey and l_linenumber = s_suppkey
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable4.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable4.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable4.sql
new file mode 100644
index 0000000..de2679c
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testJoinWithSchemaFullTable4.sql
@@ -0,0 +1,8 @@
+select
+  self_desc_table3.user.favourites_count::int8,
+  l_linenumber,
+  l_comment
+from
+  default.lineitem, self_desc_table1, self_desc_table3, default.orders, default.supplier
+where
+  self_desc_table3.user.favourites_count::int8 = (l_orderkey - 1) and l_orderkey = o_orderkey and l_linenumber = s_suppkey and self_desc_table3.user.favourites_count <> self_desc_table1.name.first_name
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect.sql
new file mode 100644
index 0000000..1b31c8a
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect.sql
@@ -0,0 +1,8 @@
+select
+  glossary.title,
+  glossary."GlossDiv".title,
+  glossary."GlossDiv".null_expected,
+  glossary."GlossDiv"."GlossList"."GlossEntry"."SortAs",
+  glossary."GlossDiv"."GlossList"."GlossEntry"."Abbrev"
+from
+  self_desc_table2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect2.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect2.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect2.sql
new file mode 100644
index 0000000..35a342d
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSelect2.sql
@@ -0,0 +1,9 @@
+select
+  glossary.title,
+  glossary."GlossDiv".title,
+  glossary."GlossDiv".null_expected,
+  glossary."GlossDiv"."GlossList"."GlossEntry"."SortAs"
+from
+  self_desc_table2
+where
+  glossary."GlossDiv"."GlossList"."GlossEntry"."Abbrev" = 'ISO 8879:1986'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSort.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSort.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSort.sql
new file mode 100644
index 0000000..98a011b
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnSelfDescTable/testSort.sql
@@ -0,0 +1,7 @@
+select
+  created_at,
+  id,
+  user.profile_sidebar_fill_color
+from
+  self_desc_table3
+order by created_at
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table1.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table1.sql b/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table1.sql
new file mode 100644
index 0000000..f636399
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table1.sql
@@ -0,0 +1 @@
+create external table schemaless (*) using json with ('compression.codec'='none') partition by column (id int8) location 'file:///schemaless'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table2.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table2.sql b/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table2.sql
new file mode 100644
index 0000000..5f2cd18
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestSQLAnalyzer/positive/create_self_desc_table2.sql
@@ -0,0 +1 @@
+create table schemaless (*) using json with ('compression.codec'='none') partition by column (id int8)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testCrossJoin.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testCrossJoin.result b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testCrossJoin.result
new file mode 100644
index 0000000..52f385a
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testCrossJoin.result
@@ -0,0 +1,22 @@
+?cast,l_linenumber,l_comment
+-------------------------------
+0,1,egular courts above the
+0,1,ongside of the furiously brave acco
+0,1,ven requests. deposits breach a
+0,2, unusual accounts. eve
+0,2,ly final dependencies: slyly bold 
+19,1,egular courts above the
+19,1,ongside of the furiously brave acco
+19,1,ven requests. deposits breach a
+19,2, unusual accounts. eve
+19,2,ly final dependencies: slyly bold 
+647,1,egular courts above the
+647,1,ongside of the furiously brave acco
+647,1,ven requests. deposits breach a
+647,2, unusual accounts. eve
+647,2,ly final dependencies: slyly bold 
+8,1,egular courts above the
+8,1,ongside of the furiously brave acco
+8,1,ven requests. deposits breach a
+8,2, unusual accounts. eve
+8,2,ly final dependencies: slyly bold 

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby.result b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby.result
new file mode 100644
index 0000000..0dc413f
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby.result
@@ -0,0 +1,5 @@
+name/first_name,?count
+-------------------------------
+Arya,1
+Eddard,1
+Syrio,1

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby2.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby2.result b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby2.result
new file mode 100644
index 0000000..e7b795f
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby2.result
@@ -0,0 +1,3 @@
+coordinates,?avg_1
+-------------------------------
+null,2.5

http://git-wip-us.apache.org/repos/asf/tajo/blob/5a155861/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby3.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby3.result b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby3.result
new file mode 100644
index 0000000..2004343
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnSelfDescTable/testGroupby3.result
@@ -0,0 +1,5 @@
+user/time_zone,?sum_1
+-------------------------------
+Central Time (US & Canada),647
+Eastern Time (US & Canada),27
+Pacific Time (US & Canada),0