You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2015/08/13 13:52:45 UTC

[8/8] tajo git commit: TAJO-1748: Refine client APIs to throw specific exceptions.

TAJO-1748: Refine client APIs to throw specific exceptions.

Closes #680


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

Branch: refs/heads/master
Commit: 903151eadc457ed7f29e96ca3acf46e73324d844
Parents: fd6a951
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Aug 13 20:50:20 2015 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Aug 13 20:50:20 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../tajo/catalog/AbstractCatalogClient.java     | 290 ++++++----
 .../src/main/proto/CatalogProtocol.proto        |   1 -
 .../org/apache/tajo/catalog/CatalogService.java | 103 ++--
 .../org/apache/tajo/catalog/CatalogUtil.java    |  14 +-
 .../org/apache/tajo/catalog/FunctionDesc.java   |  12 +-
 .../apache/tajo/catalog/MetadataProvider.java   |   2 +-
 .../java/org/apache/tajo/catalog/Schema.java    |   2 +-
 .../exception/AmbiguousFunctionException.java   |  36 --
 ...biguousPartitionDirectoryExistException.java |  30 -
 .../exception/AmbiguousTableException.java      |  33 --
 .../catalog/exception/CatalogException.java     |  35 --
 .../catalog/exception/CatalogExceptionUtil.java |  45 --
 .../exception/DuplicateColumnException.java     |  34 --
 .../exception/DuplicateDatabaseException.java   |  34 --
 .../exception/DuplicateFunctionException.java   |  32 --
 .../exception/DuplicateIndexException.java      |  34 --
 .../exception/DuplicatePartitionException.java  |  35 --
 .../exception/DuplicateTableException.java      |  35 --
 .../exception/DuplicateTablespaceException.java |  28 -
 .../InsufficientPrivilegeException.java         |  29 -
 .../catalog/exception/InvalidNameException.java |  29 -
 .../exception/MetadataConnectionException.java  |  32 --
 .../exception/UndefinedColumnException.java     |  35 --
 .../exception/UndefinedDatabaseException.java   |  35 --
 .../exception/UndefinedFunctionException.java   |  46 --
 .../exception/UndefinedIndexException.java      |  33 --
 .../exception/UndefinedPartitionException.java  |  35 --
 .../UndefinedPartitionKeyException.java         |  30 -
 .../UndefinedPartitionMethodException.java      |  30 -
 .../exception/UndefinedTableException.java      |  40 --
 .../exception/UndefinedTablespaceException.java |  34 --
 .../src/main/proto/CatalogProtos.proto          |   3 +-
 .../apache/tajo/catalog/TestFunctionDesc.java   |   8 +-
 .../org/apache/tajo/catalog/TestSchema.java     |   1 -
 .../tajo/catalog/store/HiveCatalogStore.java    | 124 ++---
 .../tajo/catalog/store/HiveCatalogUtil.java     |  11 +-
 .../org/apache/tajo/catalog/CatalogServer.java  | 336 ++----------
 .../tajo/catalog/LinkedMetadataManager.java     |   6 +-
 .../InfoSchemaMetadataDictionary.java           |   2 +-
 .../tajo/catalog/store/AbstractDBStore.java     | 543 ++++++++++---------
 .../apache/tajo/catalog/store/CatalogStore.java | 160 +++---
 .../apache/tajo/catalog/store/DerbyStore.java   |  13 +-
 .../apache/tajo/catalog/store/MariaDBStore.java |   6 +-
 .../org/apache/tajo/catalog/store/MemStore.java | 127 +++--
 .../apache/tajo/catalog/store/MySQLStore.java   |   6 +-
 .../apache/tajo/catalog/store/OracleStore.java  |  11 +-
 .../tajo/catalog/store/PostgreSQLStore.java     |  10 +-
 .../catalog/store/XMLCatalogSchemaManager.java  |  59 +-
 .../org/apache/tajo/catalog/TestCatalog.java    |  89 +--
 .../tajo/catalog/TestLinkedMetadataManager.java |  23 +-
 .../java/org/apache/tajo/cli/tsql/TajoCli.java  |  81 ++-
 .../tsql/commands/ConnectDatabaseCommand.java   |  40 +-
 .../cli/tsql/commands/DescTableCommand.java     |   3 +-
 .../cli/tsql/commands/TajoShellCommand.java     |   7 +-
 .../apache/tajo/client/CatalogAdminClient.java  |  49 +-
 .../tajo/client/CatalogAdminClientImpl.java     | 103 ++--
 .../apache/tajo/client/ClientExceptionUtil.java | 106 ----
 .../org/apache/tajo/client/QueryClient.java     |  35 +-
 .../org/apache/tajo/client/QueryClientImpl.java |  88 +--
 .../apache/tajo/client/SessionConnection.java   |  33 +-
 .../org/apache/tajo/client/TajoClientImpl.java  |  68 +--
 .../org/apache/tajo/client/TajoClientUtil.java  |   3 +-
 .../apache/tajo/client/v2/ClientDelegate.java   |   5 +-
 .../tajo/client/v2/LegacyClientDelegate.java    |  80 ++-
 .../org/apache/tajo/client/v2/TajoClient.java   |   5 +-
 .../ClientUnableToConnectException.java         |   2 +-
 .../org/apache/tajo/jdbc/WaitingResultSet.java  |   4 +
 .../org/apache/tajo/storage/RowStoreUtil.java   |   9 +-
 .../exception/AmbiguousFunctionException.java   |  33 ++
 ...biguousPartitionDirectoryExistException.java |  35 ++
 .../tajo/exception/AmbiguousTableException.java |  33 ++
 .../CatalogUpgradeRequiredException.java        |  33 ++
 .../exception/DataTypeMismatchException.java    |  34 ++
 .../tajo/exception/DefaultTajoException.java    |  34 ++
 .../exception/DuplicateColumnException.java     |  34 ++
 .../exception/DuplicateDatabaseException.java   |  33 ++
 .../exception/DuplicateFunctionException.java   |  34 ++
 .../tajo/exception/DuplicateIndexException.java |  34 ++
 .../exception/DuplicatePartitionException.java  |  35 ++
 .../tajo/exception/DuplicateTableException.java |  35 ++
 .../exception/DuplicateTablespaceException.java |  34 ++
 .../apache/tajo/exception/ErrorMessages.java    |  25 +-
 .../apache/tajo/exception/ExceptionUtil.java    | 133 ++++-
 .../InsufficientPrivilegeException.java         |  33 ++
 .../tajo/exception/InternalException.java       |  43 --
 .../exception/InvalidDataTypeException.java     |   7 +-
 .../tajo/exception/InvalidNameException.java    |  34 ++
 .../exception/InvalidOperationException.java    |   3 -
 .../InvalidTablePropertyException.java          |  33 ++
 .../LMDNoMatchedDatatypeException.java          |  34 ++
 .../exception/MetadataConnectionException.java  |  31 ++
 .../MissingTablePropertyException.java          |  33 ++
 .../tajo/exception/NotImplementedException.java |  39 ++
 .../tajo/exception/QueryFailedException.java    |  34 ++
 .../tajo/exception/QueryKilledException.java    |  34 ++
 .../tajo/exception/QueryNotFoundException.java  |  33 ++
 .../apache/tajo/exception/ReturnStateUtil.java  |  26 +-
 .../apache/tajo/exception/SQLExceptionUtil.java |   2 -
 .../apache/tajo/exception/SQLSyntaxError.java   |  35 ++
 .../org/apache/tajo/exception/TajoError.java    |   3 +-
 .../apache/tajo/exception/TajoException.java    |   4 +-
 .../tajo/exception/TajoExceptionInterface.java  |  33 --
 .../tajo/exception/TajoInternalError.java       |   1 -
 .../tajo/exception/TajoRuntimeException.java    |   2 +-
 .../UnavailableTableLocationException.java      |  33 ++
 .../exception/UndefinedColumnException.java     |  36 ++
 .../exception/UndefinedDatabaseException.java   |  36 ++
 .../exception/UndefinedFunctionException.java   |  35 ++
 .../tajo/exception/UndefinedIndexException.java |  38 ++
 .../exception/UndefinedPartitionException.java  |  35 ++
 .../UndefinedPartitionKeyException.java         |  34 ++
 .../UndefinedPartitionMethodException.java      |  35 ++
 .../tajo/exception/UndefinedTableException.java |  39 ++
 .../exception/UndefinedTablespaceException.java |  34 ++
 .../tajo/exception/UnimplementedException.java  |  34 --
 .../exception/UnknownDataFormatException.java   |  36 ++
 .../exception/UnknownDataTypeException.java     |  32 --
 .../exception/UnsupportedDataTypeException.java |  35 ++
 .../java/org/apache/tajo/storage/VTuple.java    |   6 +-
 tajo-common/src/main/proto/errors.proto         |  19 +-
 .../tajo/engine/function/FailFunction.java      |  70 +++
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |   3 +-
 .../tajo/engine/parser/SQLSyntaxError.java      |  51 --
 .../tajo/engine/planner/PhysicalPlanner.java    |   9 +-
 .../engine/planner/PhysicalPlannerImpl.java     |  10 +-
 .../engine/planner/global/GlobalPlanner.java    |  21 +-
 .../org/apache/tajo/master/GlobalEngine.java    |   8 +-
 .../java/org/apache/tajo/master/TajoMaster.java |   7 +-
 .../tajo/master/TajoMasterClientService.java    |  43 +-
 .../tajo/master/exec/CreateTableExecutor.java   |  19 +-
 .../apache/tajo/master/exec/DDLExecutor.java    |  72 ++-
 .../apache/tajo/master/exec/QueryExecutor.java  |   4 +-
 .../java/org/apache/tajo/querymaster/Query.java |  11 +-
 .../tajo/querymaster/QueryMasterTask.java       |   3 +-
 .../apache/tajo/querymaster/Repartitioner.java  |  26 +-
 .../java/org/apache/tajo/querymaster/Stage.java |   5 +-
 .../tajo/webapp/QueryExecutorServlet.java       |   9 +-
 .../org/apache/tajo/worker/TajoQueryEngine.java |   6 +-
 .../tajo/ws/rs/resources/DatabasesResource.java |  60 +-
 .../tajo/ws/rs/resources/TablesResource.java    |  24 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |  19 +-
 .../tajo/cli/tsql/TestTajoCliNegatives.java     | 146 +++++
 .../TestCatalogAdminClientExceptions.java       | 103 ++++
 .../tajo/client/TestQueryClientExceptions.java  | 126 +++++
 .../org/apache/tajo/client/TestTajoClient.java  |  22 +-
 .../tajo/client/TestTajoClientFailures.java     |  79 ---
 .../apache/tajo/client/v2/TestTajoClientV2.java |  50 +-
 .../tajo/engine/eval/TestEvalTreeUtil.java      |   3 +-
 .../tajo/engine/eval/TestSQLExpression.java     |   2 +-
 .../function/TestConditionalExpressions.java    |   2 +-
 .../planner/physical/TestPhysicalPlanner.java   |   5 +-
 .../tajo/engine/query/TestAlterTablespace.java  |   4 +-
 .../tajo/engine/query/TestHBaseTable.java       |  18 +-
 .../apache/tajo/engine/query/TestJoinQuery.java |   2 -
 .../tajo/engine/query/TestTruncateTable.java    |  14 +-
 .../queries/TestTruncateTable/table1_ddl.sql    |   2 +-
 .../testAlterTableAddDropPartition.result       |   2 +-
 .../testQueryFailure.result                     |   0
 .../TestTajoCliNegatives/testQuerySyntax.result |   3 +
 .../org/apache/tajo/plan/ExprAnnotator.java     |  15 +-
 .../org/apache/tajo/plan/ExprNormalizer.java    |   2 +-
 .../tajo/plan/LogicalPlanPreprocessor.java      |   2 +-
 .../org/apache/tajo/plan/LogicalPlanner.java    |  10 +-
 .../org/apache/tajo/plan/TypeDeterminant.java   |  10 +-
 .../plan/expr/AggregationFunctionCallEval.java  |   7 +-
 .../org/apache/tajo/plan/expr/EvalTreeUtil.java |  13 +-
 .../tajo/plan/function/AggFunctionInvoke.java   |   3 +-
 .../function/ClassBasedAggFunctionInvoke.java   |   3 +-
 .../ClassBasedScalarFunctionInvoke.java         |   5 +-
 .../tajo/plan/function/FunctionInvoke.java      |   5 +-
 .../tajo/plan/nameresolver/NameResolver.java    |   6 +-
 .../plan/nameresolver/ResolverByLegacy.java     |   3 +-
 .../tajo/plan/nameresolver/ResolverByRels.java  |   7 +-
 .../nameresolver/ResolverByRelsAndSubExprs.java |   7 +-
 .../nameresolver/ResolverBySubExprsAndRels.java |   7 +-
 .../rewrite/rules/ProjectionPushDownRule.java   |   2 +-
 .../tajo/plan/serder/EvalNodeDeserializer.java  |  10 +-
 .../plan/serder/LogicalNodeDeserializer.java    |   6 +-
 .../tajo/plan/serder/LogicalNodeSerializer.java |   7 +-
 .../org/apache/tajo/plan/util/PlannerUtil.java  |   3 +-
 .../plan/verifier/PreLogicalPlanVerifier.java   |  14 +-
 .../tajo/plan/verifier/VerificationState.java   |   1 -
 .../apache/tajo/storage/AbstractScanner.java    |  20 +-
 .../org/apache/tajo/storage/RowStoreUtil.java   |   8 +-
 .../org/apache/tajo/storage/Tablespace.java     |  14 +-
 .../storage/hbase/AbstractHBaseAppender.java    |  10 +-
 .../tajo/storage/hbase/ColumnMapping.java       |  37 +-
 .../HBaseBinarySerializerDeserializer.java      |   2 +-
 .../apache/tajo/storage/hbase/HBaseScanner.java |   8 +-
 .../tajo/storage/hbase/HBaseTablespace.java     |  79 +--
 .../hbase/HBaseTextSerializerDeserializer.java  |   2 +-
 .../storage/hbase/SortedInsertRewriter.java     |   6 +-
 .../org/apache/tajo/storage/FileTablespace.java |   2 +-
 .../tajo/storage/json/JsonLineDeserializer.java |   4 +-
 .../tajo/storage/json/JsonLineSerializer.java   |   4 +-
 .../org/apache/tajo/storage/orc/ORCScanner.java |   6 +-
 .../tajo/storage/text/DelimitedLineReader.java  |   4 +-
 198 files changed, 3541 insertions(+), 2993 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index c7a8fde..4cdd3bd 100644
