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/16 05:33:29 UTC

[2/2] tajo git commit: TAJO-1813: Allow external catalog store for unit testing.

TAJO-1813: Allow external catalog store for unit testing.

Closes #749


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

Branch: refs/heads/master
Commit: 7d97284a97a1ca7338406ef70747af4993329f02
Parents: 59fe678
Author: Jihoon Son <ji...@apache.org>
Authored: Wed Sep 16 12:33:10 2015 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Wed Sep 16 12:33:10 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../tajo/catalog/AbstractCatalogClient.java     |  32 +-
 .../org/apache/tajo/catalog/CatalogService.java |  38 +-
 .../org/apache/tajo/catalog/CatalogUtil.java    |   2 +-
 .../java/org/apache/tajo/catalog/Schema.java    |   5 +-
 .../org/apache/tajo/catalog/CatalogServer.java  |  15 +-
 .../tajo/catalog/LinkedMetadataManager.java     |  15 +
 .../apache/tajo/catalog/MiniCatalogServer.java  |  49 --
 .../tajo/catalog/store/AbstractDBStore.java     | 191 +++++--
 .../apache/tajo/catalog/store/CatalogStore.java |   2 +-
 .../src/main/resources/schemas/derby/derby.xml  |  14 +-
 .../main/resources/schemas/mariadb/mariadb.xml  |   8 +-
 .../src/main/resources/schemas/mysql/mysql.xml  |  19 +-
 .../main/resources/schemas/oracle/oracle.xml    |  21 +-
 .../resources/schemas/postgresql/postgresql.xml |  17 +-
 .../apache/tajo/catalog/CatalogTestingUtil.java | 171 ++++++-
 .../apache/tajo/catalog/MiniCatalogServer.java  | 106 ++++
 .../org/apache/tajo/catalog/TestCatalog.java    | 111 +----
 .../TestCatalogAgainstCaseSensitivity.java      | 492 +++++++++++++++++++
 .../tajo/catalog/TestCatalogExceptions.java     | 293 +++++++++++
 .../tajo/catalog/TestLinkedMetadataManager.java |  27 +-
 .../org/apache/tajo/TajoTestingCluster.java     |  16 +-
 .../apache/tajo/exception/ExceptionUtil.java    |   3 +-
 .../apache/tajo/engine/eval/ExprTestBase.java   |   3 +-
 .../tajo/engine/eval/TestEvalTreeUtil.java      |   2 +-
 .../engine/planner/TestJoinOrderAlgorithm.java  |   2 +-
 .../engine/planner/TestLogicalOptimizer.java    |   2 +-
 .../tajo/engine/planner/TestLogicalPlan.java    |   2 +-
 .../tajo/engine/planner/TestLogicalPlanner.java |   3 +-
 .../tajo/engine/planner/TestPlannerUtil.java    |   2 +-
 .../planner/physical/TestExternalSortExec.java  |   3 +-
 .../physical/TestFullOuterHashJoinExec.java     |   3 +-
 .../physical/TestFullOuterMergeJoinExec.java    |   3 +-
 .../planner/physical/TestHashAntiJoinExec.java  |   3 +-
 .../planner/physical/TestHashJoinExec.java      |   3 +-
 .../planner/physical/TestHashSemiJoinExec.java  |   3 +-
 .../physical/TestLeftOuterHashJoinExec.java     |   3 +-
 .../planner/physical/TestMergeJoinExec.java     |   3 +-
 .../planner/physical/TestPhysicalPlanner.java   |   2 +-
 .../physical/TestProgressExternalSortExec.java  |   3 +-
 .../physical/TestRightOuterHashJoinExec.java    |   3 +-
 .../physical/TestRightOuterMergeJoinExec.java   |   3 +-
 .../planner/physical/TestSortIntersectExec.java |   3 +-
 .../tajo/master/TestExecutionBlockCursor.java   |   2 +-
 .../apache/tajo/master/exec/DDLExecutor.java    |  19 +-
 .../java/org/apache/tajo/querymaster/Query.java |   8 +-
 .../apache/tajo/jdbc/TestTajoJdbcNegative.java  |   1 -
 47 files changed, 1413 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 5b85848..5603c34 100644
--- a/CHANGES
+++ b/CHANGES
@@ -539,6 +539,8 @@ Release 0.11.0 - unreleased
   
   TASKS
 
+    TAJO-1813: Allow external catalog store for unit testing. (jihoon)
+
     TAJO-1845: Enforcers in the master plan should be printed in a fixed 
     order. (jihoon)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 f74de82..1dc7a71 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
@@ -21,6 +21,7 @@ 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.TajoConstants;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.CatalogProtocol.CatalogProtocolService.BlockingInterface;
 import org.apache.tajo.catalog.CatalogProtocol.*;
@@ -42,6 +43,7 @@ import java.util.List;
 
 import static org.apache.tajo.catalog.CatalogUtil.buildTableIdentifier;
 import static org.apache.tajo.error.Errors.ResultCode.*;
+import static org.apache.tajo.exception.ExceptionUtil.throwIfError;
 import static org.apache.tajo.exception.ExceptionUtil.throwsIfThisError;
 import static org.apache.tajo.exception.ReturnStateUtil.*;
 
