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:44 UTC

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedColumnException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedColumnException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedColumnException.java
deleted file mode 100644
index 39d6130..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedColumnException.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.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class UndefinedColumnException extends CatalogException {
-	private static final long serialVersionUID = 277182608283894937L;
-
-  public UndefinedColumnException(ReturnState state) {
-    super(state);
-  }
-
-	public UndefinedColumnException(String columnName) {
-		super(ResultCode.UNDEFINED_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/UndefinedDatabaseException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedDatabaseException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedDatabaseException.java
deleted file mode 100644
index 75dcd49..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedDatabaseException.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 UndefinedDatabaseException extends CatalogException {
-	private static final long serialVersionUID = 277182608283894937L;
-
-  public UndefinedDatabaseException(ReturnState state) {
-    super(state);
-  }
-
-	public UndefinedDatabaseException(String dbName) {
-		super(Errors.ResultCode.UNDEFINED_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/UndefinedFunctionException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedFunctionException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedFunctionException.java
deleted file mode 100644
index 0ac2a93..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedFunctionException.java
+++ /dev/null
@@ -1,46 +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.ResultCode;
-import org.apache.tajo.function.FunctionUtil;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-import java.util.Collection;
-
-public class UndefinedFunctionException extends CatalogException {
-	private static final long serialVersionUID = 5062193018697228028L;
-
-  public UndefinedFunctionException(ReturnState state) {
-    super(state);
-  }
-
-  public UndefinedFunctionException(String signature) {
-    super(ResultCode.UNDEFINED_FUNCTION, signature);
-  }
-
-  public UndefinedFunctionException(String funcName, TajoDataTypes.DataType[] parameters) {
-    super(ResultCode.UNDEFINED_FUNCTION, FunctionUtil.buildSimpleFunctionSignature(funcName, parameters));
-  }
-
-	public UndefinedFunctionException(String funcName, Collection<TajoDataTypes.DataType> parameters) {
-		super(ResultCode.UNDEFINED_FUNCTION, FunctionUtil.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/UndefinedIndexException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedIndexException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedIndexException.java
deleted file mode 100644
index 036a5cc..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedIndexException.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.ResultCode;
-
-public class UndefinedIndexException extends CatalogException {
-  private static final long serialVersionUID = 3705839985189534673L;
-
-  public UndefinedIndexException(String tableName, String columnName) {
-    super(ResultCode.UNDEFINED_INDEX_FOR_COLUMNS, tableName, columnName);
-  }
-
-  public UndefinedIndexException(String indexName) {
-    super(ResultCode.UNDEFINED_INDEX_NAME, 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/UndefinedPartitionException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionException.java
deleted file mode 100644
index 1033c44..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionException.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.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class UndefinedPartitionException extends CatalogException {
-
-  private static final long serialVersionUID = 277182608283894938L;
-
-  public UndefinedPartitionException(ReturnState state) {
-    super(state);
-  }
-
-  public UndefinedPartitionException(String partitionName) {
-    super(ResultCode.UNDEFINED_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/UndefinedPartitionKeyException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionKeyException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionKeyException.java
deleted file mode 100644
index 5e6d20f..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionKeyException.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 UndefinedPartitionKeyException extends CatalogException {
-
-  private static final long serialVersionUID = 277182608283894939L;
-
-  public UndefinedPartitionKeyException(String partitionKey) {
-    super(ResultCode.UNDEFINED_PARTITION_KEY, partitionKey);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionMethodException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionMethodException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionMethodException.java
deleted file mode 100644
index 5b6eb04..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedPartitionMethodException.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 UndefinedPartitionMethodException extends CatalogException {
-
-  private static final long serialVersionUID = 277182608283894949L;
-
-  public UndefinedPartitionMethodException(String partitionName) {
-    super(ResultCode.UNDEFINED_PARTITION_METHOD, 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/UndefinedTableException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedTableException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedTableException.java
deleted file mode 100644
index bbdb69d..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedTableException.java
+++ /dev/null
@@ -1,40 +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.catalog.CatalogUtil;
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class UndefinedTableException extends CatalogException {
-	private static final long serialVersionUID = 277182608283894937L;
-
-  public UndefinedTableException(ReturnState state) {
-    super(state);
-  }
-
-  public UndefinedTableException(String dbName, String tbName) {
-		super(ResultCode.UNDEFINED_TABLE, CatalogUtil.buildFQName(dbName, tbName));
-  }
-
-	public UndefinedTableException(String relName) {
-		super(ResultCode.UNDEFINED_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/UndefinedTablespaceException.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedTablespaceException.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedTablespaceException.java
deleted file mode 100644
index f3faf6e..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/exception/UndefinedTablespaceException.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 UndefinedTablespaceException extends CatalogException {
-	private static final long serialVersionUID = 277182608283894937L;
-
-  public UndefinedTablespaceException(ReturnState state) {
-    super(state);
-  }
-
-	public UndefinedTablespaceException(String spaceName) {
-		super(Errors.ResultCode.UNDEFINED_TABLESPACE, spaceName);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index a67be97..3c93604 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -153,7 +153,8 @@ message TablePartitionProto {
 }
 
 message GetFunctionsResponse {
-  repeated FunctionDescProto functionDesc = 1;
+  required ReturnState state = 1;
+  repeated FunctionDescProto functionDesc = 2;
 }
 
 message UnregisterFunctionRequest {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
index 4a67ce6..16e85f0 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
@@ -18,13 +18,12 @@
 
 package org.apache.tajo.catalog;
 
-import org.apache.tajo.function.Function;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.function.Function;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.FileUtil;
 import org.junit.Test;
@@ -33,7 +32,6 @@ import java.io.File;
 import java.io.IOException;
 
 import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
 
 public class TestFunctionDesc {
   private static final String TEST_PATH = "target/test-data/TestFunctionDesc";
@@ -95,7 +93,7 @@ public class TestFunctionDesc {
   }
   
   @Test
-  public void testToJson() throws InternalException {
+  public void testToJson() {
 	  FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
         CatalogUtil.newSimpleDataType(Type.INT4),
         CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT8));
@@ -106,7 +104,7 @@ public class TestFunctionDesc {
   }
 
   @Test
-  public void testGetProto() throws InternalException, ClassNotFoundException {
+  public void testGetProto() throws ClassNotFoundException {
     FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
         CatalogUtil.newSimpleDataType(Type.INT4),
         CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT8));

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
index 4c251af..c4092f0 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.catalog;
 
-import org.apache.tajo.catalog.exception.DuplicateColumnException;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
 import org.apache.tajo.common.TajoDataTypes.Type;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
index a0ff5c8..b3e2b8e 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.catalog.store;
 
 import com.google.common.collect.Lists;
-
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,23 +35,13 @@ import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.tajo.BuiltinStorages;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TablePartitionProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.exception.InternalException;
-import org.apache.tajo.exception.TajoInternalError;
-import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
@@ -61,8 +50,6 @@ import org.apache.thrift.TException;
 import java.io.IOException;
 import java.util.*;
 
-import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType;
-
 public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   protected final Log LOG = LogFactory.getLog(getClass());
 
@@ -73,7 +60,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   private final HiveCatalogStoreClientPool clientPool;
   private final String defaultTableSpaceUri;
 
-  public HiveCatalogStore(final Configuration conf) throws InternalException {
+  public HiveCatalogStore(final Configuration conf) {
     if (!(conf instanceof TajoConf)) {
       throw new TajoInternalError("Invalid Configuration Type:" + conf.getClass().getSimpleName());
     }
@@ -83,7 +70,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public boolean existTable(final String databaseName, final String tableName) throws CatalogException {
+  public boolean existTable(final String databaseName, final String tableName) {
     boolean exist = false;
     org.apache.hadoop.hive.ql.metadata.Table table;
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
@@ -109,7 +96,9 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public final CatalogProtos.TableDescProto getTable(String databaseName, final String tableName) throws CatalogException {
+  public final CatalogProtos.TableDescProto getTable(String databaseName, final String tableName)
+      throws UndefinedTableException {
+
     org.apache.hadoop.hive.ql.metadata.Table table = null;
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
     Path path = null;
@@ -283,7 +272,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public final List<String> getAllTableNames(String databaseName) throws CatalogException {
+  public final List<String> getAllTableNames(String databaseName) {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
     try {
@@ -297,28 +286,28 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public void createTablespace(String spaceName, String spaceUri) throws CatalogException {
+  public void createTablespace(String spaceName, String spaceUri) {
     // SKIP
   }
 
   @Override
-  public boolean existTablespace(String spaceName) throws CatalogException {
+  public boolean existTablespace(String spaceName) {
     // SKIP
     return spaceName.equals(TajoConstants.DEFAULT_TABLESPACE_NAME);
   }
 
   @Override
-  public void dropTablespace(String spaceName) throws CatalogException {
+  public void dropTablespace(String spaceName) {
     // SKIP
   }
 
   @Override
-  public Collection<String> getAllTablespaceNames() throws CatalogException {
+  public Collection<String> getAllTablespaceNames() {
     return Lists.newArrayList(TajoConstants.DEFAULT_TABLESPACE_NAME);
   }
 
   @Override
-  public TablespaceProto getTablespace(String spaceName) throws CatalogException {
+  public TablespaceProto getTablespace(String spaceName) {
     if (spaceName.equals(TajoConstants.DEFAULT_TABLESPACE_NAME)) {
       TablespaceProto.Builder builder = TablespaceProto.newBuilder();
       builder.setSpaceName(TajoConstants.DEFAULT_TABLESPACE_NAME);
@@ -330,18 +319,17 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public void updateTableStats(CatalogProtos.UpdateTableStatsProto statsProto) throws
-    CatalogException {
+  public void updateTableStats(CatalogProtos.UpdateTableStatsProto statsProto) {
     // TODO - not implemented yet
   }
 
   @Override
-  public void alterTablespace(CatalogProtos.AlterTablespaceProto alterProto) throws CatalogException {
+  public void alterTablespace(CatalogProtos.AlterTablespaceProto alterProto) {
     throw new UnsupportedException("Tablespace in HiveMeta");
   }
 
   @Override
-  public void createDatabase(String databaseName, String tablespaceName) throws CatalogException {
+  public void createDatabase(String databaseName, String tablespaceName) throws DuplicateDatabaseException {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
     try {
@@ -364,7 +352,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public boolean existDatabase(String databaseName) throws CatalogException {
+  public boolean existDatabase(String databaseName) {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
     try {
@@ -381,7 +369,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public void dropDatabase(String databaseName) throws CatalogException {
+  public void dropDatabase(String databaseName) throws UndefinedDatabaseException {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
     try {
@@ -399,7 +387,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public Collection<String> getAllDatabaseNames() throws CatalogException {
+  public Collection<String> getAllDatabaseNames() {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
     try {
@@ -415,7 +403,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public final void createTable(final CatalogProtos.TableDescProto tableDescProto) throws CatalogException {
+  public final void createTable(final CatalogProtos.TableDescProto tableDescProto) {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
     TableDesc tableDesc = new TableDesc(tableDescProto);
@@ -566,7 +554,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public final void dropTable(String databaseName, final String tableName) throws CatalogException {
+  public final void dropTable(String databaseName, final String tableName) {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
     try {
@@ -584,7 +572,10 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
 
 
   @Override
-  public void alterTable(final CatalogProtos.AlterTableDescProto alterTableDescProto) throws CatalogException {
+  public void alterTable(final CatalogProtos.AlterTableDescProto alterTableDescProto)
+      throws DuplicateTableException, DuplicateColumnException, DuplicatePartitionException,
+      UndefinedPartitionException {
+
     final String[] split = CatalogUtil.splitFQTableName(alterTableDescProto.getTableName());
 
     if (split.length == 1) {
@@ -761,14 +752,9 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public void addPartitionMethod(CatalogProtos.PartitionMethodProto partitionMethodProto) throws CatalogException {
-    // TODO - not implemented yet
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public CatalogProtos.PartitionMethodProto getPartitionMethod(String databaseName, String tableName)
-      throws CatalogException {
+      throws UndefinedTableException, UndefinedPartitionMethodException {
+
     org.apache.hadoop.hive.ql.metadata.Table table;
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
@@ -819,7 +805,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public boolean existPartitionMethod(String databaseName, String tableName) throws CatalogException {
+  public boolean existPartitionMethod(String databaseName, String tableName) throws UndefinedTableException {
     boolean exist = false;
     org.apache.hadoop.hive.ql.metadata.Table table;
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
@@ -848,21 +834,15 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public void dropPartitionMethod(String databaseName, String tableName) throws CatalogException {
-    // TODO - not implemented yet
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public List<CatalogProtos.PartitionDescProto> getPartitions(String databaseName,
-                                                         String tableName) throws CatalogException {
+                                                         String tableName) {
     throw new UnsupportedOperationException();
   }
 
 
   @Override
   public CatalogProtos.PartitionDescProto getPartition(String databaseName, String tableName,
-                                                       String partitionName) throws CatalogException {
+                                                       String partitionName) {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
     CatalogProtos.PartitionDescProto.Builder builder = null;
 
@@ -898,75 +878,75 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public final void addFunction(final FunctionDesc func) throws CatalogException {
+  public final void addFunction(final FunctionDesc func) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public final void deleteFunction(final FunctionDesc func) throws CatalogException {
+  public final void deleteFunction(final FunctionDesc func) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public final void existFunction(final FunctionDesc func) throws CatalogException {
+  public final void existFunction(final FunctionDesc func) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public final List<String> getAllFunctionNames() throws CatalogException {
+  public final List<String> getAllFunctionNames() {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public void createIndex(CatalogProtos.IndexDescProto proto) throws CatalogException {
+  public void createIndex(CatalogProtos.IndexDescProto proto) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public void dropIndex(String databaseName, String indexName) throws CatalogException {
+  public void dropIndex(String databaseName, String indexName) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public CatalogProtos.IndexDescProto getIndexByName(String databaseName, String indexName) throws CatalogException {
+  public CatalogProtos.IndexDescProto getIndexByName(String databaseName, String indexName) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
   public CatalogProtos.IndexDescProto getIndexByColumns(String databaseName, String tableName, String[] columnNames)
-      throws CatalogException {
+      {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public boolean existIndexByName(String databaseName, String indexName) throws CatalogException {
+  public boolean existIndexByName(String databaseName, String indexName) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
   public boolean existIndexByColumns(String databaseName, String tableName, String[] columnNames)
-      throws CatalogException {
+      {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<String> getAllIndexNamesByTable(String databaseName, String tableName) throws CatalogException {
+  public List<String> getAllIndexNamesByTable(String databaseName, String tableName) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public boolean existIndexesByTable(String databaseName, String tableName) throws CatalogException {
+  public boolean existIndexesByTable(String databaseName, String tableName) {
     // TODO - not implemented yet
     throw new UnsupportedOperationException();
   }
@@ -976,7 +956,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
     clientPool.close();
   }
 
-  private boolean existColumn(final String databaseName ,final String tableName , final String columnName) throws CatalogException {
+  private boolean existColumn(final String databaseName ,final String tableName , final String columnName) {
     boolean exist = false;
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
 
@@ -1006,28 +986,28 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public List<ColumnProto> getAllColumns() throws CatalogException {
+  public List<ColumnProto> getAllColumns() {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<DatabaseProto> getAllDatabases() throws CatalogException {
+  public List<DatabaseProto> getAllDatabases() {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<IndexDescProto> getAllIndexes() throws CatalogException {
+  public List<IndexDescProto> getAllIndexes() {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<TablePartitionProto> getAllPartitions() throws CatalogException {
+  public List<TablePartitionProto> getAllPartitions() {
     throw new UnsupportedOperationException();
   }
 
   @Override
   public void addPartitions(String databaseName, String tableName, List<CatalogProtos.PartitionDescProto> partitions
-    , boolean ifNotExists) throws CatalogException {
+    , boolean ifNotExists) {
     HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null;
     List<Partition> addPartitions = TUtil.newList();
     CatalogProtos.PartitionDescProto existingPartition = null;
@@ -1074,22 +1054,22 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
   }
 
   @Override
-  public List<TableOptionProto> getAllTableProperties() throws CatalogException {
+  public List<TableOptionProto> getAllTableProperties() {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<TableStatsProto> getAllTableStats() throws CatalogException {
+  public List<TableStatsProto> getAllTableStats() {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<TableDescriptorProto> getAllTables() throws CatalogException {
+  public List<TableDescriptorProto> getAllTables() {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<TablespaceProto> getTablespaces() throws CatalogException {
+  public List<TablespaceProto> getTablespaces() {
     throw new UnsupportedOperationException();
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
index 39b3cb9..7779b4f 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
@@ -26,18 +26,19 @@ import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.tajo.BuiltinStorages;
-import org.apache.tajo.catalog.exception.CatalogException;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.exception.ExceptionUtil;
+import org.apache.tajo.exception.LMDNoMatchedDatatypeException;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoRuntimeException;
 import org.apache.thrift.TException;
 import parquet.hadoop.mapred.DeprecatedParquetOutputFormat;
 
-import static org.apache.tajo.catalog.exception.CatalogExceptionUtil.makeMDCNoMatchedDataType;
 import static org.apache.tajo.exception.ExceptionUtil.makeNotSupported;
 
 public class HiveCatalogUtil {
-  public static void validateSchema(Table tblSchema) throws CatalogException {
+  public static void validateSchema(Table tblSchema) {
     for (FieldSchema fieldSchema : tblSchema.getCols()) {
       String fieldType = fieldSchema.getType();
       if (fieldType.equalsIgnoreCase("ARRAY") || fieldType.equalsIgnoreCase("STRUCT")
@@ -47,7 +48,7 @@ public class HiveCatalogUtil {
     }
   }
 
-  public static TajoDataTypes.Type getTajoFieldType(String dataType) throws CatalogException {
+  public static TajoDataTypes.Type getTajoFieldType(String dataType) {
     Preconditions.checkNotNull(dataType);
 
     if(dataType.equalsIgnoreCase(serdeConstants.INT_TYPE_NAME)) {
@@ -73,7 +74,7 @@ public class HiveCatalogUtil {
     } else if(dataType.equalsIgnoreCase(serdeConstants.DATE_TYPE_NAME)) {
       return TajoDataTypes.Type.DATE;
     } else {
-      throw makeMDCNoMatchedDataType(dataType);
+      throw new TajoRuntimeException(new LMDNoMatchedDatatypeException(dataType));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/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 a7e8348..6d6d96f 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
@@ -34,10 +34,7 @@ import org.apache.tajo.TajoConstants;
 import org.apache.tajo.annotation.ThreadSafe;
 import org.apache.tajo.catalog.CatalogProtocol.*;
 import org.apache.tajo.catalog.dictionary.InfoSchemaMetadataDictionary;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.catalog.exception.DuplicateDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
-import org.apache.tajo.catalog.exception.UndefinedTablespaceException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.catalog.store.CatalogStore;
 import org.apache.tajo.catalog.store.DerbyStore;
@@ -45,9 +42,6 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.ReturnStateUtil;
-import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.rpc.BlockingRpcServer;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
@@ -212,9 +206,6 @@ public class CatalogServer extends AbstractService {
 
       wlock.lock();
       try {
-        if (store.existTablespace(tablespaceName)) {
-          throw new DuplicateDatabaseException(tablespaceName);
-        }
 
         store.createTablespace(tablespaceName, uri);
         LOG.info(String.format("tablespace \"%s\" (%s) is created", tablespaceName, uri));
@@ -235,15 +226,10 @@ public class CatalogServer extends AbstractService {
       wlock.lock();
       try {
         if (tablespaceName.equals(TajoConstants.DEFAULT_TABLESPACE_NAME)) {
-          throw new CatalogException(ResultCode.INSUFFICIENT_PRIVILEGE, "drop to default tablespace");
-        }
-
-        if (!store.existTablespace(tablespaceName)) {
-          throw new UndefinedTablespaceException(tablespaceName);
+          throw new InsufficientPrivilegeException("drop to default tablespace");
         }
 
         store.dropTablespace(tablespaceName);
-
         return OK;
 
       } catch (Throwable t) {
@@ -350,7 +336,6 @@ public class CatalogServer extends AbstractService {
             .build();
 
       } catch (Throwable t) {
-
         printStackTraceIfError(LOG, t);
         return GetTablespaceResponse.newBuilder()
             .setState(returnError(t))
@@ -370,10 +355,6 @@ public class CatalogServer extends AbstractService {
           return errInsufficientPrivilege("alter tablespace '"+request.getSpaceName()+"'");
         }
 
-        if (!store.existTablespace(request.getSpaceName())) {
-          return errUndefinedTablespace(request.getSpaceName());
-        }
-
         if (request.getCommandList().size() > 0) {
           for (AlterTablespaceCommand command : request.getCommandList()) {
             if (command.getType() == AlterTablespaceProto.AlterTablespaceType.LOCATION) {
@@ -417,10 +398,6 @@ public class CatalogServer extends AbstractService {
       
       wlock.lock();
       try {
-        if (store.existDatabase(databaseName)) {
-          return errDuplicateDatabase(databaseName);
-        }
-
         store.createDatabase(databaseName, tablespaceName);
         LOG.info(String.format("database \"%s\" is created", databaseName));
 
@@ -441,12 +418,7 @@ public class CatalogServer extends AbstractService {
       wlock.lock();
 
       try {
-        String [] split = CatalogUtil.splitTableName(proto.getTableName());
-        if (!store.existTable(split[0], split[1])) {
-          return errDuplicateTable(proto.getTableName());
-        }
         store.updateTableStats(proto);
-
         return OK;
 
       } catch (Throwable t) {
@@ -473,11 +445,7 @@ public class CatalogServer extends AbstractService {
       wlock.lock();
 
       try {
-        if (!store.existTable(split[0], split[1])) {
-          return errUndefinedTable(proto.getTableName());
-        }
         store.alterTable(proto);
-
         return OK;
 
       } catch (Throwable t) {
@@ -503,12 +471,7 @@ public class CatalogServer extends AbstractService {
 
       wlock.lock();
       try {
-        if (!store.existDatabase(databaseName)) {
-          return errUndefinedDatabase(databaseName);
-        }
-
         store.dropDatabase(databaseName);
-
         return OK;
 
       } catch (Throwable t) {
@@ -621,27 +584,10 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-        boolean contain;
-
-        contain = store.existDatabase(dbName);
-
-        if (contain) {
-          contain = store.existTable(dbName, tbName);
-          if (contain) {
-            return TableResponse.newBuilder()
-                .setState(OK)
-                .setTable(store.getTable(dbName, tbName))
-                .build();
-          } else {
-            return TableResponse.newBuilder()
-                .setState(errUndefinedTable(tbName))
-                .build();
-          }
-        } else {
-          return TableResponse.newBuilder()
-              .setState(errUndefinedDatabase(dbName))
-              .build();
-        }
+        return TableResponse.newBuilder()
+            .setState(OK)
+            .setTable(store.getTable(dbName, tbName))
+            .build();
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -675,13 +621,7 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-        if (store.existDatabase(dbName)) {
-          return returnStringList(store.getAllTableNames(dbName));
-        } else {
-          return StringListResponse.newBuilder()
-              .setState(errUndefinedDatabase(dbName))
-              .build();
-        }
+        return returnStringList(store.getAllTableNames(dbName));
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -698,6 +638,7 @@ public class CatalogServer extends AbstractService {
         throws ServiceException {
       Iterator<List<FunctionDescProto>> iterator = functions.values().iterator();
       GetFunctionsResponse.Builder builder = GetFunctionsResponse.newBuilder();
+      builder.setState(OK);
       while (iterator.hasNext()) {
         builder.addAllFunctionDesc(iterator.next());
       }
@@ -719,24 +660,12 @@ public class CatalogServer extends AbstractService {
       if (metaDictionary.isSystemDatabase(dbName)) {
         return errInsufficientPrivilege("create a table in database '" + dbName + "'");
       }
-      
+
       wlock.lock();
       try {
-
-        boolean contain = store.existDatabase(dbName);
-
-        if (contain) {
-          if (store.existTable(dbName, tbName)) {
-            return errDuplicateTable(tbName);
-          }
-
-          store.createTable(request);
-          LOG.info(String.format("relation \"%s\" is added to the catalog (%s)",
-              CatalogUtil.getCanonicalTableName(dbName, tbName), bindAddressStr));
-        } else {
-          return errUndefinedDatabase(dbName);
-        }
-
+        store.createTable(request);
+        LOG.info(String.format("relation \"%s\" is added to the catalog (%s)",
+            CatalogUtil.getCanonicalTableName(dbName, tbName), bindAddressStr));
         return OK;
 
       } catch (Throwable t) {
@@ -764,19 +693,9 @@ public class CatalogServer extends AbstractService {
 
       wlock.lock();
       try {
-        boolean contain = store.existDatabase(dbName);
-
-        if (contain) {
-          if (!store.existTable(dbName, tbName)) {
-            return errUndefinedTable(tbName);
-          }
-
-          store.dropTable(dbName, tbName);
-          LOG.info(String.format("relation \"%s\" is deleted from the catalog (%s)",
-              CatalogUtil.getCanonicalTableName(dbName, tbName), bindAddressStr));
-        } else {
-          return errUndefinedDatabase(dbName);
-        }
+        store.dropTable(dbName, tbName);
+        LOG.info(String.format("relation \"%s\" is deleted from the catalog (%s)",
+            CatalogUtil.getCanonicalTableName(dbName, tbName), bindAddressStr));
 
         return OK;
 
@@ -805,16 +724,10 @@ public class CatalogServer extends AbstractService {
         rlock.lock();
         try {
 
-          boolean contain = store.existDatabase(dbName);
-
-          if (contain) {
-            if (store.existTable(dbName, tbName)) {
-              return OK;
-            } else {
-              return errUndefinedTable(tbName);
-            }
+          if (store.existTable(dbName, tbName)) {
+            return OK;
           } else {
-            return errUndefinedDatabase(dbName);
+            return errUndefinedTable(tbName);
           }
 
         } catch (Throwable t) {
@@ -930,41 +843,16 @@ public class CatalogServer extends AbstractService {
       }
 
       if (metaDictionary.isSystemDatabase(dbName)) {
-        throw new ServiceException(dbName + " is a system databsae. It does not contain any partitioned tables.");
+        return GetPartitionMethodResponse.newBuilder().setState(errUndefinedPartitionMethod(tbName)).build();
       }
       
       rlock.lock();
       try {
-        boolean contain;
-
-        contain = store.existDatabase(dbName);
-
-        if (contain) {
-          contain = store.existTable(dbName, tbName);
-          if (contain) {
-
-            if (store.existPartitionMethod(dbName, tbName)) {
-
-              return GetPartitionMethodResponse.newBuilder()
-                  .setState(OK)
-                  .setPartition(store.getPartitionMethod(dbName, tbName))
-                  .build();
 
-            } else {
-              return GetPartitionMethodResponse.newBuilder()
-                  .setState(errUndefinedPartitionMethod(tbName))
-                  .build();
-            }
-          } else {
-            return GetPartitionMethodResponse.newBuilder()
-                .setState(errUndefinedTable(tbName))
-                .build();
-          }
-        } else {
-          return GetPartitionMethodResponse.newBuilder()
-              .setState(errUndefinedDatabase(tbName))
-              .build();
-        }
+        return GetPartitionMethodResponse.newBuilder()
+            .setState(OK)
+            .setPartition(store.getPartitionMethod(dbName, tbName))
+            .build();
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -999,24 +887,12 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-        boolean contain;
-
-        contain = store.existDatabase(dbName);
-
-        if (contain) {
-          contain = store.existTable(dbName, tableName);
-          if (contain) {
-            if (store.existPartitionMethod(dbName, tableName)) {
-              return OK;
-            } else {
-              return errUndefinedPartitionMethod(tableName);
-            }
-          } else {
-            return errUndefinedTable(tableName);
-          }
+        if (store.existPartitionMethod(dbName, tableName)) {
+          return OK;
         } else {
-          return errUndefinedDatabase(dbName);
+          return errUndefinedPartitionMethod(tableName);
         }
+
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
         return returnError(t);
@@ -1027,12 +903,8 @@ public class CatalogServer extends AbstractService {
     }
 
     @Override
-    public ReturnState dropPartitionMethod(RpcController controller, TableIdentifierProto request) {
-      return errFeatureNotSupported("dropPartitionMethod");
-    }
-
-    @Override
-    public GetPartitionDescResponse getPartitionByPartitionName(RpcController controller, PartitionIdentifierProto request)
+    public GetPartitionDescResponse getPartitionByPartitionName(RpcController controller,
+                                                                PartitionIdentifierProto request)
         throws ServiceException {
       String dbName = request.getDatabaseName();
       String tbName = request.getTableName();
@@ -1055,40 +927,19 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-        boolean contain;
-
-        contain = store.existDatabase(dbName);
-        if (contain) {
-          contain = store.existTable(dbName, tbName);
-          if (contain) {
-
-            if (store.existPartitionMethod(dbName, tbName)) {
-              PartitionDescProto partitionDesc = store.getPartition(dbName, tbName, partitionName);
-              if (partitionDesc != null) {
-                return GetPartitionDescResponse.newBuilder()
-                  .setState(OK)
-                  .setPartition(partitionDesc)
-                  .build();
-              } else {
-                return GetPartitionDescResponse.newBuilder()
-                  .setState(errUndefinedPartition(partitionName))
-                  .build();
-              }
-            } else {
-              return GetPartitionDescResponse.newBuilder()
-                  .setState(errUndefinedPartitionMethod(tbName))
-                  .build();
-            }
-          } else {
-            return GetPartitionDescResponse.newBuilder()
-                .setState(errUndefinedTable(tbName))
-                .build();
-          }
+
+        PartitionDescProto partitionDesc = store.getPartition(dbName, tbName, partitionName);
+        if (partitionDesc != null) {
+          return GetPartitionDescResponse.newBuilder()
+              .setState(OK)
+              .setPartition(partitionDesc)
+              .build();
         } else {
           return GetPartitionDescResponse.newBuilder()
-              .setState(errUndefinedDatabase(dbName))
+              .setState(errUndefinedPartition(partitionName))
               .build();
         }
+
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
 
@@ -1115,9 +966,7 @@ public class CatalogServer extends AbstractService {
         }
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
-        return GetPartitionsResponse.newBuilder()
-            .setState(returnError(t))
-            .build();
+        return GetPartitionsResponse.newBuilder().setState(returnError(t)).build();
       }
 
       if (metaDictionary.isSystemDatabase(dbName)) {
@@ -1126,40 +975,17 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-        boolean contain;
-
-        contain = store.existDatabase(dbName);
-        if (contain) {
-          contain = store.existTable(dbName, tbName);
-          if (contain) {
-            if (store.existPartitionMethod(dbName, tbName)) {
-              List<PartitionDescProto> partitions = store.getPartitions(dbName, tbName);
-
-              GetPartitionsResponse.Builder builder = GetPartitionsResponse.newBuilder();
-              for(PartitionDescProto partition : partitions) {
-                builder.addPartition(partition);
-              }
 
-              builder.setState(OK);
-              return builder.build();
+        List<PartitionDescProto> partitions = store.getPartitions(dbName, tbName);
 
-            } else {
-              return GetPartitionsResponse.newBuilder()
-                  .setState(errUndefinedPartitionMethod(tbName))
-                  .build();
-            }
+        GetPartitionsResponse.Builder builder = GetPartitionsResponse.newBuilder();
+        for (PartitionDescProto partition : partitions) {
+          builder.addPartition(partition);
+        }
 
-          } else {
-            return GetPartitionsResponse.newBuilder()
-                .setState(errUndefinedTable(tbName))
-                .build();
-          }
-        } else {
-          return GetPartitionsResponse.newBuilder()
-              .setState(errUndefinedDatabase(dbName))
-              .build();
+        builder.setState(OK);
+        return builder.build();
 
-        }
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
 
@@ -1173,7 +999,9 @@ public class CatalogServer extends AbstractService {
     }
 
     @Override
-    public GetTablePartitionsResponse getAllPartitions(RpcController controller, NullProto request) throws ServiceException {
+    public GetTablePartitionsResponse getAllPartitions(RpcController controller, NullProto request)
+        throws ServiceException {
+
       rlock.lock();
 
       try {
@@ -1203,24 +1031,10 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-        boolean contain;
-
-        contain = store.existDatabase(databaseName);
-        if (contain) {
-          contain = store.existTable(databaseName, tableName);
-          if (contain) {
-            if (store.existPartitionMethod(databaseName, tableName)) {
-              store.addPartitions(databaseName, tableName, request.getPartitionDescList(), request.getIfNotExists());
-              return OK;
-            } else {
-              return errUndefinedPartitionMethod(tableName);
-            }
-          } else {
-            return errUndefinedTable(tableName);
-          }
-        } else {
-          return errUndefinedDatabase(databaseName);
-        }
+
+        store.addPartitions(databaseName, tableName, request.getPartitionDescList(), request.getIfNotExists());
+        return OK;
+
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
         return returnError(t);
@@ -1245,13 +1059,7 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-        if (store.existIndexByName(
-            dbName,
-            indexDesc.getIndexName())) {
-          return errDuplicateTable(indexDesc.getIndexName());
-        }
         store.createIndex(indexDesc);
-
         return OK;
 
       } catch (Throwable t) {
@@ -1281,12 +1089,7 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-
-        if (store.existDatabase(dbName)) {
-          return store.existIndexByName(dbName, indexName) ? OK : errUndefinedIndexName(indexName);
-        } else {
-          return errUndefinedDatabase(dbName);
-        }
+        return store.existIndexByName(dbName, indexName) ? OK : errUndefinedIndexName(indexName);
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -1318,17 +1121,8 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-
-        if (store.existDatabase(databaseName)) {
-          if (store.existTable(databaseName, tableName)) {
-            return store.existIndexByColumns(databaseName, tableName,
-                columnNames.toArray(new String[columnNames.size()])) ? OK : errUndefinedIndex(tableName, columnNames);
-          } else {
-            return errUndefinedTable(tableName);
-          }
-        } else {
-          return errUndefinedDatabase(databaseName);
-        }
+        return store.existIndexByColumns(databaseName, tableName,
+            columnNames.toArray(new String[columnNames.size()])) ? OK : errUndefinedIndex(tableName, columnNames);
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -1356,16 +1150,7 @@ public class CatalogServer extends AbstractService {
 
       rlock.lock();
       try {
-
-        if (store.existDatabase(databaseName)) {
-          if (store.existTable(databaseName, tableName)) {
-            return store.existIndexesByTable(databaseName, tableName) ? OK : errUndefinedIndex(tableName);
-          } else {
-            return errUndefinedTable(tableName);
-          }
-        } else {
-          return errUndefinedDatabase(databaseName);
-        }
+        return store.existIndexesByTable(databaseName, tableName) ? OK : errUndefinedIndex(tableName);
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -1508,11 +1293,7 @@ public class CatalogServer extends AbstractService {
 
       wlock.lock();
       try {
-        if (!store.existIndexByName(dbName, indexName)) {
-          return errUndefinedIndexName(indexName);
-        }
         store.dropIndex(dbName, indexName);
-
         return OK;
 
       } catch (Throwable t) {
@@ -1695,7 +1476,8 @@ public class CatalogServer extends AbstractService {
       try {
         if (request.hasFunctionType()) {
           if (containFunction(request.getSignature(), request.getFunctionType(), request.getParameterTypesList())) {
-            function = findFunction(request.getSignature(), request.getFunctionType(), request.getParameterTypesList(), true);
+            function = findFunction(request.getSignature(), request.getFunctionType(),
+                request.getParameterTypesList(), true);
           }
         } else {
           function = findFunction(request.getSignature(), request.getParameterTypesList());

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/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 080c984..122c1af 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
@@ -23,9 +23,9 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
-import org.apache.tajo.catalog.exception.InsufficientPrivilegeException;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedTablespaceException;
+import org.apache.tajo.exception.InsufficientPrivilegeException;
+import org.apache.tajo.exception.UndefinedDatabaseException;
+import org.apache.tajo.exception.UndefinedTablespaceException;
 import org.apache.tajo.util.Pair;
 
 import javax.annotation.Nullable;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java
index 2fde0e2..e0bc54e 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.tajo.catalog.exception.UndefinedTableException;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.util.TUtil;