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 2014/12/31 11:51:27 UTC

[6/8] tajo git commit: TAJO-1176: Implements queryable virtual tables for catalog information

TAJO-1176: Implements queryable virtual tables for catalog information

Closes #273


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

Branch: refs/heads/index_support
Commit: 021a6f0b216ca9f67a5a889e72d2a8ce81c047f2
Parents: 32be38d
Author: Jihun Kang <ji...@apache.org>
Authored: Tue Dec 30 23:54:05 2014 +0900
Committer: Jihun Kang <ji...@apache.org>
Committed: Tue Dec 30 23:54:05 2014 +0900

----------------------------------------------------------------------
 .../tajo/catalog/AbstractCatalogClient.java     | 152 +++++
 .../src/main/proto/CatalogProtocol.proto        |   9 +-
 .../org/apache/tajo/catalog/CatalogService.java |  37 +-
 .../src/main/proto/CatalogProtos.proto          |  81 ++-
 .../tajo/catalog/store/HCatalogStore.java       |  53 +-
 .../org/apache/tajo/catalog/CatalogServer.java  | 247 ++++++--
 .../dictionary/AbstractTableDescriptor.java     |  90 +++
 .../catalog/dictionary/ColumnDescriptor.java    |  47 ++
 .../dictionary/ColumnsTableDescriptor.java      |  48 ++
 .../dictionary/DatabasesTableDescriptor.java    |  47 ++
 .../dictionary/IndexesTableDescriptor.java      |  52 ++
 .../InfoSchemaMetadataDictionary.java           | 124 ++++
 .../dictionary/PartitionsTableDescriptor.java   |  48 ++
 .../catalog/dictionary/TableDescriptor.java     |  29 +
 .../dictionary/TableOptionsTableDescriptor.java |  46 ++
 .../dictionary/TableStatsTableDescriptor.java   |  46 ++
 .../dictionary/TablesTableDescriptor.java       |  49 ++
 .../dictionary/TablespacesTableDescriptor.java  |  48 ++
 .../tajo/catalog/store/AbstractDBStore.java     | 295 +++++++++
 .../apache/tajo/catalog/store/CatalogStore.java |  24 +
 .../org/apache/tajo/catalog/store/MemStore.java | 189 ++++++
 .../org/apache/tajo/catalog/TestCatalog.java    |  15 +-
 .../NonForwardQueryResultFileScanner.java       | 164 +++++
 .../master/NonForwardQueryResultScanner.java    | 148 +----
 .../NonForwardQueryResultSystemScanner.java     | 616 +++++++++++++++++++
 .../tajo/master/TajoMasterClientService.java    |   2 +-
 .../apache/tajo/master/exec/QueryExecutor.java  |  27 +-
 .../TestNonForwardQueryResultSystemScanner.java | 296 +++++++++
 .../org/apache/tajo/plan/LogicalPlanner.java    |   3 +-
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  21 +
 30 files changed, 2857 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
index 6b50115..8ef1c9a 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java
@@ -19,12 +19,20 @@
 package org.apache.tajo.catalog;
 
 import com.google.protobuf.ServiceException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.CatalogProtocol.CatalogProtocolService;
 import org.apache.tajo.catalog.exception.NoSuchFunctionException;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.conf.TajoConf;
@@ -139,6 +147,24 @@ public abstract class AbstractCatalogClient implements CatalogService {
       return null;
     }
   }