--- a/CHANGES
+++ b/CHANGES
@@ -474,6 +474,8 @@ Release 0.11.0 - unreleased
 
   SUB TASKS
 
+    TAJO-1748: Refine client APIs to throw specific exceptions. (hyunsik)
+
     TAJO-1735: Implement MetadataProvider and LinkedMetadataManager. (hyunsik)
 
     TAJO-1723: INSERT INTO statement should allow nested fields as 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/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 3dca859..f74de82 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
@@ -24,15 +24,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.CatalogProtocol.CatalogProtocolService.BlockingInterface;
 import org.apache.tajo.catalog.CatalogProtocol.*;
-import org.apache.tajo.catalog.exception.AmbiguousFunctionException;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
-import org.apache.tajo.catalog.exception.UndefinedPartitionException;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.ReturnStateUtil;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListResponse;
@@ -45,6 +41,8 @@ import java.util.Collection;
 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.throwsIfThisError;
 import static org.apache.tajo.exception.ReturnStateUtil.*;
 
 /**
@@ -62,7 +60,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   abstract BlockingInterface getStub() throws ServiceException;
 
   @Override
-  public final Boolean createTablespace(final String tablespaceName, final String tablespaceUri) {
+  public final void createTablespace(final String tablespaceName, final String tablespaceUri)
+      throws DuplicateTablespaceException {
 
     try {
       final BlockingInterface stub = getStub();
@@ -70,8 +69,10 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
           .setTablespaceName(tablespaceName)
           .setTablespaceUri(tablespaceUri)
           .build();
+      final ReturnState state = stub.createTablespace(null, request);
 
-      return isSuccess(stub.createTablespace(null, request));
+      throwsIfThisError(state, DuplicateTablespaceException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -79,11 +80,14 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final Boolean dropTablespace(final String tablespaceName) {
+  public final void dropTablespace(final String tablespaceName) throws UndefinedTablespaceException {
 
     try {
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.dropTablespace(null, ProtoUtil.convertString(tablespaceName)));
+      final ReturnState state = stub.dropTablespace(null, ProtoUtil.convertString(tablespaceName));
+
+      throwsIfThisError(state, UndefinedTablespaceException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -91,19 +95,18 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final Boolean existTablespace(final String tablespaceName) {
+  public final boolean existTablespace(final String tablespaceName) {
 
     try {
       final BlockingInterface stub = getStub();
+      final ReturnState state = stub.existTablespace(null, ProtoUtil.convertString(tablespaceName));
 
-      ReturnState state = stub.existTablespace(null, ProtoUtil.convertString(tablespaceName));
-
-      if (isThisError(state, ResultCode.UNDEFINED_TABLESPACE)) {
+      if (isThisError(state, UNDEFINED_TABLESPACE)) {
         return false;
       }
-
       ensureOk(state);
       return true;
+
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
@@ -115,8 +118,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final StringListResponse response = stub.getAllTablespaceNames(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getValuesList();
 
     } catch (ServiceException e) {
@@ -130,8 +133,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final GetTablespaceListResponse response = stub.getAllTablespaces(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getTablespaceList();
 
     } catch (ServiceException e) {
@@ -140,13 +143,14 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public TablespaceProto getTablespace(final String tablespaceName) {
+  public TablespaceProto getTablespace(final String tablespaceName) throws UndefinedTablespaceException {
 
     try {
       final BlockingInterface stub = getStub();
       final GetTablespaceResponse response = stub.getTablespace(null, ProtoUtil.convertString(tablespaceName));
-      ensureOk(response.getState());
 
+      throwsIfThisError(response.getState(), UndefinedTablespaceException.class);
+      ensureOk(response.getState());
       return response.getTablespace();
 
     } catch (ServiceException e) {
@@ -155,11 +159,14 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public Boolean alterTablespace(final AlterTablespaceProto alterTablespace) {
+  public void alterTablespace(final AlterTablespaceProto alterTablespace) throws UndefinedTablespaceException {
 
     try {
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.alterTablespace(null, alterTablespace));
+      final ReturnState state = stub.alterTablespace(null, alterTablespace);
+
+      throwsIfThisError(state, UndefinedTablespaceException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -167,7 +174,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final Boolean createDatabase(final String databaseName, @Nullable final String tablespaceName) {
+  public final void createDatabase(final String databaseName, @Nullable final String tablespaceName)
+      throws DuplicateDatabaseException {
 
     try {
 
@@ -177,8 +185,10 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
       if (tablespaceName != null) {
         builder.setTablespaceName(tablespaceName);
       }
+      final ReturnState state = stub.createDatabase(null, builder.build());
 
-      return isSuccess(stub.createDatabase(null, builder.build()));
+      throwsIfThisError(state, DuplicateDatabaseException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -186,32 +196,33 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final Boolean dropDatabase(final String databaseName) {
+  public final void dropDatabase(final String databaseName) throws UndefinedDatabaseException {
 
     try {
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.dropDatabase(null, ProtoUtil.convertString(databaseName)));
+      final ReturnState state = stub.dropDatabase(null, ProtoUtil.convertString(databaseName));
+
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
-      LOG.error(e.getMessage(), e);
-      return Boolean.FALSE;
+      throw new RuntimeException(e);
     }
   }
 
   @Override
-  public final Boolean existDatabase(final String databaseName) {
+  public final boolean existDatabase(final String databaseName) {
 
     try {
       final BlockingInterface stub = getStub();
+      final ReturnState state = stub.existDatabase(null, ProtoUtil.convertString(databaseName));
 
-      ReturnState state = stub.existDatabase(null, ProtoUtil.convertString(databaseName));
-
-      if (isThisError(state, ResultCode.UNDEFINED_DATABASE)) {
+      if (isThisError(state, UNDEFINED_DATABASE)) {
         return false;
       }
-
       ensureOk(state);
       return true;
+
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
@@ -223,8 +234,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final StringListResponse response = stub.getAllDatabaseNames(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getValuesList();
 
     } catch (ServiceException e) {
@@ -238,8 +249,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final GetDatabasesResponse response = stub.getAllDatabases(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getDatabaseList();
 
     } catch (ServiceException e) {
@@ -248,15 +259,16 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final TableDesc getTableDesc(final String databaseName, final String tableName) {
+  public final TableDesc getTableDesc(final String databaseName, final String tableName)
+      throws UndefinedTableException {
 
     try {
       final BlockingInterface stub = getStub();
       final TableIdentifierProto request = buildTableIdentifier(databaseName, tableName);
+      final TableResponse response = stub.getTableDesc(null, request);
 
-      TableResponse response = stub.getTableDesc(null, request);
+      throwsIfThisError(response.getState(), UndefinedTableException.class);
       ensureOk(response.getState());
-
       return CatalogUtil.newTableDesc(response.getTable());
 
     } catch (ServiceException e) {
@@ -265,7 +277,7 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public TableDesc getTableDesc(String qualifiedName) {
+  public TableDesc getTableDesc(String qualifiedName) throws UndefinedTableException {
     String[] splitted = CatalogUtil.splitFQTableName(qualifiedName);
     return getTableDesc(splitted[0], splitted[1]);
   }
@@ -276,8 +288,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final GetTablesResponse response = stub.getAllTables(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getTableList();
 
     } catch (ServiceException e) {
@@ -291,8 +303,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final GetTablePropertiesResponse response = stub.getAllTableProperties(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getPropertiesList();
 
     } catch (ServiceException e) {
@@ -306,8 +318,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final GetTableStatsResponse response = stub.getAllTableStats(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getStatsList();
 
     } catch (ServiceException e) {
@@ -321,8 +333,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final GetColumnsResponse response = stub.getAllColumns(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getColumnList();
 
     } catch (ServiceException e) {
@@ -333,9 +345,12 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   @Override
   public List<IndexDescProto> getAllIndexes() {
     try {
-      CatalogProtocolService.BlockingInterface stub = getStub();
-      IndexListResponse response = stub.getAllIndexes(null, ProtoUtil.NULL_PROTO);
+      final BlockingInterface stub = getStub();
+      final IndexListResponse response = stub.getAllIndexes(null, ProtoUtil.NULL_PROTO);
+
+      ensureOk(response.getState());
       return response.getIndexDescList();
+
     } catch (ServiceException e) {
       LOG.error(e.getMessage(), e);
       return null;
@@ -343,14 +358,19 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final PartitionMethodDesc getPartitionMethod(final String databaseName, final String tableName) {
+  public final PartitionMethodDesc getPartitionMethod(final String databaseName, final String tableName)
+      throws UndefinedPartitionMethodException, UndefinedDatabaseException, UndefinedTableException {
 
     try {
       final BlockingInterface stub = getStub();
       final TableIdentifierProto request = buildTableIdentifier(databaseName, tableName);
       final GetPartitionMethodResponse response = stub.getPartitionMethodByTableName(null, request);
-      ensureOk(response.getState());
 
+
+      throwsIfThisError(response.getState(), UndefinedPartitionMethodException.class);
+      throwsIfThisError(response.getState(), UndefinedDatabaseException.class);
+      throwsIfThisError(response.getState(), UndefinedTableException.class);
+      ensureOk(response.getState());
       return CatalogUtil.newPartitionMethodDesc(response.getPartition());
 
     } catch (ServiceException e) {
@@ -359,11 +379,21 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final boolean existPartitionMethod(final String databaseName, final String tableName) {
+  public final boolean existPartitionMethod(final String databaseName, final String tableName)
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     try {
       final BlockingInterface stub = getStub();
       final TableIdentifierProto request = buildTableIdentifier(databaseName, tableName);
-      return isSuccess(stub.existPartitionMethod(null, request));
+      final ReturnState state = stub.existPartitionMethod(null, request);
+
+      if (isThisError(state, UNDEFINED_PARTITION_METHOD)) {
+        return false;
+      }
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      throwsIfThisError(state, UndefinedTableException.class);
+      ensureOk(state);
+      return true;
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -372,7 +402,10 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
 
   @Override
   public final PartitionDescProto getPartition(final String databaseName, final String tableName,
-                                               final String partitionName) throws UndefinedPartitionException {
+                                               final String partitionName)
+      throws UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionException,
+      UndefinedPartitionMethodException {
+
     try {
       final BlockingInterface stub = getStub();
       final PartitionIdentifierProto request = PartitionIdentifierProto.newBuilder()
@@ -380,15 +413,13 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
           .setTableName(tableName)
           .setPartitionName(partitionName)
           .build();
-
       final GetPartitionDescResponse response = stub.getPartitionByPartitionName(null, request);
 
-      if (ReturnStateUtil.isThisError(response.getState(), ResultCode.UNDEFINED_PARTITION)) {
-        throw new UndefinedPartitionException(partitionName);
-      }
-
+      throwsIfThisError(response.getState(), UndefinedDatabaseException.class);
+      throwsIfThisError(response.getState(), UndefinedTableException.class);
+      throwsIfThisError(response.getState(), UndefinedPartitionMethodException.class);
+      throwsIfThisError(response.getState(), UndefinedPartitionException.class);
       ensureOk(response.getState());
-
       return response.getPartition();
 
     } catch (ServiceException e) {
@@ -404,10 +435,9 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
           .setDatabaseName(databaseName)
           .setTableName(tableName)
           .build();
-
       final GetPartitionsResponse response = stub.getPartitionsByTableName(null, request);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getPartitionList();
 
     } catch (ServiceException e) {
@@ -420,8 +450,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final GetTablePartitionsResponse response = stub.getAllPartitions(null, ProtoUtil.NULL_PROTO);
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getPartList();
 
     } catch (ServiceException e) {
@@ -430,27 +460,33 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public boolean addPartitions(String databaseName, String tableName, List<PartitionDescProto> partitions
-    , boolean ifNotExists) {
+  public void addPartitions(String databaseName, String tableName, List<PartitionDescProto> partitions,
+                               boolean ifNotExists)
+      throws UndefinedDatabaseException, UndefinedTableException, DuplicatePartitionException,
+      UndefinedPartitionMethodException {
+
     try {
       final BlockingInterface stub = getStub();
-      final AddPartitionsProto.Builder builder = AddPartitionsProto.newBuilder();
 
-      TableIdentifierProto.Builder identifier = TableIdentifierProto.newBuilder();
-      identifier.setDatabaseName(databaseName);
-      identifier.setTableName(tableName);
+      final AddPartitionsProto.Builder builder = AddPartitionsProto.newBuilder();
+      final TableIdentifierProto.Builder identifier = TableIdentifierProto.newBuilder()
+          .setDatabaseName(databaseName)
+          .setTableName(tableName);
       builder.setTableIdentifier(identifier.build());
 
       for (PartitionDescProto partition: partitions) {
         builder.addPartitionDesc(partition);
       }
-
       builder.setIfNotExists(ifNotExists);
 
-      return isSuccess(stub.addPartitions(null, builder.build()));
+      ReturnState state = stub.addPartitions(null, builder.build());
+      throwsIfThisError(state, UndefinedTableException.class);
+      throwsIfThisError(state, UndefinedPartitionMethodException.class);
+      throwsIfThisError(state, DuplicatePartitionException.class);
+      ensureOk(state);
+
     } catch (ServiceException e) {
-      LOG.error(e.getMessage(), e);
-      return false;
+      throw new RuntimeException(e);
     }
   }
 
@@ -459,8 +495,8 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final StringListResponse response = stub.getAllTableNames(null, ProtoUtil.convertString(databaseName));
-      ensureOk(response.getState());
 
+      ensureOk(response.getState());
       return response.getValuesList();
 
     } catch (ServiceException e) {
@@ -471,31 +507,38 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   @Override
   public final Collection<FunctionDesc> getFunctions() {
     List<FunctionDesc> list = new ArrayList<FunctionDesc>();
+
     try {
-      GetFunctionsResponse response;
-      BlockingInterface stub = getStub();
-      response = stub.getFunctions(null, NullProto.newBuilder().build());
-      int size = response.getFunctionDescCount();
-      for (int i = 0; i < size; i++) {
+      final BlockingInterface stub = getStub();
+      final GetFunctionsResponse response = stub.getFunctions(null, NullProto.newBuilder().build());
+
+      ensureOk(response.getState());
+      for (int i = 0; i < response.getFunctionDescCount(); i++) {
         try {
           list.add(new FunctionDesc(response.getFunctionDesc(i)));
         } catch (ClassNotFoundException e) {
-          LOG.error(e, e);
-          return list;
+          throw new RuntimeException(e);
         }
       }
       return list;
+
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
   }
 
   @Override
-  public final boolean createTable(final TableDesc desc) {
+  public final void createTable(final TableDesc desc)
+      throws UndefinedDatabaseException, DuplicateTableException, InsufficientPrivilegeException {
+
     try {
       final BlockingInterface stub = getStub();
+      final ReturnState state = stub.createTable(null, desc.getProto());
 
-      return isSuccess(stub.createTable(null, desc.getProto()));
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      throwsIfThisError(state, DuplicateTableException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -503,7 +546,9 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public boolean dropTable(String tableName) {
+  public void dropTable(String tableName)
+      throws UndefinedDatabaseException, UndefinedTableException, InsufficientPrivilegeException {
+
     String[] splitted = CatalogUtil.splitFQTableName(tableName);
     final String databaseName = splitted[0];
     final String simpleName = splitted[1];
@@ -511,8 +556,12 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final TableIdentifierProto request = buildTableIdentifier(databaseName, simpleName);
+      final ReturnState state = stub.dropTable(null, request);
 
-      return isSuccess(stub.dropTable(null, request));
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      throwsIfThisError(state, UndefinedTableException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -529,8 +578,13 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
     try {
       final BlockingInterface stub = getStub();
       final TableIdentifierProto request = buildTableIdentifier(databaseName, tableName);
+      final ReturnState state = stub.existsTable(null, request);
 
-      return isSuccess(stub.existsTable(null, request));
+      if (isThisError(state, UNDEFINED_TABLE)) {
+        return false;
+      }
+      ensureOk(state);
+      return true;
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -708,11 +762,14 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final boolean createFunction(final FunctionDesc funcDesc) {
+  public final void createFunction(final FunctionDesc funcDesc) throws DuplicateFunctionException {
 
     try {
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.createFunction(null, funcDesc.getProto()));
+      final ReturnState state = stub.createFunction(null, funcDesc.getProto());
+
+      throwsIfThisError(state, DuplicateFunctionException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -720,15 +777,19 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final boolean dropFunction(final String signature) {
+  public final void dropFunction(final String signature) throws UndefinedFunctionException,
+      InsufficientPrivilegeException {
 
     try {
       final UnregisterFunctionRequest request = UnregisterFunctionRequest.newBuilder()
           .setSignature(signature)
           .build();
-
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.dropFunction(null, request));
+      final ReturnState state = stub.dropFunction(null, request);
+
+      throwsIfThisError(state, UndefinedFunctionException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -737,7 +798,7 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
 
   @Override
   public final FunctionDesc getFunction(final String signature, DataType... paramTypes)
-      throws AmbiguousFunctionException , UndefinedFunctionException {
+      throws AmbiguousFunctionException, UndefinedFunctionException {
     return getFunction(signature, null, paramTypes);
   }
 
@@ -754,26 +815,18 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
       builder.addParameterTypes(type);
     }
 
-    FunctionResponse response = null;
     try {
       final BlockingInterface stub = getStub();
-      response = stub.getFunctionMeta(null, builder.build());
-    } catch (ServiceException se) {
-      throw new RuntimeException(se);
-    }
-
-    if (isThisError(response.getState(), ResultCode.UNDEFINED_FUNCTION)) {
-      throw new UndefinedFunctionException(signature, paramTypes);
-    } else if (isThisError(response.getState(), ResultCode.AMBIGUOUS_FUNCTION)) {
-      throw new AmbiguousFunctionException(signature, paramTypes);
-    }
+      final FunctionResponse response = stub.getFunctionMeta(null, builder.build());
 
-    ensureOk(response.getState());
-
-    try {
+      throwsIfThisError(response.getState(), UndefinedFunctionException.class);
+      ensureOk(response.getState());
       return new FunctionDesc(response.getFunction());
+
+    } catch (ServiceException se) {
+      throw new RuntimeException(se);
     } catch (ClassNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new TajoInternalError(e);
     }
   }
 
@@ -797,7 +850,13 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
 
     try {
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.containFunction(null, builder.build()));
+      final ReturnState state  = stub.containFunction(null, builder.build());
+
+      if (isThisError(state, UNDEFINED_FUNCTION)) {
+        return false;
+      }
+      ensureOk(state);
+      return true;
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -805,11 +864,25 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public final boolean alterTable(final AlterTableDesc desc) {
+  public final void alterTable(final AlterTableDesc desc) throws DuplicateDatabaseException,
+      DuplicateTableException, DuplicateColumnException, DuplicatePartitionException,
+      UndefinedDatabaseException, UndefinedTableException, UndefinedColumnException, UndefinedPartitionMethodException,
+      InsufficientPrivilegeException, UndefinedPartitionException {
 
     try {
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.alterTable(null, desc.getProto()));
+      final ReturnState state = stub.alterTable(null, desc.getProto());
+
+      throwsIfThisError(state, DuplicateTableException.class);
+      throwsIfThisError(state, DuplicateColumnException.class);
+      throwsIfThisError(state, DuplicatePartitionException.class);
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      throwsIfThisError(state, UndefinedTableException.class);
+      throwsIfThisError(state, UndefinedColumnException.class);
+      throwsIfThisError(state, UndefinedPartitionException.class);
+      throwsIfThisError(state, UndefinedPartitionMethodException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -817,11 +890,16 @@ public abstract class AbstractCatalogClient implements CatalogService, Closeable
   }
 
   @Override
-  public boolean updateTableStats(final UpdateTableStatsProto updateTableStatsProto) {
+  public void updateTableStats(final UpdateTableStatsProto updateTableStatsProto)
+      throws InsufficientPrivilegeException, UndefinedTableException {
 
     try {
       final BlockingInterface stub = getStub();
-      return isSuccess(stub.updateTableStats(null, updateTableStatsProto));
+      final ReturnState state = stub.updateTableStats(null, updateTableStatsProto);
+
+      throwsIfThisError(state, UndefinedTableException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
+      ensureOk(state);
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/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 39201e6..8cc8e2f 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
+++ b/tajo-catalog/tajo-catalog-client/src/main/proto/CatalogProtocol.proto
@@ -119,7 +119,6 @@ service CatalogProtocolService {
 
   rpc getPartitionMethodByTableName(TableIdentifierProto) returns (GetPartitionMethodResponse);
   rpc existPartitionMethod(TableIdentifierProto) returns (ReturnState);
-  rpc dropPartitionMethod(TableIdentifierProto) returns (ReturnState);
 
   rpc getPartitionByPartitionName(PartitionIdentifierProto) returns (GetPartitionDescResponse);
   rpc getPartitionsByTableName(PartitionIdentifierProto) returns (GetPartitionsResponse);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/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 a534805..95ff641 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
@@ -18,12 +18,10 @@
 
 package org.apache.tajo.catalog;
 
-import org.apache.tajo.catalog.exception.AmbiguousFunctionException;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
-import org.apache.tajo.catalog.exception.UndefinedPartitionException;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.exception.*;
 
 import java.util.Collection;
 import java.util.List;
@@ -31,28 +29,21 @@ import java.util.List;
 public interface CatalogService {
 
   /**
-   *
    * @param tableSpaceName Tablespace name to be created
-   * @return True if tablespace is created successfully. Otherwise, it will return FALSE.
    */