@@ -80,13 +82,15 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final void dropTablespace(final String tablespaceName) throws UndefinedTablespaceException {
+  public final void dropTablespace(final String tablespaceName)
+      throws UndefinedTablespaceException, InsufficientPrivilegeException {
 
     try {
       final BlockingInterface stub = getStub();
       final ReturnState state = stub.dropTablespace(null, ProtoUtil.convertString(tablespaceName));
 
       throwsIfThisError(state, UndefinedTablespaceException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
       ensureOk(state);
 
     } catch (ServiceException e) {
@@ -196,13 +200,15 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final void dropDatabase(final String databaseName) throws UndefinedDatabaseException {
+  public final void dropDatabase(final String databaseName)
+      throws UndefinedDatabaseException, InsufficientPrivilegeException {
 
     try {
       final BlockingInterface stub = getStub();
       final ReturnState state = stub.dropDatabase(null, ProtoUtil.convertString(databaseName));
 
       throwsIfThisError(state, UndefinedDatabaseException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
       ensureOk(state);
 
     } catch (ServiceException e) {
@@ -598,18 +604,18 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final boolean createIndex(final IndexDesc index) {
+  public final void createIndex(final IndexDesc index)
+      throws DuplicateIndexException, UndefinedDatabaseException, UndefinedTableException {
 
     try {
       final BlockingInterface stub = getStub();
 
       final ReturnState state = stub.createIndex(null, index.getProto());
-      if (isSuccess(state)) {
-        return true;
-      } else {
-        // TODO
-        return false;
-      }
+
+      throwsIfThisError(state, DuplicateIndexException.class);
+      throwsIfThisError(state, UndefinedTableException.class);
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -745,7 +751,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public boolean dropIndex(final String dbName, final String indexName) {
+  public void dropIndex(final String dbName, final String indexName)
+      throws UndefinedIndexException, UndefinedDatabaseException {
     try {
       final IndexNameProto request = IndexNameProto.newBuilder()
           .setDatabaseName(dbName)
@@ -753,8 +760,11 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
           .build();
 
       final BlockingInterface stub = getStub();
+      final ReturnState state = stub.dropIndex(null, request);
 
-      return isSuccess(stub.dropIndex(null, request));
+      throwsIfThisError(state, UndefinedIndexException.class);
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 95ff641..b031313 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
@@ -137,7 +137,11 @@ public interface CatalogService {
    * @see TableDesc
    * @throws DuplicateTableException
    */
-  void createTable(TableDesc desc) throws DuplicateTableException, InsufficientPrivilegeException, DuplicateDatabaseException, UndefinedDatabaseException;
+  void createTable(TableDesc desc) throws
+      DuplicateTableException,
+      InsufficientPrivilegeException,
+      DuplicateDatabaseException,
+      UndefinedDatabaseException;
 
 
   /**
@@ -147,13 +151,19 @@ public interface CatalogService {
    * @throws UndefinedTableException
    * @throws InsufficientPrivilegeException
    */
-  void dropTable(String tableName) throws UndefinedTableException, InsufficientPrivilegeException, UndefinedDatabaseException;
+  void dropTable(String tableName) throws
+      UndefinedTableException,
+      InsufficientPrivilegeException,
+      UndefinedDatabaseException;
 
   boolean existsTable(String databaseName, String tableName);
 
   boolean existsTable(String tableName);
 
-  PartitionMethodDesc getPartitionMethod(String databaseName, String tableName) throws UndefinedPartitionMethodException, UndefinedTableException, UndefinedDatabaseException;
+  PartitionMethodDesc getPartitionMethod(String databaseName, String tableName) throws
+      UndefinedPartitionMethodException,
+      UndefinedTableException,
+      UndefinedDatabaseException;
 
   boolean existPartitionMethod(String databaseName, String tableName) throws UndefinedTableException,
       UndefinedDatabaseException;
@@ -167,9 +177,13 @@ public interface CatalogService {
   List<TablePartitionProto> getAllPartitions();
 
   void addPartitions(String databaseName, String tableName, List<PartitionDescProto> partitions
-    , boolean ifNotExists) throws UndefinedTableException, DuplicatePartitionException, UndefinedPartitionMethodException, UndefinedDatabaseException;
+    , boolean ifNotExists) throws
+      UndefinedTableException,
+      DuplicatePartitionException,
+      UndefinedPartitionMethodException,
+      UndefinedDatabaseException;
 
-  boolean createIndex(IndexDesc index);
+  void createIndex(IndexDesc index) throws DuplicateIndexException, UndefinedDatabaseException, UndefinedTableException;
 
   boolean existIndexByName(String databaseName, String indexName);
 
@@ -187,7 +201,7 @@ public interface CatalogService {
 
   Collection<IndexDesc> getAllIndexesByTable(String databaseName, String tableName);
 
-  boolean dropIndex(String databaseName, String indexName);
+  void dropIndex(String databaseName, String indexName) throws UndefinedIndexException, UndefinedDatabaseException;
 
   void createFunction(FunctionDesc funcDesc) throws DuplicateFunctionException;
 
@@ -215,8 +229,16 @@ public interface CatalogService {
    * @see AlterTableDesc
    */
   void alterTable(AlterTableDesc desc)
-      throws DuplicateColumnException, DuplicateTableException, InsufficientPrivilegeException,
-      UndefinedColumnException, UndefinedTableException, DuplicateDatabaseException, DuplicatePartitionException, UndefinedDatabaseException, UndefinedPartitionMethodException, UndefinedPartitionException;
+      throws DuplicateColumnException,
+      DuplicateTableException,
+      InsufficientPrivilegeException,
+      UndefinedColumnException,
+      UndefinedTableException,
+      DuplicateDatabaseException,
+      DuplicatePartitionException,
+      UndefinedDatabaseException,
+      UndefinedPartitionMethodException,
+      UndefinedPartitionException;
 
   void updateTableStats(UpdateTableStatsProto stats) throws UndefinedTableException, InsufficientPrivilegeException;
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
index a6487b4..4e52de1 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@ -198,7 +198,7 @@ public class CatalogUtil {
   public static String [] splitFQTableName(String qualifiedName) {
     String [] splitted = CatalogUtil.splitTableName(qualifiedName);
     if (splitted.length == 1) {
-      throw new IllegalArgumentException("createTable() requires a qualified table name, but it is \""
+      throw new IllegalArgumentException("Table name is expected to be qualified, but was \""
           + qualifiedName + "\".");
     }
     return splitted;

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
index bd64eb0..abd3bca 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
@@ -21,16 +21,14 @@ package org.apache.tajo.catalog;
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableList;
 import com.google.gson.annotations.Expose;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.catalog.SchemaUtil.ColumnVisitor;
-import org.apache.tajo.exception.DuplicateColumnException;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
 import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.exception.DuplicateColumnException;
 import org.apache.tajo.exception.TajoRuntimeException;
 import org.apache.tajo.json.GsonObject;
 import org.apache.tajo.util.StringUtils;
@@ -39,7 +37,6 @@ import org.apache.tajo.util.TUtil;
 import java.util.*;
 
 public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
-  private static final Log LOG = LogFactory.getLog(Schema.class);
 
 	@Expose protected List<Column> fields = null;
 	@Expose protected Map<String, Integer> fieldsByQualifiedName = null;

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 477e49f..dff292f 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
@@ -84,8 +84,8 @@ public class CatalogServer extends AbstractService {
   private Map<String, List<FunctionDescProto>> functions = new ConcurrentHashMap<String,
       List<FunctionDescProto>>();
 
-  private final LinkedMetadataManager linkedMetadataManager;
-  private final InfoSchemaMetadataDictionary metaDictionary = new InfoSchemaMetadataDictionary();
+  protected final LinkedMetadataManager linkedMetadataManager;
+  protected final InfoSchemaMetadataDictionary metaDictionary = new InfoSchemaMetadataDictionary();
 
   // RPC variables
   private BlockingRpcServer rpcServer;
@@ -270,7 +270,7 @@ public class CatalogServer extends AbstractService {
         return StringListResponse.newBuilder()
             .setState(OK)
             .addAllValues(linkedMetadataManager.getTablespaceNames())
-            .addAllValues(store.getAllDatabaseNames())
+            .addAllValues(store.getAllTablespaceNames())
             .build();
 
       } catch (Throwable t) {
@@ -466,13 +466,17 @@ public class CatalogServer extends AbstractService {
       String databaseName = request.getValue();
 
       if (linkedMetadataManager.existsDatabase(databaseName)) {
-        return errInsufficientPrivilege("alter a table in database '" + databaseName + "'");
+        return errInsufficientPrivilege("drop a table in database '" + databaseName + "'");
       }
 
       if (metaDictionary.isSystemDatabase(databaseName)) {
         return errInsufficientPrivilege("drop a table in database '" + databaseName + "'");
       }
 
+      if (databaseName.equals(TajoConstants.DEFAULT_DATABASE_NAME)) {
+        return errInsufficientPrivilege("drop a table in database '" + databaseName + "'");
+      }
+
       wlock.lock();
       try {
         store.dropDatabase(databaseName);
@@ -812,6 +816,7 @@ public class CatalogServer extends AbstractService {
       try {
         return GetColumnsResponse
             .newBuilder()
+            .setState(OK)
             .addAllColumn(store.getAllColumns())
             .build();
 
@@ -1275,7 +1280,7 @@ public class CatalogServer extends AbstractService {
     public IndexListResponse getAllIndexes(RpcController controller, NullProto request) throws ServiceException {
       rlock.lock();
       try {
-        return IndexListResponse.newBuilder().addAllIndexDesc(store.getAllIndexes()).build();
+        return IndexListResponse.newBuilder().setState(OK).addAllIndexDesc(store.getAllIndexes()).build();
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/LinkedMetadataManager.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/LinkedMetadataManager.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/LinkedMetadataManager.java
index 122c1af..6d4af1f 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/LinkedMetadataManager.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/LinkedMetadataManager.java
@@ -108,6 +108,21 @@ public class LinkedMetadataManager {
   }
 
   /**
+   * Check if the tablespace exists.
+   *
+   * @param tablespaceName
+   * @return True if the tablespace exists.
+   */
+  public boolean existsTablespace(String tablespaceName) {
+    for (MetadataProvider provider : providerMap.values()) {
+      if (provider.getTablespaceName().equals(tablespaceName)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
    * Return all database names
    *
    * @return A collection of database names

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/MiniCatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/MiniCatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/MiniCatalogServer.java
deleted file mode 100644
index 70ed314..0000000
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/MiniCatalogServer.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.catalog;
-
-import org.apache.tajo.conf.TajoConf;
-
-import java.io.IOException;
-
-public class MiniCatalogServer {
-  private CatalogServer catalogServers;
-  
-  public MiniCatalogServer(TajoConf conf) throws IOException {
-    catalogServers = new CatalogServer();
-    catalogServers.init(conf);
-    catalogServers.start();
-  }
-
-  public MiniCatalogServer(CatalogServer server) {
-    this.catalogServers = server;
-  }
-  
-  public void shutdown() {
-    this.catalogServers.stop();
-  }
-  
-  public CatalogServer getCatalogServer() {
-    return this.catalogServers;
-  }
-  
-  public CatalogService getCatalog() {
-    return new LocalCatalogWrapper(this.catalogServers);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 f33b621..0b1b120 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
@@ -302,7 +302,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void createTablespace(String spaceName, String spaceUri) {
+  public void createTablespace(String spaceName, String spaceUri) throws DuplicateTablespaceException {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
@@ -310,7 +310,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     try {
       conn = getConnection();
       conn.setAutoCommit(false);
-      String sql = String.format("INSERT INTO %s (SPACE_NAME, SPACE_URI) VALUES (?, ?)", TB_SPACES);
+
+      String sql = String.format("SELECT SPACE_ID FROM %s WHERE SPACE_NAME=(?)", TB_SPACES);
+      pstmt = conn.prepareStatement(sql);
+      pstmt.setString(1, spaceName);
+      res = pstmt.executeQuery();
+      if (res.next()) {
+        throw new DuplicateTablespaceException(spaceName);
+      }
+      res.close();
+      pstmt.close();
+
+      sql = String.format("INSERT INTO %s (SPACE_NAME, SPACE_URI) VALUES (?, ?)", TB_SPACES);
 
       if (LOG.isDebugEnabled()) {
         LOG.debug(sql);
@@ -476,7 +487,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     ResultSet resultSet = null;
 
     try {
-      String sql = "SELECT SPACE_NAME, SPACE_URI FROM " + TB_SPACES + " WHERE SPACE_NAME=?";
+      String sql = "SELECT * FROM " + TB_SPACES + " WHERE SPACE_NAME=?";
       conn = getConnection();
       pstmt = conn.prepareStatement(sql);
       pstmt.setString(1, spaceName);
@@ -486,13 +497,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         throw new UndefinedTablespaceException(spaceName);
       }
 
+      int spaceId = resultSet.getInt(COL_TABLESPACE_PK);
       String retrieveSpaceName = resultSet.getString("SPACE_NAME");
+      String handler = resultSet.getString("SPACE_HANDLER");
       String uri = resultSet.getString("SPACE_URI");
 
-      TablespaceProto.Builder builder = TablespaceProto.newBuilder();
-      builder.setSpaceName(retrieveSpaceName);
-      builder.setUri(uri);
-      return builder.build();
+      return TablespaceProto.newBuilder().
+          setId(spaceId).
+          setSpaceName(retrieveSpaceName).
+          setHandler(handler).
+          setUri(uri).
+          build();
+
 
     } catch (SQLException se) {
       throw new TajoInternalError(se);
@@ -502,9 +518,25 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void alterTablespace(AlterTablespaceProto alterProto) {
+  public void alterTablespace(AlterTablespaceProto alterProto) throws UndefinedTablespaceException {
     Connection conn;
     PreparedStatement pstmt = null;
+    ResultSet res = null;
+
+    try {
+      conn = getConnection();
+      String sql = String.format("SELECT SPACE_NAME FROM %s WHERE SPACE_NAME=?", TB_SPACES);
+      pstmt = conn.prepareStatement(sql);
+      pstmt.setString(1, alterProto.getSpaceName());
+      res = pstmt.executeQuery();
+      if (!res.next()) {
+        throw new UndefinedTablespaceException(alterProto.getSpaceName());
+      }
+    } catch (SQLException e) {
+      throw new TajoInternalError(e);
+    } finally {
+      CatalogUtil.closeQuietly(pstmt, res);
+    }
 
     if (alterProto.getCommandList().size() == 1) {
       AlterTablespaceCommand command = alterProto.getCommand(0);
@@ -513,7 +545,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         try {
           String sql = "UPDATE " + TB_SPACES + " SET SPACE_URI=? WHERE SPACE_NAME=?";
 
-          conn = getConnection();
           pstmt = conn.prepareStatement(sql);
           pstmt.setString(1, setLocation.getUri());
           pstmt.setString(2, alterProto.getSpaceName());
@@ -1002,7 +1033,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto)
       throws UndefinedDatabaseException, DuplicateTableException, DuplicateColumnException,
       DuplicatePartitionException, UndefinedPartitionException, UndefinedColumnException, UndefinedTableException,
-      UndefinedPartitionMethodException {
+      UndefinedPartitionMethodException, AmbiguousTableException {
 
     String[] splitted = CatalogUtil.splitTableName(alterTableDescProto.getTableName());
     if (splitted.length == 1) {
@@ -1019,13 +1050,14 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
     switch (alterTableDescProto.getAlterTableType()) {
     case RENAME_TABLE:
-      if (existTable(databaseName, alterTableDescProto.getNewTableName())) {
+      String simpleNewTableName = CatalogUtil.extractSimpleName(alterTableDescProto.getNewTableName());
+      if (existTable(databaseName, simpleNewTableName)) {
         throw new DuplicateTableException(alterTableDescProto.getNewTableName());
       }
       if (alterTableDescProto.hasNewTablePath()) {
-        renameManagedTable(tableId, alterTableDescProto.getNewTableName(), alterTableDescProto.getNewTablePath());
+        renameManagedTable(tableId, simpleNewTableName, alterTableDescProto.getNewTablePath());
       } else {
-        renameExternalTable(tableId, alterTableDescProto.getNewTableName());
+        renameExternalTable(tableId, simpleNewTableName);
       }
       break;
     case RENAME_COLUMN:
@@ -1189,7 +1221,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   private void renameColumn(final int tableId, final CatalogProtos.AlterColumnProto alterColumnProto)
-      throws UndefinedColumnException {
+      throws UndefinedColumnException, AmbiguousTableException {
 
     final String selectColumnSql =
         "SELECT COLUMN_NAME, DATA_TYPE, TYPE_LENGTH, ORDINAL_POSITION, NESTED_FIELD_NUM from " + TB_COLUMNS +
@@ -1215,10 +1247,19 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       conn = getConnection();
       conn.setAutoCommit(false);
 
+      String tableName = CatalogUtil.extractQualifier(alterColumnProto.getOldColumnName());
+      String simpleOldColumnName = CatalogUtil.extractSimpleName(alterColumnProto.getOldColumnName());
+      String simpleNewColumnName = CatalogUtil.extractSimpleName(alterColumnProto.getNewColumnName());
+
+      if (!tableName.equals(CatalogUtil.extractQualifier(alterColumnProto.getNewColumnName()))) {
+        throw new AmbiguousTableException(
+            tableName + ", " + CatalogUtil.extractQualifier(alterColumnProto.getNewColumnName()));
+      }
+
       //SELECT COLUMN
       pstmt = conn.prepareStatement(selectColumnSql);
       pstmt.setInt(1, tableId);
-      pstmt.setString(2, alterColumnProto.getOldColumnName());
+      pstmt.setString(2, simpleOldColumnName);
       resultSet = pstmt.executeQuery();
 
       CatalogProtos.ColumnProto columnProto = null;
@@ -1242,14 +1283,14 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       //DELETE COLUMN
       pstmt = conn.prepareStatement(deleteColumnNameSql);
       pstmt.setInt(1, tableId);
-      pstmt.setString(2, alterColumnProto.getOldColumnName());
+      pstmt.setString(2, simpleOldColumnName);
       pstmt.executeUpdate();
       pstmt.close();
 
       //INSERT COLUMN
       pstmt = conn.prepareStatement(insertNewColumnSql);
       pstmt.setInt(1, tableId);
-      pstmt.setString(2, CatalogUtil.extractSimpleName(columnProto.getName()));
+      pstmt.setString(2, simpleNewColumnName);
       pstmt.setInt(3, ordinalPosition);
       pstmt.setInt(4, nestedFieldNum);
       pstmt.setString(5, columnProto.getDataType().getType().name());
@@ -1266,7 +1307,14 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
-  private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) {
+  private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) throws DuplicateColumnException {
+
+    Connection conn;
+    PreparedStatement pstmt = null;
+    ResultSet resultSet = null;
+
+    final String existColumnSql =
+        String.format("SELECT COLUMN_NAME FROM %s WHERE TID=? AND COLUMN_NAME=?", TB_COLUMNS);
 
     final String insertNewColumnSql =
         "INSERT INTO " + TB_COLUMNS +
@@ -1274,17 +1322,19 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     final String columnCountSql =
         "SELECT MAX(ORDINAL_POSITION) AS POSITION FROM " + TB_COLUMNS + " WHERE TID = ?";
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(insertNewColumnSql);
-      LOG.debug(columnCountSql);
-    }
-
-    Connection conn;
-    PreparedStatement pstmt = null;
-    ResultSet resultSet = null;
-
     try {
       conn = getConnection();
+      pstmt = conn.prepareStatement(existColumnSql);
+      pstmt.setInt(1, tableId);
+      pstmt.setString(2, CatalogUtil.extractSimpleName(columnProto.getName()));
+      resultSet =  pstmt.executeQuery();
+
+      if (resultSet.next()) {
+        throw new DuplicateColumnException(columnProto.getName());
+      }
+      pstmt.close();
+      resultSet.close();
+
       pstmt = conn.prepareStatement(columnCountSql);
       pstmt.setInt(1 , tableId);
       resultSet =  pstmt.executeQuery();
@@ -1918,9 +1968,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       resultSet = stmt.executeQuery(sql);
       while (resultSet.next()) {
         ColumnProto.Builder builder = ColumnProto.newBuilder();
-        
-        builder.setTid(resultSet.getInt("TID"));
-        builder.setName(resultSet.getString("COLUMN_NAME"));
+
+        int tid = resultSet.getInt("TID");
+        String databaseName = getDatabaseNameOfTable(conn, tid);
+        String tableName = getTableName(conn, tid);
+        builder.setTid(tid);
+        builder.setName(CatalogUtil.buildFQName(databaseName, tableName, resultSet.getString("COLUMN_NAME")));
 
         int nestedFieldNum = resultSet.getInt("NESTED_FIELD_NUM");
 
@@ -2322,7 +2375,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void createIndex(final IndexDescProto proto) throws UndefinedDatabaseException, UndefinedTableException {
+  public void createIndex(final IndexDescProto proto)
+      throws UndefinedDatabaseException, UndefinedTableException, DuplicateIndexException {
     Connection conn = null;
     PreparedStatement pstmt = null;
 
@@ -2330,11 +2384,27 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     final String tableName = CatalogUtil.extractSimpleName(proto.getTableIdentifier().getTableName());
 
     try {
+
       // indexes table
       int databaseId = getDatabaseId(databaseName);
       int tableId = getTableId(databaseId, databaseName, tableName);
 
-      String sql = "INSERT INTO " + TB_INDEXES +
+      String sql = String.format("SELECT INDEX_NAME FROM %s WHERE DB_ID=? AND INDEX_NAME=?", TB_INDEXES);
+
+      conn = getConnection();
+      conn.setAutoCommit(false);
+
+      pstmt = conn.prepareStatement(sql);
+      pstmt.setInt(1, databaseId);
+      pstmt.setString(2, proto.getIndexName());
+      ResultSet res = pstmt.executeQuery();
+      if (res.next()) {
+        throw new DuplicateIndexException(proto.getIndexName());
+      }
+      pstmt.close();
+      res.close();
+
+      sql = "INSERT INTO " + TB_INDEXES +
           " (" + COL_DATABASES_PK + ", " + COL_TABLES_PK + ", INDEX_NAME, " +
           "INDEX_TYPE, PATH, COLUMN_NAMES, DATA_TYPES, ORDERS, NULL_ORDERS, IS_UNIQUE, IS_CLUSTERED) " +
           "VALUES (?,?,?,?,?,?,?,?,?,?,?)";
@@ -2362,12 +2432,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       }
       columnNamesBuilder.deleteCharAt(columnNamesBuilder.length()-1);
       dataTypesBuilder.deleteCharAt(dataTypesBuilder.length()-1);
-      ordersBuilder.deleteCharAt(ordersBuilder.length()-1);
+      ordersBuilder.deleteCharAt(ordersBuilder.length() - 1);
       nullOrdersBuilder.deleteCharAt(nullOrdersBuilder.length()-1);
 
-      conn = getConnection();
-      conn.setAutoCommit(false);
-
       pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, databaseId);
       pstmt.setInt(2, tableId);
@@ -2390,19 +2457,34 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void dropIndex(String databaseName, final String indexName) throws UndefinedDatabaseException {
-    Connection conn = null;
+  public void dropIndex(String databaseName, final String indexName)
+      throws UndefinedDatabaseException, UndefinedIndexException {
+    Connection conn;
     PreparedStatement pstmt = null;
 
     try {
       int databaseId = getDatabaseId(databaseName);
-      String sql = "DELETE FROM " + TB_INDEXES + " WHERE " + COL_DATABASES_PK + "=? AND INDEX_NAME=?";
+
+      String sql = String.format("SELECT INDEX_NAME FROM %s WHERE %s=? AND INDEX_NAME=?", TB_INDEXES, COL_DATABASES_PK);
+
+      conn = getConnection();
+      pstmt = conn.prepareStatement(sql);
+      pstmt.setInt(1, databaseId);
+      pstmt.setString(2, indexName);
+      ResultSet res = pstmt.executeQuery();
+      if (!res.next()) {
+        throw new UndefinedIndexException(CatalogUtil.buildFQName(databaseName, indexName));
+      }
+      pstmt.close();
+      res.close();
+
+      sql = "DELETE FROM " + TB_INDEXES + " WHERE " + COL_DATABASES_PK + "=? AND INDEX_NAME=?";
 
       if (LOG.isDebugEnabled()) {
         LOG.debug(sql);
       }
 
-      conn = getConnection();
+
       pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, databaseId);
       pstmt.setString(2, indexName);
@@ -2432,6 +2514,35 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
+  public static String getDatabaseNameOfTable(Connection conn, int tid) throws SQLException {
+    ResultSet res = null;
+    PreparedStatement pstmt = null;
+
+    try {
+      pstmt =
+          conn.prepareStatement("SELECT " + COL_DATABASES_PK + " FROM " + TB_TABLES + " WHERE " + COL_TABLES_PK + "=?");
+      pstmt.setInt(1, tid);
+      res = pstmt.executeQuery();
+      if (!res.next()) {
+        throw new TajoInternalError("Inconsistent data: no table corresponding to TID " + tid);
+      }
+      int dbId = res.getInt(1);
+      res.close();
+      pstmt.close();
+
+      pstmt = conn.prepareStatement("SELECT DB_NAME FROM " + TB_DATABASES + " WHERE " + COL_DATABASES_PK + "=?");
+      pstmt.setInt(1, dbId);
+      res = pstmt.executeQuery();
+      if (!res.next()) {
+        throw new TajoInternalError("Inconsistent data: no database corresponding to DB_ID " + dbId);
+      }
+
+      return res.getString(1);
+    } finally {
+      CatalogUtil.closeQuietly(pstmt, res);
+    }
+  }
+
   final static String GET_INDEXES_SQL =
       "SELECT * FROM " + TB_INDEXES;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 38d734f..a067a53 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
@@ -71,7 +71,7 @@ public interface CatalogStore extends Closeable {
 
   void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws UndefinedDatabaseException,
       DuplicateTableException, DuplicateColumnException, DuplicatePartitionException, UndefinedPartitionException,
-      UndefinedTableException, UndefinedColumnException, UndefinedPartitionMethodException;
+      UndefinedTableException, UndefinedColumnException, UndefinedPartitionMethodException, AmbiguousTableException;
 
   List<TableDescriptorProto> getAllTables();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 3ce7e2f..c48c078 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
+      * 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)
       * 6 - 2015-07-24: Merge the index branch into the master branch (TAJO-1300)
@@ -28,7 +29,7 @@
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-	<tns:base version="8">
+	<tns:base version="9">
 		<tns:objects>
 			<tns:Object order="0" type="table" name="META">
 				<tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -54,8 +55,8 @@
 				)]]>
 				</tns:sql>
 			</tns:Object>
-			<tns:Object order="3" type="index" name="IDX_DATABASE_DB_ID" dependsOn="DATABASES_">
-				<tns:sql><![CDATA[CREATE UNIQUE INDEX idx_database_db_id on DATABASES_ (DB_ID)]]></tns:sql>
+			<tns:Object order="3" type="index" name="IDX_DATABASE_DB_NAME" dependsOn="DATABASES_">
+				<tns:sql><![CDATA[CREATE UNIQUE INDEX idx_database_db_id on DATABASES_ (DB_NAME)]]></tns:sql>
 			</tns:Object>
 			<tns:Object order="4" type="table" name="TABLES">
 				<tns:sql><![CDATA[
@@ -89,6 +90,11 @@
   				CONSTRAINT COLUMNS_PK PRIMARY KEY (TID, ORDINAL_POSITION)
 				)]]>
 				</tns:sql>
+				<!--
+					The constraint for guaranteeing the unique column name should be added to COLUMNS table. However, it can
+					make this table's schema complicated due to nested fields. Fortunately, columns are always converted
+					into 'schema' which has a functionality of checking duplicated column names.
+				-->
 			</tns:Object>
 			<tns:Object order="8" type="index" name="IDX_FK_COLUMNS_TABLE_NAME" dependsOn="COLUMNS">
 				<tns:sql><![CDATA[CREATE UNIQUE INDEX idx_fk_columns_table_name on COLUMNS (TID, ORDINAL_POSITION)]]></tns:sql>
@@ -99,7 +105,7 @@
   				TID INT NOT NULL REFERENCES TABLES (TID) ON DELETE CASCADE,
   				KEY_ VARCHAR(255) NOT NULL,
   				VALUE_ VARCHAR(4000) NOT NULL,
-  				CONSTRAINT C_OPTIONS_UNIQUE UNIQUE (TID, KEY_, VALUE_)
+  				CONSTRAINT OPTIONS_PKEY PRIMARY KEY (TID, KEY_)
 				)]]>
 				</tns:sql>
 			</tns:Object>

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 e71043e..0bde634 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
+      * 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)
       * 6 - 2015-07-24: Merge the index branch into the master branch (TAJO-1300)
@@ -28,7 +29,7 @@
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-  <tns:base version="8">
+  <tns:base version="9">
     <tns:objects>
       <tns:Object order="0" type="table" name="META">
         <tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -83,6 +84,11 @@
           FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
         )]]>
         </tns:sql>
+        <!--
+		The constraint for guaranteeing the unique column name should be added to COLUMNS table. However, it can
+		make this table's schema complicated due to nested fields. Fortunately, columns are always converted
+		into 'schema' which has a functionality of checking duplicated column names.
+		-->
       </tns:Object>
       <tns:Object order="5" type="table" name="OPTIONS">
         <tns:sql><![CDATA[

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 ccf71b0..1251693 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
+    * 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)
     * 6 - 2015-07-24: Merge the index branch into the master branch (TAJO-1300)
@@ -28,7 +29,7 @@
     * 2 - 2014-06-09: First versioning
     * 1-  Before 2013-03-20
   -->
-  <tns:base version="8">
+  <tns:base version="9">
     <tns:objects>
       <tns:Object order="0" type="table" name="META">
         <tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -83,6 +84,11 @@
           FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
         )]]>
         </tns:sql>
+        <!--
+		The constraint for guaranteeing the unique column name should be added to COLUMNS table. However, it can
+		make this table's schema complicated due to nested fields. Fortunately, columns are always converted
+		into 'schema' which has a functionality of checking duplicated column names.
+		-->
       </tns:Object>
       <tns:Object order="5" type="table" name="OPTIONS">
         <tns:sql><![CDATA[
@@ -117,7 +123,10 @@
         )]]>
         </tns:sql>
       </tns:Object>
-      <tns:Object order="7" type="table" name="STATS">
+      <tns:Object order="7" type="index" name="INDEXES_IDX_TID_COLUMN_NAME" dependsOn="INDEXES">
+        <tns:sql><![CDATA[CREATE INDEX INDEXES_IDX_TID_COLUMN_NAME on INDEXES (DB_ID, TID, COLUMN_NAMES)]]></tns:sql>
+      </tns:Object>
+      <tns:Object order="8" type="table" name="STATS">
         <tns:sql><![CDATA[
         CREATE TABLE STATS (
           TID INT NOT NULL PRIMARY KEY,
@@ -127,7 +136,7 @@
         )]]>
         </tns:sql>
       </tns:Object>
-      <tns:Object order="8" type="table" name="PARTITION_METHODS">
+      <tns:Object order="9" type="table" name="PARTITION_METHODS">
         <tns:sql><![CDATA[
         CREATE TABLE PARTITION_METHODS (
           TID INT NOT NULL PRIMARY KEY,
@@ -138,7 +147,7 @@
         )]]>
         </tns:sql>
       </tns:Object>
-      <tns:Object order="9" type="table" name="PARTITIONS">
+      <tns:Object order="10" type="table" name="PARTITIONS">
         <tns:sql><![CDATA[
         CREATE TABLE PARTITIONS (
           PARTITION_ID INT NOT NULL AUTO_INCREMENT PRIMARY KEY,
@@ -150,7 +159,7 @@
         )]]>
         </tns:sql>
       </tns:Object>
-      <tns:Object order="10" type="table" name="PARTITION_KEYS">
+      <tns:Object order="11" type="table" name="PARTITION_KEYS">
         <tns:sql><![CDATA[
         CREATE TABLE PARTITION_KEYS (
           PARTITION_ID INT NOT NULL,

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 01cf1ae..1c09231 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
+      * 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)
       * 6 - 2015-07-24: Merge the index branch into the master branch (TAJO-1300)
@@ -28,7 +29,7 @@
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-  <tns:base version="8">
+  <tns:base version="9">
     <tns:objects>
   		<tns:Object order="0" type="table" name="meta">
   			<tns:sql><![CDATA[
@@ -95,7 +96,8 @@
 					TABLE_TYPE VARCHAR2(128) NOT NULL,
 					PATH VARCHAR2(4000),
 					STORE_TYPE CHAR(16),
-					FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID)
+					FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID),
+					CONSTRAINT C_TABLE_ID_UNIQ UNIQUE (DB_ID, TABLE_NAME)
 				)]]>
 				</tns:sql>
   		</tns:Object>
@@ -135,6 +137,11 @@
 					FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
 				)]]>
 				</tns:sql>
+			<!--
+				The constraint for guaranteeing the unique column name should be added to COLUMNS table. However, it can
+				make this table's schema complicated due to nested fields. Fortunately, columns are always converted
+				into 'schema' which has a functionality of checking duplicated column names.
+			-->
   		</tns:Object>
   		<tns:Object order="13" type="table" name="OPTIONS">
   			<tns:sql><![CDATA[
@@ -164,12 +171,13 @@
   				IS_CLUSTERED CHAR NOT NULL,
   				IS_ASCENDING CHAR NOT NULL,
   				FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID) ON DELETE CASCADE,
-  				FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+  				FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE,
+  				CONSTRAINT C_INDEXES_UNIQ UNIQUE (DB_ID, INDEX_NAME)
 				)]]>
 				</tns:sql>
   		</tns:Object>
-  		<tns:Object order="15" type="index" name="INDEXES_IDX_TID_COLUMN_NAME" dependsOn="INDEXES">
-  			<tns:sql><![CDATA[CREATE INDEX INDEXES_IDX_TID_COLUMN_NAME on INDEXES (TID, COLUMN_NAME)]]></tns:sql>
+  		<tns:Object order="15" type="index" name="IDX_INDEXES_COLUMNS" dependsOn="INDEXES">
+			<tns:sql><![CDATA[CREATE INDEX idx_col_names ON INDEXES (DB_ID,TID,column_names)]]></tns:sql>
   		</tns:Object>
   		<tns:Object order="16" type="table" name="STATS">
   			<tns:sql><![CDATA[
@@ -199,7 +207,8 @@
 					TID INT NOT NULL,
 					PARTITION_NAME VARCHAR2(767),
 					PATH VARCHAR2(4000),
-					FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+					FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE,
+					CONSTRAINT C_PARTITIONS_UNIQ UNIQUE (TID, PARTITION_NAME)
 				)]]>
 				</tns:sql>
   		</tns:Object>

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/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 8c73ee4..feb6656 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
+      * 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)
       * 6 - 2015-07-24: Merge the index branch into the master branch (TAJO-1300)
@@ -31,7 +32,7 @@ xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-	<tns:base version="8">
+	<tns:base version="9">
 		<tns:objects>
 			<tns:Object name="META" type="table" order="0">
 				<tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>
@@ -72,7 +73,8 @@ xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.
   				TABLE_TYPE VARCHAR(128) NOT NULL,
   				PATH VARCHAR(4096),
   				STORE_TYPE CHAR(16),
-  				FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID)
+  				FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID),
+  				UNIQUE (DB_ID, TABLE_NAME)
 				)]]>
 				</tns:sql>
 			</tns:Object>
@@ -95,6 +97,11 @@ xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.
   				FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
 				)]]>
 				</tns:sql>
+				<!--
+				The constraint for guaranteeing the unique column name should be added to COLUMNS table. However, it can
+				make this table's schema complicated due to nested fields. Fortunately, columns are always converted
+				into 'schema' which has a functionality of checking duplicated column names.
+				-->
 			</tns:Object>
 			<tns:Object name="OPTIONS" type="table" order="9">
 				<tns:sql><![CDATA[
@@ -123,7 +130,8 @@ xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.
   					IS_UNIQUE BOOLEAN NOT NULL,
   					IS_CLUSTERED BOOLEAN NOT NULL,
   					FOREIGN KEY (DB_ID) REFERENCES DATABASES_ (DB_ID) ON DELETE CASCADE,
-  					FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+  					FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE,
+  					UNIQUE (DB_ID, INDEX_NAME)
 				)]]>
 				</tns:sql>
 			</tns:Object>
@@ -162,7 +170,8 @@ xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.
   				PARTITION_NAME VARCHAR(128),
   				PARTITION_VALUE VARCHAR(1024),
   				PATH VARCHAR(4096),
-  				FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE
+  				FOREIGN KEY (TID) REFERENCES TABLES (TID) ON DELETE CASCADE,
+  				UNIQUE (TID, PARTITION_NAME)
 				)]]>
 				</tns:sql>
 			</tns:Object>

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/CatalogTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/CatalogTestingUtil.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/CatalogTestingUtil.java
index 22fc23a..07130cc 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/CatalogTestingUtil.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/CatalogTestingUtil.java
@@ -18,12 +18,32 @@
 
 package org.apache.tajo.catalog;
 
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.annotation.NotNull;
 import org.apache.tajo.annotation.Nullable;
+import org.apache.tajo.catalog.partition.PartitionDesc;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
+import org.apache.tajo.catalog.proto.CatalogProtos.PartitionKeyProto;
+import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.catalog.store.*;
+import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.exception.UnsupportedCatalogStore;
+import org.apache.tajo.util.KeyValueSet;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
 
 public class CatalogTestingUtil {
 
@@ -32,9 +52,36 @@ public class CatalogTestingUtil {
     String driverClassName = System.getProperty(CatalogConstants.STORE_CLASS);
     final boolean useDefaultCatalog = driverClassName == null;
 
-    conf = initializeDerbyStore(conf, testDirPath);
+    if (useDefaultCatalog) {
+      conf = initializeDerbyStore(conf, testDirPath);
+
+    } else {
+      Class clazz;
+      try {
+        clazz = Class.forName(driverClassName);
+      } catch (ClassNotFoundException e) {
+        throw new UnsupportedCatalogStore(driverClassName);
+      }
+      Class<? extends CatalogStore> catalogClass = clazz;
+
+      String catalogURI = System.getProperty(CatalogConstants.CATALOG_URI);
+      if (catalogURI == null) {
+        catalogURI = getCatalogURI(catalogClass, null, testDirPath);
+      }
 
-    // TODO: if useDefaultCatalog is false, use external database as catalog
+      configureCatalogClassAndUri(conf, catalogClass, catalogURI);
+
+      if (requireAuth(catalogClass)) {
+        String connectionId = System.getProperty(CatalogConstants.CONNECTION_ID);
+        String connectionPasswd = System.getProperty(CatalogConstants.CONNECTION_PASSWORD);
+
+        assert connectionId != null;
+        conf.set(CatalogConstants.CONNECTION_ID, connectionId);
+        if (connectionPasswd != null) {
+          conf.set(CatalogConstants.CONNECTION_PASSWORD, connectionPasswd);
+        }
+      }
+    }
     return conf;
   }
 
@@ -90,4 +137,124 @@ public class CatalogTestingUtil {
       throw new UnsupportedCatalogStore(clazz.getCanonicalName());
     }
   }
+
+  public static PartitionDesc buildPartitionDesc(String partitionName) {
+    PartitionDesc partitionDesc = new PartitionDesc();
+    partitionDesc.setPartitionName(partitionName);
+
+    String[] partitionNames = partitionName.split("/");
+
+    List<PartitionKeyProto> partitionKeyList = new ArrayList<>();
+    for(int i = 0; i < partitionNames.length; i++) {
+      String [] splits = partitionNames[i].split("=");
+      String columnName = "", partitionValue = "";
+      if (splits.length == 2) {
+        columnName = splits[0];
+        partitionValue = splits[1];
+      } else if (splits.length == 1) {
+        if (partitionNames[i].charAt(0) == '=') {
+          partitionValue = splits[0];
+        } else {
+          columnName = "";
+        }
+      }
+
+      PartitionKeyProto.Builder builder = PartitionKeyProto.newBuilder();
+      builder.setColumnName(partitionValue);
+      builder.setPartitionValue(columnName);
+      partitionKeyList.add(builder.build());
+    }
+
+    partitionDesc.setPartitionKeys(partitionKeyList);
+
+    partitionDesc.setPath("hdfs://xxx.com/warehouse/" + partitionName);
+    return partitionDesc;
+  }
+
+  public static void prepareBaseData(CatalogService catalog, String testDir) throws Exception {
+    catalog.createTablespace("space1", "hdfs://xxx.com/warehouse");
+    catalog.createTablespace("SpAcE1", "hdfs://xxx.com/warehouse");
+
+    catalog.createDatabase("TestDatabase1", "space1");
+    catalog.createDatabase("testDatabase1", "SpAcE1");
+
+    catalog.createTable(buildTableDesc("TestDatabase1", "TestTable1", testDir));
+    catalog.createTable(buildTableDesc("TestDatabase1", "testTable1", testDir));
+    catalog.createTable(buildPartitionTableDesc("TestDatabase1", "TestPartition1", testDir));
+    catalog.createTable(buildPartitionTableDesc("TestDatabase1", "testPartition1", testDir));
+  }
+
+  public static void cleanupBaseData(CatalogService catalog) throws Exception {
+    catalog.dropTable(CatalogUtil.buildFQName("TestDatabase1", "testPartition1"));
+    catalog.dropTable(CatalogUtil.buildFQName("TestDatabase1", "TestPartition1"));
+    catalog.dropTable(CatalogUtil.buildFQName("TestDatabase1", "TestTable1"));
+    catalog.dropTable(CatalogUtil.buildFQName("TestDatabase1", "testTable1"));
+
+    catalog.dropDatabase("TestDatabase1");
+    catalog.dropDatabase("testDatabase1");
+
+    catalog.dropTablespace("space1");
+    catalog.dropTablespace("SpAcE1");
+  }
+
+  public static TableDesc buildTableDesc(String databaseName, String tableName, String testDir) throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn(CatalogUtil.buildFQName(tableName, "Column"), Type.BLOB);
+    schema.addColumn(CatalogUtil.buildFQName(tableName, "column"), Type.INT4);
+    schema.addColumn(CatalogUtil.buildFQName(tableName, "cOlumn"), Type.INT8);
+    Path path = new Path(testDir + "/" + UUID.randomUUID().toString(), tableName);
+    TableDesc desc = new TableDesc(
+        CatalogUtil.buildFQName(databaseName, tableName),
+        schema,
+        new TableMeta("TEXT", new KeyValueSet()),
+        path.toUri(), true);
+    desc.setStats(buildTableStats());
+    desc.getMeta().setOptions(buildOptions());
+    return desc;
+  }
+
+  public static TableDesc buildPartitionTableDesc(String databaseName, String tableName, String testDir) throws Exception {
+    Schema partSchema = new Schema();
+    partSchema.addColumn(CatalogUtil.buildFQName(tableName, "DaTe"), Type.TEXT);
+    partSchema.addColumn(CatalogUtil.buildFQName(tableName, "dAtE"), Type.TEXT);
+    PartitionMethodDesc partitionMethodDesc =
+        new PartitionMethodDesc(DEFAULT_DATABASE_NAME, tableName,
+            CatalogProtos.PartitionType.COLUMN, "id,name", partSchema);
+    TableDesc desc = buildTableDesc(databaseName, tableName, testDir);
+    desc.setPartitionMethod(partitionMethodDesc);
+    return desc;
+  }
+
+  public static TableStats buildTableStats() {
+    TableStats stats = new TableStats();
+    stats.setAvgRows(1000);
+    stats.setNumBlocks(100);
+    stats.setNumBytes(10000);
+    stats.setNumRows(5000);
+    stats.setNumShuffleOutputs(40);
+    stats.setReadBytes(200);
+    return stats;
+  }
+
+  public static KeyValueSet buildOptions() {
+    KeyValueSet options = new KeyValueSet();
+    options.set("testString", "ThisIsTest");
+    options.setBool("testBool", true);
+    options.setFloat("testFloat", 0.2f);
+    options.setInt("testInt", 60);
+    options.setLong("testLong", 800l);
+    return options;
+  }
+
+  public static IndexDesc buildIndexDescs(String databaseName, String indexName, TableDesc table, Column... cols)
+      throws IOException, URISyntaxException {
+    Preconditions.checkArgument(cols.length > 0);
+    SortSpec[] colSpecs = new SortSpec[cols.length];
+    for (int i = 0; i < cols.length; i++) {
+      colSpecs[i] = new SortSpec(cols[i], true, false);
+    }
+    return new IndexDesc(databaseName, CatalogUtil.extractSimpleName(table.getName()),
+        indexName, new URI("idx_test"), colSpecs,
+        IndexMethod.TWO_LEVEL_BIN_TREE, true, true, table.getSchema());
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/MiniCatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/MiniCatalogServer.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/MiniCatalogServer.java
new file mode 100644
index 0000000..ef398ab
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/MiniCatalogServer.java
@@ -0,0 +1,106 @@
+/**
+ * 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;
+
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.exception.*;
+import org.apache.tajo.util.CommonTestingUtil;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Set;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+
+public class MiniCatalogServer extends CatalogServer {
+
+  private TajoConf conf;
+  private String testDir;
+  private CatalogService catalog;
+
+  public MiniCatalogServer() throws IOException {
+    super();
+    initAndStart();
+  }
+
+  public MiniCatalogServer(Set<MetadataProvider> metadataProviders, Collection<FunctionDesc> sqlFuncs)
+      throws IOException {
+    super(metadataProviders, sqlFuncs);
+    initAndStart();
+  }
+
+  public String getTestDir() {
+    return testDir;
+  }
+
+  private void initAndStart() {
+    try {
+      testDir = CommonTestingUtil.getTestDir().toString();
+      conf = CatalogTestingUtil.configureCatalog(new TajoConf(), testDir);
+      this.init(conf);
+      this.start();
+      catalog = new LocalCatalogWrapper(this);
+      if (!catalog.existTablespace(TajoConstants.DEFAULT_TABLESPACE_NAME)) {
+        catalog.createTablespace(TajoConstants.DEFAULT_TABLESPACE_NAME, testDir.toString());
+      }
+      if (!catalog.existDatabase(DEFAULT_DATABASE_NAME)) {
+        catalog.createDatabase(DEFAULT_DATABASE_NAME, TajoConstants.DEFAULT_TABLESPACE_NAME);
+      }
+
+    } catch (DuplicateDatabaseException
+        | UnsupportedCatalogStore
+        | IOException
+        | DuplicateTablespaceException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    cleanup();
+
+    super.serviceStop();
+    CommonTestingUtil.cleanupTestDir(testDir);
+  }
+
+  public void cleanup() throws UndefinedTableException, InsufficientPrivilegeException, UndefinedDatabaseException,
+      UndefinedTablespaceException {
+    for (String table : catalog.getAllTableNames(DEFAULT_DATABASE_NAME)) {
+      catalog.dropTable(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, table));
+    }
+    for (String database : catalog.getAllDatabaseNames()) {
+      if (!database.equals(TajoConstants.DEFAULT_DATABASE_NAME) &&
+          !linkedMetadataManager.existsDatabase(database) &&
+          !metaDictionary.isSystemDatabase(database)) {
+        catalog.dropDatabase(database);
+      }
+    }
+    for (String tablespace : catalog.getAllTablespaceNames()) {
+      if (!tablespace.equals(TajoConstants.DEFAULT_TABLESPACE_NAME) &&
+          !linkedMetadataManager.existsTablespace(tablespace)) {
+        catalog.dropTablespace(tablespace);
+      }
+    }
+  }
+
+  public CatalogService getCatalogService() {
+    return catalog;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/7d97284a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
index 2fb715c..8720105 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
@@ -27,17 +27,13 @@ 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.catalog.proto.CatalogProtos.IndexMethod;
-import org.apache.tajo.catalog.proto.CatalogProtos.PartitionKeyProto;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.UndefinedFunctionException;
-import org.apache.tajo.exception.UnsupportedCatalogStore;
 import org.apache.tajo.function.Function;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
-import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -57,47 +53,23 @@ import static org.junit.Assert.*;
 public class TestCatalog {
   static final String FieldName1="f1";
 	static final String FieldName2="f2";
-	static final String FieldName3="f3";	
+	static final String FieldName3="f3";
 
 	Schema schema1;
 	
 	static CatalogServer server;
 	static CatalogService catalog;
-  static String testDir;
-
-  public static Pair<TajoConf, String> newTajoConfForCatalogTest() throws IOException, UnsupportedCatalogStore {
-    String testDir = CommonTestingUtil.getTestDir().toString();
-    return new Pair<>(CatalogTestingUtil.configureCatalog(new TajoConf(), testDir), testDir);
-  }
 
 	@BeforeClass
 	public static void setUp() throws Exception {
 
-
-    Path defaultTableSpace = CommonTestingUtil.getTestDir();
-    Pair<TajoConf, String> confAndTestDir = newTajoConfForCatalogTest();
-    testDir = confAndTestDir.getSecond();
-
-	  server = new CatalogServer();
-    server.init(confAndTestDir.getFirst());
-    server.start();
+    server = new MiniCatalogServer();
     catalog = new LocalCatalogWrapper(server);
-    if (!catalog.existTablespace(TajoConstants.DEFAULT_TABLESPACE_NAME)) {
-      catalog.createTablespace(TajoConstants.DEFAULT_TABLESPACE_NAME, defaultTableSpace.toUri().toString());
-    }
-    if (!catalog.existDatabase(DEFAULT_DATABASE_NAME)) {
-      catalog.createDatabase(DEFAULT_DATABASE_NAME, TajoConstants.DEFAULT_TABLESPACE_NAME);
-    }
-
-    for(String table : catalog.getAllTableNames(DEFAULT_DATABASE_NAME)) {
-      catalog.dropTable(table);
-    }
 	}
 
 	@AfterClass
 	public static void tearDown() throws IOException {
 	  server.stop();
-    CommonTestingUtil.cleanupTestDir(testDir);
 	}
 
   @Test
@@ -178,20 +150,6 @@ public class TestCatalog {
     assertTrue(catalog.existDatabase("testCreateAndDropDatabases"));
     catalog.dropDatabase("testCreateAndDropDatabases");
   }
-  
-  @Test
-  public void testCreateAndDropDatabaseWithCharacterSensitivity() throws Exception {
-    assertFalse(catalog.existDatabase("TestDatabase1"));
-    assertFalse(catalog.existDatabase("testDatabase1"));
-    catalog.createDatabase("TestDatabase1", TajoConstants.DEFAULT_TABLESPACE_NAME);
-    assertTrue(catalog.existDatabase("TestDatabase1"));
-    assertFalse(catalog.existDatabase("testDatabase1"));
-    catalog.createDatabase("testDatabase1", TajoConstants.DEFAULT_TABLESPACE_NAME);
-    assertTrue(catalog.existDatabase("TestDatabase1"));
-    assertTrue(catalog.existDatabase("testDatabase1"));
-    catalog.dropDatabase("TestDatabase1");
-    catalog.dropDatabase("testDatabase1");
-  }
 
   @Test
   public void testCreateAndDropManyDatabases() throws Exception {
@@ -269,43 +227,6 @@ public class TestCatalog {
     catalog.dropDatabase("tmpdb2");
     assertFalse(catalog.existDatabase("tmpdb2"));
   }
-  
-  @Test
-  public void testCreateAndDropTableWithCharacterSensivity() throws Exception {
-    String databaseName = "TestDatabase1";
-    catalog.createDatabase(databaseName, TajoConstants.DEFAULT_TABLESPACE_NAME);
-    assertTrue(catalog.existDatabase(databaseName));
-    
-    String tableName = "TestTable1";
-    Schema schema = new Schema();
-    schema.addColumn("Column", Type.BLOB);
-    schema.addColumn("column", Type.INT4);
-    schema.addColumn("cOlumn", Type.INT8);
-    Path path = new Path(CommonTestingUtil.getTestDir(), tableName);
-    TableDesc table = new TableDesc(
-        CatalogUtil.buildFQName(databaseName, tableName),
-        schema,
-        new TableMeta("TEXT", new KeyValueSet()),
-        path.toUri(), true);
-    
-    catalog.createTable(table);
-    
-    tableName = "testTable1";
-    schema = new Schema();
-    schema.addColumn("Column", Type.BLOB);
-    schema.addColumn("column", Type.INT4);
-    schema.addColumn("cOlumn", Type.INT8);
-    path = new Path(CommonTestingUtil.getTestDir(), tableName);
-    table = new TableDesc(
-        CatalogUtil.buildFQName(databaseName, tableName),
-        schema,
-        new TableMeta("TEXT", new KeyValueSet()),
-        path.toUri(), true);
-    
-    catalog.createTable(table);
-    
-    catalog.dropDatabase(databaseName);
-  }
 
   static String dbPrefix = "db_";
   static String tablePrefix = "tb_";
@@ -915,32 +836,13 @@ public class TestCatalog {
     alterTableDesc.setTableName(tableName);
     alterTableDesc.setAlterTableType(AlterTableType.ADD_PARTITION);
 
-    PartitionDesc partitionDesc = new PartitionDesc();
-    partitionDesc.setPartitionName(partitionName);
-
-    String[] partitionNames = partitionName.split("/");
-
-    List<PartitionKeyProto> partitionKeyList = new ArrayList<PartitionKeyProto>();
-    for(int i = 0; i < partitionNames.length; i++) {
-      String columnName = partitionNames[i].split("=")[0];
-      String partitionValue = partitionNames[i].split("=")[1];
-
-      PartitionKeyProto.Builder builder = PartitionKeyProto.newBuilder();
-      builder.setColumnName(partitionValue);
-      builder.setPartitionValue(columnName);
-      partitionKeyList.add(builder.build());
-    }
-
-    partitionDesc.setPartitionKeys(partitionKeyList);
-
-    partitionDesc.setPath("hdfs://xxx.com/warehouse/" + partitionName);
-
-    alterTableDesc.setPartitionDesc(partitionDesc);
+    alterTableDesc.setPartitionDesc(CatalogTestingUtil.buildPartitionDesc(partitionName));
 
     catalog.alterTable(alterTableDesc);
 
-    CatalogProtos.PartitionDescProto resultDesc = catalog.getPartition(DEFAULT_DATABASE_NAME,
-      "addedtable", partitionName);
+    String [] split = CatalogUtil.splitFQTableName(tableName);
+
+    CatalogProtos.PartitionDescProto resultDesc = catalog.getPartition(split[0], split[1], partitionName);
 
     assertNotNull(resultDesc);
     assertEquals(resultDesc.getPartitionName(), partitionName);
@@ -949,7 +851,6 @@ public class TestCatalog {
     assertEquals(resultDesc.getPartitionKeysCount(), 2);
   }
 
-
   private void testDropPartition(String tableName, String partitionName) throws Exception {
     AlterTableDesc alterTableDesc = new AlterTableDesc();
     alterTableDesc.setTableName(tableName);