+  
+  @Override
+  public List<TablespaceProto> getAllTablespaces() {
+    try {
+      return new ServerCallable<List<TablespaceProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<TablespaceProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          CatalogProtos.GetTablespacesProto response = stub.getAllTablespaces(null, ProtoUtil.NULL_PROTO);
+          return response.getTablespaceList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
 
   @Override
   public TablespaceProto getTablespace(final String tablespaceName) {
@@ -236,6 +262,24 @@ public abstract class AbstractCatalogClient implements CatalogService {
       return null;
     }
   }
+  
+  @Override
+  public List<DatabaseProto> getAllDatabases() {
+    try {
+      return new ServerCallable<List<DatabaseProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<DatabaseProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          GetDatabasesProto response = stub.getAllDatabases(null, ProtoUtil.NULL_PROTO);
+          return response.getDatabaseList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
 
   @Override
   public final TableDesc getTableDesc(final String databaseName, final String tableName) {
@@ -261,6 +305,78 @@ public abstract class AbstractCatalogClient implements CatalogService {
     String [] splitted = CatalogUtil.splitFQTableName(qualifiedName);
     return getTableDesc(splitted[0], splitted[1]);
   }
+  
+  @Override
+  public List<TableDescriptorProto> getAllTables() {
+    try {
+      return new ServerCallable<List<TableDescriptorProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<TableDescriptorProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          GetTablesProto response = stub.getAllTables(null, ProtoUtil.NULL_PROTO);
+          return response.getTableList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
+  
+  @Override
+  public List<TableOptionProto> getAllTableOptions() {
+    try {
+      return new ServerCallable<List<TableOptionProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<TableOptionProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          GetTableOptionsProto response = stub.getAllTableOptions(null, ProtoUtil.NULL_PROTO);
+          return response.getTableOptionList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
+  
+  @Override
+  public List<TableStatsProto> getAllTableStats() {
+    try {
+      return new ServerCallable<List<TableStatsProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<TableStatsProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          GetTableStatsProto response = stub.getAllTableStats(null, ProtoUtil.NULL_PROTO);
+          return response.getStatList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
+  
+  @Override
+  public List<ColumnProto> getAllColumns() {
+    try {
+      return new ServerCallable<List<ColumnProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<ColumnProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          GetColumnsProto response = stub.getAllColumns(null, ProtoUtil.NULL_PROTO);
+          return response.getColumnList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
 
   @Override
   public final PartitionMethodDesc getPartitionMethod(final String databaseName, final String tableName) {
@@ -301,6 +417,24 @@ public abstract class AbstractCatalogClient implements CatalogService {
       return false;
     }
   }
+  
+  @Override
+  public List<TablePartitionProto> getAllPartitions() {
+    try {
+      return new ServerCallable<List<TablePartitionProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<TablePartitionProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          GetTablePartitionsProto response = stub.getAllPartitions(null, ProtoUtil.NULL_PROTO);
+          return response.getPartList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
 
   @Override
   public final Collection<String> getAllTableNames(final String databaseName) {
@@ -529,6 +663,24 @@ public abstract class AbstractCatalogClient implements CatalogService {
       return false;
     }
   }
+  
+  @Override
+  public List<IndexProto> getAllIndexes() {
+    try {
+      return new ServerCallable<List<IndexProto>>(pool, getCatalogServerAddr(), CatalogProtocol.class, false) {
+
+        @Override
+        public List<IndexProto> call(NettyClientBase client) throws Exception {
+          CatalogProtocolService.BlockingInterface stub = getStub(client);
+          GetIndexesProto response = stub.getAllIndexes(null, ProtoUtil.NULL_PROTO);
+          return response.getIndexList();
+        }
+      }.withRetries();
+    } catch (ServiceException e) {
+      LOG.error(e.getMessage(), e);
+      return null;
+    }
+  }
 
   @Override
   public final boolean createFunction(final FunctionDesc funcDesc) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
index adf0740..cae5d88 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
@@ -29,23 +29,28 @@ service CatalogProtocolService {
   rpc createTablespace(CreateTablespaceRequest) returns (BoolProto);
   rpc dropTablespace(StringProto) returns (BoolProto);
   rpc existTablespace(StringProto) returns (BoolProto);
+  rpc getAllTablespaces(NullProto) returns (GetTablespacesProto);
   rpc getAllTablespaceNames(NullProto) returns (StringListProto);
   rpc getTablespace(StringProto) returns (TablespaceProto);
   rpc alterTablespace(AlterTablespaceProto) returns (BoolProto);
   rpc alterTable(AlterTableDescProto) returns (BoolProto);
   rpc updateTableStats(UpdateTableStatsProto) returns (BoolProto);
 
-
   rpc createDatabase(CreateDatabaseRequest) returns (BoolProto);
   rpc dropDatabase(StringProto) returns (BoolProto);
   rpc existDatabase(StringProto) returns (BoolProto);
   rpc getAllDatabaseNames(NullProto) returns (StringListProto);
+  rpc getAllDatabases(NullProto) returns (GetDatabasesProto);
 
   rpc createTable(TableDescProto) returns (BoolProto);
   rpc dropTable(TableIdentifierProto) returns (BoolProto);
   rpc existsTable(TableIdentifierProto) returns (BoolProto);
   rpc getTableDesc(TableIdentifierProto) returns (TableDescProto);
   rpc getAllTableNames(StringProto) returns (StringListProto);
+  rpc getAllTables(NullProto) returns (GetTablesProto);
+  rpc getAllTableOptions(NullProto) returns (GetTableOptionsProto);
+  rpc getAllTableStats(NullProto) returns (GetTableStatsProto);
+  rpc getAllColumns(NullProto) returns (GetColumnsProto);
 
   rpc getPartitionMethodByTableName(TableIdentifierProto) returns (PartitionMethodProto);
   rpc existPartitionMethod(TableIdentifierProto) returns (BoolProto);
@@ -56,6 +61,7 @@ service CatalogProtocolService {
   rpc getPartitionByPartitionName(StringProto) returns (PartitionDescProto);
   rpc getPartitionsByTableName(StringProto) returns (PartitionsProto);
   rpc delAllPartitions(StringProto) returns (PartitionsProto);
+  rpc getAllPartitions(NullProto) returns (GetTablePartitionsProto);
 
   rpc createIndex(IndexDescProto) returns (BoolProto);
   rpc dropIndex(IndexNameProto) returns (BoolProto);
@@ -63,6 +69,7 @@ service CatalogProtocolService {
   rpc existIndexByColumn(GetIndexByColumnRequest) returns (BoolProto);
   rpc getIndexByName(IndexNameProto) returns (IndexDescProto);
   rpc getIndexByColumn(GetIndexByColumnRequest) returns (IndexDescProto);
+  rpc getAllIndexes(NullProto) returns (GetIndexesProto);
 
   rpc createFunction(FunctionDescProto) returns (BoolProto);
   rpc dropFunction(UnregisterFunctionRequest) returns (BoolProto);

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
index b41b636..eb11272 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogService.java
@@ -19,10 +19,17 @@
 package org.apache.tajo.catalog;
 
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
-import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TablePartitionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 
 import java.util.Collection;
+import java.util.List;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto;
 import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
@@ -58,6 +65,11 @@ public interface CatalogService {
    * @return All tablespace names
    */
   Collection<String> getAllTablespaceNames();
+  
+  /**
+   * 
+   */
+  List<TablespaceProto> getAllTablespaces();
 
   /**
    *
@@ -99,6 +111,11 @@ public interface CatalogService {
    * @return All database names
    */
   Collection<String> getAllDatabaseNames();
+  
+  /**
+   * 
+   */
+  List<DatabaseProto> getAllDatabases();
 
   /**
    * Get a table description by name
@@ -122,6 +139,20 @@ public interface CatalogService {
    * @return All table names which belong to a given database.
    */
   Collection<String> getAllTableNames(String databaseName);
+  
+  /**
+   * 
+   */
+  List<TableDescriptorProto> getAllTables();
+  
+  List<TableOptionProto> getAllTableOptions();
+  
+  List<TableStatsProto> getAllTableStats();
+  
+  /**
+   * 
+   */
+  List<ColumnProto> getAllColumns();
 
   /**
    *
@@ -152,6 +183,8 @@ public interface CatalogService {
   PartitionMethodDesc getPartitionMethod(String databaseName, String tableName);
 
   boolean existPartitionMethod(String databaseName, String tableName);
+  
+  List<TablePartitionProto> getAllPartitions();
 
   boolean createIndex(IndexDesc index);
 
@@ -164,6 +197,8 @@ public interface CatalogService {
   IndexDesc getIndexByColumn(String databaseName, String tableName, String columnName);
 
   boolean dropIndex(String databaseName, String indexName);
+  
+  List<IndexProto> getAllIndexes();
 
   boolean createFunction(FunctionDesc funcDesc);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/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 946b563..a204685 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -38,6 +38,7 @@ enum StoreType {
   TEXTFILE = 10;
   JSON = 11;
   HBASE = 12;
+  SYSTEM = 13;
 }
 
 enum OrderType {
@@ -61,6 +62,7 @@ enum AlterTableType {
 
 message ColumnProto {
   required string name = 1;
+  optional int32 tid = 2;
   required DataType dataType = 3;
 }
 
@@ -101,7 +103,9 @@ message TableDescProto {
 
 message TableIdentifierProto {
   required string database_name = 1;
-  required string table_name = 3;
+  required string table_name = 2;
+  optional int32 dbId = 3;
+  optional int32 tid = 4;
 }
 
 message NamespaceProto {
@@ -130,6 +134,63 @@ message GetAllTableNamesResponse {
   repeated string tableName = 1;
 }
 
+message GetTablespacesProto {
+  repeated TablespaceProto tablespace = 1;
+}
+
+message GetDatabasesProto {
+  repeated DatabaseProto database = 1;
+}
+
+message GetTablesProto {
+  repeated TableDescriptorProto table = 1;
+}
+
+message GetColumnsProto {
+  repeated ColumnProto column = 1;
+}
+
+message GetIndexesProto {
+  repeated IndexProto index = 1;
+}
+
+message GetTableOptionsProto {
+  repeated TableOptionProto tableOption = 1;
+}
+
+message GetTableStatsProto {
+  repeated TableStatsProto stat = 1;
+}
+
+message GetTablePartitionsProto {
+  repeated TablePartitionProto part = 1;
+}
+
+message IndexProto {
+  required int32 dbId = 1;
+  required int32 tId = 2;
+  required string indexName = 3;
+  required string columnName = 4;
+  required string dataType = 5;
+  required string indexType = 6;
+  optional bool isUnique = 7 [default = false];
+  optional bool isClustered = 8 [default = false];
+  optional bool isAscending = 9 [default = false];
+}
+
+message TableOptionProto {
+  required int32 tid = 1;
+  required KeyValueProto keyval = 2;
+}
+
+message TablePartitionProto {
+  required int32 pid = 1;
+  required int32 tid = 2;
+  optional string partitionName = 3;
+  required int32  ordinalPosition = 4;
+  optional string path = 5;
+}
+
 message GetIndexByColumnRequest {
   required TableIdentifierProto tableIdentifier = 1;
   required string columnName = 2;
@@ -169,6 +230,7 @@ message TableStatsProto {
   optional int64 avgRows = 6;
   optional int64 readBytes = 7;
   repeated ColumnStatsProto colStat = 8;
+  optional int32 tid = 9;
 }
 
 message ColumnStatsProto {
@@ -240,6 +302,23 @@ message PartitionDescProto {
 message TablespaceProto {
   required string spaceName = 1;
   required string uri = 2;
+  optional string handler = 3;
+  optional int32 id = 4; 
+}
+
+message DatabaseProto {
+	required int32 spaceId = 1;
+	required int32 id = 2;
+	required string name = 3;
+}
+
+message TableDescriptorProto {
+	required int32 dbId = 1;
+	required int32 tid = 2;
+	required string name = 3;
+	optional string tableType = 4;
+	required string path = 5;
+	required string storeType = 6;
 }
 
 message AlterTablespaceProto {

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
index ad0aee3..89c0fdd 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.catalog.store;
 
 import com.google.common.collect.Lists;
+
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,10 +38,17 @@ import org.apache.hcatalog.data.schema.HCatFieldSchema;
 import org.apache.hcatalog.data.schema.HCatSchema;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.exception.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TablePartitionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.exception.NotImplementedException;
@@ -54,7 +62,6 @@ import java.io.IOException;
 import java.util.*;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType;
-import static org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
 
 public class HCatalogStore extends CatalogConstants implements CatalogStore {
   protected final Log LOG = LogFactory.getLog(getClass());
@@ -236,7 +243,7 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
       List<FieldSchema> partitionKeys = table.getPartitionKeys();
 
       if (null != partitionKeys) {
-        Schema expressionSchema = new Schema();
+        org.apache.tajo.catalog.Schema expressionSchema = new org.apache.tajo.catalog.Schema();
         StringBuilder sb = new StringBuilder();
         if (partitionKeys.size() > 0) {
           for (int i = 0; i < partitionKeys.size(); i++) {
@@ -841,4 +848,44 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
 
     return exist;
   }
+
+  @Override
+  public List<ColumnProto> getAllColumns() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<DatabaseProto> getAllDatabases() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<IndexProto> getAllIndexes() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<TablePartitionProto> getAllPartitions() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<TableOptionProto> getAllTableOptions() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<TableStatsProto> getAllTableStats() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<TableDescriptorProto> getAllTables() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<TablespaceProto> getTablespaces() throws CatalogException {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
index 57086e2..30b1767 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
@@ -23,6 +23,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -30,6 +31,7 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.annotation.ThreadSafe;
 import org.apache.tajo.catalog.CatalogProtocol.CatalogProtocolService;
+import org.apache.tajo.catalog.dictionary.InfoSchemaMetadataDictionary;
 import org.apache.tajo.catalog.exception.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.catalog.store.CatalogStore;
@@ -79,6 +81,7 @@ public class CatalogServer extends AbstractService {
   private CatalogStore store;
   private Map<String, List<FunctionDescProto>> functions = new ConcurrentHashMap<String,
       List<FunctionDescProto>>();
+  private final InfoSchemaMetadataDictionary metaDictionary = new InfoSchemaMetadataDictionary();
 
   // RPC variables
   private BlockingRpcServer rpcServer;
@@ -298,6 +301,18 @@ public class CatalogServer extends AbstractService {
         rlock.unlock();
       }
     }
+    
+    @Override
+    public GetTablespacesProto getAllTablespaces(RpcController controller, NullProto request) throws ServiceException {
+      rlock.lock();
+      try {
+        return GetTablespacesProto.newBuilder().addAllTablespace(store.getTablespaces()).build();
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      } finally {
+        rlock.unlock();
+      }
+    }
 
     @Override
     public TablespaceProto getTablespace(RpcController controller, StringProto request) throws ServiceException {
@@ -349,6 +364,10 @@ public class CatalogServer extends AbstractService {
       String databaseName = request.getDatabaseName();
       String tablespaceName = request.getTablespaceName();
 
+      if (metaDictionary.isSystemDatabase(databaseName)) {
+        throw new ServiceException(databaseName + " is a system database name.");
+      }
+      
       wlock.lock();
       try {
         if (store.existDatabase(databaseName)) {
@@ -389,9 +408,14 @@ public class CatalogServer extends AbstractService {
 
     @Override
     public BoolProto alterTable(RpcController controller, AlterTableDescProto proto) throws ServiceException {
+      String [] split = CatalogUtil.splitTableName(proto.getTableName());
+      
+      if (metaDictionary.isSystemDatabase(split[0])) {
+        throw new ServiceException(split[0] + " is a system database.");
+      }
+      
       wlock.lock();
       try {
-        String [] split = CatalogUtil.splitTableName(proto.getTableName());
         if (!store.existTable(split[0], split[1])) {
           throw new NoSuchTableException(proto.getTableName());
         }
@@ -410,6 +434,10 @@ public class CatalogServer extends AbstractService {
     @Override
     public BoolProto dropDatabase(RpcController controller, StringProto request) throws ServiceException {
       String databaseName = request.getValue();
+      
+      if (metaDictionary.isSystemDatabase(databaseName)) {
+        throw new ServiceException(databaseName + " is a system database.");
+      }
 
       wlock.lock();
       try {
@@ -432,13 +460,33 @@ public class CatalogServer extends AbstractService {
     public BoolProto existDatabase(RpcController controller, StringProto request) throws ServiceException {
       String databaseName = request.getValue();
 
+      if (!metaDictionary.isSystemDatabase(databaseName)) {
+        rlock.lock();
+        try {
+          if (store.existDatabase(databaseName)) {
+            return ProtoUtil.TRUE;
+          } else {
+            return ProtoUtil.FALSE;
+          }
+        } catch (Exception e) {
+          LOG.error(e);
+          throw new ServiceException(e);
+        } finally {
+          rlock.unlock();
+        }
+      } else {
+        return ProtoUtil.TRUE;
+      }
+    }
+
+    @Override
+    public StringListProto getAllDatabaseNames(RpcController controller, NullProto request) throws ServiceException {
       rlock.lock();
       try {
-        if (store.existDatabase(databaseName)) {
-          return ProtoUtil.TRUE;
-        } else {
-          return ProtoUtil.FALSE;
-        }
+        StringListProto.Builder builder = StringListProto.newBuilder();
+        builder.addAllValues(store.getAllDatabaseNames());
+        builder.addValues(metaDictionary.getSystemDatabaseName());
+        return builder.build();
       } catch (Exception e) {
         LOG.error(e);
         throw new ServiceException(e);
@@ -446,14 +494,13 @@ public class CatalogServer extends AbstractService {
         rlock.unlock();
       }
     }
-
+    
     @Override
-    public StringListProto getAllDatabaseNames(RpcController controller, NullProto request) throws ServiceException {
+    public GetDatabasesProto getAllDatabases(RpcController controller, NullProto request) throws ServiceException {
       rlock.lock();
       try {
-        return ProtoUtil.convertStrings(store.getAllDatabaseNames());
+        return GetDatabasesProto.newBuilder().addAllDatabase(store.getAllDatabases()).build();
       } catch (Exception e) {
-        LOG.error(e);
         throw new ServiceException(e);
       } finally {
         rlock.unlock();
@@ -466,27 +513,31 @@ public class CatalogServer extends AbstractService {
       String databaseName = request.getDatabaseName();
       String tableName = request.getTableName();
 
-      rlock.lock();
-      try {
-        boolean contain;
+      if (metaDictionary.isSystemDatabase(databaseName)){
+        return metaDictionary.getTableDesc(tableName);
+      } else {
+        rlock.lock();
+        try {
+          boolean contain;
 
-        contain = store.existDatabase(databaseName);
+          contain = store.existDatabase(databaseName);
 
-        if (contain) {
-          contain = store.existTable(databaseName, tableName);
           if (contain) {
-            return store.getTable(databaseName, tableName);
+            contain = store.existTable(databaseName, tableName);
+            if (contain) {
+              return store.getTable(databaseName, tableName);
+            } else {
+              throw new NoSuchTableException(tableName);
+            }
           } else {
-            throw new NoSuchTableException(tableName);
+            throw new NoSuchDatabaseException(databaseName);
           }
-        } else {
-          throw new NoSuchDatabaseException(databaseName);
+        } catch (Exception e) {
+          LOG.error(e);
+          throw new ServiceException(e);
+        } finally {
+          rlock.unlock();
         }
-      } catch (Exception e) {
-        LOG.error(e);
-        throw new ServiceException(e);
-      } finally {
-        rlock.unlock();
       }
     }
 
@@ -496,18 +547,22 @@ public class CatalogServer extends AbstractService {
 
       String databaseName = request.getValue();
 
-      rlock.lock();
-      try {
-        if (store.existDatabase(databaseName)) {
-          return ProtoUtil.convertStrings(store.getAllTableNames(databaseName));
-        } else {
-          throw new NoSuchDatabaseException(databaseName);
+      if (metaDictionary.isSystemDatabase(databaseName)) {
+        return ProtoUtil.convertStrings(metaDictionary.getAllSystemTables());
+      } else {
+        rlock.lock();
+        try {
+          if (store.existDatabase(databaseName)) {
+            return ProtoUtil.convertStrings(store.getAllTableNames(databaseName));
+          } else {
+            throw new NoSuchDatabaseException(databaseName);
+          }
+        } catch (Exception e) {
+          LOG.error(e);
+          throw new ServiceException(e);
+        } finally {
+          rlock.unlock();
         }
-      } catch (Exception e) {
-        LOG.error(e);
-        throw new ServiceException(e);
-      } finally {
-        rlock.unlock();
       }
     }
 
@@ -532,6 +587,10 @@ public class CatalogServer extends AbstractService {
       String databaseName = splitted[0];
       String tableName = splitted[1];
 
+      if (metaDictionary.isSystemDatabase(databaseName)) {
+        throw new ServiceException(databaseName + " is a system database.");
+      }
+      
       wlock.lock();
       try {
 
@@ -563,6 +622,10 @@ public class CatalogServer extends AbstractService {
 
       String databaseName = request.getDatabaseName();
       String tableName = request.getTableName();
+      
+      if (metaDictionary.isSystemDatabase(databaseName)) {
+        throw new ServiceException(databaseName + " is a system database.");
+      }
 
       wlock.lock();
       try {
@@ -595,27 +658,83 @@ public class CatalogServer extends AbstractService {
       String databaseName = request.getDatabaseName();
       String tableName = request.getTableName();
 
-      rlock.lock();
-      try {
+      if (!metaDictionary.isSystemDatabase(databaseName)) {
+        rlock.lock();
+        try {
 
-        boolean contain = store.existDatabase(databaseName);
+          boolean contain = store.existDatabase(databaseName);
 
-        if (contain) {
-          if (store.existTable(databaseName, tableName)) {
-            return BOOL_TRUE;
+          if (contain) {
+            if (store.existTable(databaseName, tableName)) {
+              return BOOL_TRUE;
+            } else {
+              return BOOL_FALSE;
+            }
           } else {
-            return BOOL_FALSE;
+            throw new NoSuchDatabaseException(databaseName);
           }
+        } catch (Exception e) {
+          LOG.error(e);
+          throw new ServiceException(e);
+        } finally {
+          rlock.unlock();
+        }
+      } else {
+        if (metaDictionary.existTable(tableName)) {
+          return BOOL_TRUE;
         } else {
-          throw new NoSuchDatabaseException(databaseName);
+          return BOOL_FALSE;
         }
+      }
+
+    }
+    
+    @Override
+    public GetTablesProto getAllTables(RpcController controller, NullProto request) throws ServiceException {
+      rlock.lock();
+      try {
+        return GetTablesProto.newBuilder().addAllTable(store.getAllTables()).build();
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      } finally {
+        rlock.unlock();
+      }
+    }
+    
+    @Override
+    public GetTableOptionsProto getAllTableOptions(RpcController controller, NullProto request) throws ServiceException {
+      rlock.lock();
+      try {
+        return GetTableOptionsProto.newBuilder().addAllTableOption(store.getAllTableOptions()).build();
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      } finally {
+        rlock.unlock();
+      }
+    }
+    
+    @Override
+    public GetTableStatsProto getAllTableStats(RpcController controller, NullProto request) throws ServiceException {
+      rlock.lock();
+      try {
+        return GetTableStatsProto.newBuilder().addAllStat(store.getAllTableStats()).build();
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      } finally {
+        rlock.unlock();
+      }
+    }
+    
+    @Override
+    public GetColumnsProto getAllColumns(RpcController controller, NullProto request) throws ServiceException {
+      rlock.lock();
+      try {
+        return GetColumnsProto.newBuilder().addAllColumn(store.getAllColumns()).build();
       } catch (Exception e) {
-        LOG.error(e);
         throw new ServiceException(e);
       } finally {
         rlock.unlock();
       }
-
     }
 
     @Override
@@ -625,6 +744,10 @@ public class CatalogServer extends AbstractService {
       String databaseName = request.getDatabaseName();
       String tableName = request.getTableName();
 
+      if (metaDictionary.isSystemDatabase(databaseName)) {
+        throw new ServiceException(databaseName + " is a system databsae. It does not contain any partitioned tables.");
+      }
+      
       rlock.lock();
       try {
         boolean contain;
@@ -658,6 +781,10 @@ public class CatalogServer extends AbstractService {
         throws ServiceException {
       String databaseName = request.getDatabaseName();
       String tableName = request.getTableName();
+      
+      if (metaDictionary.isSystemDatabase(databaseName)) {
+        throw new ServiceException(databaseName + " is a system database. Partition Method does not support yet.");
+      }
 
       rlock.lock();
       try {
@@ -721,14 +848,28 @@ public class CatalogServer extends AbstractService {
         throws ServiceException {
       return null;
     }
+    
+    @Override
+    public GetTablePartitionsProto getAllPartitions(RpcController controller, NullProto request) throws ServiceException {
+      rlock.lock();
+      try {
+        return GetTablePartitionsProto.newBuilder().addAllPart(store.getAllPartitions()).build();
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      } finally {
+        rlock.unlock();
+      }
+    }
 
     @Override
     public BoolProto createIndex(RpcController controller, IndexDescProto indexDesc)
         throws ServiceException {
+      String databaseName = indexDesc.getTableIdentifier().getDatabaseName();
+      
       rlock.lock();
       try {
         if (store.existIndexByName(
-            indexDesc.getTableIdentifier().getDatabaseName(),
+            databaseName,
             indexDesc.getIndexName())) {
           throw new AlreadyExistsIndexException(indexDesc.getIndexName());
         }
@@ -847,6 +988,18 @@ public class CatalogServer extends AbstractService {
 
       return BOOL_TRUE;
     }
+    
+    @Override
+    public GetIndexesProto getAllIndexes(RpcController controller, NullProto request) throws ServiceException {
+      rlock.lock();
+      try {
+        return GetIndexesProto.newBuilder().addAllIndex(store.getAllIndexes()).build();
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      } finally {
+        rlock.unlock();
+      }
+    }
 
     public boolean checkIfBuiltin(FunctionType type) {
       return type == GENERAL || type == AGGREGATION || type == DISTINCT_AGGREGATION;

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/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
new file mode 100644
index 0000000..a8b384c
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java
@@ -0,0 +1,90 @@
+/**
+ * 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.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.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
+
+abstract class AbstractTableDescriptor implements TableDescriptor {
+  
+  protected InfoSchemaMetadataDictionary dictionary;
+
+  public AbstractTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    dictionary = metadataDictionary;
+  }
+
+  protected abstract ColumnDescriptor[] getColumnDescriptors();
+  
+  protected SchemaProto getSchemaProto() {
+    SchemaProto.Builder schemaBuilder = SchemaProto.newBuilder();
+    ColumnProto.Builder columnBuilder = null;
+    
+    for (ColumnDescriptor columnDescriptor: getColumnDescriptors()) {
+      columnBuilder = ColumnProto.newBuilder();
+      
+      columnBuilder.setName(columnDescriptor.getName().toLowerCase());
+      if (columnDescriptor.getLength() > 0) {
+        columnBuilder.setDataType(CatalogUtil.newDataTypeWithLen(columnDescriptor.getType(),
+            columnDescriptor.getLength()));
+      } else {
+        columnBuilder.setDataType(CatalogUtil.newSimpleDataType(columnDescriptor.getType()));
+      }
+      
+      schemaBuilder.addFields(columnBuilder.build());
+    }
+    
+    return schemaBuilder.build();
+  }
+  
+  protected TableProto getTableProto() {
+    TableProto.Builder metaBuilder = TableProto.newBuilder();
+    metaBuilder.setStoreType(StoreType.SYSTEM);
+    metaBuilder.setParams(KeyValueSetProto.newBuilder().build());
+    return metaBuilder.build();
+  }
+  
+  protected TableStatsProto getTableStatsProto() {
+    TableStatsProto.Builder statBuilder = TableStatsProto.newBuilder();
+    statBuilder.setNumRows(0l);
+    statBuilder.setNumBytes(0l);
+    return statBuilder.build();
+  }
+  
+  @Override
+  public TableDescProto getTableDescription() {
+    TableDescProto.Builder tableBuilder = TableDescProto.newBuilder();
+    
+    tableBuilder.setTableName(CatalogUtil.buildFQName(dictionary.getSystemDatabaseName(), getTableNameString()));
+    tableBuilder.setPath(dictionary.getTablePath());
+    
+    tableBuilder.setSchema(CatalogUtil.getQualfiedSchema(
+        dictionary.getSystemDatabaseName() + "." + getTableNameString(),
+        getSchemaProto()));
+    tableBuilder.setMeta(getTableProto());
+    tableBuilder.setStats(getTableStatsProto());
+    return tableBuilder.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnDescriptor.java
new file mode 100644
index 0000000..0b3928a
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnDescriptor.java
@@ -0,0 +1,47 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class ColumnDescriptor {
+
+  private final String name;
+  private final Type type;
+  private final int length;
+  
+  public ColumnDescriptor(String columnName, Type columnType, int dataLength) {
+    name = columnName;
+    type = columnType;
+    length = dataLength;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Type getType() {
+    return type;
+  }
+
+  public int getLength() {
+    return length;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnsTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnsTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnsTableDescriptor.java
new file mode 100644
index 0000000..85b8f20
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/ColumnsTableDescriptor.java
@@ -0,0 +1,48 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class ColumnsTableDescriptor extends AbstractTableDescriptor {
+  
+  private static final String TABLENAME = "columns";
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("tid", Type.INT4, 0),
+      new ColumnDescriptor("column_name", Type.TEXT, 0),
+      new ColumnDescriptor("ordinal_position", Type.INT4, 0),
+      new ColumnDescriptor("data_type", Type.TEXT, 0),
+      new ColumnDescriptor("type_length", Type.INT4, 0)
+  };
+  
+  public ColumnsTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/DatabasesTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/DatabasesTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/DatabasesTableDescriptor.java
new file mode 100644
index 0000000..951f6b2
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/DatabasesTableDescriptor.java
@@ -0,0 +1,47 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class DatabasesTableDescriptor extends AbstractTableDescriptor {
+  
+  private static final String TABLENAME = "databases";
+  
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("db_id", Type.INT4, 0),
+      new ColumnDescriptor("db_name", Type.TEXT, 0),
+      new ColumnDescriptor("space_id", Type.INT4, 0)
+  };
+  
+  public DatabasesTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/IndexesTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/IndexesTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/IndexesTableDescriptor.java
new file mode 100644
index 0000000..a079a93
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/IndexesTableDescriptor.java
@@ -0,0 +1,52 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class IndexesTableDescriptor extends AbstractTableDescriptor {
+  
+  private static final String TABLENAME = "indexes";
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("db_id", Type.INT4, 0),
+      new ColumnDescriptor("tid", Type.INT4, 0),
+      new ColumnDescriptor("index_name", Type.TEXT, 0),
+      new ColumnDescriptor("column_name", Type.TEXT, 0),
+      new ColumnDescriptor("data_type", Type.TEXT, 0),
+      new ColumnDescriptor("index_type", Type.TEXT, 0),
+      new ColumnDescriptor("is_unique", Type.BOOLEAN, 0),
+      new ColumnDescriptor("is_clustered", Type.BOOLEAN, 0),
+      new ColumnDescriptor("is_ascending", Type.BOOLEAN, 0)
+  };
+
+  public IndexesTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java
new file mode 100644
index 0000000..de79caa
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java
@@ -0,0 +1,124 @@
+/**
+ * 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.catalog.dictionary;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.tajo.catalog.exception.NoSuchTableException;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.util.TUtil;
+
+public class InfoSchemaMetadataDictionary {
+  
+  private static final String DATABASE_NAME = "information_schema";
+  
+  private static enum DEFINED_TABLES {
+    TABLESPACES,
+    DATABASES,
+    TABLES,
+    COLUMNS,
+    INDEXES,
+    TABLEOPTIONS,
+    TABLESTATS,
+    PARTITIONS,
+    MAX_TABLE;
+  }
+  
+  private List<TableDescriptor> schemaInfoTableDescriptors = new ArrayList<TableDescriptor>(
+      Collections.nCopies(DEFINED_TABLES.MAX_TABLE.ordinal(), (TableDescriptor)null));
+  
+  public InfoSchemaMetadataDictionary() {
+    createSystemTableDescriptors();
+  }
+  
+  private void createSystemTableDescriptors() {
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.TABLESPACES.ordinal(), new TablespacesTableDescriptor(this));
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.DATABASES.ordinal(), new DatabasesTableDescriptor(this));
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.TABLES.ordinal(), new TablesTableDescriptor(this));
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.COLUMNS.ordinal(), new ColumnsTableDescriptor(this));
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.INDEXES.ordinal(), new IndexesTableDescriptor(this));
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.TABLEOPTIONS.ordinal(), new TableOptionsTableDescriptor(this));
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.TABLESTATS.ordinal(), new TableStatsTableDescriptor(this));
+    schemaInfoTableDescriptors.set(DEFINED_TABLES.PARTITIONS.ordinal(), new PartitionsTableDescriptor(this));
+  }
+
+  public boolean isSystemDatabase(String databaseName) {
+    boolean result = false;
+    
+    if (databaseName != null && !databaseName.isEmpty()) {
+      result = DATABASE_NAME.equalsIgnoreCase(databaseName);
+    }
+    
+    return result;
+  }
+  
+  public String getSystemDatabaseName() {
+    return DATABASE_NAME;
+  }
+  
+  public List<String> getAllSystemTables() {
+    List<String> systemTableNames = TUtil.newList();
+    
+    for (TableDescriptor descriptor: schemaInfoTableDescriptors) {
+      systemTableNames.add(descriptor.getTableNameString());
+    }
+    
+    return systemTableNames;
+  }
+  
+  private TableDescriptor getTableDescriptor(String tableName) {
+    TableDescriptor tableDescriptor = null;
+    
+    if (tableName == null || tableName.isEmpty()) {
+      throw new NoSuchTableException(tableName);
+    }
+    
+    tableName = tableName.toUpperCase();
+    for (int idx = 0; idx < schemaInfoTableDescriptors.size(); idx++) {
+      TableDescriptor testDescriptor = schemaInfoTableDescriptors.get(idx);
+      if (testDescriptor.getTableNameString().equalsIgnoreCase(tableName)) {
+        tableDescriptor = testDescriptor;
+        break;
+      }
+    }
+    
+    return tableDescriptor;
+  }
+  
+  public CatalogProtos.TableDescProto getTableDesc(String tableName) {
+    TableDescriptor tableDescriptor;
+    
+    tableDescriptor = getTableDescriptor(tableName);
+    if (tableDescriptor == null) {
+      throw new NoSuchTableException(DATABASE_NAME, tableName);
+    }
+    
+    return tableDescriptor.getTableDescription();
+  }
+  
+  public boolean existTable(String tableName) {
+    return getTableDescriptor(tableName) != null;
+  }
+  
+  protected String getTablePath() {
+    return "SYSTEM";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/PartitionsTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/PartitionsTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/PartitionsTableDescriptor.java
new file mode 100644
index 0000000..d69c93e
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/PartitionsTableDescriptor.java
@@ -0,0 +1,48 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class PartitionsTableDescriptor extends AbstractTableDescriptor {
+  
+  private static final String TABLENAME = "partitions";
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("pid", Type.INT4, 0),
+      new ColumnDescriptor("tid", Type.INT4, 0),
+      new ColumnDescriptor("partition_name", Type.TEXT, 0),
+      new ColumnDescriptor("ordinal_position", Type.INT4, 0),
+      new ColumnDescriptor("path", Type.TEXT, 0)
+  };
+
+  public PartitionsTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableDescriptor.java
new file mode 100644
index 0000000..4bfe4c1
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableDescriptor.java
@@ -0,0 +1,29 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.catalog.proto.CatalogProtos;
+
+public interface TableDescriptor {
+
+  public String getTableNameString();
+  
+  public CatalogProtos.TableDescProto getTableDescription();
+  
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableOptionsTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableOptionsTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableOptionsTableDescriptor.java
new file mode 100644
index 0000000..de08111
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableOptionsTableDescriptor.java
@@ -0,0 +1,46 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class TableOptionsTableDescriptor extends AbstractTableDescriptor {
+  
+  private static final String TABLENAME = "table_options";
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("tid", Type.INT4, 0),
+      new ColumnDescriptor("key_", Type.TEXT, 0),
+      new ColumnDescriptor("value_", Type.TEXT, 0)
+  };
+
+  public TableOptionsTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableStatsTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableStatsTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableStatsTableDescriptor.java
new file mode 100644
index 0000000..39fd364
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TableStatsTableDescriptor.java
@@ -0,0 +1,46 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class TableStatsTableDescriptor extends AbstractTableDescriptor {
+  
+  private static final String TABLENAME = "table_stats";
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("tid", Type.INT4, 0),
+      new ColumnDescriptor("num_rows", Type.INT8, 0),
+      new ColumnDescriptor("num_bytes", Type.INT8, 0)
+  };
+
+  public TableStatsTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablesTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablesTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablesTableDescriptor.java
new file mode 100644
index 0000000..7485248
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablesTableDescriptor.java
@@ -0,0 +1,49 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class TablesTableDescriptor extends AbstractTableDescriptor {
+  
+  private static final String TABLENAME = "tables";
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("tid", Type.INT4, 0),
+      new ColumnDescriptor("db_id", Type.INT4, 0),
+      new ColumnDescriptor("table_name", Type.TEXT, 0),
+      new ColumnDescriptor("table_type", Type.TEXT, 0),
+      new ColumnDescriptor("path", Type.TEXT, 0),
+      new ColumnDescriptor("store_type", Type.TEXT, 0)
+  };
+
+  public TablesTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablespacesTableDescriptor.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablespacesTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablespacesTableDescriptor.java
new file mode 100644
index 0000000..4c21eb6
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/TablespacesTableDescriptor.java
@@ -0,0 +1,48 @@
+/**
+ * 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.catalog.dictionary;
+
+import org.apache.tajo.common.TajoDataTypes.Type;
+
+class TablespacesTableDescriptor extends AbstractTableDescriptor {
+
+  private static final String TABLENAME = "tablespace";
+  
+  private final ColumnDescriptor[] columns = new ColumnDescriptor[] {
+      new ColumnDescriptor("space_id", Type.INT4, 0),
+      new ColumnDescriptor("space_name", Type.TEXT, 0),
+      new ColumnDescriptor("space_handler", Type.TEXT, 0),
+      new ColumnDescriptor("space_uri", Type.TEXT, 0)
+  };
+  
+  public TablespacesTableDescriptor(InfoSchemaMetadataDictionary metadataDictionary) {
+    super(metadataDictionary);
+  }
+
+  @Override
+  public String getTableNameString() {
+    return TABLENAME;
+  }
+
+  @Override
+  protected ColumnDescriptor[] getColumnDescriptors() {
+    return columns;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/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 c7d55eb..a239a92 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
@@ -38,12 +38,14 @@ import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.exception.UnimplementedException;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.Pair;
+import org.apache.tajo.util.TUtil;
 
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.*;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto.AlterTablespaceCommand;
@@ -403,6 +405,37 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
     return tablespaceNames;
   }
+  
+  @Override
+  public List<TablespaceProto> getTablespaces() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+    List<TablespaceProto> tablespaces = TUtil.newList();
+
+    try {
+      String sql = "SELECT SPACE_ID, SPACE_NAME, SPACE_HANDLER, SPACE_URI FROM " + TB_SPACES ;
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+
+      while (resultSet.next()) {
+        TablespaceProto.Builder builder = TablespaceProto.newBuilder();
+        builder.setId(resultSet.getInt("SPACE_ID"));
+        builder.setSpaceName(resultSet.getString("SPACE_NAME"));
+        builder.setHandler(resultSet.getString("SPACE_HANDLER"));
+        builder.setUri(resultSet.getString("SPACE_URI"));
+        
+        tablespaces.add(builder.build());
+      }
+      return tablespaces;
+
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+  }
 
   @Override
   public TablespaceProto getTablespace(String spaceName) throws CatalogException {
@@ -593,6 +626,38 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
     return databaseNames;
   }
+  
+  @Override
+  public List<DatabaseProto> getAllDatabases() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+
+    List<DatabaseProto> databases = new ArrayList<DatabaseProto>();
+
+    try {
+      String sql = "SELECT DB_ID, DB_NAME, SPACE_ID FROM " + TB_DATABASES;
+
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+      while (resultSet.next()) {
+        DatabaseProto.Builder builder = DatabaseProto.newBuilder();
+        
+        builder.setId(resultSet.getInt("DB_ID"));
+        builder.setName(resultSet.getString("DB_NAME"));
+        builder.setSpaceId(resultSet.getInt("SPACE_ID"));
+        
+        databases.add(builder.build());
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+    
+    return databases;
+  }
 
   private static class TableSpaceInternal {
     private final int spaceId;
@@ -1450,6 +1515,163 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
     return tables;
   }
+  
+  @Override
+  public List<TableDescriptorProto> getAllTables() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+
+    List<TableDescriptorProto> tables = new ArrayList<TableDescriptorProto>();
+
+    try {
+      String sql = "SELECT t.TID, t.DB_ID, t." + COL_TABLES_NAME + ", t.TABLE_TYPE, t.PATH, t.STORE_TYPE, " +
+          " s.SPACE_URI FROM " + TB_TABLES + " t, " + TB_DATABASES + " d, " + TB_SPACES +
+          " s WHERE t.DB_ID = d.DB_ID AND d.SPACE_ID = s.SPACE_ID";
+
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+      while (resultSet.next()) {
+        TableDescriptorProto.Builder builder = TableDescriptorProto.newBuilder();
+        
+        builder.setTid(resultSet.getInt("TID"));
+        builder.setDbId(resultSet.getInt("DB_ID"));
+        String tableName = resultSet.getString(COL_TABLES_NAME);
+        builder.setName(tableName);
+        String tableTypeString = resultSet.getString("TABLE_TYPE");
+        TableType tableType = TableType.valueOf(tableTypeString);
+        builder.setTableType(tableTypeString);
+
+        if (tableType == TableType.BASE_TABLE) {
+          builder.setPath(resultSet.getString("SPACE_URI") + "/" + tableName);
+        } else {
+          builder.setPath(resultSet.getString("PATH"));
+        }
+        String storeType = resultSet.getString("STORE_TYPE");
+        if (storeType != null) {
+          storeType = storeType.trim();
+          builder.setStoreType(storeType);
+        }
+        
+        tables.add(builder.build());
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+    
+    return tables;
+  }
+  
+  @Override
+  public List<TableOptionProto> getAllTableOptions() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+
+    List<TableOptionProto> options = new ArrayList<TableOptionProto>();
+
+    try {
+      String sql = "SELECT tid, key_, value_ FROM " + TB_OPTIONS;
+
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+      while (resultSet.next()) {
+        TableOptionProto.Builder builder = TableOptionProto.newBuilder();
+        
+        builder.setTid(resultSet.getInt("TID"));
+        
+        KeyValueProto.Builder keyValueBuilder = KeyValueProto.newBuilder();
+        keyValueBuilder.setKey(resultSet.getString("KEY_"));
+        keyValueBuilder.setValue(resultSet.getString("VALUE_"));
+        builder.setKeyval(keyValueBuilder.build());
+        
+        options.add(builder.build());
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+    
+    return options;
+  }
+  
+  @Override
+  public List<TableStatsProto> getAllTableStats() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+
+    List<TableStatsProto> stats = new ArrayList<TableStatsProto>();
+
+    try {
+      String sql = "SELECT tid, num_rows, num_bytes FROM " + TB_STATISTICS;
+
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+      while (resultSet.next()) {
+        TableStatsProto.Builder builder = TableStatsProto.newBuilder();
+        
+        builder.setTid(resultSet.getInt("TID"));
+        builder.setNumRows(resultSet.getLong("NUM_ROWS"));
+        builder.setNumBytes(resultSet.getLong("NUM_BYTES"));
+        
+        stats.add(builder.build());
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+    
+    return stats;
+  }
+  
+  @Override
+  public List<ColumnProto> getAllColumns() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+
+    List<ColumnProto> columns = new ArrayList<ColumnProto>();
+
+    try {
+      String sql = "SELECT TID, COLUMN_NAME, ORDINAL_POSITION, DATA_TYPE, TYPE_LENGTH FROM " + TB_COLUMNS +
+          " ORDER BY TID ASC, ORDINAL_POSITION ASC";
+
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+      while (resultSet.next()) {
+        ColumnProto.Builder builder = ColumnProto.newBuilder();
+        
+        builder.setTid(resultSet.getInt("TID"));
+        builder.setName(resultSet.getString("COLUMN_NAME"));
+        
+        Type type = getDataType(resultSet.getString("DATA_TYPE").trim());
+        int typeLength = resultSet.getInt("TYPE_LENGTH");
+        
+        if (typeLength > 0) {
+          builder.setDataType(CatalogUtil.newDataTypeWithLen(type, typeLength));
+        } else {
+          builder.setDataType(CatalogUtil.newSimpleDataType(type));
+        }
+        
+        columns.add(builder.build());
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+    
+    return columns;
+  }
 
   private static final String ADD_PARTITION_SQL =
       "INSERT INTO " + TB_PARTTIONS + " (TID, PARTITION_NAME, ORDINAL_POSITION, PATH) VALUES (?,?,?,?)";
@@ -1705,6 +1927,40 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       CatalogUtil.closeQuietly(pstmt);
     }
   }
+  
+  @Override
+  public List<TablePartitionProto> getAllPartitions() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+
+    List<TablePartitionProto> partitions = new ArrayList<TablePartitionProto>();
+
+    try {
+      String sql = "SELECT PID, TID, PARTITION_NAME, ORDINAL_POSITION, PATH FROM " + TB_PARTTIONS;
+
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+      while (resultSet.next()) {
+        TablePartitionProto.Builder builder = TablePartitionProto.newBuilder();
+        
+        builder.setPid(resultSet.getInt("PID"));
+        builder.setTid(resultSet.getInt("TID"));
+        builder.setPartitionName(resultSet.getString("PARTITION_NAME"));
+        builder.setOrdinalPosition(resultSet.getInt("ORDINAL_POSITION"));
+        builder.setPath(resultSet.getString("PATH"));
+        
+        partitions.add(builder.build());
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+    
+    return partitions;
+  }
 
 
   @Override
@@ -1984,6 +2240,45 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
     return protos.toArray(new IndexDescProto[protos.size()]);
   }
+  
+  @Override
+  public List<IndexProto> getAllIndexes() throws CatalogException {
+    Connection conn = null;
+    Statement stmt = null;
+    ResultSet resultSet = null;
+
+    List<IndexProto> indexes = new ArrayList<IndexProto>();
+
+    try {
+      String sql = "SELECT " + COL_DATABASES_PK + ", " + COL_TABLES_PK + ", INDEX_NAME, " +
+        "COLUMN_NAME, DATA_TYPE, INDEX_TYPE, IS_UNIQUE, IS_CLUSTERED, IS_ASCENDING FROM " + TB_INDEXES;
+
+      conn = getConnection();
+      stmt = conn.createStatement();
+      resultSet = stmt.executeQuery(sql);
+      while (resultSet.next()) {
+        IndexProto.Builder builder = IndexProto.newBuilder();
+        
+        builder.setDbId(resultSet.getInt(COL_DATABASES_PK));
+        builder.setTId(resultSet.getInt(COL_TABLES_PK));
+        builder.setIndexName(resultSet.getString("INDEX_NAME"));
+        builder.setColumnName(resultSet.getString("COLUMN_NAME"));
+        builder.setDataType(resultSet.getString("DATA_TYPE"));
+        builder.setIndexType(resultSet.getString("INDEX_TYPE"));
+        builder.setIsUnique(resultSet.getBoolean("IS_UNIQUE"));
+        builder.setIsClustered(resultSet.getBoolean("IS_CLUSTERED"));
+        builder.setIsAscending(resultSet.getBoolean("IS_ASCENDING"));
+        
+        indexes.add(builder.build());
+      }
+    } catch (SQLException se) {
+      throw new CatalogException(se);
+    } finally {
+      CatalogUtil.closeQuietly(stmt, resultSet);
+    }
+    
+    return indexes;
+  }
 
   private void resultToIndexDescProtoBuilder(IndexDescProto.Builder builder,
                                              final ResultSet res) throws SQLException {

http://git-wip-us.apache.org/repos/asf/tajo/blob/021a6f0b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
index 041fc52..ed6fedc 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
@@ -20,9 +20,17 @@ package org.apache.tajo.catalog.store;
 
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TablePartitionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
 
 import java.io.Closeable;
+
 import org.apache.tajo.catalog.exception.CatalogException;
 
 import java.util.Collection;
@@ -41,6 +49,8 @@ public interface CatalogStore extends Closeable {
   void dropTablespace(String spaceName) throws CatalogException;
 
   Collection<String> getAllTablespaceNames() throws CatalogException;
+  
+  List<TablespaceProto> getTablespaces() throws CatalogException;
 
   TablespaceProto getTablespace(String spaceName) throws CatalogException;
 
@@ -54,6 +64,8 @@ public interface CatalogStore extends Closeable {
   void dropDatabase(String databaseName) throws CatalogException;
 
   Collection<String> getAllDatabaseNames() throws CatalogException;
+  
+  List<DatabaseProto> getAllDatabases() throws CatalogException;
 
   /*************************** TABLE ******************************/
   void createTable(CatalogProtos.TableDescProto desc) throws CatalogException;
@@ -67,6 +79,14 @@ public interface CatalogStore extends Closeable {
   List<String> getAllTableNames(String databaseName) throws CatalogException;
 
   void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws CatalogException;
+  
+  List<TableDescriptorProto> getAllTables() throws CatalogException;
+  
+  List<TableOptionProto> getAllTableOptions() throws CatalogException;
+  
+  List<TableStatsProto> getAllTableStats() throws CatalogException;
+  
+  List<ColumnProto> getAllColumns() throws CatalogException;
 
   void updateTableStats(CatalogProtos.UpdateTableStatsProto statsProto) throws CatalogException;
 
@@ -100,6 +120,8 @@ public interface CatalogStore extends Closeable {
   void delPartition(String partitionName) throws CatalogException;
 
   void dropPartitions(String tableName) throws CatalogException;
+  
+  List<TablePartitionProto> getAllPartitions() throws CatalogException;
 
   /**************************** INDEX *******************************/
   void createIndex(IndexDescProto proto) throws CatalogException;
@@ -117,6 +139,8 @@ public interface CatalogStore extends Closeable {
       throws CatalogException;
 
   IndexDescProto [] getIndexes(String databaseName, String tableName) throws CatalogException;
+  
+  List<IndexProto> getAllIndexes() throws CatalogException;
 
   /************************** FUNCTION *****************************/