-  Boolean createTablespace(String tableSpaceName, String uri);
+  void createTablespace(String tableSpaceName, String uri) throws DuplicateTablespaceException;
 
   /**
-   *
    * @param tableSpaceName Tablespace name to be created
-   * @return True if tablespace is created successfully. Otherwise, it will return FALSE.
    */
-  Boolean existTablespace(String tableSpaceName);
+  boolean existTablespace(String tableSpaceName);
 
   /**
-   *
    * @param tableSpaceName Tablespace name to be created
-   * @return True if tablespace is created successfully. Otherwise, it will return FALSE.
    */
-  Boolean dropTablespace(String tableSpaceName);
+  void dropTablespace(String tableSpaceName) throws UndefinedTablespaceException, InsufficientPrivilegeException;
 
   /**
-   *
    * @return All tablespace names
    */
   Collection<String> getAllTablespaceNames();
@@ -67,35 +58,30 @@ public interface CatalogService {
    * @param tablespaceName Tablespace name to get
    * @return Tablespace description
    */
-  TablespaceProto getTablespace(String tablespaceName);
+  TablespaceProto getTablespace(String tablespaceName) throws UndefinedTablespaceException;
 
   /**
-   *
    * @param alterTablespace AlterTablespace
-   * @return True if update is successfully.
    */
-  Boolean alterTablespace(AlterTablespaceProto alterTablespace);
+  void alterTablespace(AlterTablespaceProto alterTablespace)
+      throws UndefinedTablespaceException, InsufficientPrivilegeException;
 
   /**
-   *
    * @param databaseName Database name to be created
-   * @return True if database is created successfully. Otherwise, it will return FALSE.
    */
-  Boolean createDatabase(String databaseName, String tablespaceName);
+  void createDatabase(String databaseName, String tablespaceName) throws DuplicateDatabaseException;
 
   /**
-   *
    * @param databaseName Database name to be dropped
-   * @return True if database is dropped successfully. Otherwise, it will return FALSE.
    */
-  Boolean dropDatabase(String databaseName);
+  void dropDatabase(String databaseName) throws UndefinedDatabaseException, InsufficientPrivilegeException;
 
   /**
    *
    * @param databaseName Database name to be checked
    * @return True if database exists. Otherwise, it will return FALSE.
    */
-  Boolean existDatabase(String databaseName);
+  boolean existDatabase(String databaseName);
 
   /**
    *
@@ -113,17 +99,17 @@ public interface CatalogService {
    * @param tableName table name
    * @return a table description
    * @see TableDesc
-   * @throws Throwable
+   * @throws UndefinedTableException
    */
-  TableDesc getTableDesc(String databaseName, String tableName);
+  TableDesc getTableDesc(String databaseName, String tableName) throws UndefinedTableException;
 
   /**
    * Get a table description by name
    * @return a table description
    * @see TableDesc
-   * @throws Throwable
+   * @throws UndefinedTableException
    */
-  TableDesc getTableDesc(String qualifiedName);
+  TableDesc getTableDesc(String qualifiedName) throws UndefinedTableException;
 
   /**
    *
@@ -131,24 +117,17 @@ public interface CatalogService {
    */
   Collection<String> getAllTableNames(String databaseName);
   
-  /**
-   * 
-   */
   List<TableDescriptorProto> getAllTables();
   
   List<TableOptionProto> getAllTableOptions();
   
   List<TableStatsProto> getAllTableStats();
   
-  /**
-   * 
-   */
   List<ColumnProto> getAllColumns();
 
   List<IndexDescProto> getAllIndexes();
 
   /**
-   *
    * @return All FunctionDescs
    */
   Collection<FunctionDesc> getFunctions();
@@ -156,36 +135,39 @@ public interface CatalogService {
   /**
    * Add a table via table description
    * @see TableDesc
-   * @throws Throwable
+   * @throws DuplicateTableException
    */
-  boolean createTable(TableDesc desc);
+  void createTable(TableDesc desc) throws DuplicateTableException, InsufficientPrivilegeException, DuplicateDatabaseException, UndefinedDatabaseException;
 
 
   /**
    * Drop a table by name
    *
    * @param tableName table name
-   * @throws Throwable
+   * @throws UndefinedTableException
+   * @throws InsufficientPrivilegeException
    */
-  boolean dropTable(String tableName);
+  void dropTable(String tableName) throws UndefinedTableException, InsufficientPrivilegeException, UndefinedDatabaseException;
 
   boolean existsTable(String databaseName, String tableName);
 
   boolean existsTable(String tableName);
 
-  PartitionMethodDesc getPartitionMethod(String databaseName, String tableName);
+  PartitionMethodDesc getPartitionMethod(String databaseName, String tableName) throws UndefinedPartitionMethodException, UndefinedTableException, UndefinedDatabaseException;
 
-  boolean existPartitionMethod(String databaseName, String tableName);
+  boolean existPartitionMethod(String databaseName, String tableName) throws UndefinedTableException,
+      UndefinedDatabaseException;
 
   PartitionDescProto getPartition(String databaseName, String tableName, String partitionName)
-      throws UndefinedPartitionException;
+      throws UndefinedPartitionException, UndefinedPartitionMethodException, UndefinedDatabaseException,
+      UndefinedTableException;
 
   List<PartitionDescProto> getPartitions(String databaseName, String tableName);
 
   List<TablePartitionProto> getAllPartitions();
 
-  boolean addPartitions(String databaseName, String tableName, List<PartitionDescProto> partitions
-    , boolean ifNotExists);
+  void addPartitions(String databaseName, String tableName, List<PartitionDescProto> partitions
+    , boolean ifNotExists) throws UndefinedTableException, DuplicatePartitionException, UndefinedPartitionMethodException, UndefinedDatabaseException;
 
   boolean createIndex(IndexDesc index);
 
@@ -207,27 +189,34 @@ public interface CatalogService {
 
   boolean dropIndex(String databaseName, String indexName);
 
-  boolean createFunction(FunctionDesc funcDesc);
+  void createFunction(FunctionDesc funcDesc) throws DuplicateFunctionException;
 
-  boolean dropFunction(String signature);
+  void dropFunction(String signature) throws UndefinedFunctionException, InsufficientPrivilegeException;
 
-  FunctionDesc getFunction(String signature, DataType... paramTypes) throws AmbiguousFunctionException, UndefinedFunctionException;
+  FunctionDesc getFunction(String signature, DataType... paramTypes)
+      throws AmbiguousFunctionException, UndefinedFunctionException;
 
-  FunctionDesc getFunction(String signature, FunctionType funcType, DataType... paramTypes) throws AmbiguousFunctionException, UndefinedFunctionException;
+  FunctionDesc getFunction(String signature, FunctionType funcType, DataType... paramTypes)
+      throws AmbiguousFunctionException, UndefinedFunctionException;
 
   boolean containFunction(String signature, DataType... paramTypes);
 
   boolean containFunction(String signature, FunctionType funcType, DataType... paramTypes);
 
   /**
-  * Add a table via table description
-  * @see AlterTableDesc
-  * @throws Throwable
-  */
-  boolean alterTable(AlterTableDesc desc);
-
-  boolean updateTableStats(UpdateTableStatsProto stats);
-
-
+   * Add a table via table description
+   *
+   * @throws DuplicateColumnException
+   * @throws DuplicateTableException
+   * @throws InsufficientPrivilegeException
+   * @throws UndefinedColumnException
+   * @throws UndefinedTableException
+   *
+   * @see AlterTableDesc
+   */
+  void alterTable(AlterTableDesc desc)
+      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/903151ea/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 378bbcb..63228f6 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
@@ -500,9 +500,13 @@ public class CatalogUtil {
           basisTypeOfVarLengthType = givenTypes.get(j).getType();
         } else if (basisTypeOfVarLengthType != null) {
           // If there are more than one type, we choose the most widen type as the basis type.
-          basisTypeOfVarLengthType =
-              getWidestType(CatalogUtil.newSimpleDataTypeArray(basisTypeOfVarLengthType, givenTypes.get(j).getType()))
-              .getType();
+          try {
+            basisTypeOfVarLengthType =
+                getWidestType(CatalogUtil.newSimpleDataTypeArray(basisTypeOfVarLengthType, givenTypes.get(j).getType()))
+                .getType();
+          } catch (UndefinedOperatorException e) {
+            continue;
+          }
         }
       }
 
@@ -676,7 +680,7 @@ public class CatalogUtil {
    * @param types A list of DataTypes
    * @return The widest DataType
    */
-  public static DataType getWidestType(DataType...types) {
+  public static DataType getWidestType(DataType...types) throws UndefinedOperatorException {
     DataType widest = types[0];
     for (int i = 1; i < types.length; i++) {
 
@@ -688,7 +692,7 @@ public class CatalogUtil {
       if (types[i].getType() != Type.NULL_TYPE) {
         Type candidate = TUtil.getFromNestedMap(OPERATION_CASTING_MAP, widest.getType(), types[i].getType());
         if (candidate == null) {
-          throw new TajoRuntimeException(new UndefinedOperatorException(StringUtils.join(types)));
+          throw new UndefinedOperatorException(StringUtils.join(types));
         }
         widest = newSimpleDataType(candidate);
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
index 9f71e8e..a2d4b75 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
@@ -21,14 +21,14 @@ package org.apache.tajo.catalog;
 import com.google.common.base.Objects;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.annotation.NotNull;
-import org.apache.tajo.function.*;
-import org.apache.tajo.json.GsonObject;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.function.*;
+import org.apache.tajo.json.GsonObject;
 
 import java.lang.reflect.Constructor;
 
@@ -85,16 +85,14 @@ public class FunctionDesc implements ProtoObject<FunctionDescProto>, Cloneable,
   }
 
   /**
-   * 
    * @return Function Instance
-   * @throws org.apache.tajo.exception.InternalException
    */
-  public Function newInstance() throws InternalException {
+  public Function newInstance() {
     try {
       Constructor<? extends Function> cons = getLegacyFuncClass().getConstructor();
       return cons.newInstance();
     } catch (Exception ioe) {
-      throw new InternalException("Cannot initiate function " + signature);
+      throw new TajoInternalError("Cannot initiate function " + signature);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/MetadataProvider.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/MetadataProvider.java
index 216d33c..cafe0a1 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/MetadataProvider.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/MetadataProvider.java
@@ -18,7 +18,7 @@
 
 package org.apache.tajo.catalog;
 
-import org.apache.tajo.catalog.exception.UndefinedTablespaceException;
+import org.apache.tajo.exception.UndefinedTablespaceException;
 
 import javax.annotation.Nullable;
 import java.net.URI;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/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 7bffe3a..bd64eb0 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
@@ -24,7 +24,7 @@ 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.catalog.exception.DuplicateColumnException;
+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;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousFunctionException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousFunctionException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousFunctionException.java
deleted file mode 100644
index 56c11e1..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousFunctionException.java
+++ /dev/null
@@ -1,36 +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.exception;
-
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-
-import static org.apache.tajo.function.FunctionUtil.buildSimpleFunctionSignature;
-
-public class AmbiguousFunctionException extends CatalogException {
-
-  public AmbiguousFunctionException(PrimitiveProtos.ReturnState state) {
-    super(state);
-  }
-
-  public AmbiguousFunctionException(String funcName, DataType[] parameters) {
-    super(Errors.ResultCode.AMBIGUOUS_FUNCTION, buildSimpleFunctionSignature(funcName, parameters));
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousPartitionDirectoryExistException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousPartitionDirectoryExistException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousPartitionDirectoryExistException.java
deleted file mode 100644
index 0c99a4f..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousPartitionDirectoryExistException.java
+++ /dev/null
@@ -1,30 +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.exception;
-
-
-import org.apache.tajo.error.Errors.ResultCode;
-
-public class AmbiguousPartitionDirectoryExistException extends CatalogException {
-	private static final long serialVersionUID = 277182608283894931L;
-
-	public AmbiguousPartitionDirectoryExistException(String columnName) {
-		super(ResultCode.AMBIGUOUS_PARTITION_DIRECTORY, columnName);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousTableException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousTableException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousTableException.java
deleted file mode 100644
index 27f5534..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/AmbiguousTableException.java
+++ /dev/null
@@ -1,33 +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.exception;
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class AmbiguousTableException extends CatalogException {
-
-  public AmbiguousTableException(ReturnState state) {
-    super(state);
-  }
-
-  public AmbiguousTableException(String tableName) {
-    super(Errors.ResultCode.AMBIGUOUS_TABLE, tableName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogException.java
deleted file mode 100644
index e19199c..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogException.java
+++ /dev/null
@@ -1,35 +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.exception;
-
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class CatalogException extends TajoException {
-  private static final long serialVersionUID = -26362412527118618L;
-
-  public CatalogException(ReturnState state) {
-    super(state);
-  }
-
-  public CatalogException(ResultCode code, String...args) {
-    super(code, args);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogExceptionUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogExceptionUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogExceptionUtil.java
deleted file mode 100644
index 182a3f5..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/CatalogExceptionUtil.java
+++ /dev/null
@@ -1,45 +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.exception;
-
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.function.FunctionUtil;
-
-import java.util.Collection;
-
-public class CatalogExceptionUtil {
-
-  public static CatalogException makeUndefinedTable(String tbName) {
-    return new CatalogException(ResultCode.UNDEFINED_TABLE, tbName);
-  }
-
-  public static CatalogException makeDuplicateTable(String tbName) {
-    return new CatalogException(ResultCode.DUPLICATE_TABLE, tbName);
-  }
-
-  public static CatalogException makeCatalogUpgrade() {
-    return new CatalogException(ResultCode.CAT_UPGRADE_REQUIRED);
-  }
-
-  public static CatalogException makeMDCNoMatchedDataType(String dataType) {
-    return new CatalogException(ResultCode.MDC_NO_MATCHED_DATATYPE, dataType);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateColumnException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateColumnException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateColumnException.java
deleted file mode 100644
index 88b067a..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateColumnException.java
+++ /dev/null
@@ -1,34 +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.exception;
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class DuplicateColumnException extends CatalogException {
-	private static final long serialVersionUID = 6766228091940775275L;
-
-  public DuplicateColumnException(ReturnState state) {
-    super(state);
-  }
-
-	public DuplicateColumnException(String columnName) {
-		super(Errors.ResultCode.DUPLICATE_COLUMN, columnName);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateDatabaseException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateDatabaseException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateDatabaseException.java
deleted file mode 100644
index 8725d49..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateDatabaseException.java
+++ /dev/null
@@ -1,34 +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.exception;
-
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class DuplicateDatabaseException extends CatalogException {
-
-  public DuplicateDatabaseException(ReturnState state) {
-    super(state);
-  }
-	public DuplicateDatabaseException(String dbName) {
-		super(Errors.ResultCode.DUPLICATE_DATABASE, dbName);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateFunctionException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateFunctionException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateFunctionException.java
deleted file mode 100644
index b5cde13..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateFunctionException.java
+++ /dev/null
@@ -1,32 +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.exception;
-
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.error.Errors;
-
-import static org.apache.tajo.function.FunctionUtil.buildSimpleFunctionSignature;
-
-public class DuplicateFunctionException extends CatalogException {
-	private static final long serialVersionUID = 3224521585413794703L;
-
-	public DuplicateFunctionException(String funcName, DataType[] parameters) {
-		super(Errors.ResultCode.DUPLICATE_FUNCTION, buildSimpleFunctionSignature(funcName, parameters));
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateIndexException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateIndexException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateIndexException.java
deleted file mode 100644
index b9e71c2..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateIndexException.java
+++ /dev/null
@@ -1,34 +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.exception;
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class DuplicateIndexException extends CatalogException {
-  private static final long serialVersionUID = 3705839985189534673L;
-
-  public DuplicateIndexException(ReturnState state) {
-    super(state);
-  }
-
-  public DuplicateIndexException(String indexName) {
-    super(Errors.ResultCode.DUPLICATE_INDEX, indexName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicatePartitionException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicatePartitionException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicatePartitionException.java
deleted file mode 100644
index bbb50b9..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicatePartitionException.java
+++ /dev/null
@@ -1,35 +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.exception;
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class DuplicatePartitionException extends CatalogException {
-  private static final long serialVersionUID = 277182608283894930L;
-
-  public DuplicatePartitionException(ReturnState state) {
-    super(state);
-  }
-
-  public DuplicatePartitionException(String partitionName) {
-    super(Errors.ResultCode.DUPLICATE_PARTITION, partitionName);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTableException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTableException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTableException.java
deleted file mode 100644
index 2111186..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTableException.java
+++ /dev/null
@@ -1,35 +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.exception;
-
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class DuplicateTableException extends CatalogException {
-	private static final long serialVersionUID = -641623770742392865L;
-
-  public DuplicateTableException(ReturnState state) {
-    super(state);
-  }
-
-  public DuplicateTableException(String relName) {
-    super(Errors.ResultCode.DUPLICATE_TABLE, relName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTablespaceException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTablespaceException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTablespaceException.java
deleted file mode 100644
index 2e03e3c..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/DuplicateTablespaceException.java
+++ /dev/null
@@ -1,28 +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.exception;
-
-
-import org.apache.tajo.error.Errors;
-
-public class DuplicateTablespaceException extends CatalogException {
-	public DuplicateTablespaceException(String spaceName) {
-		super(Errors.ResultCode.DUPLICATE_TABLESPACE, spaceName);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InsufficientPrivilegeException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InsufficientPrivilegeException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InsufficientPrivilegeException.java
deleted file mode 100644
index 9bcc866..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InsufficientPrivilegeException.java
+++ /dev/null
@@ -1,29 +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.exception;
-
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.TajoException;
-
-public class InsufficientPrivilegeException extends TajoException {
-
-  public InsufficientPrivilegeException(String towhat) {
-    super(ResultCode.INSUFFICIENT_PRIVILEGE, towhat);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InvalidNameException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InvalidNameException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InvalidNameException.java
deleted file mode 100644
index 1692777..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/InvalidNameException.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Lisensed 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.exception;
-
-
-import org.apache.tajo.error.Errors;
-
-public class InvalidNameException extends CatalogException {
-
-	public InvalidNameException(String databaseName) {
-		super(Errors.ResultCode.INVALID_NAME, databaseName);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/MetadataConnectionException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/MetadataConnectionException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/MetadataConnectionException.java
deleted file mode 100644
index e155bf1..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/MetadataConnectionException.java
+++ /dev/null
@@ -1,32 +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.exception;
-
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.TajoError;
-
-/**
- * Tajo Metadata Connector's connection error
- */
-public class MetadataConnectionException extends TajoError {
-
-  public MetadataConnectionException(String uri, Throwable t) {
-    super(ResultCode.CAT_CANNOT_CONNECT, t, uri, t.getMessage());
-  }
-}