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

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

Repository: tajo
Updated Branches:
  refs/heads/master fd6a95180 -> 903151ead


http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
index 968601c..0094310 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
@@ -128,7 +128,7 @@ public abstract class Tablespace {
    * @throws java.io.IOException
    */
   public abstract List<Fragment> getSplits(String fragmentId, TableDesc tableDesc,
-                                           ScanNode scanNode) throws IOException;
+                                           ScanNode scanNode) throws IOException, TajoException;
 
   /**
    * It returns the splits that will serve as input for the non-forward query scanner such as 'select * from table1'.
@@ -200,7 +200,7 @@ public abstract class Tablespace {
    * @return The list of input fragments.
    * @throws java.io.IOException
    */
-  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc) throws IOException {
+  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc) throws IOException, TajoException {
     return getSplits(fragmentId, tableDesc, null);
   }
 
@@ -345,7 +345,7 @@ public abstract class Tablespace {
    * @param outSchema  The output schema of select query for inserting.
    * @throws java.io.IOException
    */
-  public abstract void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException;
+  public abstract void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws TajoException;
 
   /**
    * Rewrite the logical plan. It is assumed that the final plan will be given in this method.
@@ -366,7 +366,7 @@ public abstract class Tablespace {
    * @param ifNotExists Creates the table only when the table does not exist.
    * @throws java.io.IOException
    */
-  public abstract void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException;
+  public abstract void createTable(TableDesc tableDesc, boolean ifNotExists) throws TajoException, IOException;
 
   /**
    * This method is called after executing "DROP TABLE" statement with the 'PURGE' option
@@ -375,7 +375,7 @@ public abstract class Tablespace {
    * @param tableDesc
    * @throws java.io.IOException
    */
-  public abstract void purgeTable(TableDesc tableDesc) throws IOException;
+  public abstract void purgeTable(TableDesc tableDesc) throws IOException, TajoException;
 
   /**
    * This method is called before executing 'INSERT' or 'CREATE TABLE as SELECT'.
@@ -386,7 +386,7 @@ public abstract class Tablespace {
    * @param node The child node of the root node.
    * @throws java.io.IOException
    */
-  public abstract void prepareTable(LogicalNode node) throws IOException;
+  public abstract void prepareTable(LogicalNode node) throws IOException, TajoException;
 
   /**
    * Finalizes result data. Tajo stores result data in the staging directory.
@@ -406,7 +406,7 @@ public abstract class Tablespace {
                                    LogicalPlan plan, Schema schema,
                                    TableDesc tableDesc) throws IOException;
 
-  public abstract void rollbackTable(LogicalNode node) throws IOException;
+  public abstract void rollbackTable(LogicalNode node) throws IOException, TajoException;
 
   @Override
   public boolean equals(Object obj) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
index 0fc2922..af78d13 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
@@ -26,6 +26,10 @@ import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.datum.Datum;
+import org.apache.tajo.exception.InvalidTablePropertyException;
+import org.apache.tajo.exception.MissingTablePropertyException;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.storage.Appender;
 import org.apache.tajo.storage.TableStatistics;
 import org.apache.tajo.storage.Tuple;
@@ -89,7 +93,11 @@ public abstract class AbstractHBaseAppender implements Appender {
     if (enabledStats) {
       stats = new TableStatistics(this.schema);
     }
-    columnMapping = new ColumnMapping(schema, meta.getOptions());
+    try {
+      columnMapping = new ColumnMapping(schema, meta.getOptions());
+    } catch (TajoException e) {
+      throw new TajoInternalError(e);
+    }
 
     mappingColumnFamilies = columnMapping.getMappingColumns();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
index 0314e8e..7df7e9c 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
@@ -20,6 +20,9 @@ package org.apache.tajo.storage.hbase;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.exception.InvalidTablePropertyException;
+import org.apache.tajo.exception.MissingTablePropertyException;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.util.BytesUtils;
 import org.apache.tajo.util.KeyValueSet;
 
@@ -45,13 +48,14 @@ public class ColumnMapping {
 
   private int numRowKeys;
 
-  public ColumnMapping(Schema schema, KeyValueSet tableProperty) throws IOException{
+  public ColumnMapping(Schema schema, KeyValueSet tableProperty)
+      throws MissingTablePropertyException, InvalidTablePropertyException {
     this.schema = schema;
     this.tableProperty = tableProperty;
     init();
   }
 
-  public void init() throws IOException {
+  public void init() throws MissingTablePropertyException, InvalidTablePropertyException {
     hbaseTableName = tableProperty.get(HBaseStorageConstants.META_TABLE_KEY);
     String delim = tableProperty.get(HBaseStorageConstants.META_ROWKEY_DELIMITER, "").trim();
     if (delim.length() > 0) {
@@ -71,13 +75,14 @@ public class ColumnMapping {
 
     String columnMapping = tableProperty.get(HBaseStorageConstants.META_COLUMNS_KEY, "");
     if (columnMapping == null || columnMapping.isEmpty()) {
-      throw new IOException("'columns' property is required.");
+      throw new MissingTablePropertyException(HBaseStorageConstants.META_COLUMNS_KEY, hbaseTableName);
     }
 
     String[] columnMappingTokens = columnMapping.split(",");
 
     if (columnMappingTokens.length != schema.getRootColumns().size()) {
-      throw new IOException("The number of mapped HBase columns is great than the number of Tajo table columns");
+      throw new InvalidTablePropertyException(
+          "mapping column pairs must be more than number of columns in the schema", hbaseTableName);
     }
 
     int index = 0;
@@ -89,8 +94,9 @@ public class ColumnMapping {
       if (mappingTokens.length == 3) {
         if (mappingTokens[0].length == 0) {
           // cfname
-          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
-              "or '<cfname>:value:' or '<cfname>:value:#b'");
+          throw new InvalidTablePropertyException(eachToken +
+              " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
+              "or '<cfname>:value:' or '<cfname>:value:#b'", hbaseTableName);
         }
         //<cfname>:key: or <cfname>:value:
         if (mappingTokens[2].length != 0) {
@@ -98,8 +104,9 @@ public class ColumnMapping {
           if ("#b".equals(binaryOption)) {
             isBinaryColumns[index] = true;
           } else {
-            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
-                "or '<cfname>:value:' or '<cfname>:value:#b'");
+            throw new InvalidTablePropertyException(eachToken +
+                " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
+                "or '<cfname>:value:' or '<cfname>:value:#b'", hbaseTableName);
           }
         }
         mappingColumns[index][0] = mappingTokens[0];
@@ -109,7 +116,9 @@ public class ColumnMapping {
         } else if (HBaseStorageConstants.VALUE_COLUMN_MAPPING.equalsIgnoreCase(keyOrValue)) {
           isColumnValues[index] = true;
         } else {
-          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
+          throw new InvalidTablePropertyException(eachToken +
+              " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'",
+              hbaseTableName);
         }
       } else if (mappingTokens.length == 2) {
         //<cfname>: or <cfname>:<qualifier> or :key
@@ -122,7 +131,8 @@ public class ColumnMapping {
           isBinaryColumns[index] = rowKeyMapping.isBinary();
           if (!cfName.isEmpty()) {
             if (rowKeyDelimiter == 0) {
-              throw new IOException("hbase.rowkey.delimiter is required.");
+              throw new InvalidTablePropertyException("hbase.rowkey.delimiter is required.",
+                  hbaseTableName);
             }
             rowKeyFieldIndexes[index] = Integer.parseInt(cfName);
           } else {
@@ -130,7 +140,9 @@ public class ColumnMapping {
           }
         } else {
           if (cfName.isEmpty()) {
-            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
+            throw new InvalidTablePropertyException(eachToken +
+                " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'",
+                hbaseTableName);
           }
           if (cfName != null) {
             mappingColumns[index][0] = Bytes.toBytes(cfName);
@@ -149,7 +161,8 @@ public class ColumnMapping {
           }
         }
       } else {
-        throw new IOException(eachToken + " 'column' attribute '[cfname]:[qualfier]:'");
+        throw new InvalidTablePropertyException(eachToken + " 'column' attribute '[cfname]:[qualfier]:'"
+            , hbaseTableName);
       }
 
       index++;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
index 40c4aea..ff6fa59 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
@@ -66,7 +66,7 @@ public class HBaseBinarySerializerDeserializer {
     return datum;
   }
 
-  public static byte[] serialize(Column col, Datum datum) throws IOException {
+  public static byte[] serialize(Column col, Datum datum) {
     if (datum == null || datum instanceof NullDatum) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
index 1626526..beae592 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
@@ -36,7 +36,7 @@ import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.Fragment;
@@ -123,7 +123,11 @@ public class HBaseScanner implements Scanner {
 
     outTuple = new VTuple(targets.length);
 
-    columnMapping = new ColumnMapping(schema, meta.getOptions());
+    try {
+      columnMapping = new ColumnMapping(schema, meta.getOptions());
+    } catch (TajoException e) {
+      new TajoInternalError(e);
+    }
     targetIndexes = new int[targets.length];
     int index = 0;
     for (Column eachTargetColumn: targets) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
index 35c974b..f613b88 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
@@ -42,6 +42,9 @@ import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.exception.DataTypeMismatchException;
+import org.apache.tajo.exception.InvalidTablePropertyException;
+import org.apache.tajo.exception.MissingTablePropertyException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.expr.*;
@@ -50,6 +53,7 @@ import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.ScanNode;
 import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRuleContext;
+import org.apache.tajo.plan.verifier.SyntaxErrorUtil;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.*;
@@ -120,7 +124,7 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
+  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws TajoException, IOException {
     createTable(tableDesc.getUri(), tableDesc.getMeta(), tableDesc.getSchema(), tableDesc.isExternal(), ifNotExists);
     TableStats stats = new TableStats();
     stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
@@ -128,17 +132,17 @@ public class HBaseTablespace extends Tablespace {
   }
 
   private void createTable(URI uri, TableMeta tableMeta, Schema schema,
-                           boolean isExternal, boolean ifNotExists) throws IOException {
+                           boolean isExternal, boolean ifNotExists) throws TajoException, IOException {
     String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
     if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
-      throw new IOException("HBase mapped table is required a '" +
-          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
+      throw new MissingTablePropertyException(HBaseStorageConstants.META_TABLE_KEY, "hbase");
     }
     TableName hTableName = TableName.valueOf(hbaseTableName);
 
     String mappedColumns = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
     if (mappedColumns != null && mappedColumns.split(",").length > schema.size()) {
-      throw new IOException("Columns property has more entry than Tajo table columns");
+      throw new InvalidTablePropertyException(HBaseStorageConstants.META_COLUMNS_KEY,
+          "mapping column pairs must be more than number of columns in the schema");
     }
 
     ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta.getOptions());
@@ -152,17 +156,17 @@ public class HBaseTablespace extends Tablespace {
     if (numRowKeys > 1) {
       for (int i = 0; i < isRowKeyMappings.length; i++) {
         if (isRowKeyMappings[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
-          throw new IOException("Key field type should be TEXT type.");
+          throw SyntaxErrorUtil.makeSyntaxError("Key field type should be TEXT type.");
         }
       }
     }
 
     for (int i = 0; i < isRowKeyMappings.length; i++) {
       if (columnMapping.getIsColumnKeys()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
-        throw new IOException("Column key field('<cfname>:key:') type should be TEXT type.");
+        throw SyntaxErrorUtil.makeSyntaxError("Column key field('<cfname>:key:') type should be TEXT type.");
       }
       if (columnMapping.getIsColumnValues()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
-        throw new IOException("Column value field(('<cfname>:value:') type should be TEXT type.");
+        throw SyntaxErrorUtil.makeSyntaxError("Column value field(('<cfname>:value:') type should be TEXT type.");
       }
     }
 
@@ -172,8 +176,7 @@ public class HBaseTablespace extends Tablespace {
       if (isExternal) {
         // If tajo table is external table, only check validation.
         if (mappedColumns == null || mappedColumns.isEmpty()) {
-          throw new IOException("HBase mapped table is required a '" +
-              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
+          throw new MissingTablePropertyException(HBaseStorageConstants.META_COLUMNS_KEY, hbaseTableName);
         }
         if (!hAdmin.tableExists(hTableName)) {
           throw new IOException("HBase table [" + hbaseTableName + "] not exists. " +
@@ -187,8 +190,7 @@ public class HBaseTablespace extends Tablespace {
 
         Collection<String> mappingColumnFamilies =columnMapping.getColumnFamilyNames();
         if (mappingColumnFamilies.isEmpty()) {
-          throw new IOException("HBase mapped table is required a '" +
-              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
+          throw new MissingTablePropertyException(HBaseStorageConstants.META_COLUMNS_KEY, hbaseTableName);
         }
 
         for (String eachMappingColumnFamily : mappingColumnFamilies) {
@@ -207,7 +209,7 @@ public class HBaseTablespace extends Tablespace {
         // Creating hbase table
         HTableDescriptor hTableDescriptor = parseHTableDescriptor(tableMeta, schema);
 
-        byte[][] splitKeys = getSplitKeys(conf, schema, tableMeta);
+        byte[][] splitKeys = getSplitKeys(conf, hbaseTableName, schema, tableMeta);
         if (splitKeys == null) {
           hAdmin.createTable(hTableDescriptor);
         } else {
@@ -228,7 +230,9 @@ public class HBaseTablespace extends Tablespace {
    * @return
    * @throws java.io.IOException
    */
-  private byte[][] getSplitKeys(TajoConf conf, Schema schema, TableMeta meta) throws IOException {
+  private byte[][] getSplitKeys(TajoConf conf, String hbaseTableName, Schema schema, TableMeta meta)
+      throws MissingTablePropertyException, InvalidTablePropertyException, IOException {
+
     String splitRowKeys = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_KEY, "");
     String splitRowKeysFile = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_FILE_KEY, "");
 
@@ -255,8 +259,8 @@ public class HBaseTablespace extends Tablespace {
     }
 
     if (rowkeyBinary && numRowKeys > 1) {
-      throw new IOException("If rowkey is mapped to multi column and a rowkey is binary, " +
-          "Multiple region for creation is not support.");
+      throw new InvalidTablePropertyException("If rowkey is mapped to multi column and a rowkey is binary, " +
+          "Multiple region for creation is not support.", hbaseTableName);
     }
 
     if (splitRowKeys != null && !splitRowKeys.isEmpty()) {
@@ -277,7 +281,8 @@ public class HBaseTablespace extends Tablespace {
       Path path = new Path(splitRowKeysFile);
       FileSystem fs = path.getFileSystem(conf);
       if (!fs.exists(path)) {
-        throw new IOException("hbase.split.rowkeys.file=" + path.toString() + " not exists.");
+        throw new MissingTablePropertyException("hbase.split.rowkeys.file=" + path.toString() + " not exists.",
+            hbaseTableName);
       }
 
       SortedSet<String> splitKeySet = new TreeSet<String>();
@@ -345,11 +350,12 @@ public class HBaseTablespace extends Tablespace {
    * @return
    * @throws java.io.IOException
    */
-  public static HTableDescriptor parseHTableDescriptor(TableMeta tableMeta, Schema schema) throws IOException {
+  public static HTableDescriptor parseHTableDescriptor(TableMeta tableMeta, Schema schema)
+      throws MissingTablePropertyException, InvalidTablePropertyException {
+
     String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
     if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
-      throw new IOException("HBase mapped table is required a '" +
-          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
+      throw new MissingTablePropertyException(HBaseStorageConstants.META_TABLE_KEY, hbaseTableName);
     }
     TableName hTableName = TableName.valueOf(hbaseTableName);
 
@@ -373,7 +379,7 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public void purgeTable(TableDesc tableDesc) throws IOException {
+  public void purgeTable(TableDesc tableDesc) throws IOException, TajoException {
     HBaseAdmin hAdmin =  new HBaseAdmin(hbaseConf);
 
     try {
@@ -398,7 +404,9 @@ public class HBaseTablespace extends Tablespace {
    * @return
    * @throws java.io.IOException
    */
-  private Column[] getIndexableColumns(TableDesc tableDesc) throws IOException {
+  private Column[] getIndexableColumns(TableDesc tableDesc) throws
+      MissingTablePropertyException, InvalidTablePropertyException {
+
     ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta().getOptions());
     boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
     int[] rowKeyIndexes = columnMapping.getRowKeyFieldIndexes();
@@ -416,7 +424,9 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc, ScanNode scanNode) throws IOException {
+  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc, ScanNode scanNode)
+      throws IOException, TajoException {
+
     ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta().getOptions());
 
     List<IndexPredication> indexPredications = getIndexPredications(columnMapping, tableDesc, scanNode);
@@ -786,7 +796,9 @@ public class HBaseTablespace extends Tablespace {
   }
 
   public List<IndexPredication> getIndexPredications(ColumnMapping columnMapping,
-                                                     TableDesc tableDesc, ScanNode scanNode) throws IOException {
+                                                     TableDesc tableDesc, ScanNode scanNode)
+      throws IOException, MissingTablePropertyException, InvalidTablePropertyException {
+
     List<IndexPredication> indexPredications = new ArrayList<IndexPredication>();
     Column[] indexableColumns = getIndexableColumns(tableDesc);
     if (indexableColumns != null && indexableColumns.length == 1) {
@@ -1112,7 +1124,7 @@ public class HBaseTablespace extends Tablespace {
     }
   }
 
-  public void prepareTable(LogicalNode node) throws IOException {
+  public void prepareTable(LogicalNode node) throws TajoException, IOException {
     if (node.getType() == NodeType.CREATE_TABLE) {
       CreateTableNode cNode = (CreateTableNode)node;
       if (!cNode.isExternal()) {
@@ -1125,7 +1137,7 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public void rollbackTable(LogicalNode node) throws IOException {
+  public void rollbackTable(LogicalNode node) throws IOException, TajoException {
     if (node.getType() == NodeType.CREATE_TABLE) {
       CreateTableNode cNode = (CreateTableNode)node;
       if (cNode.isExternal()) {
@@ -1164,19 +1176,22 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException  {
+  public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws TajoException {
     if (tableDesc != null) {
       Schema tableSchema = tableDesc.getSchema();
       if (tableSchema.size() != outSchema.size()) {
-        throw new IOException("The number of table columns is different from SELECT columns");
+        throw SyntaxErrorUtil.makeSyntaxError("Target columns and projected columns are mismatched to each other");
       }
 
       for (int i = 0; i < tableSchema.size(); i++) {
         if (!tableSchema.getColumn(i).getDataType().equals(outSchema.getColumn(i).getDataType())) {
-          throw new IOException(outSchema.getColumn(i).getQualifiedName() +
-              "(" + outSchema.getColumn(i).getDataType().getType() + ")" +
-              " is different column type with " + tableSchema.getColumn(i).getSimpleName() +
-              "(" + tableSchema.getColumn(i).getDataType().getType() + ")");
+          final Column tableColumn = tableSchema.getColumn(i);
+          final Column outColumn = outSchema.getColumn(i);
+          throw new DataTypeMismatchException(
+              tableColumn.getQualifiedName(),
+              tableColumn.getDataType().getType().name(),
+              outColumn.getQualifiedName(),
+              outColumn.getDataType().getType().name());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
index ea5d0b0..c868cd1 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
@@ -62,7 +62,7 @@ public class HBaseTextSerializerDeserializer {
     return datum;
   }
 
-  public static byte[] serialize(Column col, Datum datum) throws IOException {
+  public static byte[] serialize(Column col, Datum datum) {
     if (datum == null || datum instanceof NullDatum) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
index 40789ac..a99d4d0 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
@@ -21,6 +21,8 @@ package org.apache.tajo.storage.hbase;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.exception.InvalidTablePropertyException;
+import org.apache.tajo.exception.MissingTablePropertyException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.plan.LogicalPlan;
@@ -53,7 +55,9 @@ public class SortedInsertRewriter implements LogicalPlanRewriteRule {
     return hbaseMode && node.getType() == NodeType.CREATE_TABLE || node.getType() == NodeType.INSERT;
   }
 
-  public static Column[] getIndexColumns(Schema tableSchema, KeyValueSet tableProperty) throws IOException {
+  public static Column[] getIndexColumns(Schema tableSchema, KeyValueSet tableProperty)
+      throws IOException, TajoException {
+
     List<Column> indexColumns = new ArrayList<Column>();
 
     ColumnMapping columnMapping = new ColumnMapping(tableSchema, tableProperty);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
index e8a6c12..678675d 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
@@ -936,7 +936,7 @@ public class FileTablespace extends Tablespace {
   }
 
   @Override
-  public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException {
+  public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
index 7ef483c..c720118 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
@@ -32,7 +32,7 @@ import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.text.TextLineDeserializer;
 import org.apache.tajo.storage.text.TextLineParsingError;
@@ -208,7 +208,7 @@ public class JsonLineDeserializer extends TextLineDeserializer {
       break;
 
     default:
-      throw new UnimplementedException(types.get(fullPath).name() + " is not supported.");
+      throw new NotImplementedException(types.get(fullPath).name() + " is not supported.");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
index 99f81a2..0fd9e02 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
@@ -27,7 +27,7 @@ import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.text.TextLineSerializer;
 
@@ -119,7 +119,7 @@ public class JsonLineSerializer extends TextLineSerializer {
       break;
 
     default:
-      throw new UnimplementedException(fieldName + "(" + types.get(fullPath).name() + ") is not supported.");
+      throw new NotImplementedException(fieldName + "(" + types.get(fullPath).name() + ") is not supported.");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java
index 9511071..dfb44d3 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java
@@ -29,7 +29,7 @@ import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.*;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.storage.FileScanner;
 import org.apache.tajo.storage.StorageConstants;
@@ -85,7 +85,7 @@ public class ORCScanner extends FileScanner {
 
       default:
         LOG.error("Not supported type for "+type.toString());
-        throw new UnimplementedException("ORC type: "+type.toString());
+        throw new NotImplementedException("ORC type: "+type.toString());
     }
   }
 
@@ -267,7 +267,7 @@ public class ORCScanner extends FileScanner {
         return NullDatum.get();
 
       default:
-        throw new UnimplementedException("ORC type: "+type.toString());
+        throw new NotImplementedException("ORC type: "+type.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
index 0443308..5b93c5c 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.SplittableCompressionCodec;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.compress.CodecPool;
@@ -72,7 +72,7 @@ public class DelimitedLineReader implements Closeable {
     this.bufferSize = bufferSize;
     if (this.codec instanceof SplittableCompressionCodec) {
       // bzip2 does not support multi-thread model
-      throw new UnimplementedException(this.getClass() + " does not support " + this.codec.getDefaultExtension());
+      throw new NotImplementedException(this.getClass() + " does not support " + this.codec.getDefaultExtension());
     }
   }
 


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

Posted by hy...@apache.org.
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());
-  }
-}


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

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
index a3eb0c3..cd8dcb6 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
@@ -22,29 +22,31 @@
 package org.apache.tajo.catalog.store;
 
 import com.google.protobuf.InvalidProtocolBufferException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tajo.annotation.Nullable;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.*;
+import org.apache.tajo.catalog.CatalogConstants;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.FunctionDesc;
+import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.exception.InternalException;
-import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TUtil;
 
 import java.io.IOException;
 import java.sql.*;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 
-import static org.apache.tajo.catalog.exception.CatalogExceptionUtil.makeCatalogUpgrade;
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto.AlterTablespaceCommand;
 import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto;
 import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
@@ -85,19 +87,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   protected abstract Connection createConnection(final Configuration conf) throws SQLException;
   
-  protected void createDatabaseDependants() throws CatalogException {
-    
+  protected void createDatabaseDependants() {
   }
   
-  protected boolean isInitialized() throws CatalogException {
+  protected boolean isInitialized() {
     return catalogSchemaManager.isInitialized(getConnection());
   }
 
-  protected boolean catalogAlreadyExists() throws CatalogException {
+  protected boolean catalogAlreadyExists() {
     return catalogSchemaManager.catalogAlreadyExists(getConnection());
   }
 
-  protected void createBaseTable() throws CatalogException {
+  protected void createBaseTable() {
     createDatabaseDependants();
     
     catalogSchemaManager.createBaseSchema(getConnection());
@@ -105,11 +106,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     insertSchemaVersion();
   }
 
-  protected void dropBaseTable() throws CatalogException {
+  protected void dropBaseTable() {
     catalogSchemaManager.dropBaseSchema(getConnection());
   }
 
-  public AbstractDBStore(Configuration conf) throws InternalException {
+  public AbstractDBStore(Configuration conf) {
     this.conf = conf;
 
     if (conf.get(CatalogConstants.DEPRECATED_CATALOG_URI) != null) {
@@ -169,9 +170,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
           try {
             createBaseTable();
             LOG.info("The base tables of CatalogServer are created.");
-          } catch (CatalogException ce) {
+          } catch (Throwable e) {
             dropBaseTable();
-            throw ce;
+            throw e;
           }
         }
      }
@@ -184,7 +185,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     return catalogSchemaManager.getCatalogStore().getSchema().getVersion();
   }
 
-  public String readSchemaFile(String path) throws CatalogException {
+  public String readSchemaFile(String path) {
     try {
       return FileUtil.readTextFileFromResource("schemas/" + path);
     } catch (IOException e) {
@@ -248,7 +249,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     return schemaVersion;
   }
 
-  private void verifySchemaVersion() throws CatalogException {
+  private void verifySchemaVersion() throws CatalogUpgradeRequiredException {
     int schemaVersion = -1;
 
     schemaVersion = getSchemaVersion();
@@ -264,7 +265,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       LOG.error("| In order to learn how to migration Apache Tajo instance, |");
       LOG.error("| please refer http://tajo.apache.org/docs/current/backup_and_restore/catalog.html |");
       LOG.error("=========================================================================");
-      throw makeCatalogUpgrade();
+      throw new CatalogUpgradeRequiredException();
     }
 
     LOG.info(String.format("The compatibility of the catalog schema (version: %d) has been verified.",
@@ -274,7 +275,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   /**
    * Insert the version of the current catalog schema
    */
-  protected void insertSchemaVersion() throws CatalogException {
+  protected void insertSchemaVersion() {
     Connection conn;
     PreparedStatement pstmt = null;
     try {
@@ -290,7 +291,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void createTablespace(String spaceName, String spaceUri) throws CatalogException {
+  public void createTablespace(String spaceName, String spaceUri) {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
@@ -325,7 +326,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public boolean existTablespace(String tableSpaceName) throws CatalogException {
+  public boolean existTablespace(String tableSpaceName) {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
@@ -353,8 +354,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void dropTablespace(String tableSpaceName) throws CatalogException {
-
+  public void dropTablespace(String tableSpaceName) throws UndefinedTablespaceException {
 
     Connection conn = null;
     PreparedStatement pstmt = null;
@@ -366,7 +366,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       conn.setAutoCommit(false);
 
       for (String databaseName : databaseNames) {
-        dropDatabase(databaseName);
+        try {
+          dropDatabase(databaseName);
+        } catch (UndefinedDatabaseException e) {
+          LOG.warn(e);
+          continue;
+        }
       }
 
       String sql = "DELETE FROM " + TB_SPACES + " WHERE " + COL_TABLESPACE_PK + "= ?";
@@ -389,11 +394,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public Collection<String> getAllTablespaceNames() throws CatalogException {
+  public Collection<String> getAllTablespaceNames() {
     return getAllTablespaceNamesInternal(null);
   }
 
-  private Collection<String> getAllTablespaceNamesInternal(@Nullable String whereCondition) throws CatalogException {
+  private Collection<String> getAllTablespaceNamesInternal(@Nullable String whereCondition) {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet resultSet = null;
@@ -423,7 +428,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
   
   @Override
-  public List<TablespaceProto> getTablespaces() throws CatalogException {
+  public List<TablespaceProto> getTablespaces() {
     Connection conn = null;
     Statement stmt = null;
     ResultSet resultSet = null;
@@ -454,7 +459,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public TablespaceProto getTablespace(String spaceName) throws CatalogException {
+  public TablespaceProto getTablespace(String spaceName) throws UndefinedTablespaceException {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet resultSet = null;
@@ -486,7 +491,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void alterTablespace(AlterTablespaceProto alterProto) throws CatalogException {
+  public void alterTablespace(AlterTablespaceProto alterProto) {
     Connection conn;
     PreparedStatement pstmt = null;
 
@@ -512,11 +517,17 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void createDatabase(String databaseName, String tablespaceName) throws CatalogException {
+  public void createDatabase(String databaseName, String tablespaceName)
+      throws UndefinedTablespaceException, DuplicateDatabaseException {
+
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
 
+    if (existDatabase(databaseName)) {
+      throw new DuplicateDatabaseException(databaseName);
+    }
+
     try {
       TableSpaceInternal spaceInfo = getTableSpaceInfo(tablespaceName);
 
@@ -549,7 +560,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public boolean existDatabase(String databaseName) throws CatalogException {
+  public boolean existDatabase(String databaseName) {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
@@ -577,7 +588,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void dropDatabase(String databaseName) throws CatalogException {
+  public void dropDatabase(String databaseName) throws UndefinedDatabaseException {
     Collection<String> tableNames = getAllTableNames(databaseName);
 
     Connection conn = null;
@@ -587,7 +598,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       conn.setAutoCommit(false);
 
       for (String tableName : tableNames) {
-        dropTableInternal(conn, databaseName, tableName);
+        try {
+          dropTableInternal(conn, databaseName, tableName);
+        } catch (UndefinedTableException e) {
+          LOG.warn(e);
+        }
       }
 
       String sql = "DELETE FROM " + TB_DATABASES + " WHERE DB_NAME = ?";
@@ -610,11 +625,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public Collection<String> getAllDatabaseNames() throws CatalogException {
+  public Collection<String> getAllDatabaseNames() {
     return getAllDatabaseNamesInternal(null);
   }
 
-  private Collection<String> getAllDatabaseNamesInternal(@Nullable String whereCondition) throws CatalogException {
+  private Collection<String> getAllDatabaseNamesInternal(@Nullable String whereCondition) {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet resultSet = null;
@@ -644,7 +659,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
   
   @Override
-  public List<DatabaseProto> getAllDatabases() throws CatalogException {
+  public List<DatabaseProto> getAllDatabases() {
     Connection conn = null;
     Statement stmt = null;
     ResultSet resultSet = null;
@@ -711,17 +726,17 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       pstmt.setString(1, spaceName);
       res = pstmt.executeQuery();
       if (!res.next()) {
-        throw new TajoInternalError("There is no SPACE_ID matched to the space name '" + spaceName + "'");
+        throw new UndefinedTablespaceException(spaceName);
       }
       return new TableSpaceInternal(res.getInt(1), res.getString(2), res.getString(3));
     } catch (SQLException se) {
-      throw new UndefinedTablespaceException(spaceName);
+      throw new TajoInternalError(se);
     } finally {
       CatalogUtil.closeQuietly(pstmt, res);
     }
   }
 
-  private int getTableId(int databaseId, String databaseName, String tableName) {
+  private int getTableId(int databaseId, String databaseName, String tableName) throws UndefinedTableException {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
@@ -734,7 +749,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       pstmt.setString(2, tableName);
       res = pstmt.executeQuery();
       if (!res.next()) {
-        throw new TajoInternalError("There is no tid matched to '" + tableName + "'");
+        throw new UndefinedTableException(databaseName, tableName);
       }
       return res.getInt(1);
     } catch (SQLException se) {
@@ -750,25 +765,30 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void createTable(final CatalogProtos.TableDescProto table) throws CatalogException {
+  public void createTable(final CatalogProtos.TableDescProto table)
+      throws UndefinedDatabaseException, DuplicateTableException {
+
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
 
+    String[] splitted = CatalogUtil.splitTableName(table.getTableName());
+    if (splitted.length == 1) {
+      throw new TajoInternalError(
+          "createTable() requires a qualified table name, but it is '" + table.getTableName() + "'");
+    }
+    final String databaseName = splitted[0];
+    final String tableName = splitted[1];
+
+    if (existTable(databaseName, tableName)) {
+      throw new DuplicateTableException(tableName);
+    }
+    final int dbid = getDatabaseId(databaseName);
+
     try {
       conn = getConnection();
       conn.setAutoCommit(false);
 
-      String[] splitted = CatalogUtil.splitTableName(table.getTableName());
-      if (splitted.length == 1) {
-        throw new TajoInternalError(
-            "createTable() requires a qualified table name, but it is '" + table.getTableName() + "'");
-      }
-      String databaseName = splitted[0];
-      String tableName = splitted[1];
-
-      int dbid = getDatabaseId(databaseName);
-
       String sql = "INSERT INTO TABLES (DB_ID, " + COL_TABLES_NAME +
           ", TABLE_TYPE, PATH, STORE_TYPE) VALUES(?, ?, ?, ?, ?) ";
 
@@ -900,26 +920,26 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void updateTableStats(final CatalogProtos.UpdateTableStatsProto statsProto) throws
-    CatalogException {
+  public void updateTableStats(final CatalogProtos.UpdateTableStatsProto statsProto)
+      throws UndefinedDatabaseException, UndefinedTableException {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
 
+    String[] splitted = CatalogUtil.splitTableName(statsProto.getTableName());
+    if (splitted.length == 1) {
+      throw new IllegalArgumentException("updateTableStats() requires a qualified table name, but it is \""
+          + statsProto.getTableName() + "\".");
+    }
+    final String databaseName = splitted[0];
+    final String tableName = splitted[1];
+
+    final int dbid = getDatabaseId(databaseName);
+
     try {
       conn = getConnection();
       conn.setAutoCommit(false);
 
-      String[] splitted = CatalogUtil.splitTableName(statsProto.getTableName());
-      if (splitted.length == 1) {
-        throw new IllegalArgumentException("updateTableStats() requires a qualified table name, but it is \""
-          + statsProto.getTableName() + "\".");
-      }
-      String databaseName = splitted[0];
-      String tableName = splitted[1];
-
-      int dbid = getDatabaseId(databaseName);
-
       String tidSql =
         "SELECT TID from " + TB_TABLES + " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_NAME + "=?";
       pstmt = conn.prepareStatement(tidSql);
@@ -928,7 +948,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       res = pstmt.executeQuery();
 
       if (!res.next()) {
-        throw new TajoInternalError("There is no TID matched to '" + statsProto.getTableName() + "'");
+        throw new UndefinedTableException(statsProto.getTableName());
       }
 
       int tableId = res.getInt("TID");
@@ -968,66 +988,66 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws CatalogException {
+  public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto)
+      throws UndefinedDatabaseException, DuplicateTableException, DuplicateColumnException,
+      DuplicatePartitionException, UndefinedPartitionException, UndefinedColumnException, UndefinedTableException,
+      UndefinedPartitionMethodException {
 
     String[] splitted = CatalogUtil.splitTableName(alterTableDescProto.getTableName());
     if (splitted.length == 1) {
       throw new IllegalArgumentException("alterTable() requires a qualified table name, but it is \""
           + alterTableDescProto.getTableName() + "\".");
     }
-    String databaseName = splitted[0];
-    String tableName = splitted[1];
+    final String databaseName = splitted[0];
+    final String tableName = splitted[1];
     String partitionName = null;
     CatalogProtos.PartitionDescProto partitionDesc = null;
-    try {
 
-      int databaseId = getDatabaseId(databaseName);
-      int tableId = getTableId(databaseId, databaseName, tableName);
-
-      switch (alterTableDescProto.getAlterTableType()) {
-        case RENAME_TABLE:
-          if (existTable(databaseName,alterTableDescProto.getNewTableName())) {
-            throw new DuplicateTableException(alterTableDescProto.getNewTableName());
-          }
-          renameTable(tableId, alterTableDescProto.getNewTableName());
-          break;
-        case RENAME_COLUMN:
-          if (existColumn(tableId, alterTableDescProto.getAlterColumnName().getNewColumnName())) {
-            throw new DuplicateColumnException(alterTableDescProto.getAlterColumnName().getNewColumnName());
-          }
-          renameColumn(tableId, alterTableDescProto.getAlterColumnName());
-          break;
-        case ADD_COLUMN:
-          if (existColumn(tableId, alterTableDescProto.getAddColumn().getName())) {
-            throw new DuplicateColumnException(alterTableDescProto.getAddColumn().getName());
-          }
-          addNewColumn(tableId, alterTableDescProto.getAddColumn());
-          break;
-        case ADD_PARTITION:
-          partitionName = alterTableDescProto.getPartitionDesc().getPartitionName();
-          partitionDesc = getPartition(databaseName, tableName, partitionName);
-          if(partitionDesc != null) {
-            throw new DuplicatePartitionException(partitionName);
-          }
-          addPartition(tableId, alterTableDescProto.getPartitionDesc());
-          break;
-        case DROP_PARTITION:
-          partitionName = alterTableDescProto.getPartitionDesc().getPartitionName();
-          partitionDesc = getPartition(databaseName, tableName, partitionName);
-          if(partitionDesc == null) {
-            throw new UndefinedPartitionException(partitionName);
-          }
-          dropPartition(partitionDesc.getId());
-          break;
-        case SET_PROPERTY:
-          setProperties(tableId, alterTableDescProto.getParams());
-          break;
-        default:
+    int databaseId = getDatabaseId(databaseName);
+    int tableId = getTableId(databaseId, databaseName, tableName);
+
+    switch (alterTableDescProto.getAlterTableType()) {
+    case RENAME_TABLE:
+      if (existTable(databaseName, alterTableDescProto.getNewTableName())) {
+        throw new DuplicateTableException(alterTableDescProto.getNewTableName());
+      }
+      renameTable(tableId, alterTableDescProto.getNewTableName());
+      break;
+    case RENAME_COLUMN:
+      if (existColumn(tableId, alterTableDescProto.getAlterColumnName().getNewColumnName())) {
+        throw new DuplicateColumnException(alterTableDescProto.getAlterColumnName().getNewColumnName());
+      }
+      renameColumn(tableId, alterTableDescProto.getAlterColumnName());
+      break;
+    case ADD_COLUMN:
+      if (existColumn(tableId, alterTableDescProto.getAddColumn().getName())) {
+        throw new DuplicateColumnException(alterTableDescProto.getAddColumn().getName());
+      }
+      addNewColumn(tableId, alterTableDescProto.getAddColumn());
+      break;
+    case ADD_PARTITION:
+      partitionName = alterTableDescProto.getPartitionDesc().getPartitionName();
+      try {
+        // check if it exists
+        getPartition(databaseName, tableName, partitionName);
+        throw new DuplicatePartitionException(partitionName);
+      } catch (UndefinedPartitionException e) {
       }
-    } catch (SQLException sqlException) {
-      throw new TajoInternalError(sqlException);
+      addPartition(tableId, alterTableDescProto.getPartitionDesc());
+      break;
+    case DROP_PARTITION:
+      partitionName = alterTableDescProto.getPartitionDesc().getPartitionName();
+      partitionDesc = getPartition(databaseName, tableName, partitionName);
+      if (partitionDesc == null) {
+        throw new UndefinedPartitionException(partitionName);
+      }
+      dropPartition(partitionDesc.getId());
+      break;
+    case SET_PROPERTY:
+      setProperties(tableId, alterTableDescProto.getParams());
+      break;
+    default:
     }
-
   }
 
   private Map<String, String> getTableOptions(final int tableId) {
@@ -1099,7 +1119,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
-  private void renameTable(final int tableId, final String tableName) throws CatalogException {
+  private void renameTable(final int tableId, final String tableName) {
 
     final String updtaeRenameTableSql = "UPDATE " + TB_TABLES + " SET " + COL_TABLES_NAME + " = ? " + " WHERE TID = ?";
 
@@ -1126,7 +1146,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   private void renameColumn(final int tableId, final CatalogProtos.AlterColumnProto alterColumnProto)
-      throws CatalogException {
+      throws UndefinedColumnException {
 
     final String selectColumnSql =
         "SELECT COLUMN_NAME, DATA_TYPE, TYPE_LENGTH, ORDINAL_POSITION, NESTED_FIELD_NUM from " + TB_COLUMNS +
@@ -1203,7 +1223,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
-  private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) throws CatalogException {
+  private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) {
 
     final String insertNewColumnSql =
         "INSERT INTO " + TB_COLUMNS +
@@ -1251,7 +1271,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
-  public void addPartition(int tableId, CatalogProtos.PartitionDescProto partition) throws CatalogException {
+  private void addPartition(int tableId, CatalogProtos.PartitionDescProto partition) {
     Connection conn = null;
     PreparedStatement pstmt1 = null, pstmt2 = null;
 
@@ -1297,7 +1317,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
-  private void dropPartition(int partitionId) throws CatalogException {
+  private void dropPartition(int partitionId) {
     Connection conn = null;
     PreparedStatement pstmt1 = null, pstmt2 = null;
 
@@ -1331,7 +1351,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
-  private int getDatabaseId(String databaseName) throws SQLException, UndefinedDatabaseException {
+  private int getDatabaseId(String databaseName) throws UndefinedDatabaseException {
     String sql = String.format("SELECT DB_ID from %s WHERE DB_NAME = ?", TB_DATABASES);
 
     if (LOG.isDebugEnabled()) {
@@ -1352,13 +1372,15 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       }
 
       return res.getInt("DB_ID");
+    } catch (SQLException e) {
+      throw new TajoInternalError(e);
     } finally {
       CatalogUtil.closeQuietly(pstmt, res);
     }
   }
 
   @Override
-  public boolean existTable(String databaseName, final String tableName) throws CatalogException {
+  public boolean existTable(String databaseName, final String tableName) throws UndefinedDatabaseException {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
@@ -1390,7 +1412,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   public void dropTableInternal(Connection conn, String databaseName, final String tableName)
-      throws SQLException, UndefinedDatabaseException {
+      throws SQLException, UndefinedDatabaseException, UndefinedTableException {
 
     PreparedStatement pstmt = null;
 
@@ -1485,7 +1507,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void dropTable(String databaseName, final String tableName) throws CatalogException {
+  public void dropTable(String databaseName, final String tableName)
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     Connection conn = null;
     try {
       conn = getConnection();
@@ -1503,8 +1527,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     }
   }
 
-  public Pair<Integer, String> getDatabaseIdAndUri(String databaseName)
-      throws SQLException, UndefinedDatabaseException {
+  public Pair<Integer, String> getDatabaseIdAndUri(String databaseName) throws UndefinedDatabaseException {
 
     String sql =
         "SELECT DB_ID, SPACE_URI from " + TB_DATABASES + " natural join " + TB_SPACES + " WHERE db_name = ?";
@@ -1526,7 +1549,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         throw new UndefinedDatabaseException(databaseName);
       }
 
-      return new Pair<Integer, String>(res.getInt(1), res.getString(2) + "/" + databaseName);
+      return new Pair<>(res.getInt(1), res.getString(2) + "/" + databaseName);
+    } catch (SQLException e) {
+      throw new TajoInternalError(e);
     } finally {
       CatalogUtil.closeQuietly(pstmt, res);
     }
@@ -1534,19 +1559,19 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   @Override
   public CatalogProtos.TableDescProto getTable(String databaseName, String tableName)
-      throws CatalogException {
-    Connection conn = null;
+      throws UndefinedDatabaseException, UndefinedTableException {
+
+    Connection conn;
     ResultSet res = null;
     PreparedStatement pstmt = null;
-
     CatalogProtos.TableDescProto.Builder tableBuilder = null;
     String storeType;
 
+    Pair<Integer, String> databaseIdAndUri = getDatabaseIdAndUri(databaseName);
+
     try {
       tableBuilder = CatalogProtos.TableDescProto.newBuilder();
 
-      Pair<Integer, String> databaseIdAndUri = getDatabaseIdAndUri(databaseName);
-
       //////////////////////////////////////////
       // Geting Table Description
       //////////////////////////////////////////
@@ -1554,7 +1579,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
           "SELECT TID, " + COL_TABLES_NAME + ", TABLE_TYPE, PATH, STORE_TYPE FROM TABLES " +
               "WHERE DB_ID = ? AND " + COL_TABLES_NAME + "=?";
 
-
       if (LOG.isDebugEnabled()) {
         LOG.debug(sql);
       }
@@ -1566,7 +1590,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       res = pstmt.executeQuery();
 
       if (!res.next()) { // there is no table of the given name.
-        return null;
+         throw new UndefinedTableException(tableName);
       }
 
       int tableId = res.getInt(1);
@@ -1666,7 +1690,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       if (res.next()) {
         tableBuilder.setPartition(resultToPartitionMethodProto(databaseName, tableName, res));
       }
-    } catch (Throwable se) {
+    } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
       CatalogUtil.closeQuietly(pstmt, res);
@@ -1685,7 +1709,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public List<String> getAllTableNames(String databaseName) throws CatalogException {
+  public List<String> getAllTableNames(String databaseName) throws UndefinedDatabaseException {
     Connection conn = null;
     PreparedStatement pstmt = null;
     ResultSet res = null;
@@ -1718,7 +1742,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
   
   @Override
-  public List<TableDescriptorProto> getAllTables() throws CatalogException {
+  public List<TableDescriptorProto> getAllTables() {
     Connection conn = null;
     Statement stmt = null;
     ResultSet resultSet = null;
@@ -1767,7 +1791,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
   
   @Override
-  public List<TableOptionProto> getAllTableProperties() throws CatalogException {
+  public List<TableOptionProto> getAllTableProperties() {
     Connection conn = null;
     Statement stmt = null;
     ResultSet resultSet = null;
@@ -1802,7 +1826,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
   
   @Override
-  public List<TableStatsProto> getAllTableStats() throws CatalogException {
+  public List<TableStatsProto> getAllTableStats() {
     Connection conn = null;
     Statement stmt = null;
     ResultSet resultSet = null;
@@ -1834,7 +1858,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
   
   @Override
-  public List<ColumnProto> getAllColumns() throws CatalogException {
+  public List<ColumnProto> getAllColumns() {
     Connection conn = null;
     Statement stmt = null;
     ResultSet resultSet = null;
@@ -1880,74 +1904,55 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void addPartitionMethod(CatalogProtos.PartitionMethodProto proto) throws CatalogException {
+  public CatalogProtos.PartitionMethodProto getPartitionMethod(String databaseName, String tableName) throws
+      UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionMethodException {
+
     Connection conn = null;
+    ResultSet res = null;
     PreparedStatement pstmt = null;
 
+    final int databaseId = getDatabaseId(databaseName);
+    final int tableId = getTableId(databaseId, databaseName, tableName);
+    ensurePartitionTable(tableName, tableId);
+
     try {
-      String sql = "INSERT INTO " + TB_PARTITION_METHODS
-        + " (" + COL_TABLES_PK + ", PARTITION_TYPE,  EXPRESSION, EXPRESSION_SCHEMA) VALUES (?,?,?,?)";
+      String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS +
+          " WHERE " + COL_TABLES_PK + " = ? ";
 
       if (LOG.isDebugEnabled()) {
         LOG.debug(sql);
       }
 
-      String databaseName = proto.getTableIdentifier().getDatabaseName();
-      String tableName = proto.getTableIdentifier().getTableName();
-
-      int databaseId = getDatabaseId(databaseName);
-      int tableId = getTableId(databaseId, databaseName, tableName);
-
       conn = getConnection();
       pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, tableId);
-      pstmt.setString(2, proto.getPartitionType().name());
-      pstmt.setString(3, proto.getExpression());
-      pstmt.setBytes(4, proto.getExpressionSchema().toByteArray());
-      pstmt.executeUpdate();
-    } catch (SQLException se) {
-      throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(pstmt);
-    }
-  }
-
-  @Override
-  public void dropPartitionMethod(String databaseName, String tableName) throws CatalogException {
-    Connection conn = null;
-    PreparedStatement pstmt = null;
-
-    try {
-      String sql = "DELETE FROM " + TB_PARTITION_METHODS + " WHERE " + COL_TABLES_PK + " = ? ";
+      res = pstmt.executeQuery();
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
+      if (res.next()) {
+        return resultToPartitionMethodProto(databaseName, tableName, res);
+      } else {
+        throw new UndefinedPartitionMethodException(tableName);
       }
 
-      int databaseId = getDatabaseId(databaseName);
-      int tableId = getTableId(databaseId, databaseName, tableName);
-
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
-      pstmt.setInt(1, tableId);
-      pstmt.executeUpdate();
-    } catch (SQLException se) {
+    } catch (Throwable se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt);
+      CatalogUtil.closeQuietly(pstmt, res);
     }
   }
 
   @Override
-  public CatalogProtos.PartitionMethodProto getPartitionMethod(String databaseName, String tableName)
-      throws CatalogException {
+  public boolean existPartitionMethod(String databaseName, String tableName)
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
+    boolean exist = false;
 
     try {
       String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS +
-          " WHERE " + COL_TABLES_PK + " = ? ";
+          " WHERE " + COL_TABLES_PK + "= ?";
 
       if (LOG.isDebugEnabled()) {
         LOG.debug(sql);
@@ -1961,23 +1966,30 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
-      if (res.next()) {
-        return resultToPartitionMethodProto(databaseName, tableName, res);
-      }
-    } catch (Throwable se) {
+      exist = res.next();
+    } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
       CatalogUtil.closeQuietly(pstmt, res);
     }
-    return null;
+    return exist;
   }
 
-  @Override
-  public boolean existPartitionMethod(String databaseName, String tableName) throws CatalogException {
-    Connection conn = null;
+  /**
+   * Ensure if the table is partitioned table.
+   *
+   * @param tbName Table name
+   * @param tableId Table id
+   * @throws UndefinedTableException
+   * @throws UndefinedDatabaseException
+   * @throws UndefinedPartitionMethodException
+   */
+  private void ensurePartitionTable(String tbName, int tableId)
+      throws UndefinedTableException, UndefinedDatabaseException, UndefinedPartitionMethodException {
+
+    Connection conn;
     ResultSet res = null;
     PreparedStatement pstmt = null;
-    boolean exist = false;
 
     try {
       String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS +
@@ -1987,26 +1999,31 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         LOG.debug(sql);
       }
 
-      int databaseId = getDatabaseId(databaseName);
-      int tableId = getTableId(databaseId, databaseName, tableName);
-
       conn = getConnection();
       pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
-      exist = res.next();
+      if (!res.next()) {
+        throw new UndefinedPartitionMethodException(tbName);
+      }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
       CatalogUtil.closeQuietly(pstmt, res);
     }
-    return exist;
   }
 
   @Override
   public CatalogProtos.PartitionDescProto getPartition(String databaseName, String tableName,
-                                                       String partitionName) throws CatalogException {
+                                                       String partitionName)
+      throws UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionMethodException,
+      UndefinedPartitionException {
+
+    final int databaseId = getDatabaseId(databaseName);
+    final int tableId = getTableId(databaseId, databaseName, tableName);
+    ensurePartitionTable(tableName, tableId);
+
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
@@ -2020,9 +2037,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         LOG.debug(sql);
       }
 
-      int databaseId = getDatabaseId(databaseName);
-      int tableId = getTableId(databaseId, databaseName, tableName);
-
       conn = getConnection();
       pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, tableId);
@@ -2036,7 +2050,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         builder.setPartitionName(partitionName);
         setPartitionKeys(res.getInt(COL_PARTITIONS_PK), builder);
       } else {
-        return null;
+        throw new UndefinedPartitionException(partitionName);
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
@@ -2046,8 +2060,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     return builder.build();
   }
 
-  private void setPartitionKeys(int pid, PartitionDescProto.Builder partitionDesc) throws
-    CatalogException {
+  private void setPartitionKeys(int pid, PartitionDescProto.Builder partitionDesc) {
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
@@ -2075,13 +2088,19 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public List<PartitionDescProto> getPartitions(String databaseName, String tableName) throws CatalogException {
+  public List<PartitionDescProto> getPartitions(String databaseName, String tableName)
+      throws UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionMethodException {
+
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
     PartitionDescProto.Builder builder = null;
     List<PartitionDescProto> partitions = new ArrayList<PartitionDescProto>();
 
+    final int databaseId = getDatabaseId(databaseName);
+    final int tableId = getTableId(databaseId, databaseName, tableName);
+    ensurePartitionTable(tableName, tableId);
+
     try {
       String sql = "SELECT PATH, PARTITION_NAME, " + COL_PARTITIONS_PK + " FROM "
         + TB_PARTTIONS +" WHERE " + COL_TABLES_PK + " = ?  ";
@@ -2090,9 +2109,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         LOG.debug(sql);
       }
 
-      int databaseId = getDatabaseId(databaseName);
-      int tableId = getTableId(databaseId, databaseName, tableName);
-
       conn = getConnection();
       pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, tableId);
@@ -2114,7 +2130,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public List<TablePartitionProto> getAllPartitions() throws CatalogException {
+  public List<TablePartitionProto> getAllPartitions() {
     Connection conn = null;
     Statement stmt = null;
     ResultSet resultSet = null;
@@ -2149,9 +2165,15 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   @Override
   public void addPartitions(String databaseName, String tableName, List<CatalogProtos.PartitionDescProto> partitions
-    , boolean ifNotExists) throws CatalogException {
-    Connection conn = null;
+    , boolean ifNotExists) throws UndefinedDatabaseException, UndefinedTableException,
+      UndefinedPartitionMethodException {
+
+    final int databaseId = getDatabaseId(databaseName);
+    final int tableId = getTableId(databaseId, databaseName, tableName);
+    ensurePartitionTable(tableName, tableId);
 
+
+    Connection conn = null;
     // To delete existing partition keys
     PreparedStatement pstmt1 = null;
     // To delete existing partition;
@@ -2164,9 +2186,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     PartitionDescProto partitionDesc = null;
 
     try {
-      int databaseId = getDatabaseId(databaseName);
-      int tableId = getTableId(databaseId, databaseName, tableName);
-
       conn = getConnection();
       conn.setAutoCommit(false);
 
@@ -2181,11 +2200,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       int batchSize = conf.getInt(TajoConf.ConfVars.PARTITION_DYNAMIC_BULK_INSERT_BATCH_SIZE.varname, 1000);
       for(currentIndex = 0; currentIndex < partitions.size(); currentIndex++) {
         PartitionDescProto partition = partitions.get(currentIndex);
-        partitionDesc = getPartition(databaseName, tableName, partition.getPartitionName());
 
-        // Delete existing partition and partition keys
-        if (partitionDesc != null) {
-          if(ifNotExists) {
+        try {
+          partitionDesc = getPartition(databaseName, tableName, partition.getPartitionName());
+          // Delete existing partition and partition keys
+          if (ifNotExists) {
             pstmt1.setInt(1, partitionDesc.getId());
             pstmt1.addBatch();
             pstmt1.clearParameters();
@@ -2193,9 +2212,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
             pstmt2.setInt(1, partitionDesc.getId());
             pstmt2.addBatch();
             pstmt2.clearParameters();
-          } else {
-            throw new DuplicatePartitionException(partition.getPartitionName());
           }
+        } catch (UndefinedPartitionException e) {
         }
 
         // Insert partition
@@ -2261,7 +2279,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void createIndex(final IndexDescProto proto) throws CatalogException {
+  public void createIndex(final IndexDescProto proto) throws UndefinedDatabaseException, UndefinedTableException {
     Connection conn = null;
     PreparedStatement pstmt = null;
 
@@ -2324,7 +2342,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public void dropIndex(String databaseName, final String indexName) throws CatalogException {
+  public void dropIndex(String databaseName, final String indexName) throws UndefinedDatabaseException {
     Connection conn = null;
     PreparedStatement pstmt = null;
 
@@ -2358,7 +2376,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
       if (!res.next()) {
-        throw new TajoInternalError("Cannot get any table name from TID");
+        throw new TajoInternalError("Inconsistent data: no table corresponding to TID " + tableId);
       }
       return res.getString(1);
     } finally {
@@ -2371,7 +2389,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   @Override
   public IndexDescProto getIndexByName(String databaseName, final String indexName)
-      throws CatalogException {
+      throws UndefinedDatabaseException, UndefinedIndexException {
+
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
@@ -2392,13 +2411,20 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       pstmt.setString(2, indexName);
       res = pstmt.executeQuery();
       if (!res.next()) {
-        throw new TajoInternalError("There is no index matched to " + indexName);
+        throw new UndefinedIndexException(indexName);
       }
       IndexDescProto.Builder builder = IndexDescProto.newBuilder();
       resultToIndexDescProtoBuilder(builder, res);
       String tableName = getTableName(conn, res.getInt(COL_TABLES_PK));
       builder.setTableIdentifier(CatalogUtil.buildTableIdentifier(databaseName, tableName));
-      builder.setTargetRelationSchema(getTable(databaseName, tableName).getSchema());
+
+      try {
+        builder.setTargetRelationSchema(getTable(databaseName, tableName).getSchema());
+      } catch (UndefinedTableException e) {
+        throw new TajoInternalError(
+            "Inconsistent table and index information: table " + tableName + " does not exists.");
+      }
+
       proto = builder.build();
     } catch (SQLException se) {
       throw new TajoInternalError(se);
@@ -2411,7 +2437,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   @Override
   public IndexDescProto getIndexByColumns(String databaseName, String tableName, String[] columnNames)
-      throws CatalogException {
+      throws UndefinedDatabaseException, UndefinedTableException, UndefinedIndexException {
+
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
@@ -2441,7 +2468,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
       pstmt.setString(3, unifiedName);
       res = pstmt.executeQuery();
       if (!res.next()) {
-        throw new TajoInternalError("ERROR: there is no index matched to " + unifiedName);
+        throw new UndefinedIndexException(unifiedName);
       }
 
       IndexDescProto.Builder builder = IndexDescProto.newBuilder();
@@ -2459,7 +2486,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public boolean existIndexByName(String databaseName, final String indexName) throws CatalogException {
+  public boolean existIndexByName(String databaseName, final String indexName) throws UndefinedDatabaseException {
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
@@ -2493,7 +2520,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   @Override
   public boolean existIndexByColumns(String databaseName, String tableName, String[] columnNames)
-      throws CatalogException {
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     Connection conn = null;
     ResultSet res = null;
     PreparedStatement pstmt = null;
@@ -2535,7 +2563,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   @Override
   public List<String> getAllIndexNamesByTable(final String databaseName, final String tableName)
-      throws CatalogException {
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     ResultSet res = null;
     PreparedStatement pstmt = null;
     final List<String> indexNames = new ArrayList<String>();
@@ -2569,10 +2598,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public boolean existIndexesByTable(String databaseName, String tableName) throws CatalogException {
+  public boolean existIndexesByTable(String databaseName, String tableName)
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     ResultSet res = null;
     PreparedStatement pstmt = null;
-    final List<String> indexNames = new ArrayList<String>();
 
     try {
       final int databaseId = getDatabaseId(databaseName);
@@ -2599,12 +2629,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public List<IndexDescProto> getAllIndexes() throws CatalogException {
+  public List<IndexDescProto> getAllIndexes() throws UndefinedDatabaseException {
     List<IndexDescProto> indexDescProtos = TUtil.newList();
     for (String databaseName : getAllDatabaseNames()) {
       for (String tableName : getAllTableNames(databaseName)) {
-        for (String indexName: getAllIndexNamesByTable(databaseName, tableName)) {
-          indexDescProtos.add(getIndexByName(databaseName, indexName));
+        try {
+          for (String indexName: getAllIndexNamesByTable(databaseName, tableName)) {
+            indexDescProtos.add(getIndexByName(databaseName, indexName));
+          }
+        } catch (UndefinedTableException e) {
+          LOG.warn(e);
+        } catch (UndefinedIndexException e) {
+          throw new TajoInternalError(e);
         }
       }
     }
@@ -2634,19 +2670,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
     builder.setIsClustered(res.getBoolean("is_clustered"));
   }
 
-  /**
-   * INDEXS table doesn't store type_length, so we need another resultToColumnProto method
-   */
-  private ColumnProto indexResultToColumnProto(final ResultSet res) throws SQLException {
-    ColumnProto.Builder builder = ColumnProto.newBuilder();
-    builder.setName(res.getString("column_name").trim());
-
-    Type type = getDataType(res.getString("data_type").trim());
-    builder.setDataType(CatalogUtil.newSimpleDataType(type));
-
-    return builder.build();
-  }
-
   private ColumnProto resultToColumnProto(final ResultSet res) throws SQLException {
     ColumnProto.Builder builder = ColumnProto.newBuilder();
     builder.setName(res.getString("column_name").trim());
@@ -2692,12 +2715,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   private CatalogProtos.PartitionMethodProto resultToPartitionMethodProto(final String databaseName,
                                                                           final String tableName,
                                                                           final ResultSet res)
-      throws SQLException, InvalidProtocolBufferException {
-    CatalogProtos.PartitionMethodProto.Builder partBuilder = CatalogProtos.PartitionMethodProto.newBuilder();
-    partBuilder.setTableIdentifier(CatalogUtil.buildTableIdentifier(databaseName, tableName));
-    partBuilder.setPartitionType(CatalogProtos.PartitionType.valueOf(res.getString("partition_type")));
-    partBuilder.setExpression(res.getString("expression"));
-    partBuilder.setExpressionSchema(SchemaProto.parseFrom(res.getBytes("expression_schema")));
+      throws SQLException {
+
+    CatalogProtos.PartitionMethodProto.Builder partBuilder;
+    try {
+      partBuilder = CatalogProtos.PartitionMethodProto.newBuilder();
+      partBuilder.setTableIdentifier(CatalogUtil.buildTableIdentifier(databaseName, tableName));
+      partBuilder.setPartitionType(CatalogProtos.PartitionType.valueOf(res.getString("partition_type")));
+      partBuilder.setExpression(res.getString("expression"));
+      partBuilder.setExpressionSchema(SchemaProto.parseFrom(res.getBytes("expression_schema")));
+    } catch (InvalidProtocolBufferException e) {
+      throw new TajoInternalError(e);
+    }
     return partBuilder.build();
   }
 
@@ -2708,27 +2737,27 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
   }
 
   @Override
-  public final void addFunction(final FunctionDesc func) throws CatalogException {
+  public final void addFunction(final FunctionDesc func) {
     // TODO - not implemented yet
   }
 
   @Override
-  public final void deleteFunction(final FunctionDesc func) throws CatalogException {
+  public final void deleteFunction(final FunctionDesc func) {
     // TODO - not implemented yet
   }
 
   @Override
-  public final void existFunction(final FunctionDesc func) throws CatalogException {
+  public final void existFunction(final FunctionDesc func) {
     // TODO - not implemented yet
   }
 
   @Override
-  public final List<String> getAllFunctionNames() throws CatalogException {
+  public final List<String> getAllFunctionNames() {
     // TODO - not implemented yet
     return null;
   }
 
-  private boolean existColumn(final int tableId, final String columnName) throws CatalogException {
+  private boolean existColumn(final int tableId, final String columnName) {
     Connection conn ;
     PreparedStatement pstmt = null;
     ResultSet res = null;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
index ef9ddd0..64a0e86 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/CatalogStore.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * 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.
@@ -18,126 +18,134 @@
 
 package org.apache.tajo.catalog.store;
 
+import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
+import org.apache.tajo.exception.*;
 
 import java.io.Closeable;
-
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto;
-
+import java.sql.SQLException;
 import java.util.Collection;
 import java.util.List;
 
 public interface CatalogStore extends Closeable {
   /*************************** Tablespace ******************************/
-  void createTablespace(String spaceName, String spaceUri) throws CatalogException;
+  void createTablespace(String spaceName, String spaceUri) throws DuplicateTablespaceException;
+
+  boolean existTablespace(String spaceName);
 
-  boolean existTablespace(String spaceName) throws CatalogException;
+  void dropTablespace(String spaceName) throws UndefinedTablespaceException, UndefinedTableException;
 
-  void dropTablespace(String spaceName) throws CatalogException;
+  Collection<String> getAllTablespaceNames();
 
-  Collection<String> getAllTablespaceNames() throws CatalogException;
-  
-  List<TablespaceProto> getTablespaces() throws CatalogException;
+  List<TablespaceProto> getTablespaces();
 
-  TablespaceProto getTablespace(String spaceName) throws CatalogException;
+  TablespaceProto getTablespace(String spaceName) throws UndefinedTablespaceException;
 
-  void alterTablespace(AlterTablespaceProto alterProto) throws CatalogException;
+  void alterTablespace(AlterTablespaceProto alterProto) throws UndefinedTablespaceException;
 
   /*************************** Database ******************************/
-  void createDatabase(String databaseName, String tablespaceName) throws CatalogException;
+  void createDatabase(String databaseName, String tablespaceName) throws UndefinedTablespaceException,
+      DuplicateDatabaseException;
 
-  boolean existDatabase(String databaseName) throws CatalogException;
+  boolean existDatabase(String databaseName);
 
-  void dropDatabase(String databaseName) throws CatalogException;
+  void dropDatabase(String databaseName) throws UndefinedDatabaseException, UndefinedTableException;
 
-  Collection<String> getAllDatabaseNames() throws CatalogException;
-  
-  List<DatabaseProto> getAllDatabases() throws CatalogException;
+  Collection<String> getAllDatabaseNames();
+
+  List<DatabaseProto> getAllDatabases();
 
   /*************************** TABLE ******************************/
-  void createTable(CatalogProtos.TableDescProto desc) throws CatalogException;
-  
-  boolean existTable(String databaseName, String tableName) throws CatalogException;
-  
-  void dropTable(String databaseName, String tableName) throws CatalogException;
-  
-  CatalogProtos.TableDescProto getTable(String databaseName, String tableName) throws CatalogException;
-  
-  List<String> getAllTableNames(String databaseName) throws CatalogException;
-
-  void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws CatalogException;
-  
-  List<TableDescriptorProto> getAllTables() throws CatalogException;
-
-  List<TableOptionProto> getAllTableProperties() throws CatalogException;
-  
-  List<TableStatsProto> getAllTableStats() throws CatalogException;
-  
-  List<ColumnProto> getAllColumns() throws CatalogException;
-
-  void updateTableStats(CatalogProtos.UpdateTableStatsProto statsProto) throws CatalogException;
+  void createTable(CatalogProtos.TableDescProto desc) throws UndefinedDatabaseException, DuplicateTableException;
 
-  /************************ PARTITION METHOD **************************/
-  void addPartitionMethod(PartitionMethodProto partitionMethodProto) throws CatalogException;
+  boolean existTable(String databaseName, String tableName) throws UndefinedDatabaseException;
+
+  void dropTable(String databaseName, String tableName) throws UndefinedDatabaseException, UndefinedTableException;
+
+  CatalogProtos.TableDescProto getTable(String databaseName, String tableName) throws UndefinedDatabaseException,
+      UndefinedTableException;
+
+  List<String> getAllTableNames(String databaseName) throws UndefinedDatabaseException;
+
+  void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws UndefinedDatabaseException,
+      DuplicateTableException, DuplicateColumnException, DuplicatePartitionException, UndefinedPartitionException,
+      UndefinedTableException, UndefinedColumnException, UndefinedPartitionMethodException;
+
+  List<TableDescriptorProto> getAllTables();
 
-  PartitionMethodProto getPartitionMethod(String databaseName, String tableName)
-      throws CatalogException;
+  List<TableOptionProto> getAllTableProperties();
 
-  boolean existPartitionMethod(String databaseName, String tableName) throws CatalogException;
+  List<TableStatsProto> getAllTableStats();
 
-  void dropPartitionMethod(String dbName, String tableName) throws CatalogException;
+  List<ColumnProto> getAllColumns();
 
+  void updateTableStats(CatalogProtos.UpdateTableStatsProto statsProto) throws UndefinedDatabaseException, UndefinedTableException;
+
+  /************************ PARTITION METHOD **************************/
+  PartitionMethodProto getPartitionMethod(String databaseName, String tableName) throws UndefinedDatabaseException,
+      UndefinedTableException, UndefinedPartitionMethodException;
+
+  boolean existPartitionMethod(String databaseName, String tableName) throws UndefinedDatabaseException,
+      UndefinedTableException;
 
   /************************** PARTITIONS *****************************/
   /**
    * Get all partitions of a table
    * @param tableName the table name
    * @return
-   * @throws CatalogException
+   * @throws TajoException
    */
-  List<CatalogProtos.PartitionDescProto> getPartitions(String databaseName, String tableName) throws CatalogException;
+  List<CatalogProtos.PartitionDescProto> getPartitions(String databaseName, String tableName) throws
+      UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionMethodException;
 
   CatalogProtos.PartitionDescProto getPartition(String databaseName, String tableName,
-                                                String partitionName) throws CatalogException;
+                                                String partitionName)
+      throws UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionException,
+      UndefinedPartitionMethodException;
 
-  List<TablePartitionProto> getAllPartitions() throws CatalogException;
+  List<TablePartitionProto> getAllPartitions();
 
   void addPartitions(String databaseName, String tableName, List<CatalogProtos.PartitionDescProto> partitions
-    , boolean ifNotExists) throws CatalogException;
+      , boolean ifNotExists) throws UndefinedDatabaseException,
+      UndefinedTableException, DuplicatePartitionException, UndefinedPartitionException,
+      UndefinedPartitionMethodException;
 
   /**************************** INDEX *******************************/
-  void createIndex(IndexDescProto proto) throws CatalogException;
-  
-  void dropIndex(String databaseName, String indexName) throws CatalogException;
-  
-  IndexDescProto getIndexByName(String databaseName, String indexName) throws CatalogException;
+  void createIndex(IndexDescProto proto) throws UndefinedDatabaseException, UndefinedTableException,
+      DuplicateIndexException;
+
+  void dropIndex(String databaseName, String indexName) throws UndefinedDatabaseException,
+      UndefinedTableException, UndefinedIndexException;
+
+  IndexDescProto getIndexByName(String databaseName, String indexName) throws UndefinedDatabaseException,
+      UndefinedIndexException;
 
-  IndexDescProto getIndexByColumns(String databaseName, String tableName, String[] columnNames)
-      throws CatalogException;
-  
-  boolean existIndexByName(String databaseName, String indexName) throws CatalogException;
+  IndexDescProto getIndexByColumns(String databaseName, String tableName, String[] columnNames) throws
+      UndefinedDatabaseException, UndefinedTableException, UndefinedIndexException
+      ;
 
-  boolean existIndexByColumns(String databaseName, String tableName, String[] columnNames)
-      throws CatalogException;
+  boolean existIndexByName(String databaseName, String indexName) throws UndefinedDatabaseException;
 
-  List<String> getAllIndexNamesByTable(String databaseName, String tableName) throws CatalogException;
+  boolean existIndexByColumns(String databaseName, String tableName, String[] columnNames) throws
+      UndefinedDatabaseException, UndefinedTableException;
 
-  boolean existIndexesByTable(String databaseName, String tableName) throws CatalogException;
+  List<String> getAllIndexNamesByTable(String databaseName, String tableName) throws UndefinedDatabaseException, UndefinedTableException;
 
-  List<IndexDescProto> getAllIndexes() throws CatalogException;
+  boolean existIndexesByTable(String databaseName, String tableName)
+      throws UndefinedDatabaseException, UndefinedTableException;
+
+  List<IndexDescProto> getAllIndexes() throws UndefinedDatabaseException;
 
   /************************** FUNCTION *****************************/
 
-  
-  void addFunction(FunctionDesc func) throws CatalogException;
-  
-  void deleteFunction(FunctionDesc func) throws CatalogException;
-  
-  void existFunction(FunctionDesc func) throws CatalogException;
-  
-  List<String> getAllFunctionNames() throws CatalogException;
+
+  void addFunction(FunctionDesc func);
+
+  void deleteFunction(FunctionDesc func);
+
+  void existFunction(FunctionDesc func);
+
+  List<String> getAllFunctionNames();
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
index d9ec3d3..19a4f13 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/DerbyStore.java
@@ -23,11 +23,12 @@ package org.apache.tajo.catalog.store;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.exception.TajoInternalError;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
 
 public class DerbyStore extends AbstractDBStore {
 
@@ -37,7 +38,7 @@ public class DerbyStore extends AbstractDBStore {
     return CATALOG_DRIVER;
   }
 
-  public DerbyStore(final Configuration conf) throws InternalException {
+  public DerbyStore(final Configuration conf) {
     super(conf);
   }
 
@@ -46,7 +47,7 @@ public class DerbyStore extends AbstractDBStore {
   }
 
   @Override
-  public String readSchemaFile(String filename) throws CatalogException {
+  public String readSchemaFile(String filename) {
     return super.readSchemaFile("derby/" + filename);
   }
 
@@ -72,7 +73,7 @@ public class DerbyStore extends AbstractDBStore {
   }
 
   @Override
-  protected void createDatabaseDependants() throws CatalogException {
+  protected void createDatabaseDependants() {
     String schemaName = catalogSchemaManager.getCatalogStore().getSchema().getSchemaName();
     Statement stmt = null;
     

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MariaDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MariaDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MariaDBStore.java
index cb661ac..8275e62 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MariaDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MariaDBStore.java
@@ -22,8 +22,6 @@
 package org.apache.tajo.catalog.store;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.exception.InternalException;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -32,7 +30,7 @@ import java.sql.SQLException;
 public class MariaDBStore extends AbstractDBStore {
   private static final String CATALOG_DRIVER = "org.mariadb.jdbc.Driver";
 
-  public MariaDBStore(Configuration conf) throws InternalException {
+  public MariaDBStore(Configuration conf) {
     super(conf);
   }
 
@@ -52,6 +50,6 @@ public class MariaDBStore extends AbstractDBStore {
   }
 
   @Override
-  protected void createDatabaseDependants() throws CatalogException {
+  protected void createDatabaseDependants() {
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
index c822482..7227366 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java
@@ -26,9 +26,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.*;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
@@ -37,7 +37,6 @@ import java.io.IOException;
 import java.util.*;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto.AlterTablespaceType;
-import static org.apache.tajo.catalog.proto.CatalogProtos.*;
 
 /**
  * CatalogServer guarantees that all operations are thread-safe.
@@ -62,7 +61,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void createTablespace(String spaceName, String spaceUri) throws CatalogException {
+  public void createTablespace(String spaceName, String spaceUri) throws DuplicateTablespaceException {
     if (tablespaces.containsKey(spaceName)) {
       throw new DuplicateTablespaceException(spaceName);
     }
@@ -71,12 +70,12 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public boolean existTablespace(String spaceName) throws CatalogException {
+  public boolean existTablespace(String spaceName) {
     return tablespaces.containsKey(spaceName);
   }
 
   @Override
-  public void dropTablespace(String spaceName) throws CatalogException {
+  public void dropTablespace(String spaceName) throws UndefinedTablespaceException {
     if (!tablespaces.containsKey(spaceName)) {
       throw new UndefinedTablespaceException(spaceName);
     }
@@ -84,12 +83,12 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public Collection<String> getAllTablespaceNames() throws CatalogException {
+  public Collection<String> getAllTablespaceNames() {
     return tablespaces.keySet();
   }
   
   @Override
-  public List<TablespaceProto> getTablespaces() throws CatalogException {
+  public List<TablespaceProto> getTablespaces() {
     List<TablespaceProto> tablespaceList = TUtil.newList();
     int tablespaceId = 0;
     
@@ -105,7 +104,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public TablespaceProto getTablespace(String spaceName) throws CatalogException {
+  public TablespaceProto getTablespace(String spaceName) throws UndefinedTablespaceException {
     if (!tablespaces.containsKey(spaceName)) {
       throw new UndefinedTablespaceException(spaceName);
     }
@@ -117,7 +116,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void alterTablespace(CatalogProtos.AlterTablespaceProto alterProto) throws CatalogException {
+  public void alterTablespace(CatalogProtos.AlterTablespaceProto alterProto) throws UndefinedTablespaceException {
     if (!tablespaces.containsKey(alterProto.getSpaceName())) {
       throw new UndefinedTablespaceException(alterProto.getSpaceName());
     }
@@ -133,7 +132,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void createDatabase(String databaseName, String tablespaceName) throws CatalogException {
+  public void createDatabase(String databaseName, String tablespaceName) throws DuplicateDatabaseException {
     if (databases.containsKey(databaseName)) {
       throw new DuplicateDatabaseException(databaseName);
     }
@@ -144,12 +143,12 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public boolean existDatabase(String databaseName) throws CatalogException {
+  public boolean existDatabase(String databaseName) {
     return databases.containsKey(databaseName);
   }
 
   @Override
-  public void dropDatabase(String databaseName) throws CatalogException {
+  public void dropDatabase(String databaseName) throws UndefinedDatabaseException {
     if (!databases.containsKey(databaseName)) {
       throw new UndefinedDatabaseException(databaseName);
     }
@@ -159,12 +158,12 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public Collection<String> getAllDatabaseNames() throws CatalogException {
+  public Collection<String> getAllDatabaseNames() {
     return databases.keySet();
   }
   
   @Override
-  public List<DatabaseProto> getAllDatabases() throws CatalogException {
+  public List<DatabaseProto> getAllDatabases() {
     List<DatabaseProto> databaseList = new ArrayList<DatabaseProto>();
     int dbId = 0;
     
@@ -194,7 +193,9 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void createTable(CatalogProtos.TableDescProto request) throws CatalogException {
+  public void createTable(CatalogProtos.TableDescProto request)
+      throws UndefinedDatabaseException, DuplicateTableException {
+
     String [] splitted = CatalogUtil.splitTableName(request.getTableName());
     if (splitted.length == 1) {
       throw new IllegalArgumentException("createTable() requires a qualified table name, but it is \""
@@ -213,7 +214,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void updateTableStats(CatalogProtos.UpdateTableStatsProto request) throws CatalogException {
+  public void updateTableStats(CatalogProtos.UpdateTableStatsProto request) throws UndefinedDatabaseException {
     String [] splitted = CatalogUtil.splitTableName(request.getTableName());
     if (splitted.length == 1) {
       throw new IllegalArgumentException("createTable() requires a qualified table name, but it is \""
@@ -230,14 +231,14 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public boolean existTable(String dbName, String tbName) throws CatalogException {
+  public boolean existTable(String dbName, String tbName) throws UndefinedDatabaseException {
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, dbName);
 
     return database.containsKey(tbName);
   }
 
   @Override
-  public void dropTable(String dbName, String tbName) throws CatalogException {
+  public void dropTable(String dbName, String tbName) throws UndefinedDatabaseException, UndefinedTableException {
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, dbName);
 
     if (database.containsKey(tbName)) {
@@ -251,7 +252,8 @@ public class MemStore implements CatalogStore {
    * @see CatalogStore#alterTable(AlterTableDesc)
    */
   @Override
-  public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) throws CatalogException {
+  public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto)
+      throws UndefinedDatabaseException, DuplicateTableException, DuplicatePartitionException, UndefinedPartitionException {
 
     String[] split = CatalogUtil.splitTableName(alterTableDescProto.getTableName());
     if (split.length == 1) {
@@ -383,7 +385,8 @@ public class MemStore implements CatalogStore {
    */
   @Override
   public CatalogProtos.TableDescProto getTable(String databaseName, String tableName)
-      throws CatalogException {
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, databaseName);
 
     if (database.containsKey(tableName)) {
@@ -403,13 +406,13 @@ public class MemStore implements CatalogStore {
    * @see CatalogStore#getAllTableNames()
    */
   @Override
-  public List<String> getAllTableNames(String databaseName) throws CatalogException {
+  public List<String> getAllTableNames(String databaseName) throws UndefinedDatabaseException {
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, databaseName);
     return new ArrayList<String>(database.keySet());
   }
   
   @Override
-  public List<TableDescriptorProto> getAllTables() throws CatalogException {
+  public List<TableDescriptorProto> getAllTables() {
     List<TableDescriptorProto> tableList = new ArrayList<CatalogProtos.TableDescriptorProto>();
     int dbId = 0, tableId = 0;
     
@@ -439,7 +442,7 @@ public class MemStore implements CatalogStore {
   }
   
   @Override
-  public List<TableOptionProto> getAllTableProperties() throws CatalogException {
+  public List<TableOptionProto> getAllTableProperties() {
     List<TableOptionProto> optionList = new ArrayList<CatalogProtos.TableOptionProto>();
     int tid = 0;
     
@@ -468,7 +471,7 @@ public class MemStore implements CatalogStore {
   }
   
   @Override
-  public List<TableStatsProto> getAllTableStats() throws CatalogException {
+  public List<TableStatsProto> getAllTableStats() {
     List<TableStatsProto> statList = new ArrayList<CatalogProtos.TableStatsProto>();
     int tid = 0;
     
@@ -494,7 +497,7 @@ public class MemStore implements CatalogStore {
   }
   
   @Override
-  public List<ColumnProto> getAllColumns() throws CatalogException {
+  public List<ColumnProto> getAllColumns() {
     List<ColumnProto> columnList = new ArrayList<CatalogProtos.ColumnProto>();
     int tid = 0;
     
@@ -521,13 +524,9 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void addPartitionMethod(CatalogProtos.PartitionMethodProto partitionMethodProto) throws CatalogException {
-    throw new RuntimeException("not supported!");
-  }
-
-  @Override
   public CatalogProtos.PartitionMethodProto getPartitionMethod(String databaseName, String tableName)
-      throws CatalogException {
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, databaseName);
 
     if (database.containsKey(tableName)) {
@@ -540,7 +539,8 @@ public class MemStore implements CatalogStore {
 
   @Override
   public boolean existPartitionMethod(String databaseName, String tableName)
-      throws CatalogException {
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     Map<String, CatalogProtos.TableDescProto> database = checkAndGetDatabaseNS(databases, databaseName);
 
     if (database.containsKey(tableName)) {
@@ -552,12 +552,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void dropPartitionMethod(String databaseName, String tableName) throws CatalogException {
-    throw new RuntimeException("not supported!");
-  }
-
-  @Override
-  public List<CatalogProtos.PartitionDescProto> getPartitions(String databaseName, String tableName) throws CatalogException {
+  public List<CatalogProtos.PartitionDescProto> getPartitions(String databaseName, String tableName) {
     List<CatalogProtos.PartitionDescProto> protos = new ArrayList<CatalogProtos.PartitionDescProto>();
 
     if (partitions.containsKey(tableName)) {
@@ -570,7 +565,7 @@ public class MemStore implements CatalogStore {
 
   @Override
   public CatalogProtos.PartitionDescProto getPartition(String databaseName, String tableName,
-                                                       String partitionName) throws CatalogException {
+                                                       String partitionName) throws UndefinedPartitionException {
     if (partitions.containsKey(tableName) && partitions.get(tableName).containsKey(partitionName)) {
       return partitions.get(tableName).get(partitionName);
     } else {
@@ -578,7 +573,7 @@ public class MemStore implements CatalogStore {
     }
   }
 
-  public List<TablePartitionProto> getAllPartitions() throws CatalogException {
+  public List<TablePartitionProto> getAllPartitions() {
     List<TablePartitionProto> protos = new ArrayList<TablePartitionProto>();
     Set<String> tables = partitions.keySet();
     for (String table : tables) {
@@ -604,14 +599,19 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void addPartitions(String databaseName, String tableName, List<CatalogProtos.PartitionDescProto> partitions
-    , boolean ifNotExists) throws CatalogException {
+  public void addPartitions(String databaseName, String tableName, List<CatalogProtos.PartitionDescProto> partitions,
+    boolean ifNotExists) throws DuplicatePartitionException {
+
     for(CatalogProtos.PartitionDescProto partition: partitions) {
       String partitionName = partition.getPartitionName();
 
       if (this.partitions.containsKey(tableName) && this.partitions.get(tableName).containsKey(partitionName)) {
         if (ifNotExists) {
-          dropPartition(databaseName, tableName, partitionName);
+          try {
+            dropPartition(databaseName, tableName, partitionName);
+          } catch (UndefinedPartitionException e) {
+            // ignore
+          }
         } else {
           throw new DuplicatePartitionException(partitionName);
         }
@@ -624,13 +624,14 @@ public class MemStore implements CatalogStore {
    * @see CatalogStore#createIndex(nta.catalog.proto.CatalogProtos.IndexDescProto)
    */
   @Override
-  public void createIndex(IndexDescProto proto) throws CatalogException {
+  public void createIndex(IndexDescProto proto) throws UndefinedDatabaseException, UndefinedTableException,
+      DuplicateIndexException {
+
     final String databaseName = proto.getTableIdentifier().getDatabaseName();
     final String tableName = CatalogUtil.extractSimpleName(proto.getTableIdentifier().getTableName());
-
+    getTable(databaseName, tableName);
     Map<String, IndexDescProto> index = checkAndGetDatabaseNS(indexes, databaseName);
     Map<String, IndexDescProto> indexByColumn = checkAndGetDatabaseNS(indexesByColumn, databaseName);
-    TableDescProto tableDescProto = getTable(databaseName, tableName);
 
     if (index.containsKey(proto.getIndexName())) {
       throw new DuplicateIndexException(proto.getIndexName());
@@ -649,7 +650,9 @@ public class MemStore implements CatalogStore {
    * @see CatalogStore#dropIndex(java.lang.String)
    */
   @Override
-  public void dropIndex(String databaseName, String indexName) throws CatalogException {
+  public void dropIndex(String databaseName, String indexName) throws UndefinedDatabaseException,
+      UndefinedIndexException, UndefinedTableException {
+
     Map<String, IndexDescProto> index = checkAndGetDatabaseNS(indexes, databaseName);
     Map<String, IndexDescProto> indexByColumn = checkAndGetDatabaseNS(indexesByColumn, databaseName);
     if (!index.containsKey(indexName)) {
@@ -657,7 +660,7 @@ public class MemStore implements CatalogStore {
     }
     IndexDescProto proto = index.get(indexName);
     final String tableName = CatalogUtil.extractSimpleName(proto.getTableIdentifier().getTableName());
-    TableDescProto tableDescProto = getTable(databaseName, tableName);
+    getTable(databaseName, tableName);
     index.remove(indexName);
     String originalTableName = proto.getTableIdentifier().getTableName();
     String simpleTableName = CatalogUtil.extractSimpleName(originalTableName);
@@ -670,7 +673,9 @@ public class MemStore implements CatalogStore {
    * @see CatalogStore#getIndexByName(java.lang.String)
    */
   @Override
-  public IndexDescProto getIndexByName(String databaseName, String indexName) throws CatalogException {
+  public IndexDescProto getIndexByName(String databaseName, String indexName)
+      throws UndefinedDatabaseException, UndefinedIndexException {
+
     Map<String, IndexDescProto> index = checkAndGetDatabaseNS(indexes, databaseName);
     if (!index.containsKey(indexName)) {
       throw new UndefinedIndexException(indexName);
@@ -680,7 +685,9 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public IndexDescProto getIndexByColumns(String databaseName, String tableName, String[] columnNames) throws CatalogException {
+  public IndexDescProto getIndexByColumns(String databaseName, String tableName, String[] columnNames)
+      throws UndefinedDatabaseException, UndefinedTableException, UndefinedIndexException {
+
     Map<String, IndexDescProto> indexByColumn = checkAndGetDatabaseNS(indexesByColumn, databaseName);
     String simpleTableName = CatalogUtil.extractSimpleName(tableName);
     TableDescProto tableDescProto = getTable(databaseName, simpleTableName);
@@ -694,13 +701,15 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public boolean existIndexByName(String databaseName, String indexName) throws CatalogException {
+  public boolean existIndexByName(String databaseName, String indexName) throws UndefinedDatabaseException {
     Map<String, IndexDescProto> index = checkAndGetDatabaseNS(indexes, databaseName);
     return index.containsKey(indexName);
   }
 
   @Override
-  public boolean existIndexByColumns(String databaseName, String tableName, String[] columnNames) throws CatalogException {
+  public boolean existIndexByColumns(String databaseName, String tableName, String[] columnNames)
+      throws UndefinedDatabaseException, UndefinedTableException {
+
     Map<String, IndexDescProto> indexByColumn = checkAndGetDatabaseNS(indexesByColumn, databaseName);
     TableDescProto tableDescProto = getTable(databaseName, tableName);
     return indexByColumn.containsKey(
@@ -709,7 +718,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public List<String> getAllIndexNamesByTable(String databaseName, String tableName) throws CatalogException {
+  public List<String> getAllIndexNamesByTable(String databaseName, String tableName) throws UndefinedDatabaseException {
     List<String> indexNames = new ArrayList<String>();
     Map<String, IndexDescProto> indexByColumn = checkAndGetDatabaseNS(indexesByColumn, databaseName);
     String simpleTableName = CatalogUtil.extractSimpleName(tableName);
@@ -723,7 +732,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public boolean existIndexesByTable(String databaseName, String tableName) throws CatalogException {
+  public boolean existIndexesByTable(String databaseName, String tableName) throws UndefinedDatabaseException {
     Map<String, IndexDescProto> indexByColumn = checkAndGetDatabaseNS(indexesByColumn, databaseName);
     String simpleTableName = CatalogUtil.extractSimpleName(tableName);
     for (IndexDescProto proto : indexByColumn.values()) {
@@ -735,7 +744,7 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public List<IndexDescProto> getAllIndexes() throws CatalogException {
+  public List<IndexDescProto> getAllIndexes() {
     List<IndexDescProto> indexDescProtos = TUtil.newList();
     for (Map<String,IndexDescProto> indexMap : indexes.values()) {
       indexDescProtos.addAll(indexMap.values());
@@ -744,22 +753,22 @@ public class MemStore implements CatalogStore {
   }
 
   @Override
-  public void addFunction(FunctionDesc func) throws CatalogException {
+  public void addFunction(FunctionDesc func) {
     // to be implemented
   }
 
   @Override
-  public void deleteFunction(FunctionDesc func) throws CatalogException {
+  public void deleteFunction(FunctionDesc func) {
     // to be implemented
   }
 
   @Override
-  public void existFunction(FunctionDesc func) throws CatalogException {
+  public void existFunction(FunctionDesc func) {
     // to be implemented
   }
 
   @Override
-  public List<String> getAllFunctionNames() throws CatalogException {
+  public List<String> getAllFunctionNames() {
     // to be implemented
     return null;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
index d6e902a..bd1ce3c 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MySQLStore.java
@@ -22,8 +22,6 @@
 package org.apache.tajo.catalog.store;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.exception.InternalException;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -32,7 +30,7 @@ import java.sql.SQLException;
 public class MySQLStore extends AbstractDBStore {
   private static final String CATALOG_DRIVER = "com.mysql.jdbc.Driver";
 
-  public MySQLStore(Configuration conf) throws InternalException {
+  public MySQLStore(Configuration conf) {
     super(conf);
   }
 
@@ -52,6 +50,6 @@ public class MySQLStore extends AbstractDBStore {
   }
 
   @Override
-  protected void createDatabaseDependants() throws CatalogException {
+  protected void createDatabaseDependants() {
   }
 }


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

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/OracleStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/OracleStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/OracleStore.java
index 4b7e6a3..1911507 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/OracleStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/OracleStore.java
@@ -18,19 +18,18 @@
 
 package org.apache.tajo.catalog.store;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.exception.TajoException;
+
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.exception.InternalException;
-
 public class OracleStore extends AbstractDBStore {
 
   private static final String CATALOG_DRIVER = "oracle.jdbc.OracleDriver";
 
-  public OracleStore(Configuration conf) throws InternalException {
+  public OracleStore(Configuration conf) {
     super(conf);
   }
 
@@ -50,7 +49,7 @@ public class OracleStore extends AbstractDBStore {
   }
 
   @Override
-  protected void createDatabaseDependants() throws CatalogException {
+  protected void createDatabaseDependants() {
     
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java
index 6089fdd..df2ddde 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/PostgreSQLStore.java
@@ -18,19 +18,17 @@
 
 package org.apache.tajo.catalog.store;
 
+import org.apache.hadoop.conf.Configuration;
+
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.exception.InternalException;
-
 public class PostgreSQLStore extends AbstractDBStore {
 
   private static final String CATALOG_DRIVER = "org.postgresql.Driver";
   
-  public PostgreSQLStore(Configuration conf) throws InternalException {
+  public PostgreSQLStore(Configuration conf) {
     super(conf);
   }
 
@@ -50,7 +48,7 @@ public class PostgreSQLStore extends AbstractDBStore {
   }
 
   @Override
-  protected void createDatabaseDependants() throws CatalogException {
+  protected void createDatabaseDependants() {
     
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/XMLCatalogSchemaManager.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/XMLCatalogSchemaManager.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/XMLCatalogSchemaManager.java
index 9d767a0..c36426e 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/XMLCatalogSchemaManager.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/XMLCatalogSchemaManager.java
@@ -18,27 +18,14 @@
 
 package org.apache.tajo.catalog.store;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.CatalogConstants;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.store.object.*;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.util.TUtil;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBElement;
@@ -48,15 +35,16 @@ import javax.xml.stream.XMLInputFactory;
 import javax.xml.stream.XMLStreamConstants;
 import javax.xml.stream.XMLStreamException;
 import javax.xml.stream.XMLStreamReader;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.catalog.CatalogConstants;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.catalog.store.object.*;
-import org.apache.tajo.exception.TajoInternalError;
-import org.apache.tajo.util.TUtil;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.sql.*;
+import java.util.*;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
 
 public class XMLCatalogSchemaManager {
   protected final Log LOG = LogFactory.getLog(getClass());
@@ -76,8 +64,7 @@ public class XMLCatalogSchemaManager {
     }
   }
   
-  protected String getDropSQL(DatabaseObjectType type, String name) 
-      throws CatalogException {
+  protected String getDropSQL(DatabaseObjectType type, String name) {
     SQLObject foundDropQuery = null;
     String sqlStatement = "DROP " + type.toString() + " " + name;
     
@@ -101,7 +88,7 @@ public class XMLCatalogSchemaManager {
     return sqlStatement;
   }
 
-  public void dropBaseSchema(Connection conn) throws CatalogException {
+  public void dropBaseSchema(Connection conn) {
     if (!isLoaded()) {
       throw new TajoInternalError("Schema files are not loaded yet.");
     }
@@ -271,7 +258,7 @@ public class XMLCatalogSchemaManager {
     return result;
   }
 
-  public void createBaseSchema(Connection conn) throws CatalogException {
+  public void createBaseSchema(Connection conn) {
     Statement stmt;
     
     if (!isLoaded()) {
@@ -347,7 +334,7 @@ public class XMLCatalogSchemaManager {
     CatalogUtil.closeQuietly(stmt);
   }
 
-  public boolean catalogAlreadyExists(Connection conn) throws CatalogException {
+  public boolean catalogAlreadyExists(Connection conn) {
     boolean result = false;
     try {
       List<String> constants = TUtil.newList();
@@ -375,7 +362,7 @@ public class XMLCatalogSchemaManager {
     return result;
   }
 
-  public boolean isInitialized(Connection conn) throws CatalogException {
+  public boolean isInitialized(Connection conn) {
     if (!isLoaded()) {
       throw new TajoInternalError("Database schema files are not loaded.");
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
index e2a096a..1d628f2 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
@@ -22,7 +22,6 @@ import com.google.common.collect.Sets;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.dictionary.InfoSchemaMetadataDictionary;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
 import org.apache.tajo.catalog.partition.PartitionDesc;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -33,7 +32,9 @@ import org.apache.tajo.catalog.store.*;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.exception.UndefinedFunctionException;
 import org.apache.tajo.function.Function;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
@@ -141,11 +142,11 @@ public class TestCatalog {
     //////////////////////////////////////////////////////////////////////////////
 
     assertFalse(catalog.existTablespace("space1"));
-    assertTrue(catalog.createTablespace("space1", "hdfs://xxx.com/warehouse"));
+    catalog.createTablespace("space1", "hdfs://xxx.com/warehouse");
     assertTrue(catalog.existTablespace("space1"));
 
     assertFalse(catalog.existTablespace("space2"));
-    assertTrue(catalog.createTablespace("space2", "hdfs://yyy.com/warehouse"));
+    catalog.createTablespace("space2", "hdfs://yyy.com/warehouse");
     assertTrue(catalog.existTablespace("space2"));
 
     //////////////////////////////////////////////////////////////////////////////
@@ -199,32 +200,32 @@ public class TestCatalog {
     //////////////////////////////////////////////////////////////////////////////
     // Clean up
     //////////////////////////////////////////////////////////////////////////////
-    assertTrue(catalog.dropTablespace("space1"));
+    catalog.dropTablespace("space1");
     assertFalse(catalog.existTablespace("space1"));
-    assertTrue(catalog.dropTablespace("space2"));
+    catalog.dropTablespace("space2");
     assertFalse(catalog.existTablespace("space2"));
   }
 
   @Test
   public void testCreateAndDropDatabases() throws Exception {
     assertFalse(catalog.existDatabase("testCreateAndDropDatabases"));
-    assertTrue(catalog.createDatabase("testCreateAndDropDatabases", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    catalog.createDatabase("testCreateAndDropDatabases", TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase("testCreateAndDropDatabases"));
-    assertTrue(catalog.dropDatabase("testCreateAndDropDatabases"));
+    catalog.dropDatabase("testCreateAndDropDatabases");
   }
   
   @Test
   public void testCreateAndDropDatabaseWithCharacterSensitivity() throws Exception {
     assertFalse(catalog.existDatabase("TestDatabase1"));
     assertFalse(catalog.existDatabase("testDatabase1"));
-    assertTrue(catalog.createDatabase("TestDatabase1", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    catalog.createDatabase("TestDatabase1", TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase("TestDatabase1"));
     assertFalse(catalog.existDatabase("testDatabase1"));
-    assertTrue(catalog.createDatabase("testDatabase1", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    catalog.createDatabase("testDatabase1", TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase("TestDatabase1"));
     assertTrue(catalog.existDatabase("testDatabase1"));
-    assertTrue(catalog.dropDatabase("TestDatabase1"));
-    assertTrue(catalog.dropDatabase("testDatabase1"));
+    catalog.dropDatabase("TestDatabase1");
+    catalog.dropDatabase("testDatabase1");
   }
 
   @Test
@@ -236,7 +237,7 @@ public class TestCatalog {
     for (int i = 0; i < NUM; i++) {
       String databaseName = namePrefix + i;
       assertFalse(catalog.existDatabase(databaseName));
-      assertTrue(catalog.createDatabase(databaseName, TajoConstants.DEFAULT_TABLESPACE_NAME));
+      catalog.createDatabase(databaseName, TajoConstants.DEFAULT_TABLESPACE_NAME);
       assertTrue(catalog.existDatabase(databaseName));
       createdDatabases.add(databaseName);
     }
@@ -252,7 +253,7 @@ public class TestCatalog {
     Collections.shuffle(createdDatabases);
     for (String tobeDropped : createdDatabases) {
       assertTrue(catalog.existDatabase(tobeDropped));
-      assertTrue(catalog.dropDatabase(tobeDropped));
+      catalog.dropDatabase(tobeDropped);
       assertFalse(catalog.existDatabase(tobeDropped));
     }
   }
@@ -273,16 +274,16 @@ public class TestCatalog {
 
   @Test
   public void testCreateAndDropTable() throws Exception {
-    assertTrue(catalog.createDatabase("tmpdb1", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    catalog.createDatabase("tmpdb1", TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase("tmpdb1"));
-    assertTrue(catalog.createDatabase("tmpdb2", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    catalog.createDatabase("tmpdb2", TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase("tmpdb2"));
 
     TableDesc table1 = createMockupTable("tmpdb1", "table1");
-    assertTrue(catalog.createTable(table1));
+    catalog.createTable(table1);
 
     TableDesc table2 = createMockupTable("tmpdb2", "table2");
-    assertTrue(catalog.createTable(table2));
+    catalog.createTable(table2);
 
     Set<String> tmpdb1 = Sets.newHashSet(catalog.getAllTableNames("tmpdb1"));
     assertEquals(1, tmpdb1.size());
@@ -293,21 +294,21 @@ public class TestCatalog {
     assertEquals(1, tmpdb2.size());
     assertTrue(tmpdb2.contains("table2"));
 
-    assertTrue(catalog.dropDatabase("tmpdb1"));
+    catalog.dropDatabase("tmpdb1");
     assertFalse(catalog.existDatabase("tmpdb1"));
 
     tmpdb2 = Sets.newHashSet(catalog.getAllTableNames("tmpdb2"));
     assertEquals(1, tmpdb2.size());
     assertTrue(tmpdb2.contains("table2"));
 
-    assertTrue(catalog.dropDatabase("tmpdb2"));
+    catalog.dropDatabase("tmpdb2");
     assertFalse(catalog.existDatabase("tmpdb2"));
   }
   
   @Test
   public void testCreateAndDropTableWithCharacterSensivity() throws Exception {
     String databaseName = "TestDatabase1";
-    assertTrue(catalog.createDatabase(databaseName, TajoConstants.DEFAULT_TABLESPACE_NAME));
+    catalog.createDatabase(databaseName, TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase(databaseName));
     
     String tableName = "TestTable1";
@@ -322,7 +323,7 @@ public class TestCatalog {
         new TableMeta("TEXT", new KeyValueSet()),
         path.toUri(), true);
     
-    assertTrue(catalog.createTable(table));
+    catalog.createTable(table);
     
     tableName = "testTable1";
     schema = new Schema();
@@ -336,9 +337,9 @@ public class TestCatalog {
         new TableMeta("TEXT", new KeyValueSet()),
         path.toUri(), true);
     
-    assertTrue(catalog.createTable(table));
+    catalog.createTable(table);
     
-    assertTrue(catalog.dropDatabase(databaseName));
+    catalog.dropDatabase(databaseName);
   }
 
   static String dbPrefix = "db_";
@@ -347,7 +348,7 @@ public class TestCatalog {
   static final int TABLE_NUM_PER_DB = 3;
   static final int TOTAL_TABLE_NUM = DB_NUM * TABLE_NUM_PER_DB;
 
-  private Map<String, List<String>> createBaseDatabaseAndTables() throws IOException {
+  private Map<String, List<String>> createBaseDatabaseAndTables() throws IOException, TajoException {
 
     Map<String, List<String>> createdDatabaseAndTablesMap = new HashMap<String, List<String>>();
 
@@ -357,12 +358,12 @@ public class TestCatalog {
       String databaseName = dbPrefix + dbIdx;
 
       if (!catalog.existDatabase(databaseName)) {
-        assertTrue(catalog.createDatabase(databaseName, TajoConstants.DEFAULT_TABLESPACE_NAME));
+        catalog.createDatabase(databaseName, TajoConstants.DEFAULT_TABLESPACE_NAME);
       }
 
       String tableName = tablePrefix + tableId;
       TableDesc table = createMockupTable(databaseName, tableName);
-      assertTrue(catalog.createTable(table));
+      catalog.createTable(table);
 
       TUtil.putToNestedList(createdDatabaseAndTablesMap, databaseName, tableName);
     }
@@ -438,7 +439,7 @@ public class TestCatalog {
   /**
    * It asserts the equality between an original table desc and a restored table desc.
    */
-  private static void assertSchemaEquality(String tableName, Schema schema) throws IOException {
+  private static void assertSchemaEquality(String tableName, Schema schema) throws IOException, TajoException {
     Path path = new Path(CommonTestingUtil.getTestDir(), tableName);
     TableDesc tableDesc = new TableDesc(
         CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, tableName),
@@ -577,30 +578,30 @@ public class TestCatalog {
   }
 
   @Test
-  public void testCreateSameTables() throws IOException {
-    assertTrue(catalog.createDatabase("tmpdb3", TajoConstants.DEFAULT_TABLESPACE_NAME));
+  public void testCreateSameTables() throws IOException, TajoException {
+    catalog.createDatabase("tmpdb3", TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase("tmpdb3"));
-    assertTrue(catalog.createDatabase("tmpdb4", TajoConstants.DEFAULT_TABLESPACE_NAME));
+    catalog.createDatabase("tmpdb4", TajoConstants.DEFAULT_TABLESPACE_NAME);
     assertTrue(catalog.existDatabase("tmpdb4"));
 
     TableDesc table1 = createMockupTable("tmpdb3", "table1");
-    assertTrue(catalog.createTable(table1));
+    catalog.createTable(table1);
     TableDesc table2 = createMockupTable("tmpdb3", "table2");
-    assertTrue(catalog.createTable(table2));
+    catalog.createTable(table2);
     assertTrue(catalog.existsTable("tmpdb3", "table1"));
     assertTrue(catalog.existsTable("tmpdb3", "table2"));
 
     TableDesc table3 = createMockupTable("tmpdb4", "table1");
-    assertTrue(catalog.createTable(table3));
+    catalog.createTable(table3);
     TableDesc table4 = createMockupTable("tmpdb4", "table2");
-    assertTrue(catalog.createTable(table4));
+    catalog.createTable(table4);
     assertTrue(catalog.existsTable("tmpdb4", "table1"));
     assertTrue(catalog.existsTable("tmpdb4", "table2"));
 
-    assertTrue(catalog.dropTable("tmpdb3.table1"));
-    assertTrue(catalog.dropTable("tmpdb3.table2"));
-    assertTrue(catalog.dropTable("tmpdb4.table1"));
-    assertTrue(catalog.dropTable("tmpdb4.table2"));
+    catalog.dropTable("tmpdb3.table1");
+    catalog.dropTable("tmpdb3.table2");
+    catalog.dropTable("tmpdb4.table1");
+    catalog.dropTable("tmpdb4.table2");
 
     assertFalse(catalog.existsTable("tmpdb3.table1"));
     assertFalse(catalog.existsTable("tmpdb3.table2"));
@@ -612,7 +613,7 @@ public class TestCatalog {
 	public void testAddAndDelIndex() throws Exception {
 	  TableDesc desc = prepareTable();
     prepareIndexDescs();
-	  assertTrue(catalog.createTable(desc));
+	  catalog.createTable(desc);
 	  
 	  assertFalse(catalog.existIndexByName(DEFAULT_DATABASE_NAME, desc1.getName()));
 	  assertFalse(catalog.existIndexByColumnNames(DEFAULT_DATABASE_NAME, "indexed", new String[]{"id"}));
@@ -1111,7 +1112,7 @@ public class TestCatalog {
     FunctionDesc meta = new FunctionDesc("testint", TestIntFunc.class, FunctionType.GENERAL,
         CatalogUtil.newSimpleDataType(Type.INT4),
         CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT4));
-    assertTrue(catalog.createFunction(meta));
+    catalog.createFunction(meta);
 
     // UPGRADE TO INT4 SUCCESS==> LOOK AT SECOND PARAM BELOW
     FunctionDesc retrieved = catalog.getFunction("testint", CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT2));
@@ -1127,7 +1128,7 @@ public class TestCatalog {
     FunctionDesc meta = new FunctionDesc("testintinvalid", TestIntFunc.class, FunctionType.GENERAL,
         CatalogUtil.newSimpleDataType(Type.INT4),
         CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT4));
-    assertTrue(catalog.createFunction(meta));
+    catalog.createFunction(meta);
 
     //UPGRADE TO INT8 WILL FAIL ==> LOOK AT SECOND PARAM BELOW
     catalog.getFunction("testintinvalid", CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT8));
@@ -1139,7 +1140,7 @@ public class TestCatalog {
     FunctionDesc meta = new FunctionDesc("testfloat", TestFloatFunc.class, FunctionType.GENERAL,
         CatalogUtil.newSimpleDataType(Type.INT4),
         CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8, Type.INT4));
-    assertTrue(catalog.createFunction(meta));
+    catalog.createFunction(meta);
 
     //UPGRADE TO FLOAT 8 SUCCESS==> LOOK AT FIRST PARAM BELOW
     FunctionDesc retrieved = catalog.getFunction("testfloat",
@@ -1156,7 +1157,7 @@ public class TestCatalog {
     FunctionDesc meta = new FunctionDesc("testfloatinvalid", TestFloatFunc.class, FunctionType.GENERAL,
         CatalogUtil.newSimpleDataType(Type.INT4),
         CatalogUtil.newSimpleDataTypeArray(Type.FLOAT8, Type.INT4));
-    assertTrue(catalog.createFunction(meta));
+    catalog.createFunction(meta);
 
     // UPGRADE TO DECIMAL WILL FAIL ==> LOOK AT FIRST PARAM BELOW
     catalog.getFunction("testfloatinvalid", CatalogUtil.newSimpleDataTypeArray(Type.NUMERIC, Type.INT4));
@@ -1168,7 +1169,7 @@ public class TestCatalog {
     FunctionDesc meta = new FunctionDesc("testany", TestAnyParamFunc.class, FunctionType.GENERAL,
         CatalogUtil.newSimpleDataType(Type.INT4),
         CatalogUtil.newSimpleDataTypeArray(Type.ANY));
-    assertTrue(catalog.createFunction(meta));
+    catalog.createFunction(meta);
 
     FunctionDesc retrieved = catalog.getFunction("testany", CatalogUtil.newSimpleDataTypeArray(Type.INT1));
     assertEquals(retrieved.getFunctionName(), "testany");

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestLinkedMetadataManager.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestLinkedMetadataManager.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestLinkedMetadataManager.java
index 4ddf7ab..d8f894e 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestLinkedMetadataManager.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestLinkedMetadataManager.java
@@ -24,12 +24,11 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.TajoConstants;
-import org.apache.tajo.catalog.exception.UndefinedTablespaceException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.error.Errors;
-import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.junit.AfterClass;
@@ -153,7 +152,7 @@ public class TestLinkedMetadataManager {
   static CatalogService catalog;
 
   @BeforeClass
-  public static void setUp() throws IOException {
+  public static void setUp() throws IOException, DuplicateTablespaceException, DuplicateDatabaseException {
     TajoConf conf = new TajoConf();
     conf.setVar(TajoConf.ConfVars.CATALOG_ADDRESS, "127.0.0.1:0");
 
@@ -227,7 +226,7 @@ public class TestLinkedMetadataManager {
     assertEquals(Sets.newHashSet("table1", "table2"), Sets.newHashSet(catalog.getAllTableNames("space1")));
   }
 
-  @Test
+  @Test(expected = InsufficientPrivilegeException.class)
   public void testCreateTable() throws Exception {
     TableDesc tb = new TableDesc(
         "space1.errortable",
@@ -235,22 +234,12 @@ public class TestLinkedMetadataManager {
         new TableMeta("x", new KeyValueSet()),
         URI.create("file:///"));
 
-    try {
-      catalog.createTable(tb);
-      fail();
-    } catch (TajoInternalError e) {
-      assertEquals(Errors.ResultCode.INTERNAL_ERROR, e.getErrorCode());
-    }
+    catalog.createTable(tb);
   }
 
-  @Test
+  @Test(expected = InsufficientPrivilegeException.class)
   public void testDropTable() throws Exception {
-    try {
-      catalog.dropTable("space1.table1");
-      fail();
-    } catch (TajoInternalError e) {
-      assertEquals(Errors.ResultCode.INTERNAL_ERROR, e.getErrorCode());
-    }
+    catalog.dropTable("space1.table1");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
index eadd4df..affd128 100644
--- a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
@@ -33,8 +33,10 @@ import org.apache.tajo.cli.tsql.commands.*;
 import org.apache.tajo.client.*;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.exception.ExceptionUtil;
 import org.apache.tajo.exception.ReturnStateUtil;
 import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.DefaultTajoException;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.service.ServiceTrackerFactory;
 import org.apache.tajo.util.FileUtil;
@@ -444,7 +446,7 @@ public class TajoCli {
   }
 
   private int executeParsedResults(Collection<ParsedResult> parsedResults) throws Exception {
-    int exitCode;
+    int exitCode = 0;
     for (ParsedResult parsedResult : parsedResults) {
       if (parsedResult.getType() == StatementType.META) {
         exitCode = executeMetaCommand(parsedResult.getStatement());
@@ -457,10 +459,10 @@ public class TajoCli {
       }
     }
 
-    return 0;
+    return exitCode;
   }
 
-  public int executeMetaCommand(String line) throws Exception {
+  public int executeMetaCommand(String line) {
     String [] metaCommands = line.split(";");
     for (String metaCommand : metaCommands) {
       String arguments [] = metaCommand.split(" ");
@@ -474,8 +476,8 @@ public class TajoCli {
 
       try {
         invoked.invoke(arguments);
-      } catch (Exception e) {
-        onError(null, e);
+      } catch (Throwable t) {
+        onError(t);
         return -1;
       } finally {
         context.getOutput().flush();
@@ -493,11 +495,8 @@ public class TajoCli {
 
     long startTime = System.currentTimeMillis();
     ClientProtos.SubmitQueryResponse response = client.executeQueryWithJson(json);
-    if (response == null) {
-      onError("response is null", null);
-
-    } else if (ReturnStateUtil.isSuccess(response.getState())) {
 
+    if (ReturnStateUtil.isSuccess(response.getState())) {
       switch (response.getResultType()) {
       case FETCH:
         QueryId queryId = new QueryId(response.getQueryId());
@@ -511,9 +510,7 @@ public class TajoCli {
       }
 
     } else {
-      if (ReturnStateUtil.isError(response.getState())) {
-        onError(response.getState().getMessage(), null);
-      }
+      onError((Throwable) ExceptionUtil.toTajoExceptionCommon(response.getState()));
     }
   }
 
@@ -523,33 +520,33 @@ public class TajoCli {
     ClientProtos.SubmitQueryResponse response = null;
     try{
       response = client.executeQuery(statement);
-    } catch(Throwable te){
-      onError(null, te);
+    } catch(Throwable t){
+      onError(t);
     }
 
-    if (response != null) {
 
-      if (ReturnStateUtil.isSuccess(response.getState())) {
+    if (ReturnStateUtil.isSuccess(response.getState())) {
 
-        switch (response.getResultType()) {
-        case FETCH:
-          QueryId queryId = new QueryId(response.getQueryId());
-          waitForQueryCompleted(queryId);
-          break;
-        case ENCLOSED:
-          localQueryCompleted(response, startTime);
-          break;
-        default:
-          displayFormatter.printMessage(sout, "OK");
-        }
+      switch (response.getResultType()) {
 
-      } else {
-        if (ReturnStateUtil.isError(response.getState())) {
-          onError(response.getState().getMessage(), null);
-        }
+      case FETCH:
+        QueryId queryId = new QueryId(response.getQueryId());
+        waitForQueryCompleted(queryId);
+        break;
+
+      case ENCLOSED:
+        localQueryCompleted(response, startTime);
+        break;
+
+      default:
+        displayFormatter.printMessage(sout, "OK");
       }
+
+    } else {
+      onError((Throwable) ExceptionUtil.toTajoExceptionCommon(response.getState()));
     }
 
+
     return wasError ? -1 : 0;
   }
 
@@ -570,7 +567,7 @@ public class TajoCli {
         displayFormatter.printResult(sout, sin, desc, responseTime, res);
       }
     } catch (Throwable t) {
-      onError(null, t);
+      onError(t);
     } finally {
       if (res != null) {
         try {
@@ -637,7 +634,7 @@ public class TajoCli {
         }
       }
     } catch (Throwable t) {
-      onError(null, t);
+      onError(t);
     } finally {
       if (res != null) {
         try {
@@ -667,19 +664,13 @@ public class TajoCli {
     sout.println("Invalid command " + command + ". Try \\? for help.");
   }
 
-  private void onError(String message, Throwable t) {
+  private void onError(Throwable t) {
+    Preconditions.checkNotNull(t);
+
     wasError = true;
-    if (t == null) {
-      displayFormatter.printErrorMessage(sout, message);
-    } else {
-      if (t instanceof SQLException) {
-        displayFormatter.printErrorMessage(sout, t.getMessage());
-      } else {
-        displayFormatter.printErrorMessage(sout, t);
-      }
-    }
-    if (reconnect && (t instanceof InvalidClientSessionException ||
-        (message != null && message.startsWith("org.apache.tajo.session.InvalidSessionException")))) {
+    displayFormatter.printErrorMessage(sout, t.getMessage());
+
+    if (reconnect && (t instanceof InvalidClientSessionException)) {
       try {
         ((SessionConnection)client).reconnect();
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java
index 3682a74..819f848 100644
--- a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java
@@ -18,9 +18,11 @@
 
 package org.apache.tajo.cli.tsql.commands;
 
+import com.google.common.base.Preconditions;
 import com.google.protobuf.ServiceException;
 import org.apache.tajo.cli.tsql.TajoCli;
 import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.UndefinedDatabaseException;
 
 import java.sql.SQLException;
 
@@ -37,28 +39,28 @@ public class ConnectDatabaseCommand extends TajoShellCommand {
 
   @Override
   public void invoke(String[] cmd) throws Exception {
-    if (cmd.length == 1) {
+
+    if (cmd.length == 1) { // no given database name
+
       context.getOutput().write(String.format("You are now connected to database \"%s\" as user \"%s\".%n",
           client.getCurrentDatabase(), client.getUserInfo().getUserName()));
+
     } else if (cmd.length == 2) {
-      String databaseName = cmd[1];
-      databaseName = databaseName.replace("\"", "");
-      if (!client.existDatabase(databaseName)) {
-        context.getOutput().write("Database '" + databaseName + "'  not found\n");
-      } else {
-        try {
-          if (client.selectDatabase(databaseName)) {
-            context.setCurrentDatabase(client.getCurrentDatabase());
-            context.getOutput().write(String.format("You are now connected to database \"%s\" as user \"%s\".%n",
-                context.getCurrentDatabase(), client.getUserInfo().getUserName()));
-          }
-        } catch (TajoException se) {
-          if (se.getMessage() != null) {
-            context.getOutput().write(se.getMessage());
-          } else {
-            context.getOutput().write(String.format("cannot connect the database \"%s\"", databaseName));
-          }
-        }
+      final String databaseName = cmd[1].replace("\"", "");
+
+      try {
+        client.selectDatabase(databaseName);
+        Preconditions.checkState(databaseName.equals(client.getCurrentDatabase()));
+
+        context.setCurrentDatabase(client.getCurrentDatabase());
+        context.getOutput().write(String.format(
+                "You are now connected to database \"%s\" as user \"%s\".%n",
+                context.getCurrentDatabase(),
+                client.getUserInfo().getUserName())
+        );
+
+      } catch (TajoException se) {
+        context.getOutput().write("ERROR: " + se.getMessage());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
index 4b7006d..b17a336 100644
--- a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
@@ -28,6 +28,7 @@ import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.SortSpecProto;
 import org.apache.tajo.cli.tsql.TajoCli;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.StringUtils;
 
@@ -45,7 +46,7 @@ public class DescTableCommand extends TajoShellCommand {
   }
 
   @Override
-  public void invoke(String[] cmd) throws Exception {
+  public void invoke(String[] cmd) throws TajoException {
     if (cmd.length == 2) {
       String tableName = cmd[1];
       tableName = tableName.replace("\"", "");

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
index 2ac5854..22fa829 100644
--- a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
@@ -28,8 +28,11 @@ public abstract class TajoShellCommand {
     return new String[] {};
   }
   public abstract void invoke(String [] command) throws Exception;
+
   public abstract String getUsage();
+
   public abstract String getDescription();
+
   public void printHelp() {
     context.getOutput().print(getCommand());
     context.getOutput().print(" - ");
@@ -78,10 +81,6 @@ public abstract class TajoShellCommand {
     }
   }
 
-  protected void printCenter(String message) {
-    printCenter(message, maxColumn, true);
-  }
-
   protected void print(char c, int count) {
     for(int i = 0; i < count; i++) {
       context.getOutput().print(c);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClient.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClient.java b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClient.java
index bc63f84..7acc9b9 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClient.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClient.java
@@ -22,15 +22,13 @@ import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
-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.IndexDescProto;
-import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.*;
 
 import java.io.Closeable;
 import java.net.URI;
-import java.sql.SQLException;
 import java.util.List;
 
 public interface CatalogAdminClient extends Closeable {
@@ -39,15 +37,14 @@ public interface CatalogAdminClient extends Closeable {
    *
    * @param databaseName The database name to be created. This name is case sensitive.
    * @return True if created successfully.
-   * @throws java.sql.SQLException
+   * @throws DuplicateDatabaseException
    */
-  boolean createDatabase(final String databaseName) throws DuplicateDatabaseException;
+  void createDatabase(final String databaseName) throws DuplicateDatabaseException;
   /**
    * Does the database exist?
    *
    * @param databaseName The database name to be checked. This name is case sensitive.
    * @return True if so.
-   * @throws java.sql.SQLException
    */
   boolean existDatabase(final String databaseName);
   /**
@@ -55,9 +52,9 @@ public interface CatalogAdminClient extends Closeable {
    *
    * @param databaseName The database name to be dropped. This name is case sensitive.
    * @return True if the database is dropped successfully.
-   * @throws java.sql.SQLException
+   * @throws UndefinedDatabaseException
    */
-  boolean dropDatabase(final String databaseName) throws UndefinedDatabaseException;
+  void dropDatabase(final String databaseName) throws UndefinedDatabaseException, InsufficientPrivilegeException;
 
   List<String> getAllDatabaseNames();
 
@@ -78,10 +75,11 @@ public interface CatalogAdminClient extends Closeable {
    * @param path The external table location
    * @param meta Table meta
    * @return the created table description.
-   * @throws java.sql.SQLException
+   * @throws DuplicateTableException
    */
   TableDesc createExternalTable(final String tableName, final Schema schema, final URI path,
-                                       final TableMeta meta) throws DuplicateTableException;
+                                       final TableMeta meta)
+      throws DuplicateTableException, UnavailableTableLocationException, InsufficientPrivilegeException;
 
   /**
    * Create an external table.
@@ -93,20 +91,20 @@ public interface CatalogAdminClient extends Closeable {
    * @param meta Table meta
    * @param partitionMethodDesc Table partition description
    * @return the created table description.
-   * @throws java.sql.SQLException
+   * @throws DuplicateTableException
    */
   TableDesc createExternalTable(final String tableName, final Schema schema, final URI path,
                                        final TableMeta meta, final PartitionMethodDesc partitionMethodDesc)
-      throws DuplicateTableException;
+      throws DuplicateTableException, InsufficientPrivilegeException, UnavailableTableLocationException;
 
   /**
    * Drop a table
    *
    * @param tableName The table name to be dropped. This name is case sensitive.
    * @return True if the table is dropped successfully.
-   * @throws java.sql.SQLException
+   * @throws InsufficientPrivilegeException
    */
-  boolean dropTable(final String tableName) throws UndefinedTableException;
+  void dropTable(final String tableName) throws UndefinedTableException, InsufficientPrivilegeException;
 
   /**
    * Drop a table.
@@ -114,9 +112,11 @@ public interface CatalogAdminClient extends Closeable {
    * @param tableName The table name to be dropped. This name is case sensitive.
    * @param purge If purge is true, this call will remove the entry in catalog as well as the table contents.
    * @return True if the table is dropped successfully.
-   * @throws java.sql.SQLException
+   * @throws UndefinedTableException
+   * @throws InsufficientPrivilegeException
    */
-  boolean dropTable(final String tableName, final boolean purge) throws UndefinedTableException;
+  void dropTable(final String tableName, final boolean purge) throws UndefinedTableException,
+      InsufficientPrivilegeException;
 
   /**
    * Get a list of table names.
@@ -124,7 +124,6 @@ public interface CatalogAdminClient extends Closeable {
    * @param databaseName The database name to show all tables. This name is case sensitive.
    *                     If it is null, this method will show all tables
    *                     in the current database of this session.
-   * @throws java.sql.SQLException
    */
   List<String> getTableList(@Nullable final String databaseName);
 
@@ -133,23 +132,23 @@ public interface CatalogAdminClient extends Closeable {
    *
    * @param tableName The table name to get. This name is case sensitive.
    * @return Table description
-   * @throws java.sql.SQLException
+   * @throws UndefinedTableException
    */
   TableDesc getTableDesc(final String tableName) throws UndefinedTableException;
 
   List<CatalogProtos.FunctionDescProto> getFunctions(final String functionName);
 
-  IndexDescProto getIndex(final String indexName) throws SQLException;
+  IndexDescProto getIndex(final String indexName);
 
-  boolean existIndex(final String indexName) throws SQLException;
+  boolean existIndex(final String indexName);
 
-  List<IndexDescProto> getIndexes(final String tableName) throws SQLException;
+  List<IndexDescProto> getIndexes(final String tableName);
 
-  boolean hasIndexes(final String tableName) throws SQLException;
+  boolean hasIndexes(final String tableName);
 
-  IndexDescProto getIndex(final String tableName, final String[] columnNames) throws SQLException;
+  IndexDescProto getIndex(final String tableName, final String[] columnNames);
 
-  boolean existIndex(final String tableName, final String[] columnName) throws SQLException;
+  boolean existIndex(final String tableName, final String[] columnName);
 
-  boolean dropIndex(final String indexName) throws SQLException;
+  boolean dropIndex(final String indexName);
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
index d4d8a86..af1278d 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
@@ -24,25 +24,24 @@ import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
-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.*;
-import org.apache.tajo.error.Errors;
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.DropTableRequest;
 import org.apache.tajo.ipc.ClientProtos.GetIndexWithColumnsRequest;
 import org.apache.tajo.rpc.NettyClientBase;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListResponse;
 
 import java.io.IOException;
 import java.net.URI;
-import java.sql.SQLException;
 import java.util.List;
 
+import static org.apache.tajo.exception.ExceptionUtil.throwsIfThisError;
 import static org.apache.tajo.exception.ReturnStateUtil.*;
-import static org.apache.tajo.exception.SQLExceptionUtil.throwIfError;
 import static org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService.BlockingInterface;
 
 public class CatalogAdminClientImpl implements CatalogAdminClient {
@@ -53,19 +52,16 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   }
 
   @Override
-  public boolean createDatabase(final String databaseName) throws DuplicateDatabaseException {
+  public void createDatabase(final String databaseName) throws DuplicateDatabaseException {
 
-    final BlockingInterface stub = conn.getTMStub();
 
-    try {
-      PrimitiveProtos.ReturnState state = stub.createDatabase(null, conn.getSessionedString(databaseName));
 
-      if (isThisError(state, Errors.ResultCode.DUPLICATE_DATABASE)) {
-        throw new DuplicateDatabaseException(state);
-      }
+    try {
+      final BlockingInterface stub = conn.getTMStub();
+      final ReturnState state = stub.createDatabase(null, conn.getSessionedString(databaseName));
 
+      throwsIfThisError(state, DuplicateDatabaseException.class);
       ensureOk(state);
-      return true;
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
@@ -75,27 +71,33 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   @Override
   public boolean existDatabase(final String databaseName) {
 
-    final BlockingInterface stub = conn.getTMStub();
-
     try {
-      return isSuccess(stub.existDatabase(null, conn.getSessionedString(databaseName)));
+      final BlockingInterface stub = conn.getTMStub();
+      final ReturnState state = stub.existDatabase(null, conn.getSessionedString(databaseName));
+
+      if (isThisError(state, ResultCode.UNDEFINED_DATABASE)) {
+        return false;
+      }
+      ensureOk(state);
+      return true;
+
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
   }
 
   @Override
-  public boolean dropDatabase(final String databaseName) throws UndefinedDatabaseException {
-
-    final BlockingInterface stub = conn.getTMStub();
+  public void dropDatabase(final String databaseName) throws UndefinedDatabaseException {
 
     try {
-      PrimitiveProtos.ReturnState state = stub.dropDatabase(null, conn.getSessionedString(databaseName));
-      if (isThisError(state, Errors.ResultCode.UNDEFINED_DATABASE)) {
+      final BlockingInterface stub = conn.getTMStub();
+      final ReturnState state = stub.dropDatabase(null, conn.getSessionedString(databaseName));
+
+      if (isThisError(state, ResultCode.UNDEFINED_DATABASE)) {
         throw new UndefinedDatabaseException(state);
       }
       ensureOk(state);
-      return true;
+
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
@@ -117,14 +119,14 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
 
     final BlockingInterface stub = conn.getTMStub();
 
-    PrimitiveProtos.ReturnState state;
+    ReturnState state;
     try {
       state = stub.existTable(null, conn.getSessionedString(tableName));
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
 
-    if (isThisError(state, Errors.ResultCode.UNDEFINED_TABLE)) {
+    if (isThisError(state, ResultCode.UNDEFINED_TABLE)) {
       return false;
     }
 
@@ -134,19 +136,19 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
 
   @Override
   public TableDesc createExternalTable(String tableName, Schema schema, URI path, TableMeta meta)
-      throws DuplicateTableException {
+      throws DuplicateTableException, UnavailableTableLocationException, InsufficientPrivilegeException {
     return createExternalTable(tableName, schema, path, meta, null);
   }
 
   public TableDesc createExternalTable(final String tableName, final Schema schema, final URI path,
                                        final TableMeta meta, final PartitionMethodDesc partitionMethodDesc)
-      throws DuplicateTableException {
+      throws DuplicateTableException, InsufficientPrivilegeException, UnavailableTableLocationException {
 
-    NettyClientBase client = conn.getTajoMasterConnection();
+    final NettyClientBase client = conn.getTajoMasterConnection();
     conn.checkSessionAndGet(client);
-    BlockingInterface tajoMasterService = client.getStub();
+    final BlockingInterface tajoMasterService = client.getStub();
 
-    ClientProtos.CreateTableRequest.Builder builder = ClientProtos.CreateTableRequest.newBuilder();
+    final ClientProtos.CreateTableRequest.Builder builder = ClientProtos.CreateTableRequest.newBuilder();
     builder.setSessionId(conn.sessionId);
     builder.setName(tableName);
     builder.setSchema(schema.getProto());
@@ -164,21 +166,22 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
       throw new RuntimeException(e);
     }
 
-    if (isThisError(res.getState(), Errors.ResultCode.DUPLICATE_TABLE)) {
-      throw new DuplicateTableException(res.getState());
-    }
+    throwsIfThisError(res.getState(), DuplicateTableException.class);
+    throwsIfThisError(res.getState(), InsufficientPrivilegeException.class);
+    throwsIfThisError(res.getState(), UnavailableTableLocationException.class);
 
     ensureOk(res.getState());
     return CatalogUtil.newTableDesc(res.getTable());
   }
 
   @Override
-  public boolean dropTable(String tableName) throws UndefinedTableException {
-    return dropTable(tableName, false);
+  public void dropTable(String tableName) throws UndefinedTableException, InsufficientPrivilegeException {
+    dropTable(tableName, false);
   }
 
   @Override
-  public boolean dropTable(final String tableName, final boolean purge) throws UndefinedTableException {
+  public void dropTable(final String tableName, final boolean purge)
+      throws UndefinedTableException, InsufficientPrivilegeException {
 
     final BlockingInterface stub = conn.getTMStub();
     final DropTableRequest request = DropTableRequest.newBuilder()
@@ -188,19 +191,17 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
         .build();
 
 
-    PrimitiveProtos.ReturnState state;
+    ReturnState state;
     try {
       state = stub.dropTable(null, request);
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
 
-    if (isThisError(state, Errors.ResultCode.UNDEFINED_TABLE)) {
-      throw new UndefinedTableException(state);
-    }
+    throwsIfThisError(state, UndefinedTableException.class);
+    throwsIfThisError(state, InsufficientPrivilegeException.class);
 
     ensureOk(state);
-    return true;
   }
 
   @Override
@@ -231,7 +232,7 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
       throw new RuntimeException(e);
     }
 
-    if (isThisError(res.getState(), Errors.ResultCode.UNDEFINED_TABLE)) {
+    if (isThisError(res.getState(), ResultCode.UNDEFINED_TABLE)) {
       throw new UndefinedTableException(res.getState());
     }
 
@@ -257,7 +258,7 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   }
 
   @Override
-  public IndexDescProto getIndex(final String indexName) throws SQLException {
+  public IndexDescProto getIndex(final String indexName) {
     final BlockingInterface stub = conn.getTMStub();
 
     IndexResponse res;
@@ -267,12 +268,12 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
       throw new RuntimeException(e);
     }
 
-    throwIfError(res.getState());
+    ensureOk(res.getState());
     return res.getIndexDesc();
   }
 
   @Override
-  public boolean existIndex(final String indexName) throws SQLException {
+  public boolean existIndex(final String indexName){
     final BlockingInterface stub = conn.getTMStub();
 
     try {
@@ -283,7 +284,7 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   }
 
   @Override
-  public List<IndexDescProto> getIndexes(final String tableName) throws SQLException {
+  public List<IndexDescProto> getIndexes(final String tableName) {
     final BlockingInterface stub = conn.getTMStub();
 
     IndexListResponse response;
@@ -294,12 +295,12 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
       throw new RuntimeException(e);
     }
 
-    throwIfError(response.getState());
+    ensureOk(response.getState());
     return response.getIndexDescList();
   }
 
   @Override
-  public boolean hasIndexes(final String tableName) throws SQLException {
+  public boolean hasIndexes(final String tableName) {
     final BlockingInterface stub = conn.getTMStub();
 
     try {
@@ -310,7 +311,7 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   }
 
   @Override
-  public IndexDescProto getIndex(final String tableName, final String[] columnNames) throws SQLException {
+  public IndexDescProto getIndex(final String tableName, final String[] columnNames) {
     final BlockingInterface stub = conn.getTMStub();
 
     GetIndexWithColumnsRequest.Builder builder = GetIndexWithColumnsRequest.newBuilder();
@@ -327,12 +328,12 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
       throw new RuntimeException(e);
     }
 
-    throwIfError(response.getState());
+    ensureOk(response.getState());
     return response.getIndexDesc();
   }
 
   @Override
-  public boolean existIndex(final String tableName, final String[] columnName) throws SQLException {
+  public boolean existIndex(final String tableName, final String[] columnName) {
     final BlockingInterface stub = conn.getTMStub();
 
     GetIndexWithColumnsRequest.Builder builder = GetIndexWithColumnsRequest.newBuilder();
@@ -350,7 +351,7 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   }
 
   @Override
-  public boolean dropIndex(final String indexName) throws SQLException {
+  public boolean dropIndex(final String indexName) {
     final BlockingInterface stub = conn.getTMStub();
 
     try {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/ClientExceptionUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/ClientExceptionUtil.java b/tajo-client/src/main/java/org/apache/tajo/client/ClientExceptionUtil.java
deleted file mode 100644
index 2ecc078..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/client/ClientExceptionUtil.java
+++ /dev/null
@@ -1,106 +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.client;
-
-import com.google.common.collect.Maps;
-import org.apache.tajo.catalog.exception.*;
-import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.*;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-import java.lang.reflect.Constructor;
-import java.util.Map;
-
-import static org.apache.tajo.error.Errors.ResultCode.*;
-import static org.apache.tajo.exception.ReturnStateUtil.isError;
-
-/**
- * Exception related utilities. Especially, it provides a way to recover @{link ReturnState} into TajoException.
- */
-public class ClientExceptionUtil {
-
-  static Map<ResultCode, Class<? extends TajoExceptionInterface>> EXCEPTIONS = Maps.newHashMap();
-
-  static {
-
-    // General Errors
-    ADD_EXCEPTION(INTERNAL_ERROR, TajoInternalError.class);
-    ADD_EXCEPTION(FEATURE_NOT_SUPPORTED, UnsupportedException.class);
-
-    ADD_EXCEPTION(UNDEFINED_TABLESPACE, UndefinedTablespaceException.class);
-    ADD_EXCEPTION(UNDEFINED_DATABASE, UndefinedDatabaseException.class);
-    // ADD_EXCEPTION(UNDEFINED_SCHEMA, );
-    ADD_EXCEPTION(UNDEFINED_TABLE, UndefinedTableException.class);
-    ADD_EXCEPTION(UNDEFINED_COLUMN, UndefinedColumnException.class);
-    ADD_EXCEPTION(UNDEFINED_FUNCTION, UndefinedFunctionException.class);
-    ADD_EXCEPTION(UNDEFINED_PARTITION, UndefinedPartitionException.class);
-    ADD_EXCEPTION(UNDEFINED_OPERATOR, UndefinedOperatorException.class);
-
-    ADD_EXCEPTION(DUPLICATE_TABLESPACE, DuplicateTableException.class);
-    ADD_EXCEPTION(DUPLICATE_DATABASE, DuplicateDatabaseException.class);
-    // ADD_EXCEPTION(DUPLICATE_SCHEMA, );
-    ADD_EXCEPTION(DUPLICATE_TABLE, DuplicateTableException.class);
-    ADD_EXCEPTION(DUPLICATE_COLUMN, DuplicateColumnException.class);
-    // ADD_EXCEPTION(DUPLICATE_ALIAS, );
-    ADD_EXCEPTION(DUPLICATE_INDEX, DuplicateIndexException.class);
-    ADD_EXCEPTION(DUPLICATE_PARTITION, DuplicatePartitionException.class);
-
-    ADD_EXCEPTION(AMBIGUOUS_TABLE, AmbiguousTableException.class);
-    ADD_EXCEPTION(AMBIGUOUS_COLUMN, AmbiguousColumnException.class);
-  }
-
-  private static void ADD_EXCEPTION(ResultCode code, Class<? extends TajoExceptionInterface> cls) {
-    EXCEPTIONS.put(code, cls);
-  }
-
-  public static void throwIfError(ReturnState state) throws TajoException {
-    if (isError(state)) {
-      throw toTajoException(state);
-    }
-  }
-
-  public static TajoException toTajoException(ReturnState state) {
-
-    if (state.getReturnCode() == ResultCode.INTERNAL_ERROR) {
-      throw new TajoInternalError(state);
-
-    } else if (EXCEPTIONS.containsKey(state.getReturnCode())) {
-      Object exception = null;
-      try {
-        Class clazz = EXCEPTIONS.get(state.getReturnCode());
-        Constructor c = clazz.getConstructor(ReturnState.class);
-        exception = c.newInstance(new Object[]{state});
-      } catch (Throwable t) {
-        throw new TajoInternalError(t);
-      }
-
-      if (exception instanceof TajoException) {
-        return (TajoException) exception;
-      } else if (exception instanceof TajoRuntimeException) {
-        throw ((TajoRuntimeException) exception);
-      } else {
-        throw ((TajoError) exception);
-      }
-
-    } else {
-      throw new TajoInternalError("Unregistred Exception (" + state.getReturnCode().name() +"): "
-          + state.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java b/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
index 966bddf..ad9bfc5 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/QueryClient.java
@@ -20,10 +20,12 @@ package org.apache.tajo.client;
 
 import org.apache.tajo.QueryId;
 import org.apache.tajo.auth.UserRoleInfo;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
+import org.apache.tajo.exception.UndefinedDatabaseException;
 import org.apache.tajo.exception.NoSuchSessionVariableException;
+import org.apache.tajo.exception.QueryNotFoundException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.ClientProtos.GetQueryResultResponse;
 import org.apache.tajo.ipc.ClientProtos.QueryHistoryProto;
 import org.apache.tajo.ipc.ClientProtos.QueryInfoProto;
 import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
@@ -31,7 +33,6 @@ import org.apache.tajo.jdbc.TajoMemoryResultSet;
 
 import java.io.Closeable;
 import java.sql.ResultSet;
-import java.sql.SQLException;
 import java.util.List;
 import java.util.Map;
 
@@ -60,19 +61,19 @@ public interface QueryClient extends Closeable {
    */
   void closeQuery(final QueryId queryId);
 
-  void closeNonForwardQuery(final QueryId queryId) throws SQLException;
+  void closeNonForwardQuery(final QueryId queryId);
 
   String getCurrentDatabase();
 
-  Boolean selectDatabase(final String databaseName) throws UndefinedDatabaseException;
+  void selectDatabase(final String databaseName) throws UndefinedDatabaseException;
 
-  Map<String, String> updateSessionVariables(final Map<String, String> variables) throws NoSuchSessionVariableException;
+  Map<String, String> updateSessionVariables(final Map<String, String> variables);
 
-  Map<String, String> unsetSessionVariables(final List<String> variables) throws NoSuchSessionVariableException;
+  Map<String, String> unsetSessionVariables(final List<String> variables);
 
   String getSessionVariable(final String varname) throws NoSuchSessionVariableException;
 
-  Boolean existSessionVariable(final String varname);
+  boolean existSessionVariable(final String varname);
 
   Map<String, String> getAllSessionVariables();
 
@@ -97,29 +98,29 @@ public interface QueryClient extends Closeable {
 
   ResultSet executeJsonQueryAndGetResult(final String json) throws TajoException;
 
-  QueryStatus getQueryStatus(QueryId queryId);
+  QueryStatus getQueryStatus(QueryId queryId) throws QueryNotFoundException;
 
-  ResultSet getQueryResult(QueryId queryId) throws SQLException;
+  ResultSet getQueryResult(QueryId queryId) throws TajoException;
 
-  ResultSet createNullResultSet(QueryId queryId) throws SQLException;
+  ResultSet createNullResultSet(QueryId queryId);
 
-  ClientProtos.GetQueryResultResponse getResultResponse(QueryId queryId) throws SQLException;
+  GetQueryResultResponse getResultResponse(QueryId queryId) throws TajoException;
 
-  TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws SQLException;
+  TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws TajoException;
 
   boolean updateQuery(final String sql) throws TajoException;
 
   boolean updateQueryWithJson(final String json) throws TajoException;
 
-  List<ClientProtos.BriefQueryInfo> getRunningQueryList() throws SQLException;
+  List<ClientProtos.BriefQueryInfo> getRunningQueryList();
 
   List<ClientProtos.BriefQueryInfo> getFinishedQueryList();
 
-  List<ClientProtos.WorkerResourceInfo> getClusterInfo() throws SQLException;
+  List<ClientProtos.WorkerResourceInfo> getClusterInfo();
 
-  QueryStatus killQuery(final QueryId queryId);
+  QueryStatus killQuery(final QueryId queryId) throws QueryNotFoundException;
 
-  QueryInfoProto getQueryInfo(final QueryId queryId) throws SQLException;
+  QueryInfoProto getQueryInfo(final QueryId queryId) throws QueryNotFoundException;
 
-  QueryHistoryProto getQueryHistory(final QueryId queryId) throws SQLException;
+  QueryHistoryProto getQueryHistory(final QueryId queryId) throws QueryNotFoundException;
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
index 02110c0..20e56ac 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/QueryClientImpl.java
@@ -21,16 +21,17 @@ package org.apache.tajo.client;
 import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.*;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoIdProtos.SessionIdProto;
+import org.apache.tajo.TajoProtos;
 import org.apache.tajo.auth.UserRoleInfo;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
-import org.apache.tajo.exception.NoSuchSessionVariableException;
-import org.apache.tajo.exception.SQLExceptionUtil;
-import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.client.v2.exception.ClientUnableToConnectException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.QueryMasterClientProtocol;
 import org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService.BlockingInterface;
@@ -38,20 +39,20 @@ import org.apache.tajo.jdbc.FetchResultSet;
 import org.apache.tajo.jdbc.TajoMemoryResultSet;
 import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.RpcClientManager;
+import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.util.ProtoUtil;
 
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.sql.ResultSet;
-import java.sql.SQLException;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.tajo.exception.ExceptionUtil.throwIfError;
+import static org.apache.tajo.exception.ExceptionUtil.throwsIfThisError;
 import static org.apache.tajo.exception.ReturnStateUtil.ensureOk;
 import static org.apache.tajo.exception.ReturnStateUtil.isSuccess;
-import static org.apache.tajo.exception.ReturnStateUtil.returnError;
-import static org.apache.tajo.exception.SQLExceptionUtil.throwIfError;
 import static org.apache.tajo.ipc.ClientProtos.*;
 import static org.apache.tajo.ipc.QueryMasterClientProtocol.QueryMasterClientProtocolService;
 
@@ -118,17 +119,17 @@ public class QueryClientImpl implements QueryClient {
   }
 
   @Override
-  public Boolean selectDatabase(String databaseName) throws UndefinedDatabaseException {
-    return conn.selectDatabase(databaseName);
+  public void selectDatabase(String databaseName) throws UndefinedDatabaseException {
+    conn.selectDatabase(databaseName);
   }
 
   @Override
-  public Map<String, String> updateSessionVariables(Map<String, String> variables) throws NoSuchSessionVariableException {
+  public Map<String, String> updateSessionVariables(Map<String, String> variables) {
     return conn.updateSessionVariables(variables);
   }
 
   @Override
-  public Map<String, String> unsetSessionVariables(List<String> variables) throws NoSuchSessionVariableException {
+  public Map<String, String> unsetSessionVariables(List<String> variables) {
     return conn.unsetSessionVariables(variables);
   }
 
@@ -138,7 +139,7 @@ public class QueryClientImpl implements QueryClient {
   }
 
   @Override
-  public Boolean existSessionVariable(String varname) {
+  public boolean existSessionVariable(String varname) {
     return conn.existSessionVariable(varname);
   }
 
@@ -184,7 +185,7 @@ public class QueryClientImpl implements QueryClient {
   public ResultSet executeQueryAndGetResult(String sql) throws TajoException {
 
     ClientProtos.SubmitQueryResponse response = executeQuery(sql);
-    ensureOk(response.getState());
+    throwIfError(response.getState());
 
     QueryId queryId = new QueryId(response.getQueryId());
 
@@ -202,7 +203,7 @@ public class QueryClientImpl implements QueryClient {
   public ResultSet executeJsonQueryAndGetResult(final String json) throws TajoException {
 
     ClientProtos.SubmitQueryResponse response = executeQueryWithJson(json);
-    ensureOk(response.getState());
+    throwIfError(response.getState());
 
     QueryId queryId = new QueryId(response.getQueryId());
 
@@ -216,7 +217,7 @@ public class QueryClientImpl implements QueryClient {
     }
   }
 
-  public ResultSet getQueryResultAndWait(QueryId queryId) {
+  public ResultSet getQueryResultAndWait(QueryId queryId) throws QueryNotFoundException {
 
     if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
       return createNullResultSet(queryId);
@@ -254,12 +255,11 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(t);
     }
 
-    ensureOk(res.getState());
     return res;
   }
 
   @Override
-  public QueryStatus getQueryStatus(QueryId queryId) {
+  public QueryStatus getQueryStatus(QueryId queryId) throws QueryNotFoundException {
 
     final BlockingInterface stub = conn.getTMStub();
     final GetQueryStatusRequest request = GetQueryStatusRequest.newBuilder()
@@ -274,19 +274,20 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(t);
     }
 
+    throwsIfThisError(res.getState(), QueryNotFoundException.class);
     ensureOk(res.getState());
     return new QueryStatus(res);
   }
 
   @Override
-  public ResultSet getQueryResult(QueryId queryId) {
+  public ResultSet getQueryResult(QueryId queryId) throws QueryNotFoundException {
 
     if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
       return createNullResultSet(queryId);
     }
 
     GetQueryResultResponse response = getResultResponse(queryId);
-    ensureOk(response.getState());
+
     TableDesc tableDesc = CatalogUtil.newTableDesc(response.getTableDesc());
     return new FetchResultSet(this, tableDesc.getLogicalSchema(), queryId, defaultFetchRows);
   }
@@ -297,7 +298,7 @@ public class QueryClientImpl implements QueryClient {
   }
 
   @Override
-  public GetQueryResultResponse getResultResponse(QueryId queryId) {
+  public GetQueryResultResponse getResultResponse(QueryId queryId) throws QueryNotFoundException {
     if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
       return null;
     }
@@ -315,12 +316,13 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(t);
     }
 
+    throwsIfThisError(response.getState(), QueryNotFoundException.class);
     ensureOk(response.getState());
     return response;
   }
 
   @Override
-  public TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws SQLException {
+  public TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws TajoException {
 
     final BlockingInterface stub = conn.getTMStub();
     final GetQueryResultDataRequest request = GetQueryResultDataRequest.newBuilder()
@@ -359,7 +361,7 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(e);
     }
 
-    ClientExceptionUtil.throwIfError(response.getState());
+    throwIfError(response.getState());
     conn.updateSessionVarsCache(ProtoUtil.convertToMap(response.getSessionVars()));
 
     return true;
@@ -378,12 +380,12 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(e);
     }
 
-    ClientExceptionUtil.throwIfError(response.getState());
+    throwIfError(response.getState());
     return true;
   }
 
   @Override
-  public List<ClientProtos.BriefQueryInfo> getRunningQueryList() throws SQLException {
+  public List<ClientProtos.BriefQueryInfo> getRunningQueryList() {
 
     final BlockingInterface stmb = conn.getTMStub();
 
@@ -394,7 +396,7 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(e);
     }
 
-    throwIfError(res.getState());
+    ensureOk(res.getState());
     return res.getQueryListList();
   }
 
@@ -415,7 +417,7 @@ public class QueryClientImpl implements QueryClient {
   }
 
   @Override
-  public List<ClientProtos.WorkerResourceInfo> getClusterInfo() throws SQLException {
+  public List<ClientProtos.WorkerResourceInfo> getClusterInfo() {
 
     final BlockingInterface stub = conn.getTMStub();
     final GetClusterInfoRequest request = GetClusterInfoRequest.newBuilder()
@@ -429,18 +431,18 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(e);
     }
 
-    throwIfError(res.getState());
+    ensureOk(res.getState());
     return res.getWorkerListList();
   }
 
   @Override
-  public QueryStatus killQuery(final QueryId queryId) {
+  public QueryStatus killQuery(final QueryId queryId) throws QueryNotFoundException {
 
     final BlockingInterface stub = conn.getTMStub();
     QueryStatus status = getQueryStatus(queryId);
 
     /* send a kill to the TM */
-    QueryIdRequest request = buildQueryIdRequest(queryId);
+    final QueryIdRequest request = buildQueryIdRequest(queryId);
     try {
       stub.killQuery(null, request);
     } catch (ServiceException e) {
@@ -475,7 +477,7 @@ public class QueryClientImpl implements QueryClient {
   	return this.maxRows;
   }
 
-  public QueryInfoProto getQueryInfo(final QueryId queryId) throws SQLException {
+  public QueryInfoProto getQueryInfo(final QueryId queryId) throws QueryNotFoundException {
 
     final BlockingInterface stub = conn.getTMStub();
     final QueryIdRequest request = buildQueryIdRequest(queryId);
@@ -487,11 +489,12 @@ public class QueryClientImpl implements QueryClient {
       throw new RuntimeException(e);
     }
 
-    throwIfError(res.getState());
+    throwsIfThisError(res.getState(), QueryNotFoundException.class);
+    ensureOk(res.getState());
     return res.getQueryInfo();
   }
 
-  public QueryHistoryProto getQueryHistory(final QueryId queryId) throws SQLException {
+  public QueryHistoryProto getQueryHistory(final QueryId queryId) throws QueryNotFoundException {
     final QueryInfoProto queryInfo = getQueryInfo(queryId);
 
     if (queryInfo.getHostNameOfQM() == null || queryInfo.getQueryMasterClientPort() == 0) {
@@ -504,6 +507,7 @@ public class QueryClientImpl implements QueryClient {
     RpcClientManager manager = RpcClientManager.getInstance();
     NettyClientBase qmClient = null;
 
+
     try {
 
       qmClient = manager.newClient(
@@ -516,6 +520,7 @@ public class QueryClientImpl implements QueryClient {
           false
       );
 
+
       conn.checkSessionAndGet(conn.getTajoMasterConnection());
 
       QueryIdRequest request = QueryIdRequest.newBuilder()
@@ -531,19 +536,18 @@ public class QueryClientImpl implements QueryClient {
         throw new RuntimeException(e);
       }
 
-      throwIfError(res.getState());
+      ensureOk(res.getState());
       return res.getQueryHistory();
 
+    } catch (NoSuchMethodException | ClassNotFoundException e) {
+      throw new TajoInternalError(e);
     } catch (ConnectException e) {
-      throw SQLExceptionUtil.makeUnableToEstablishConnection(e);
-    } catch (ClassNotFoundException e) {
-      throw SQLExceptionUtil.makeUnableToEstablishConnection(e);
-    } catch (NoSuchMethodException e) {
-      throw SQLExceptionUtil.makeUnableToEstablishConnection(e);
-    } catch (SQLException e) {
-      throw e;
+      throw new TajoRuntimeException(
+          new ClientUnableToConnectException(NetUtils.normalizeInetSocketAddress(qmAddress)));
     } finally {
-      qmClient.close();
+      if (qmClient != null) {
+        qmClient.close();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
index a97cb33..105830f 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/SessionConnection.java
@@ -26,10 +26,10 @@ import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.annotation.NotNull;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.auth.UserRoleInfo;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
+import org.apache.tajo.exception.ExceptionUtil;
+import org.apache.tajo.exception.UndefinedDatabaseException;
 import org.apache.tajo.client.v2.exception.ClientConnectionException;
 import org.apache.tajo.exception.NoSuchSessionVariableException;
-import org.apache.tajo.exception.SQLExceptionUtil;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.SessionUpdateResponse;
 import org.apache.tajo.ipc.ClientProtos.UpdateSessionVariableRequest;
@@ -61,7 +61,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import static org.apache.tajo.error.Errors.ResultCode.NO_SUCH_SESSION_VARIABLE;
 import static org.apache.tajo.error.Errors.ResultCode.UNDEFINED_DATABASE;
 import static org.apache.tajo.exception.ReturnStateUtil.*;
-import static org.apache.tajo.exception.SQLExceptionUtil.throwIfError;
 import static org.apache.tajo.exception.SQLExceptionUtil.toSQLException;
 import static org.apache.tajo.ipc.ClientProtos.CreateSessionRequest;
 import static org.apache.tajo.ipc.ClientProtos.CreateSessionResponse;
@@ -226,7 +225,7 @@ public class SessionConnection implements Closeable {
     return Collections.unmodifiableMap(sessionVarsCache);
   }
 
-  public Map<String, String> unsetSessionVariables(final List<String> variables) throws NoSuchSessionVariableException {
+  public Map<String, String> unsetSessionVariables(final List<String> variables) {
 
     final BlockingInterface stub = getTMStub();
     final UpdateSessionVariableRequest request = UpdateSessionVariableRequest.newBuilder()
@@ -241,10 +240,6 @@ public class SessionConnection implements Closeable {
       throw new RuntimeException(e);
     }
 
-    if (isThisError(response.getState(), NO_SUCH_SESSION_VARIABLE)) {
-      throw new NoSuchSessionVariableException(response.getState());
-    }
-
     ensureOk(response.getState());
     updateSessionVarsCache(ProtoUtil.convertToMap(response.getSessionVars()));
     return Collections.unmodifiableMap(sessionVarsCache);
@@ -285,7 +280,7 @@ public class SessionConnection implements Closeable {
     return response.getValue();
   }
 
-  public Boolean existSessionVariable(final String varname) {
+  public boolean existSessionVariable(final String varname) {
 
     ReturnState state;
     try {
@@ -319,27 +314,19 @@ public class SessionConnection implements Closeable {
     return ProtoUtil.convertToMap(response.getValue());
   }
 
-  public Boolean selectDatabase(final String dbName) throws UndefinedDatabaseException {
+  public void selectDatabase(final String dbName) throws UndefinedDatabaseException {
 
-    BlockingInterface stub = getTMStub();
-    boolean selected;
     try {
-      ReturnState state = stub.selectDatabase(null, getSessionedString(dbName));
-
-      if (isThisError(state, UNDEFINED_DATABASE)) {
-        throw new UndefinedDatabaseException(dbName);
-      }
+      final BlockingInterface stub = getTMStub();
+      final ReturnState state = stub.selectDatabase(null, getSessionedString(dbName));
 
-      selected = ensureOk(state);
+      ExceptionUtil.throwsIfThisError(state, UndefinedDatabaseException.class);
+      ensureOk(state);
+      this.baseDatabase = dbName;
 
     } catch (ServiceException e) {
       throw new RuntimeException(e);
     }
-
-    if (selected) {
-      this.baseDatabase = dbName;
-    }
-    return selected;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/TajoClientImpl.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientImpl.java
index 8c167a4..a6c07ea 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientImpl.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientImpl.java
@@ -26,11 +26,10 @@ import org.apache.tajo.annotation.ThreadSafe;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.exception.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexDescProto;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.ipc.ClientProtos.*;
 import org.apache.tajo.jdbc.TajoMemoryResultSet;
 import org.apache.tajo.service.ServiceTracker;
@@ -96,7 +95,7 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
     queryClient.closeQuery(queryId);
   }
 
-  public void closeNonForwardQuery(final QueryId queryId) throws SQLException {
+  public void closeNonForwardQuery(final QueryId queryId) {
     queryClient.closeNonForwardQuery(queryId);
   }
 
@@ -116,23 +115,23 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
     return queryClient.executeJsonQueryAndGetResult(json);
   }
 
-  public QueryStatus getQueryStatus(QueryId queryId) {
+  public QueryStatus getQueryStatus(QueryId queryId) throws QueryNotFoundException {
     return queryClient.getQueryStatus(queryId);
   }
 
-  public ResultSet getQueryResult(QueryId queryId) throws SQLException {
+  public ResultSet getQueryResult(QueryId queryId) throws TajoException {
     return queryClient.getQueryResult(queryId);
   }
 
-  public ResultSet createNullResultSet(QueryId queryId) throws SQLException {
+  public ResultSet createNullResultSet(QueryId queryId) {
     return TajoClientUtil.createNullResultSet(queryId);
   }
 
-  public GetQueryResultResponse getResultResponse(QueryId queryId) throws SQLException {
+  public GetQueryResultResponse getResultResponse(QueryId queryId) throws TajoException {
     return queryClient.getResultResponse(queryId);
   }
 
-  public TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws SQLException {
+  public TajoMemoryResultSet fetchNextQueryResult(final QueryId queryId, final int fetchRowNum) throws TajoException {
     return queryClient.fetchNextQueryResult(queryId, fetchRowNum);
   }
 
@@ -144,11 +143,11 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
     return queryClient.updateQueryWithJson(json);
   }
 
-  public QueryStatus killQuery(final QueryId queryId) {
+  public QueryStatus killQuery(final QueryId queryId) throws QueryNotFoundException {
     return queryClient.killQuery(queryId);
   }
 
-  public List<BriefQueryInfo> getRunningQueryList() throws SQLException {
+  public List<BriefQueryInfo> getRunningQueryList() {
     return queryClient.getRunningQueryList();
   }
 
@@ -156,15 +155,15 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
     return queryClient.getFinishedQueryList();
   }
 
-  public List<WorkerResourceInfo> getClusterInfo() throws SQLException {
+  public List<WorkerResourceInfo> getClusterInfo() {
     return queryClient.getClusterInfo();
   }
 
-  public QueryInfoProto getQueryInfo(final QueryId queryId) throws SQLException {
+  public QueryInfoProto getQueryInfo(final QueryId queryId) throws QueryNotFoundException {
     return queryClient.getQueryInfo(queryId);
   }
 
-  public QueryHistoryProto getQueryHistory(final QueryId queryId) throws SQLException {
+  public QueryHistoryProto getQueryHistory(final QueryId queryId) throws QueryNotFoundException {
     return queryClient.getQueryHistory(queryId);
   }
 
@@ -180,16 +179,18 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
   // CatalogClient wrappers
   /*------------------------------------------------------------------------*/
 
-  public boolean createDatabase(final String databaseName) throws DuplicateDatabaseException {
-    return catalogClient.createDatabase(databaseName);
+  public void createDatabase(final String databaseName) throws DuplicateDatabaseException {
+    catalogClient.createDatabase(databaseName);
   }
 
   public boolean existDatabase(final String databaseName) {
     return catalogClient.existDatabase(databaseName);
   }
 
-  public boolean dropDatabase(final String databaseName) throws UndefinedDatabaseException {
-    return catalogClient.dropDatabase(databaseName);
+  public void dropDatabase(final String databaseName) throws UndefinedDatabaseException,
+      InsufficientPrivilegeException {
+
+    catalogClient.dropDatabase(databaseName);
   }
 
   public List<String> getAllDatabaseNames() {
@@ -200,23 +201,28 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
     return catalogClient.existTable(tableName);
   }
 
-  public TableDesc createExternalTable(final String tableName, final Schema schema, final URI path,
-                                       final TableMeta meta) throws DuplicateTableException {
+  public TableDesc createExternalTable(final String tableName,
+                                       final Schema schema,
+                                       final URI path,
+                                       final TableMeta meta)
+      throws DuplicateTableException, UnavailableTableLocationException, InsufficientPrivilegeException {
+
     return catalogClient.createExternalTable(tableName, schema, path, meta);
   }
 
   public TableDesc createExternalTable(final String tableName, final Schema schema, final URI path,
                                        final TableMeta meta, final PartitionMethodDesc partitionMethodDesc)
-      throws DuplicateTableException {
+      throws DuplicateTableException, UnavailableTableLocationException, InsufficientPrivilegeException {
     return catalogClient.createExternalTable(tableName, schema, path, meta, partitionMethodDesc);
   }
 
-  public boolean dropTable(final String tableName) throws UndefinedTableException {
-    return dropTable(tableName, false);
+  public void dropTable(final String tableName) throws UndefinedTableException, InsufficientPrivilegeException {
+    dropTable(tableName, false);
   }
 
-  public boolean dropTable(final String tableName, final boolean purge) throws UndefinedTableException {
-    return catalogClient.dropTable(tableName, purge);
+  public void dropTable(final String tableName, final boolean purge) throws UndefinedTableException,
+      InsufficientPrivilegeException {
+    catalogClient.dropTable(tableName, purge);
   }
 
   public List<String> getTableList(@Nullable final String databaseName) {
@@ -232,37 +238,37 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
   }
 
   @Override
-  public IndexDescProto getIndex(String indexName) throws SQLException {
+  public IndexDescProto getIndex(String indexName) {
     return catalogClient.getIndex(indexName);
   }
 
   @Override
-  public boolean existIndex(String indexName) throws SQLException {
+  public boolean existIndex(String indexName) {
     return catalogClient.existIndex(indexName);
   }
 
   @Override
-  public List<IndexDescProto> getIndexes(String tableName) throws SQLException {
+  public List<IndexDescProto> getIndexes(String tableName) {
     return catalogClient.getIndexes(tableName);
   }
 
   @Override
-  public boolean hasIndexes(String tableName) throws SQLException {
+  public boolean hasIndexes(String tableName) {
     return catalogClient.hasIndexes(tableName);
   }
 
   @Override
-  public IndexDescProto getIndex(String tableName, String[] columnNames) throws SQLException {
+  public IndexDescProto getIndex(String tableName, String[] columnNames) {
     return catalogClient.getIndex(tableName, columnNames);
   }
 
   @Override
-  public boolean existIndex(String tableName, String[] columnName) throws SQLException {
+  public boolean existIndex(String tableName, String[] columnName) {
     return catalogClient.existIndex(tableName, columnName);
   }
 
   @Override
-  public boolean dropIndex(String indexName) throws SQLException {
+  public boolean dropIndex(String indexName) {
     return catalogClient.dropIndex(indexName);
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
index c79b756..3f30f97 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoClientUtil.java
@@ -25,6 +25,7 @@ import org.apache.tajo.TajoProtos;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.exception.QueryNotFoundException;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.jdbc.FetchResultSet;
 import org.apache.tajo.jdbc.TajoMemoryResultSet;
@@ -58,7 +59,7 @@ public class TajoClientUtil {
     return !isQueryWaitingForSchedule(state) && !isQueryRunning(state);
   }
 
-  public static QueryStatus waitCompletion(QueryClient client, QueryId queryId) {
+  public static QueryStatus waitCompletion(QueryClient client, QueryId queryId) throws QueryNotFoundException {
     QueryStatus status = client.getQueryStatus(queryId);
 
     while(!isQueryComplete(status.getState())) {


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

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCliNegatives.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCliNegatives.java b/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCliNegatives.java
new file mode 100644
index 0000000..edbeff6
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCliNegatives.java
@@ -0,0 +1,146 @@
+/*
+ * 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.cli.tsql;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ConfigKey;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.client.QueryStatus;
+import org.apache.tajo.util.FileUtil;
+import org.junit.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTajoCliNegatives extends QueryTestCaseBase {
+  private static TajoCli tajoCli;
+  private static ByteArrayOutputStream out;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    out = new ByteArrayOutputStream();
+    tajoCli = new TajoCli(testingCluster.getConfiguration(), new String[]{}, System.in, out);
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    out.close();
+    if (tajoCli != null) {
+      tajoCli.close();
+    }
+  }
+
+  @Before
+  public void resetConsole() throws IOException {
+    out.reset();
+  }
+
+  public void assertMetaCommandFailures(String cmd) throws Exception {
+    Path resultFile = getResultFile(getMethodName() + ".result");
+    String expected = FileUtil.readTextFile(new File(resultFile.toUri()));
+
+    tajoCli.executeMetaCommand(cmd);
+    String consoleResult = new String(out.toByteArray());
+    assertEquals(expected, consoleResult);
+  }
+
+  public void assertMetaCommandFailure(String cmd, String expectedMsg) throws Exception {
+    tajoCli.executeMetaCommand(cmd);
+    String consoleResult = new String(out.toByteArray());
+    assertEquals(expectedMsg, consoleResult);
+  }
+
+  public void assertScriptFailure(String cmd) throws Exception {
+    Path resultFile = getResultFile(getMethodName() + ".result");
+    String expected = FileUtil.readTextFile(new File(resultFile.toUri()));
+
+    tajoCli.executeScript(cmd);
+    String consoleResult = new String(out.toByteArray());
+    assertEquals(expected, consoleResult);
+  }
+
+  public void assertScriptFailure(String cmd, String expectedMsg) throws Exception {
+    tajoCli.executeScript(cmd);
+    String consoleResult = new String(out.toByteArray());
+    assertEquals(expectedMsg, consoleResult);
+  }
+
+  @Test
+  public void testConnectDatabase() throws Exception {
+    assertMetaCommandFailure("\\c unknown_db", "ERROR: database 'unknown_db' does not exist");
+  }
+
+  @Test
+  public void testDescTable() throws Exception {
+    assertMetaCommandFailure("\\d unknown_table", "ERROR: relation 'unknown_table' does not exist\n");
+  }
+
+  @Test
+  public void testQueryVerification() throws Exception {
+    assertScriptFailure("select * from unknown_table", "ERROR: relation 'default.unknown_table' does not exist\n");
+  }
+
+  @Test
+  public void testQuerySyntax() throws Exception {
+    assertScriptFailure("select * from unknown-table");
+  }
+
+  private static void setVar(TajoCli cli, ConfigKey key, String val) throws Exception {
+    cli.executeMetaCommand("\\set " + key.keyname() + " " + val);
+  }
+
+  public static class TajoCliOutputTestFormatter extends DefaultTajoCliOutputFormatter {
+    @Override
+    protected String getResponseTimeReadable(float responseTime) {
+      return "";
+    }
+    @Override
+    public void printProgress(PrintWriter sout, QueryStatus status) {
+      //nothing to do
+    }
+  }
+
+  @Test
+  public void testQueryNotImplementedFeature() throws Exception {
+
+    try {
+      client.updateQuery("CREATE DATABASE TestTajoCliNegatives");
+      client.updateQuery("CREATE TABLE TestTajoCliNegatives.table12u79 ( name RECORD(last TEXT, first TEXT) )");
+
+      assertScriptFailure("select name FROM TestTajoCliNegatives.table12u79",
+          "ERROR: not implemented feature: record field projection\n");
+
+    } finally {
+      client.updateQuery("DROP TABLE IF EXISTS TestTajoCliNegatives.table12u79");
+      client.updateQuery("DROP DATABASE IF EXISTS TestTajoCliNegatives");
+    }
+  }
+
+  @Test
+  public void testQueryFailure() throws Exception {
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
+    assertScriptFailure("select fail(3, l_orderkey, 'testQueryFailure') from default.lineitem" ,
+        "ERROR: No error message\n");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/client/TestCatalogAdminClientExceptions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestCatalogAdminClientExceptions.java b/tajo-core/src/test/java/org/apache/tajo/client/TestCatalogAdminClientExceptions.java
new file mode 100644
index 0000000..a92af58
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestCatalogAdminClientExceptions.java
@@ -0,0 +1,103 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.exception.*;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.KeyValueSet;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.junit.Assert.assertFalse;
+
+public class TestCatalogAdminClientExceptions extends QueryTestCaseBase {
+  private static TajoTestingCluster cluster;
+  private static TajoClient client;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = TpchTestBase.getInstance().getTestingCluster();
+    client = cluster.newTajoClient();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    client.close();
+  }
+
+  @Test(expected = DuplicateDatabaseException.class)
+  public final void testCreateDatabase() throws TajoException {
+    client.createDatabase("default"); // duplicate database
+  }
+
+  @Test
+  public final void testExistDatabase() {
+    assertFalse(client.existDatabase("unknown-database")); // unknown database
+  }
+
+  @Test(expected = UndefinedDatabaseException.class)
+  public final void testDropDatabase() throws TajoException {
+    client.dropDatabase("unknown-database"); // unknown database
+  }
+
+  @Test(expected = UnavailableTableLocationException.class)
+  public final void testCreateExternalTableUnavailableLocation() throws TajoException {
+    client.createExternalTable("table128237", new Schema(), URI.create("/tajo/test1bcd"),
+        new TableMeta("TEXT", new KeyValueSet()));
+  }
+
+  @Test(expected = DuplicateTableException.class)
+  public final void testCreateExternalTableDuplicated() throws TajoException {
+    client.createExternalTable("default.lineitem", new Schema(), URI.create("/"),
+        new TableMeta("TEXT", new KeyValueSet()));
+  }
+
+  @Test(expected = InsufficientPrivilegeException.class)
+  public final void testCreateExternalTableInsufficientPrivilege() throws TajoException {
+    Path p = TajoConf.getWarehouseDir(conf);
+    client.createExternalTable("information_schema.table1237891", new Schema(), p.toUri(),
+        new TableMeta("TEXT", new KeyValueSet()));
+  }
+
+  @Test(expected = UndefinedTableException.class)
+  public final void testDropTableAbsent() throws UndefinedTableException, InsufficientPrivilegeException {
+    client.dropTable("unknown-table"); // unknown table
+  }
+
+  @Test(expected = InsufficientPrivilegeException.class)
+  public final void testDropTableInsufficient() throws UndefinedTableException, InsufficientPrivilegeException {
+    client.dropTable("information_schema.tables"); // cannot be dropped
+  }
+
+  @Test(expected = UndefinedTableException.class)
+  public final void testGetTableDesc() throws UndefinedTableException {
+    client.getTableDesc("unknown-table");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/client/TestQueryClientExceptions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestQueryClientExceptions.java b/tajo-core/src/test/java/org/apache/tajo/client/TestQueryClientExceptions.java
new file mode 100644
index 0000000..6b11846
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestQueryClientExceptions.java
@@ -0,0 +1,126 @@
+/**
+ * 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.client;
+
+import com.facebook.presto.hive.shaded.com.google.common.collect.Maps;
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.exception.*;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+@NotThreadSafe
+public class TestQueryClientExceptions {
+  private static TajoTestingCluster cluster;
+  private static TajoClient client;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = TpchTestBase.getInstance().getTestingCluster();
+    client = cluster.newTajoClient();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    client.close();
+  }
+
+  @Test
+  public void testExecuteQuery() {
+    // This is just an error propagation unit test. Specified SQL errors will be addressed in other unit tests.
+    ReturnState state = client.executeQuery("select * from unknown_table").getState();
+    assertEquals(Errors.ResultCode.UNDEFINED_TABLE, state.getReturnCode());
+
+    state = client.executeQuery("create table default.lineitem (name int);").getState();
+    assertEquals(Errors.ResultCode.DUPLICATE_TABLE, state.getReturnCode());
+  }
+
+  @Test(expected = DuplicateTableException.class)
+  public void testUpdateQuery() throws TajoException {
+    client.updateQuery("create table default.lineitem (name int);");
+  }
+
+  @Test(expected = UndefinedTableException.class)
+  public void testExecuteQueryAndGetResult() throws TajoException {
+    // This is just an error propagation unit test. Specified SQL errors will be addressed in other unit tests.
+    client.executeQueryAndGetResult("select * from unknown_table");
+  }
+
+  @Test
+  public void testCloseQuery() {
+    // absent query id
+    client.closeQuery(LocalTajoTestingUtility.newQueryId());
+    client.closeNonForwardQuery(LocalTajoTestingUtility.newQueryId());
+  }
+
+  @Test(expected = UndefinedDatabaseException .class)
+  public void testSelectDatabase() throws UndefinedDatabaseException {
+    // absent database name
+    client.selectDatabase("unknown_db");
+  }
+
+  @Test(expected = NoSuchSessionVariableException.class)
+  public void testGetSessionVar() throws NoSuchSessionVariableException {
+    // absent session variable
+    client.getSessionVariable("unknown-var");
+  }
+
+  @Test(expected = QueryNotFoundException.class)
+  public void testGetQueryResult() throws TajoException {
+    // absent query id
+    client.getQueryResult(LocalTajoTestingUtility.newQueryId());
+  }
+
+  @Test(expected = QueryNotFoundException.class)
+  public void testGetResultResponse() throws TajoException {
+    // absent query id
+    client.getResultResponse(LocalTajoTestingUtility.newQueryId());
+  }
+
+  @Test(expected = QueryNotFoundException.class)
+  public void testFetchNextQueryResult() throws TajoException {
+    client.fetchNextQueryResult(LocalTajoTestingUtility.newQueryId(), 100);
+  }
+
+  @Test(expected = QueryNotFoundException.class)
+  public void testKillQuery() throws QueryNotFoundException {
+    client.killQuery(LocalTajoTestingUtility.newQueryId());
+  }
+
+  @Test(expected = QueryNotFoundException.class)
+  public void testGetQueryInfo() throws QueryNotFoundException {
+    client.getQueryInfo(LocalTajoTestingUtility.newQueryId());
+  }
+
+  @Test(expected = QueryNotFoundException.class)
+  public void testGetQueryHistory() throws QueryNotFoundException {
+    client.getQueryHistory(LocalTajoTestingUtility.newQueryId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
index 0ba0d76..13e9de9 100644
--- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -96,7 +96,7 @@ public class TestTajoClient {
 
       // test existence
       assertFalse(client.existDatabase(prefix + i));
-      assertTrue(client.createDatabase(prefix + i));
+      client.createDatabase(prefix + i);
       assertTrue(client.existDatabase(prefix + i));
 
       // test allDatabaseNames
@@ -108,7 +108,7 @@ public class TestTajoClient {
     for (int i = 0; i < 10; i++) {
       assertTrue(client.existDatabase(prefix + i));
       assertTrue(client.getAllDatabaseNames().contains(prefix + i));
-      assertTrue(client.dropDatabase(prefix + i));
+      client.dropDatabase(prefix + i);
       assertFalse(client.existDatabase(prefix + i));
       assertFalse(client.getAllDatabaseNames().contains(prefix + i));
     }
@@ -122,13 +122,13 @@ public class TestTajoClient {
     assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase());
 
     String databaseName = CatalogUtil.normalizeIdentifier("testcurrentdatabase");
-    assertTrue(client.createDatabase(databaseName));
+    client.createDatabase(databaseName);
     assertEquals(currentNum + 1, client.getAllDatabaseNames().size());
     assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase());
-    assertTrue(client.selectDatabase(databaseName));
+    client.selectDatabase(databaseName);
     assertEquals(databaseName, client.getCurrentDatabase());
-    assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME));
-    assertTrue(client.dropDatabase(databaseName));
+    client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME);
+    client.dropDatabase(databaseName);
 
     assertEquals(currentNum, client.getAllDatabaseNames().size());
   }
@@ -139,7 +139,7 @@ public class TestTajoClient {
     assertFalse(client.existDatabase("invaliddatabase"));
 
     try {
-      assertTrue(client.selectDatabase("invaliddatabase"));
+      client.selectDatabase("invaliddatabase");
       assertFalse(true);
     } catch (Throwable t) {
       assertFalse(false);
@@ -152,8 +152,8 @@ public class TestTajoClient {
   public final void testDropCurrentDatabase() throws IOException, TajoException, InterruptedException {
     int currentNum = client.getAllDatabaseNames().size();
     String databaseName = CatalogUtil.normalizeIdentifier("testdropcurrentdatabase");
-    assertTrue(client.createDatabase(databaseName));
-    assertTrue(client.selectDatabase(databaseName));
+    client.createDatabase(databaseName);
+    client.selectDatabase(databaseName);
     assertEquals(databaseName, client.getCurrentDatabase());
 
     try {
@@ -163,8 +163,8 @@ public class TestTajoClient {
       assertFalse(false);
     }
 
-    assertTrue(client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME));
-    assertTrue(client.dropDatabase(databaseName));
+    client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME);
+    client.dropDatabase(databaseName);
     assertEquals(currentNum, client.getAllDatabaseNames().size());
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClientFailures.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClientFailures.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClientFailures.java
deleted file mode 100644
index b745caa..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClientFailures.java
+++ /dev/null
@@ -1,79 +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.client;
-
-import net.jcip.annotations.NotThreadSafe;
-import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.TpchTestBase;
-import org.apache.tajo.catalog.exception.DuplicateDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.sql.SQLException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-@NotThreadSafe
-public class TestTajoClientFailures {
-  private static TajoTestingCluster cluster;
-  private static TajoClient client;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    cluster = TpchTestBase.getInstance().getTestingCluster();
-    client = cluster.newTajoClient();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    client.close();
-  }
-
-  @Test(expected = DuplicateDatabaseException.class)
-  public final void testCreateDatabase() throws TajoException {
-    assertFalse(client.createDatabase("default")); // duplicate database
-  }
-
-  @Test(expected = UndefinedDatabaseException.class)
-  public final void testDropDatabase() throws TajoException {
-    assertFalse(client.dropDatabase("unknown-database")); // unknown database
-  }
-
-  @Test(expected = UndefinedTableException.class)
-  public final void testDropTable() throws UndefinedTableException {
-    assertFalse(client.dropTable("unknown-table")); // unknown table
-  }
-
-  @Test
-  public void testExecuteSQL() throws SQLException {
-    // This is just an error propagation unit test. Specified SQL errors will be addressed in other unit tests.
-    ReturnState state = client.executeQuery("select * from unknown_table").getState();
-    assertEquals(Errors.ResultCode.UNDEFINED_TABLE, state.getReturnCode());
-
-    state = client.executeQuery("create table default.lineitem (name int);").getState();
-    assertEquals(Errors.ResultCode.DUPLICATE_TABLE, state.getReturnCode());
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java b/tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java
index 99b7c15..e1fca63 100644
--- a/tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java
+++ b/tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java
@@ -20,10 +20,7 @@ package org.apache.tajo.client.v2;
 
 import com.facebook.presto.hive.shaded.com.google.common.collect.Lists;
 import org.apache.tajo.QueryTestCaseBase;
-import org.apache.tajo.catalog.exception.DuplicateDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
-import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.service.ServiceTracker;
 import org.apache.tajo.service.ServiceTrackerFactory;
 import org.junit.AfterClass;
@@ -196,8 +193,8 @@ public class TestTajoClientV2 extends QueryTestCaseBase {
     resultContainer.get(0).close();
   }
 
-  @Test(timeout = 10 * 1000)
-  public void testQueryFutureKill() throws TajoException, ExecutionException, InterruptedException, SQLException {
+  @Test(expected = QueryKilledException.class, timeout = 10 * 1000)
+  public void testQueryFutureKill() throws Throwable {
     QueryFuture future = clientv2.executeQueryAsync("select sleep(1) from lineitem where l_orderkey > 4");
 
     assertTrue(future.isOk());
@@ -216,6 +213,14 @@ public class TestTajoClientV2 extends QueryTestCaseBase {
     assertFalse(future.isSuccessful());
     assertFalse(future.isFailed());
     assertTrue(future.isKilled());
+
+    try {
+      future.get();
+    } catch (ExecutionException e) {
+      throw e.getCause();
+    } finally {
+      future.release();
+    }
   }
 
 
@@ -233,4 +238,37 @@ public class TestTajoClientV2 extends QueryTestCaseBase {
   public void testErrorOnExecuteQueryAsync() throws TajoException {
     clientv2.executeQueryAsync("select * from unknown_table");
   }
+
+  @Test(expected = SQLSyntaxError.class)
+  public void testSyntaxErrorOnUpdateQuery() throws TajoException {
+    clientv2.executeUpdate("drap table unknown-table");
+  }
+
+  @Test(expected = SQLSyntaxError.class)
+  public void testSyntaxErrorOnExecuteQuery() throws TajoException {
+    clientv2.executeQuery("select fail(3, ");
+  }
+
+  @Test(expected = SQLSyntaxError.class)
+  public void testSyntaxErrorOnExecuteQueryAsync() throws TajoException {
+    clientv2.executeQueryAsync("select fail(3, ");
+  }
+
+  @Test(expected = QueryFailedException.class)
+  public void testFailedExecuteQuery() throws TajoException {
+    clientv2.executeQuery("select fail(3, l_orderkey, 'testQueryFailure') from default.lineitem");
+  }
+
+  @Test(expected = QueryFailedException.class)
+  public void testFailedExecuteQueryAsync() throws Throwable {
+    QueryFuture future = clientv2.executeQueryAsync(
+            "select fail(3, l_orderkey, 'testQueryFailure') from default.lineitem");
+    try {
+      future.get();
+    } catch (ExecutionException e) {
+      throw e.getCause();
+    } finally {
+      future.release();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
index c08ea87..8c31550 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
@@ -33,7 +33,6 @@ import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.engine.function.FunctionLoader;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
@@ -212,7 +211,7 @@ public class TestEvalTreeUtil {
   };
   
   @Test
-  public final void testGetSchemaFromTargets() throws InternalException {
+  public final void testGetSchemaFromTargets() {
     Target [] targets = getRawTargets(QUERIES[0]);
     Schema schema = EvalTreeUtil.getSchemaByTargets(null, targets);
     Column col1 = schema.getColumn(0);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
index 684f0f2..fe51aa4 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLExpression.java
@@ -21,7 +21,7 @@ package org.apache.tajo.engine.eval;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
+import org.apache.tajo.exception.UndefinedFunctionException;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.TimestampDatum;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
index 7e63bc1..c214f66 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestConditionalExpressions.java
@@ -20,7 +20,7 @@ package org.apache.tajo.engine.function;
 
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
+import org.apache.tajo.exception.UndefinedFunctionException;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.engine.eval.ExprTestBase;
 import org.apache.tajo.exception.TajoException;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index 0d4e6a4..69b36c5 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -41,7 +41,7 @@ import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.plan.LogicalOptimizer;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
@@ -173,7 +173,8 @@ public class TestPhysicalPlanner {
     createLargeScoreTable();
   }
 
-  public static void createLargeScoreTable() throws IOException {
+  public static void createLargeScoreTable() throws IOException, TajoException {
+
     // Preparing a large table
     Path scoreLargePath = new Path(testDir, "score_large");
     CommonTestingUtil.cleanupTestDir(scoreLargePath.toString());

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java
index b0ec92d..8509b07 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestAlterTablespace.java
@@ -37,7 +37,7 @@ public class TestAlterTablespace extends QueryTestCaseBase {
       //////////////////////////////////////////////////////////////////////////////
 
       assertFalse(catalog.existTablespace("space1"));
-      assertTrue(catalog.createTablespace("space1", "hdfs://xxx.com/warehouse"));
+      catalog.createTablespace("space1", "hdfs://xxx.com/warehouse");
       assertTrue(catalog.existTablespace("space1"));
 
       // pre verification
@@ -52,7 +52,7 @@ public class TestAlterTablespace extends QueryTestCaseBase {
       assertEquals("space1", space1.getSpaceName());
       assertEquals("hdfs://yyy.com/warehouse", space1.getUri());
 
-      assertTrue(catalog.dropTablespace("space1"));
+      catalog.dropTablespace("space1");
       assertFalse(catalog.existTablespace("space1"));
     }
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
index 7adb237..3a17c7b 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
@@ -35,6 +35,9 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.ScanNode;
 import org.apache.tajo.storage.StorageConstants;
@@ -98,10 +101,9 @@ public class TestHBaseTable extends QueryTestCaseBase {
   public void testVerifyCreateHBaseTableRequiredMeta() throws Exception {
     try {
       executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) TABLESPACE cluster1 USING hbase").close();
-
       fail("hbase table must have 'table' meta");
-    } catch (Exception e) {
-      assertTrue(e.getMessage().indexOf("HBase mapped table") >= 0);
+    } catch (TajoException e) {
+      assertEquals(e.getErrorCode(), ResultCode.MISSING_TABLE_PROPERTY);
     }
 
     try {
@@ -110,8 +112,8 @@ public class TestHBaseTable extends QueryTestCaseBase {
           "WITH ('table'='hbase_table')").close();
 
       fail("hbase table must have 'columns' meta");
-    } catch (Exception e) {
-      assertTrue(e.getMessage().indexOf("'columns' property is required") >= 0);
+    } catch (TajoException e) {
+      assertEquals(e.getErrorCode(), ResultCode.MISSING_TABLE_PROPERTY);
     }
   }
 
@@ -152,7 +154,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
     try {
       executeString(sql).close();
       fail("External table should be a existed table.");
-    } catch (Exception e) {
+    } catch (Throwable e) {
       assertTrue(e.getMessage().indexOf("External table should be a existed table.") >= 0);
     }
   }
@@ -1126,8 +1128,8 @@ public class TestHBaseTable extends QueryTestCaseBase {
       executeString("insert into hbase_mapped_table " +
           "select id, name from base_table ").close();
       fail("If inserting data type different with target table data type, should throw exception");
-    } catch (Exception e) {
-      assertTrue(e.getMessage().indexOf("is different column type with") >= 0);
+    } catch (TajoException e) {
+      assertEquals(ResultCode.DATATYPE_MISMATCH, e.getErrorCode());
     } finally {
       executeString("DROP TABLE base_table PURGE").close();
       executeString("DROP TABLE hbase_mapped_table PURGE").close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
index 4fb2d31..2fddbfa 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.engine.query;
 
-import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
@@ -31,7 +30,6 @@ import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.datum.Datum;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java
index 1be21e4..3ae0c60 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTruncateTable.java
@@ -18,23 +18,20 @@
 
 package org.apache.tajo.engine.query;
 
-import org.apache.tajo.IntegrationTest;
 import org.apache.tajo.QueryTestCaseBase;
-import org.apache.tajo.TajoConstants;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-@Category(IntegrationTest.class)
+
 public class TestTruncateTable extends QueryTestCaseBase {
-  public TestTruncateTable() {
-    super(TajoConstants.DEFAULT_DATABASE_NAME);
-  }
+
   @Test
   public final void testTruncateTable() throws Exception {
     try {
@@ -66,7 +63,7 @@ public class TestTruncateTable extends QueryTestCaseBase {
   }
 
   @Test
-  public final void testTruncateExternalTable() throws Exception {
+  public final void testTruncateExternalTable() throws TajoException, SQLException {
     try {
       List<String> createdNames = executeDDL("table2_ddl.sql", "truncate_table2", "truncate_table2");
       assertTableExists(createdNames.get(0));
@@ -80,7 +77,6 @@ public class TestTruncateTable extends QueryTestCaseBase {
       res.close();
 
       executeString("truncate table truncate_table2");
-
       fail("Can't truncate external table");
     } catch (Exception e) {
       // succeeded

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/resources/queries/TestTruncateTable/table1_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTruncateTable/table1_ddl.sql b/tajo-core/src/test/resources/queries/TestTruncateTable/table1_ddl.sql
index dc67e75..b6196ec 100644
--- a/tajo-core/src/test/resources/queries/TestTruncateTable/table1_ddl.sql
+++ b/tajo-core/src/test/resources/queries/TestTruncateTable/table1_ddl.sql
@@ -1 +1 @@
-CREATE TABLE truncate_table1 AS SELECT * FROM lineitem;
\ No newline at end of file
+CREATE TABLE truncate_table1 AS SELECT * FROM default.lineitem;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/resources/results/TestTajoCli/testAlterTableAddDropPartition.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCli/testAlterTableAddDropPartition.result b/tajo-core/src/test/resources/results/TestTajoCli/testAlterTableAddDropPartition.result
index b09f134..fdb620d 100644
--- a/tajo-core/src/test/resources/results/TestTajoCli/testAlterTableAddDropPartition.result
+++ b/tajo-core/src/test/resources/results/TestTajoCli/testAlterTableAddDropPartition.result
@@ -1,5 +1,5 @@
 OK
-ERROR: 'key2' column is not the partition key
+ERROR: 'key2' column is not a partition key
 OK
 OK
 ERROR: partition 'key=0.1' does not exist

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/resources/results/TestTajoCliNegatives/testQueryFailure.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCliNegatives/testQueryFailure.result b/tajo-core/src/test/resources/results/TestTajoCliNegatives/testQueryFailure.result
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/resources/results/TestTajoCliNegatives/testQuerySyntax.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCliNegatives/testQuerySyntax.result b/tajo-core/src/test/resources/results/TestTajoCliNegatives/testQuerySyntax.result
new file mode 100644
index 0000000..86b0b59
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTajoCliNegatives/testQuerySyntax.result
@@ -0,0 +1,3 @@
+ERROR: mismatched input '-' expecting {<EOF>, EXCEPT, GROUP, HAVING, INTERSECT, LIMIT, ORDER, UNION, WHERE, WINDOW, ';', ','}
+LINE 1:21 select * from unknown-table
+                               ^

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
index bc6acbe..a44b526 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java
@@ -27,12 +27,12 @@ import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.FunctionDesc;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
+import org.apache.tajo.exception.UndefinedFunctionException;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.*;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.plan.algebra.BaseAlgebraVisitor;
 import org.apache.tajo.plan.expr.*;
@@ -53,6 +53,7 @@ import static org.apache.tajo.algebra.WindowSpec.WindowFrameStartBoundType;
 import static org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import static org.apache.tajo.common.TajoDataTypes.DataType;
 import static org.apache.tajo.common.TajoDataTypes.Type;
+import static org.apache.tajo.function.FunctionUtil.buildSimpleFunctionSignature;
 import static org.apache.tajo.plan.logical.WindowSpec.*;
 import static org.apache.tajo.plan.verifier.SyntaxErrorUtil.makeSyntaxError;
 
@@ -386,7 +387,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
 
   @Override
   public EvalNode visitExistsPredicate(Context ctx, Stack<Expr> stack, ExistsPredicate expr) throws TajoException {
-    throw new UnimplementedException("EXISTS clause");
+    throw new NotImplementedException("EXISTS clause");
   }
 
   ///////////////////////////////////////////////////////////////////////////////////////////////////////////
@@ -573,7 +574,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     stack.pop(); // <--- Pop
 
     if (!catalog.containFunction(expr.getSignature(), paramTypes)) {
-      throw new UndefinedFunctionException(expr.getSignature(), paramTypes);
+      throw new UndefinedFunctionException(buildSimpleFunctionSignature(expr.getSignature(), paramTypes));
     }
 
     FunctionDesc funcDesc = catalog.getFunction(expr.getSignature(), paramTypes);
@@ -623,7 +624,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     FunctionDesc countRows = catalog.getFunction("count", FunctionType.AGGREGATION,
         new DataType[] {});
     if (countRows == null) {
-      throw new UndefinedFunctionException(expr.getSignature(), new DataType[]{});
+      throw new UndefinedFunctionException(buildSimpleFunctionSignature(expr.getSignature(), new DataType[]{}));
     }
 
     ctx.currentBlock.setAggregationRequire();
@@ -648,7 +649,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     }
 
     if (!catalog.containFunction(setFunction.getSignature(), functionType, paramTypes)) {
-      throw new UndefinedFunctionException(setFunction.getSignature(), paramTypes);
+      throw new UndefinedFunctionException(buildSimpleFunctionSignature(setFunction.getSignature(), paramTypes));
     }
 
     FunctionDesc funcDesc = catalog.getFunction(setFunction.getSignature(), functionType, paramTypes);
@@ -737,7 +738,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor<ExprAnnotator.Context, Eva
     }
 
     if (!catalog.containFunction(windowFunc.getSignature(), functionType, paramTypes)) {
-      throw new UndefinedFunctionException(funcName, paramTypes);
+      throw new UndefinedFunctionException(buildSimpleFunctionSignature(funcName, paramTypes));
     }
 
     FunctionDesc funcDesc = catalog.getFunction(funcName, functionType, paramTypes);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/ExprNormalizer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprNormalizer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprNormalizer.java
index c89f10a..ff7eabd 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprNormalizer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprNormalizer.java
@@ -21,7 +21,7 @@ package org.apache.tajo.plan;
 import com.google.common.collect.Sets;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedColumnException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.plan.nameresolver.NameResolver;
 import org.apache.tajo.plan.nameresolver.NameResolvingMode;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanPreprocessor.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanPreprocessor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanPreprocessor.java
index f3743ee..76907f2 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanPreprocessor.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanPreprocessor.java
@@ -21,7 +21,7 @@ package org.apache.tajo.plan;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedColumnException;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.plan.LogicalPlan.QueryBlock;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
index b1d3e2e..9b114f1 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
@@ -35,8 +35,8 @@ import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.algebra.WindowSpec;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
+import org.apache.tajo.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
@@ -46,7 +46,7 @@ import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.exception.ExceptionUtil;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.plan.LogicalPlan.QueryBlock;
 import org.apache.tajo.plan.algebra.BaseAlgebraVisitor;
 import org.apache.tajo.plan.expr.*;
@@ -553,7 +553,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       throws TajoException {
     for (Target t : projectable.getTargets()) {
       if (t.getEvalTree().getValueType().getType() == TajoDataTypes.Type.RECORD) {
-        throw new UnimplementedException("record field projection");
+        throw new NotImplementedException("record field projection");
       }
     }
   }
@@ -2003,7 +2003,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       partitionMethodDesc = new PartitionMethodDesc(context.queryContext.get(SessionVars.CURRENT_DATABASE), tableName,
           CatalogProtos.PartitionType.COLUMN, partitionExpression, convertColumnsToSchema(partition.getColumns()));
     } else {
-      throw new UnimplementedException("partition type '" + expr.getPartitionType() + "'");
+      throw new NotImplementedException("partition type '" + expr.getPartitionType() + "'");
     }
     return partitionMethodDesc;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java b/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java
index a526efc..dfa8079 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/TypeDeterminant.java
@@ -25,11 +25,12 @@ import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.FunctionDesc;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
+import org.apache.tajo.exception.UndefinedFunctionException;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.function.FunctionUtil;
 import org.apache.tajo.plan.visitor.SimpleAlgebraVisitor;
 
 import java.util.Stack;
@@ -37,6 +38,7 @@ import java.util.Stack;
 import static org.apache.tajo.common.TajoDataTypes.DataType;
 import static org.apache.tajo.common.TajoDataTypes.Type.BOOLEAN;
 import static org.apache.tajo.common.TajoDataTypes.Type.NULL_TYPE;
+import static org.apache.tajo.function.FunctionUtil.buildSimpleFunctionSignature;
 
 public class TypeDeterminant extends SimpleAlgebraVisitor<LogicalPlanner.PlanContext, DataType> {
   private DataType BOOL_TYPE = CatalogUtil.newSimpleDataType(BOOLEAN);
@@ -171,7 +173,7 @@ public class TypeDeterminant extends SimpleAlgebraVisitor<LogicalPlanner.PlanCon
     stack.pop(); // <--- Pop
 
     if (!catalog.containFunction(expr.getSignature(), paramTypes)) {
-      throw new UndefinedFunctionException(expr.getSignature(), paramTypes);
+      throw new UndefinedFunctionException(FunctionUtil.buildSimpleFunctionSignature(expr.getSignature(), paramTypes));
     }
 
     FunctionDesc funcDesc = catalog.getFunction(expr.getSignature(), paramTypes);
@@ -207,7 +209,7 @@ public class TypeDeterminant extends SimpleAlgebraVisitor<LogicalPlanner.PlanCon
     stack.pop(); // <-- pop
 
     if (!catalog.containFunction(setFunction.getSignature(), functionType, paramTypes)) {
-      throw new UndefinedFunctionException(setFunction.getSignature(), paramTypes);
+      throw new UndefinedFunctionException(buildSimpleFunctionSignature(setFunction.getSignature(), paramTypes));
     }
 
     FunctionDesc funcDesc = catalog.getFunction(setFunction.getSignature(), functionType, paramTypes);
@@ -257,7 +259,7 @@ public class TypeDeterminant extends SimpleAlgebraVisitor<LogicalPlanner.PlanCon
     }
 
     if (!catalog.containFunction(windowFunc.getSignature(), functionType, paramTypes)) {
-      throw new UndefinedFunctionException(funcName, paramTypes);
+      throw new UndefinedFunctionException(FunctionUtil.buildSimpleFunctionSignature(funcName, paramTypes));
     }
 
     FunctionDesc funcDesc = catalog.getFunction(funcName, functionType, paramTypes);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java
index 1c29a6d..3ff0065 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AggregationFunctionCallEval.java
@@ -23,7 +23,6 @@ import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.plan.function.AggFunctionInvoke;
 import org.apache.tajo.plan.function.FunctionContext;
 import org.apache.tajo.plan.function.FunctionInvokeContext;
@@ -45,11 +44,7 @@ public class AggregationFunctionCallEval extends FunctionEval implements Cloneab
   protected AggregationFunctionCallEval(EvalType type, FunctionDesc desc, EvalNode[] givenArgs) {
     super(type, desc, givenArgs);
     this.invokeContext = new FunctionInvokeContext(null, getParamType());
-    try {
-      this.functionInvoke = AggFunctionInvoke.newInstance(funcDesc);
-    } catch (InternalException e) {
-      throw new RuntimeException(e);
-    }
+    this.functionInvoke = AggFunctionInvoke.newInstance(funcDesc);
   }
 
   public AggregationFunctionCallEval(FunctionDesc desc, EvalNode[] givenArgs) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java
index 7ff695b..51534e1 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java
@@ -29,10 +29,10 @@ import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.exception.InternalException;
-import org.apache.tajo.plan.util.ExprFinder;
+import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.Target;
+import org.apache.tajo.plan.util.ExprFinder;
 import org.apache.tajo.util.TUtil;
 
 import java.util.*;
@@ -144,8 +144,7 @@ public class EvalTreeUtil {
     return finder.getColumnRefs();
   }
   
-  public static Schema getSchemaByTargets(Schema inputSchema, Target[] targets)
-      throws InternalException {
+  public static Schema getSchemaByTargets(Schema inputSchema, Target[] targets) {
     Schema schema = new Schema();
     for (Target target : targets) {
       schema.addColumn(
@@ -167,8 +166,7 @@ public class EvalTreeUtil {
     return sb.toString();
   }
   
-  public static DataType getDomainByExpr(Schema inputSchema, EvalNode expr)
-      throws InternalException {
+  public static DataType getDomainByExpr(Schema inputSchema, EvalNode expr) {
     switch (expr.getType()) {
     case AND:      
     case OR:
@@ -192,8 +190,7 @@ public class EvalTreeUtil {
 
       
     default:
-      throw new InternalException("Unknown expr type: " 
-          + expr.getType().toString());
+      throw new TajoInternalError("Unknown expr type: " + expr.getType().toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/function/AggFunctionInvoke.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/AggFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/AggFunctionInvoke.java
index 2c2afbe..4930938 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/AggFunctionInvoke.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/AggFunctionInvoke.java
@@ -22,7 +22,6 @@ import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.storage.Tuple;
 
@@ -35,7 +34,7 @@ public abstract class AggFunctionInvoke implements Cloneable {
     this.functionDesc = functionDesc;
   }
 
-  public static AggFunctionInvoke newInstance(FunctionDesc desc) throws InternalException {
+  public static AggFunctionInvoke newInstance(FunctionDesc desc) {
     // TODO: The below line is due to the bug in the function type. The type of class-based functions is not set properly.
     if (desc.getInvocation().hasLegacy()) {
       return new ClassBasedAggFunctionInvoke(desc);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedAggFunctionInvoke.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedAggFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedAggFunctionInvoke.java
index 6657871..7caaef0 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedAggFunctionInvoke.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedAggFunctionInvoke.java
@@ -22,7 +22,6 @@ import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.storage.Tuple;
 
 import java.io.IOException;
@@ -33,7 +32,7 @@ import java.io.IOException;
 public class ClassBasedAggFunctionInvoke extends AggFunctionInvoke implements Cloneable {
   @Expose private AggFunction function;
 
-  public ClassBasedAggFunctionInvoke(FunctionDesc functionDesc) throws InternalException {
+  public ClassBasedAggFunctionInvoke(FunctionDesc functionDesc) {
     super(functionDesc);
     function = (AggFunction) functionDesc.newInstance();
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedScalarFunctionInvoke.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedScalarFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedScalarFunctionInvoke.java
index c3f4ad9..fcdb261 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedScalarFunctionInvoke.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/ClassBasedScalarFunctionInvoke.java
@@ -21,7 +21,6 @@ package org.apache.tajo.plan.function;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.TUtil;
 
@@ -35,13 +34,13 @@ public class ClassBasedScalarFunctionInvoke extends FunctionInvoke implements Cl
 
   }
 
-  public ClassBasedScalarFunctionInvoke(FunctionDesc funcDesc) throws InternalException {
+  public ClassBasedScalarFunctionInvoke(FunctionDesc funcDesc) {
     super(funcDesc);
     function = (GeneralFunction) funcDesc.newInstance();
   }
 
   @Override
-  public void setFunctionDesc(FunctionDesc desc) throws InternalException {
+  public void setFunctionDesc(FunctionDesc desc) {
     super.setFunctionDesc(desc);
     function = (GeneralFunction) functionDesc.newInstance();
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java
index b8b5cfe..b591b0a 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java
@@ -21,7 +21,6 @@ package org.apache.tajo.plan.function;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.storage.Tuple;
 
@@ -42,7 +41,7 @@ public abstract class FunctionInvoke implements Cloneable {
     this.functionDesc = functionDesc;
   }
 
-  public static FunctionInvoke newInstance(FunctionDesc desc) throws InternalException {
+  public static FunctionInvoke newInstance(FunctionDesc desc) {
     if (desc.getInvocation().hasLegacy()) {
       return new ClassBasedScalarFunctionInvoke(desc);
     } else if (desc.getInvocation().hasPython()) {
@@ -52,7 +51,7 @@ public abstract class FunctionInvoke implements Cloneable {
     }
   }
 
-  public void setFunctionDesc(FunctionDesc functionDesc) throws InternalException {
+  public void setFunctionDesc(FunctionDesc functionDesc) {
     this.functionDesc = functionDesc;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/NameResolver.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/NameResolver.java b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/NameResolver.java
index c346ce9..3eb51ba 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/NameResolver.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/NameResolver.java
@@ -26,9 +26,9 @@ import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.NestedPathUtil;
 import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.exception.AmbiguousTableException;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
+import org.apache.tajo.exception.AmbiguousTableException;
+import org.apache.tajo.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.exception.AmbiguousColumnException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.plan.LogicalPlan;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByLegacy.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByLegacy.java b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByLegacy.java
index 3bbb2be..8991b60 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByLegacy.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByLegacy.java
@@ -22,11 +22,10 @@ import org.apache.tajo.algebra.ColumnReferenceExpr;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedColumnException;
 import org.apache.tajo.exception.AmbiguousColumnException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.RelationNode;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRels.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRels.java b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRels.java
index aee131b..82fbc21 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRels.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRels.java
@@ -20,12 +20,11 @@ package org.apache.tajo.plan.nameresolver;
 
 import org.apache.tajo.algebra.ColumnReferenceExpr;
 import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.exception.AmbiguousTableException;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
+import org.apache.tajo.exception.AmbiguousTableException;
+import org.apache.tajo.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.exception.AmbiguousColumnException;
 import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.PlanningException;
 
 public class ResolverByRels extends NameResolver {
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRelsAndSubExprs.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRelsAndSubExprs.java b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRelsAndSubExprs.java
index 560ae50..4bfdaf6 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRelsAndSubExprs.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverByRelsAndSubExprs.java
@@ -20,12 +20,11 @@ package org.apache.tajo.plan.nameresolver;
 
 import org.apache.tajo.algebra.ColumnReferenceExpr;
 import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.exception.AmbiguousTableException;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
+import org.apache.tajo.exception.AmbiguousTableException;
+import org.apache.tajo.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.exception.AmbiguousColumnException;
 import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.PlanningException;
 
 public class ResolverByRelsAndSubExprs extends NameResolver {
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverBySubExprsAndRels.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverBySubExprsAndRels.java b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverBySubExprsAndRels.java
index 39458ec..ea291f4 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverBySubExprsAndRels.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/nameresolver/ResolverBySubExprsAndRels.java
@@ -20,12 +20,11 @@ package org.apache.tajo.plan.nameresolver;
 
 import org.apache.tajo.algebra.ColumnReferenceExpr;
 import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.exception.AmbiguousTableException;
-import org.apache.tajo.catalog.exception.UndefinedColumnException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
+import org.apache.tajo.exception.AmbiguousTableException;
+import org.apache.tajo.exception.UndefinedColumnException;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.exception.AmbiguousColumnException;
 import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.PlanningException;
 
 public class ResolverBySubExprsAndRels extends NameResolver {
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java
index 18e001e..a7cf85e 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/ProjectionPushDownRule.java
@@ -26,7 +26,7 @@ import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.catalog.SortSpec;
-import org.apache.tajo.catalog.exception.DuplicateColumnException;
+import org.apache.tajo.exception.DuplicateColumnException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.plan.LogicalPlan;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java
index 73a07d5..3a1d257 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java
@@ -27,21 +27,21 @@ import org.apache.tajo.algebra.WindowSpec.WindowFrameStartBoundType;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.SortSpec;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
+import org.apache.tajo.exception.UndefinedFunctionException;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionSignatureProto;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.*;
-import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.function.python.PythonScriptEngine;
 import org.apache.tajo.plan.logical.WindowSpec;
 import org.apache.tajo.plan.serder.PlanProto.WinFunctionEvalSpec;
 
-import java.io.IOException;
 import java.util.*;
 
+import static org.apache.tajo.function.FunctionUtil.buildSimpleFunctionSignature;
+
 /**
  * It deserializes a serialized eval tree consisting of a number of EvalNodes.
  *
@@ -238,7 +238,9 @@ public class EvalNodeDeserializer {
             parameterTypes = funcSignatureProto.getParameterTypesList().toArray(
                 new DataType[funcSignatureProto.getParameterTypesCount()]);
           }
-          throw new TajoInternalError(new UndefinedFunctionException(functionName, parameterTypes));
+          throw new TajoInternalError(
+              new UndefinedFunctionException(buildSimpleFunctionSignature(functionName, parameterTypes))
+          );
         }
       } else {
         throw new TajoInternalError("Unknown EvalType: " + type.name());

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
index d298cc8..dad9893 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
@@ -30,7 +30,7 @@ import org.apache.tajo.catalog.SortSpec;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.plan.Target;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.*;
@@ -596,7 +596,7 @@ public class LogicalNodeDeserializer {
       alterTablespace.setLocation(alterTablespaceProto.getSetLocation().getLocation());
       break;
     default:
-      throw new UnimplementedException("Unknown SET type in ALTER TABLE: " + alterTablespaceProto.getSetType().name());
+      throw new NotImplementedException("Unknown SET type in ALTER TABLE: " + alterTablespaceProto.getSetType().name());
     }
 
     return alterTablespace;
@@ -645,7 +645,7 @@ public class LogicalNodeDeserializer {
       alterTable.setIfExists(alterPartition.getIfExists());
       break;
     default:
-      throw new UnimplementedException("Unknown SET type in ALTER TABLE: " + alterTableProto.getSetType().name());
+      throw new NotImplementedException("Unknown SET type in ALTER TABLE: " + alterTableProto.getSetType().name());
     }
 
     return alterTable;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
index fe69fc1..ae74e30 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
@@ -23,10 +23,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.tajo.algebra.JoinType;
 import org.apache.tajo.catalog.SortSpec;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.SortSpecProto;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.Target;
 import org.apache.tajo.plan.logical.*;
@@ -567,7 +566,7 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe
       break;
 
     default:
-      throw new UnimplementedException("Unknown SET type in ALTER TABLESPACE: " + node.getSetType().name());
+      throw new NotImplementedException("Unknown SET type in ALTER TABLESPACE: " + node.getSetType().name());
     }
 
     PlanProto.LogicalNode.Builder nodeBuilder = createNodeBuilder(context, node);
@@ -632,7 +631,7 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe
       alterTableBuilder.setAlterPartition(partitionBuilder);
       break;
     default:
-      throw new UnimplementedException("Unknown SET type in ALTER TABLE: " + node.getAlterTableOpType().name());
+      throw new NotImplementedException("Unknown SET type in ALTER TABLE: " + node.getAlterTableOpType().name());
     }
 
     PlanProto.LogicalNode.Builder nodeBuilder = createNodeBuilder(context, node);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
index 445dc8a..e4bf8bc 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
@@ -27,6 +27,7 @@ import org.apache.tajo.catalog.*;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.plan.InvalidQueryException;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.Target;
@@ -923,7 +924,7 @@ public class PlannerUtil {
     }
   }
 
-  public static TableDesc getTableDesc(CatalogService catalog, LogicalNode node) throws IOException {
+  public static TableDesc getTableDesc(CatalogService catalog, LogicalNode node) throws UndefinedTableException {
     if (node.getType() == NodeType.ROOT) {
       node = ((LogicalRootNode)node).getChild();
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
index cfdb005..0ff6aba 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/PreLogicalPlanVerifier.java
@@ -26,14 +26,7 @@ import org.apache.tajo.algebra.*;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.exception.CatalogExceptionUtil;
-import org.apache.tajo.catalog.exception.DuplicateDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
-import org.apache.tajo.exception.ExceptionUtil;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.exception.TajoInternalError;
-import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.plan.algebra.BaseAlgebraVisitor;
 import org.apache.tajo.plan.util.ExprFinder;
 import org.apache.tajo.util.TUtil;
@@ -43,7 +36,6 @@ import java.util.Collection;
 import java.util.Set;
 import java.util.Stack;
 
-import static org.apache.tajo.catalog.exception.CatalogExceptionUtil.makeUndefinedTable;
 import static org.apache.tajo.plan.verifier.SyntaxErrorUtil.makeSyntaxError;
 
 public class PreLogicalPlanVerifier extends BaseAlgebraVisitor<PreLogicalPlanVerifier.Context, Expr> {
@@ -162,7 +154,7 @@ public class PreLogicalPlanVerifier extends BaseAlgebraVisitor<PreLogicalPlanVer
     }
 
     if (!catalog.existsTable(qualifiedName)) {
-      context.state.addVerification(makeUndefinedTable(qualifiedName));
+      context.state.addVerification(new UndefinedTableException(qualifiedName));
       return false;
     }
     return true;
@@ -180,7 +172,7 @@ public class PreLogicalPlanVerifier extends BaseAlgebraVisitor<PreLogicalPlanVer
       System.out.println("A");
     }
     if (catalog.existsTable(qualifiedName)) {
-      context.state.addVerification(CatalogExceptionUtil.makeDuplicateTable(qualifiedName));
+      context.state.addVerification(new DuplicateTableException(qualifiedName));
       return false;
     }
     return true;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/VerificationState.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/VerificationState.java b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/VerificationState.java
index fd16d11..8a6aea1 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/VerificationState.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/verifier/VerificationState.java
@@ -23,7 +23,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.exception.TajoError;
 import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.exception.TajoExceptionInterface;
 import org.apache.tajo.exception.TajoRuntimeException;
 import org.apache.tajo.util.TUtil;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/AbstractScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/AbstractScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/AbstractScanner.java
index 07fe353..b57782c 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/AbstractScanner.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/AbstractScanner.java
@@ -21,7 +21,7 @@ package org.apache.tajo.storage;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.plan.expr.EvalNode;
 
@@ -34,22 +34,22 @@ public abstract class AbstractScanner implements Scanner {
 
   @Override
   public void init() throws IOException {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
   public void reset() throws IOException {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
   public void close() throws IOException {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
   public boolean isProjectable() {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
@@ -59,7 +59,7 @@ public abstract class AbstractScanner implements Scanner {
 
   @Override
   public boolean isSelectable() {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
@@ -69,21 +69,21 @@ public abstract class AbstractScanner implements Scanner {
 
   @Override
   public boolean isSplittable() {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
   public float getProgress() {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
   public TableStats getInputStats() {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 
   @Override
   public Schema getSchema() {
-    throw new UnimplementedException();
+    throw new NotImplementedException();
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
index bfe4e55..42c015e 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
@@ -24,7 +24,7 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.IntervalDatum;
 import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.exception.UnknownDataTypeException;
+import org.apache.tajo.exception.UnsupportedDataTypeException;
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.exception.ValueTooLongForTypeCharactersException;
 import org.apache.tajo.tuple.offheap.RowWriter;
@@ -160,7 +160,7 @@ public class RowStoreUtil {
             // TODO - to be implemented
             throw new UnsupportedException(type.getType().name());
           default:
-            throw new RuntimeException(new UnknownDataTypeException(type.getType().name()));
+            throw new RuntimeException(new UnsupportedDataTypeException(type.getType().name()));
         }
       }
       return tuple;
@@ -261,7 +261,7 @@ public class RowStoreUtil {
           bb.put(tuple.getBytes(i));
           break;
         default:
-          throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
+          throw new RuntimeException(new UnsupportedDataTypeException(col.getDataType().getType().name()));
         }
       }
 
@@ -322,7 +322,7 @@ public class RowStoreUtil {
           size += tuple.getBytes(i).length;
           break;
         default:
-          throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
+          throw new RuntimeException(new UnsupportedDataTypeException(col.getDataType().getType().name()));
         }
       }
 


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

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDelegate.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDelegate.java b/tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDelegate.java
index 8dce7c4..80c0172 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDelegate.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/v2/ClientDelegate.java
@@ -18,7 +18,8 @@
 
 package org.apache.tajo.client.v2;
 
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
+import org.apache.tajo.exception.QueryFailedException;
+import org.apache.tajo.exception.UndefinedDatabaseException;
 import org.apache.tajo.exception.TajoException;
 
 import java.io.Closeable;
@@ -31,7 +32,7 @@ public interface ClientDelegate extends Closeable {
 
   int executeUpdate(String sql) throws TajoException;
 
-  ResultSet executeSQL(String sql) throws TajoException;
+  ResultSet executeSQL(String sql) throws TajoException, QueryFailedException;
 
   QueryFuture executeSQLAsync(String sql) throws TajoException;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/v2/LegacyClientDelegate.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/v2/LegacyClientDelegate.java b/tajo-client/src/main/java/org/apache/tajo/client/v2/LegacyClientDelegate.java
index a17311b..697c1ac 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/v2/LegacyClientDelegate.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/v2/LegacyClientDelegate.java
@@ -18,17 +18,17 @@
 
 package org.apache.tajo.client.v2;
 
-import com.google.common.base.Optional;
 import com.google.common.util.concurrent.AbstractFuture;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.annotation.ThreadSafe;
 import org.apache.tajo.auth.UserRoleInfo;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
-import org.apache.tajo.client.*;
+import org.apache.tajo.client.DummyServiceTracker;
+import org.apache.tajo.client.QueryClientImpl;
+import org.apache.tajo.client.SessionConnection;
+import org.apache.tajo.client.TajoClientUtil;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.GetQueryStatusResponse;
 import org.apache.tajo.service.ServiceTracker;
@@ -72,10 +72,20 @@ public class LegacyClientDelegate extends SessionConnection implements ClientDel
   }
 
   @Override
-  public ResultSet executeSQL(String sql) throws TajoException {
+  public ResultSet executeSQL(String sql) throws TajoException, QueryFailedException, QueryKilledException {
     try {
       return executeSQLAsync(sql).get();
-    } catch (InterruptedException | ExecutionException e) {
+
+    } catch (ExecutionException e) {
+
+      if (e.getCause() instanceof TajoException) {
+        throw (TajoException) e.getCause();
+      } else if (e.getCause() instanceof TajoRuntimeException) {
+        throw new TajoException((TajoRuntimeException)e.getCause());
+      } else {
+        throw new TajoInternalError(e);
+      }
+    } catch (InterruptedException e) {
       throw new RuntimeException(e);
     }
   }
@@ -83,7 +93,7 @@ public class LegacyClientDelegate extends SessionConnection implements ClientDel
   @Override
   public QueryFuture executeSQLAsync(String sql) throws TajoException {
     ClientProtos.SubmitQueryResponse response = queryClient.executeQuery(sql);
-    ClientExceptionUtil.throwIfError(response.getState());
+    ExceptionUtil.throwIfError(response.getState());
 
     QueryId queryId = new QueryId(response.getQueryId());
 
@@ -270,7 +280,11 @@ public class LegacyClientDelegate extends SessionConnection implements ClientDel
 
     @Override
     public boolean isKilled() {
-      return queryClient.getQueryStatus(queryId).getState() == TajoProtos.QueryState.QUERY_KILLED;
+      try {
+        return queryClient.getQueryStatus(queryId).getState() == TajoProtos.QueryState.QUERY_KILLED;
+      } catch (QueryNotFoundException e) {
+        throw new TajoInternalError(e);
+      }
     }
 
     @Override
@@ -295,7 +309,11 @@ public class LegacyClientDelegate extends SessionConnection implements ClientDel
 
     @Override
     public void kill() {
-      queryClient.killQuery(queryId).getState();
+      try {
+        queryClient.killQuery(queryId).getState();
+      } catch (QueryNotFoundException e) {
+        throw new TajoInternalError(e);
+      }
     }
 
     @Override
@@ -337,7 +355,8 @@ public class LegacyClientDelegate extends SessionConnection implements ClientDel
     }
 
     GetQueryStatusResponse waitCompletion() {
-      GetQueryStatusResponse response = queryClient.getRawQueryStatus(queryId);
+      GetQueryStatusResponse response;
+      response = queryClient.getRawQueryStatus(queryId);
       ensureOk(response.getState());
       updateState(response);
 
@@ -367,13 +386,26 @@ public class LegacyClientDelegate extends SessionConnection implements ClientDel
 
       if (finalResponse.getQueryState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
         if (finalResponse.hasHasResult()) {
-          set(queryClient.getQueryResult(queryId));
+          try {
+            set(queryClient.getQueryResult(queryId));
+          } catch (QueryNotFoundException e) {
+            setException(e);
+            return;
+          }
         } else { // when update
           set(TajoClientUtil.NULL_RESULT_SET);
         }
+
+      } else if (finalResponse.getQueryState() == TajoProtos.QueryState.QUERY_KILLED) {
+        setException(new QueryKilledException());
+
       } else {
-        cancel(false); // failed
-        set(TajoClientUtil.NULL_RESULT_SET);
+        if (finalResponse.hasErrorMessage()) {
+          setException(new QueryFailedException(finalResponse.getErrorMessage()));
+        } else {
+          setException(new QueryFailedException(
+              "internal error. See master and worker logs in ${tajo-install-dir}/logs for the cause of this error"));
+        }
       }
     }
   }
@@ -402,52 +434,52 @@ public class LegacyClientDelegate extends SessionConnection implements ClientDel
 
     @Override
     public InetSocketAddress getResourceTrackerAddress() throws ServiceTrackerException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public InetSocketAddress getCatalogAddress() throws ServiceTrackerException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public InetSocketAddress getMasterHttpInfo() throws ServiceTrackerException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public int getState(String masterName, TajoConf conf) throws ServiceTrackerException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public int formatHA(TajoConf conf) throws ServiceTrackerException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public List<String> getMasters(TajoConf conf) throws ServiceTrackerException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public void register() throws IOException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public void delete() throws IOException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public boolean isActiveMaster() {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
 
     @Override
     public List<TajoMasterInfo> getMasters() throws IOException {
-      throw new UnimplementedException();
+      throw new NotImplementedException();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/v2/TajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/v2/TajoClient.java b/tajo-client/src/main/java/org/apache/tajo/client/v2/TajoClient.java
index 08a921d..f9401cb 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/v2/TajoClient.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/v2/TajoClient.java
@@ -18,7 +18,8 @@
 
 package org.apache.tajo.client.v2;
 
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
+import org.apache.tajo.exception.QueryFailedException;
+import org.apache.tajo.exception.UndefinedDatabaseException;
 import org.apache.tajo.client.v2.exception.ClientUnableToConnectException;
 import org.apache.tajo.exception.TajoException;
 
@@ -114,7 +115,7 @@ public class TajoClient implements Closeable {
    * @return QueryHandler
    * @throws TajoException
    */
-  public ResultSet executeQuery(String sql) throws TajoException {
+  public ResultSet executeQuery(String sql) throws TajoException, QueryFailedException {
     return delegate.executeSQL(sql);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/client/v2/exception/ClientUnableToConnectException.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/v2/exception/ClientUnableToConnectException.java b/tajo-client/src/main/java/org/apache/tajo/client/v2/exception/ClientUnableToConnectException.java
index e567d7d..dfa9c01 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/v2/exception/ClientUnableToConnectException.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/v2/exception/ClientUnableToConnectException.java
@@ -22,7 +22,7 @@ import org.apache.tajo.error.Errors;
 import org.apache.tajo.exception.TajoException;
 
 public class ClientUnableToConnectException extends TajoException {
-  public ClientUnableToConnectException() {
+  public ClientUnableToConnectException(String address) {
     super(Errors.ResultCode.CLIENT_UNABLE_TO_ESTABLISH_CONNECTION);
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/jdbc/WaitingResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/WaitingResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/WaitingResultSet.java
index b9f8df5..68cc2fc 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/WaitingResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/WaitingResultSet.java
@@ -27,6 +27,8 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryClient;
 import org.apache.tajo.client.QueryStatus;
 import org.apache.tajo.client.TajoClientUtil;
+import org.apache.tajo.exception.SQLExceptionUtil;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.ipc.ClientProtos;
 
 import java.sql.SQLException;
@@ -66,6 +68,8 @@ public class WaitingResultSet extends FetchResultSet {
       return tableDesc.getLogicalSchema();
     } catch (ServiceException e) {
       throw new SQLException(e);
+    } catch (TajoException e) {
+      throw SQLExceptionUtil.toSQLException(e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-client/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/storage/RowStoreUtil.java b/tajo-client/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
index 87282a0..13b0201 100644
--- a/tajo-client/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
+++ b/tajo-client/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
@@ -23,8 +23,7 @@ import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.IntervalDatum;
-import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.exception.UnknownDataTypeException;
+import org.apache.tajo.exception.UnsupportedDataTypeException;
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.exception.ValueTooLongForTypeCharactersException;
 import org.apache.tajo.util.BitArray;
@@ -158,7 +157,7 @@ public class RowStoreUtil {
             // TODO - to be implemented
             throw new UnsupportedException(type.getType().name());
           default:
-            throw new RuntimeException(new UnknownDataTypeException(type.getType().name()));
+            throw new RuntimeException(new UnsupportedDataTypeException(type.getType().name()));
         }
       }
       return tuple;
@@ -259,7 +258,7 @@ public class RowStoreUtil {
             bb.put(tuple.getBytes(i));
             break;
           default:
-            throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
+            throw new RuntimeException(new UnsupportedDataTypeException(col.getDataType().getType().name()));
         }
       }
 
@@ -320,7 +319,7 @@ public class RowStoreUtil {
             size += tuple.getBytes(i).length;
             break;
           default:
-            throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
+            throw new RuntimeException(new UnsupportedDataTypeException(col.getDataType().getType().name()));
         }
       }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousFunctionException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousFunctionException.java b/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousFunctionException.java
new file mode 100644
index 0000000..7945963
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousFunctionException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class AmbiguousFunctionException extends TajoException {
+
+  public AmbiguousFunctionException(ReturnState state) {
+    super(state);
+  }
+
+  public AmbiguousFunctionException(String signature) {
+    super(Errors.ResultCode.AMBIGUOUS_FUNCTION, signature);
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousTableException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousTableException.java b/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousTableException.java
new file mode 100644
index 0000000..136a07f
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/AmbiguousTableException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class AmbiguousTableException extends TajoException {
+
+  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-common/src/main/java/org/apache/tajo/exception/CatalogUpgradeRequiredException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/CatalogUpgradeRequiredException.java b/tajo-common/src/main/java/org/apache/tajo/exception/CatalogUpgradeRequiredException.java
new file mode 100644
index 0000000..901e2dd
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/CatalogUpgradeRequiredException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class CatalogUpgradeRequiredException extends TajoException {
+
+  public CatalogUpgradeRequiredException() {
+    super(ResultCode.CAT_UPGRADE_REQUIRED);
+  }
+
+  public CatalogUpgradeRequiredException(ReturnState e) {
+    super(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/DataTypeMismatchException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DataTypeMismatchException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DataTypeMismatchException.java
new file mode 100644
index 0000000..3a41c7e
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DataTypeMismatchException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DataTypeMismatchException extends TajoException {
+
+  public DataTypeMismatchException(ReturnState e) {
+    super(e);
+  }
+
+  public DataTypeMismatchException(String columnName, String columnType,
+                                   String expression, String expressionDataType) {
+    super(ResultCode.DATATYPE_MISMATCH, columnName, columnType, expression, expressionDataType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/DefaultTajoException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DefaultTajoException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DefaultTajoException.java
new file mode 100644
index 0000000..8bce9e4
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DefaultTajoException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+
+/**
+ * Abstracted exception or error interface. TajoException and
+ * TajoRuntimeException always have a ResultCode and a message.
+ * This interface helps routines access both TajoException and
+ * TajoRuntimeException in a common way.
+ */
+public interface DefaultTajoException {
+
+  ResultCode getErrorCode();
+
+  String getMessage();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateColumnException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateColumnException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateColumnException.java
new file mode 100644
index 0000000..623a18b
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateColumnException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DuplicateColumnException extends TajoException {
+	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-common/src/main/java/org/apache/tajo/exception/DuplicateDatabaseException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateDatabaseException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateDatabaseException.java
new file mode 100644
index 0000000..2ead97d
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateDatabaseException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exception;
+
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DuplicateDatabaseException extends TajoException {
+
+  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-common/src/main/java/org/apache/tajo/exception/DuplicateFunctionException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateFunctionException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateFunctionException.java
new file mode 100644
index 0000000..cc86bbb
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateFunctionException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DuplicateFunctionException extends TajoException {
+  private static final long serialVersionUID = 3224521585413794703L;
+
+  public DuplicateFunctionException(ReturnState state) {
+    super(state);
+  }
+
+  public DuplicateFunctionException(String signature) {
+    super(Errors.ResultCode.DUPLICATE_FUNCTION, signature);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateIndexException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateIndexException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateIndexException.java
new file mode 100644
index 0000000..19239e4
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateIndexException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DuplicateIndexException extends TajoException {
+  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-common/src/main/java/org/apache/tajo/exception/DuplicatePartitionException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DuplicatePartitionException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicatePartitionException.java
new file mode 100644
index 0000000..b1ca2ee
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicatePartitionException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DuplicatePartitionException extends TajoException {
+  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-common/src/main/java/org/apache/tajo/exception/DuplicateTableException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateTableException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateTableException.java
new file mode 100644
index 0000000..873b927
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateTableException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.exception;
+
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DuplicateTableException extends TajoException {
+	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-common/src/main/java/org/apache/tajo/exception/DuplicateTablespaceException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateTablespaceException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateTablespaceException.java
new file mode 100644
index 0000000..e3ad86f
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DuplicateTablespaceException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exception;
+
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class DuplicateTablespaceException extends TajoException {
+
+  public DuplicateTablespaceException(ReturnState state) {
+    super(state);
+  }
+
+	public DuplicateTablespaceException(String spaceName) {
+		super(Errors.ResultCode.DUPLICATE_TABLESPACE, spaceName);
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/ErrorMessages.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/ErrorMessages.java b/tajo-common/src/main/java/org/apache/tajo/exception/ErrorMessages.java
index 3b646e1..ad5776c 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/ErrorMessages.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/ErrorMessages.java
@@ -42,7 +42,9 @@ public class ErrorMessages {
     ADD_MESSAGE(INVALID_RPC_CALL, "invalid RPC Call: %s", 1);
 
     // Query Management and Scheduler
-    ADD_MESSAGE(NO_SUCH_QUERYID, "query %s does not exist", 1);
+    ADD_MESSAGE(QUERY_FAILED, "query has been failed due to %s", 1);
+    ADD_MESSAGE(QUERY_KILLED, "query has been killed");
+    ADD_MESSAGE(QUERY_NOT_FOUND, "query %s does not exist", 1);
     ADD_MESSAGE(NO_DATA, "no data for %s due to query failure or error", 1);
     ADD_MESSAGE(INCOMPLETE_QUERY, "query %s is stilling running", 1);
 
@@ -52,9 +54,9 @@ public class ErrorMessages {
     ADD_MESSAGE(INVALID_SESSION_VARIABLE, "invalid session variable '%s': %s", 2);
 
 
-
+    // Syntax Error or Access Rule Violation
     ADD_MESSAGE(SYNTAX_ERROR, "%s", 1);
-    ADD_MESSAGE(INSUFFICIENT_PRIVILEGE, "Insufficient privilege to %s");
+    ADD_MESSAGE(INSUFFICIENT_PRIVILEGE, "Insufficient privilege to %s", 1);
     ADD_MESSAGE(INVALID_NAME, "Invalid name '%s'");
 
     ADD_MESSAGE(UNDEFINED_TABLESPACE, "tablespace '%s' does not exist", 1);
@@ -63,8 +65,9 @@ public class ErrorMessages {
     ADD_MESSAGE(UNDEFINED_TABLE, "relation '%s' does not exist", 1);
     ADD_MESSAGE(UNDEFINED_COLUMN, "column '%s' does not exist", 1);
     ADD_MESSAGE(UNDEFINED_FUNCTION, "function does not exist: %s", 1);
+    ADD_MESSAGE(UNDEFINED_PARTITION_METHOD, "table '%s' is not a partitioned table", 1);
     ADD_MESSAGE(UNDEFINED_PARTITION, "partition '%s' does not exist", 1);
-    ADD_MESSAGE(UNDEFINED_PARTITION_KEY, "'%s' column is not the partition key", 1);
+    ADD_MESSAGE(UNDEFINED_PARTITION_KEY, "'%s' column is not a partition key", 1);
     ADD_MESSAGE(UNDEFINED_OPERATOR, "operator does not exist: '%s'", 1);
     ADD_MESSAGE(UNDEFINED_INDEX_FOR_TABLE, "index ''%s' does not exist", 1);
     ADD_MESSAGE(UNDEFINED_INDEX_FOR_COLUMNS, "index does not exist for '%s' columns of '%s' table", 2);
@@ -81,11 +84,12 @@ public class ErrorMessages {
 
     ADD_MESSAGE(AMBIGUOUS_TABLE, "table name '%s' is ambiguous", 1);
     ADD_MESSAGE(AMBIGUOUS_COLUMN, "column name '%s' is ambiguous", 1);
+    ADD_MESSAGE(AMBIGUOUS_FUNCTION, "function '%s' is ambiguous", 1);
 
     ADD_MESSAGE(DIVISION_BY_ZERO, "Division by zero: %s", 1);
 
     ADD_MESSAGE(DATATYPE_MISMATCH,
-        "column \"%s\" is of type %s but expression %s is of type %s", 4);
+        "column '%s' is of type %s but expression %s is of type %s", 4);
 
     ADD_MESSAGE(SET_OPERATION_SCHEMA_MISMATCH, "each %s query must have the same number of columns", 1);
     ADD_MESSAGE(SET_OPERATION_DATATYPE_MISMATCH, "%s types %s and %s cannot be matched");
@@ -93,9 +97,14 @@ public class ErrorMessages {
     ADD_MESSAGE(CAT_UPGRADE_REQUIRED, "catalog must be upgraded");
     ADD_MESSAGE(CAT_CANNOT_CONNECT, "cannot connect metadata store '%s': %s", 2);
 
-    ADD_MESSAGE(MDC_NO_MATCHED_DATATYPE, "no matched type for %s", 1);
+    ADD_MESSAGE(LMD_NO_MATCHED_DATATYPE, "no matched type for %s", 1);
 
-    ADD_MESSAGE(UNKNOWN_DATAFORMAT, "Unknown data format: '%s'", 1);
+    // Storage and Data Format
+    ADD_MESSAGE(UNAVAILABLE_TABLE_LOCATION, "unavailable table location '%s': %s", 2);
+    ADD_MESSAGE(UNKNOWN_DATAFORMAT, "unknown data format: '%s'", 1);
+    ADD_MESSAGE(UNSUPPORTED_DATATYPE, "unsupported data type: '%s'", 1);
+    ADD_MESSAGE(INVALID_TABLE_PROPERTY, "invalid table property '%s': '%s'", 2);
+    ADD_MESSAGE(MISSING_TABLE_PROPERTY, "table property '%s' required for '%s'", 2);
 
     ADD_MESSAGE(AMBIGUOUS_PARTITION_DIRECTORY, "There is a directory which is assumed to be a partitioned directory" +
       " : '%s'", 1);
@@ -119,7 +128,7 @@ public class ErrorMessages {
 
   public static String getInternalErrorMessage(Throwable t) {
     if (t.getMessage() != null) {
-      return MESSAGES.get(INTERNAL_ERROR).getFirst() + ": " + t.getMessage();
+      return String.format(MESSAGES.get(INTERNAL_ERROR).getFirst(), t.getMessage());
     } else {
       return getInternalErrorMessage();
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/ExceptionUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/ExceptionUtil.java b/tajo-common/src/main/java/org/apache/tajo/exception/ExceptionUtil.java
index bc01cb9..71dcfc4 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/ExceptionUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/ExceptionUtil.java
@@ -18,12 +18,143 @@
 
 package org.apache.tajo.exception;
 
+import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+import java.lang.reflect.Constructor;
+import java.util.Map;
+
+import static org.apache.tajo.error.Errors.ResultCode.*;
+import static org.apache.tajo.exception.ReturnStateUtil.isError;
 
 public class ExceptionUtil {
 
+  static Map<Errors.ResultCode, Class<? extends DefaultTajoException>> EXCEPTIONS = Maps.newHashMap();
+
+  static {
+
+    // General Errors
+    ADD_EXCEPTION(INTERNAL_ERROR, TajoInternalError.class);
+    ADD_EXCEPTION(FEATURE_NOT_SUPPORTED, UnsupportedException.class);
+    ADD_EXCEPTION(NOT_IMPLEMENTED, NotImplementedException.class);
+
+    // Query Management and Scheduler
+    ADD_EXCEPTION(QUERY_NOT_FOUND, QueryNotFoundException.class);
+
+    // Syntax Error or Access Rule Violation
+    ADD_EXCEPTION(SYNTAX_ERROR_OR_ACCESS_RULE_VIOLATION, SQLSyntaxError.class);
+    ADD_EXCEPTION(SYNTAX_ERROR, SQLSyntaxError.class);
+    ADD_EXCEPTION(INSUFFICIENT_PRIVILEGE, InsufficientPrivilegeException.class);
+
+    ADD_EXCEPTION(UNDEFINED_TABLESPACE, UndefinedTablespaceException.class);
+    ADD_EXCEPTION(UNDEFINED_DATABASE, UndefinedDatabaseException.class);
+    // ADD_EXCEPTION(UNDEFINED_SCHEMA, );
+    ADD_EXCEPTION(UNDEFINED_TABLE, UndefinedTableException.class);
+    ADD_EXCEPTION(UNDEFINED_COLUMN, UndefinedColumnException.class);
+    ADD_EXCEPTION(UNDEFINED_FUNCTION, UndefinedFunctionException.class);
+    ADD_EXCEPTION(UNDEFINED_PARTITION, UndefinedPartitionException.class);
+    ADD_EXCEPTION(UNDEFINED_PARTITION_KEY, UndefinedPartitionKeyException.class);
+    ADD_EXCEPTION(UNDEFINED_OPERATOR, UndefinedOperatorException.class);
+
+    ADD_EXCEPTION(DUPLICATE_TABLESPACE, DuplicateTableException.class);
+    ADD_EXCEPTION(DUPLICATE_DATABASE, DuplicateDatabaseException.class);
+    // ADD_EXCEPTION(DUPLICATE_SCHEMA, );
+    ADD_EXCEPTION(DUPLICATE_TABLE, DuplicateTableException.class);
+    ADD_EXCEPTION(DUPLICATE_COLUMN, DuplicateColumnException.class);
+    // ADD_EXCEPTION(DUPLICATE_ALIAS, );
+    ADD_EXCEPTION(DUPLICATE_INDEX, DuplicateIndexException.class);
+    ADD_EXCEPTION(DUPLICATE_PARTITION, DuplicatePartitionException.class);
+
+    ADD_EXCEPTION(AMBIGUOUS_TABLE, AmbiguousTableException.class);
+    ADD_EXCEPTION(AMBIGUOUS_COLUMN, AmbiguousColumnException.class);
+    ADD_EXCEPTION(AMBIGUOUS_FUNCTION, AmbiguousFunctionException.class);
+
+    ADD_EXCEPTION(DATATYPE_MISMATCH, DataTypeMismatchException.class);
+
+    ADD_EXCEPTION(UNAVAILABLE_TABLE_LOCATION, UnavailableTableLocationException.class);
+    ADD_EXCEPTION(UNKNOWN_DATAFORMAT, UnknownDataFormatException.class);
+    ADD_EXCEPTION(UNSUPPORTED_DATATYPE, UnsupportedDataTypeException.class);
+    ADD_EXCEPTION(INVALID_TABLE_PROPERTY, InvalidTablePropertyException.class);
+    ADD_EXCEPTION(MISSING_TABLE_PROPERTY, MissingTablePropertyException.class);
+  }
+
+  private static void ADD_EXCEPTION(Errors.ResultCode code, Class<? extends DefaultTajoException> cls) {
+    EXCEPTIONS.put(code, cls);
+  }
+
+  /**
+   * If the exception is equivalent to the error corresponding to the expected exception, throws the exception.
+   * It is used to throw an exception for a error.
+   *
+   * @param state ReturnState
+   * @param clazz Exception class corresponding to the expected
+   * @param <T> Exception class
+   * @throws T Exception
+   */
+  public static <T extends TajoException> void throwsIfThisError(ReturnState state, Class<T> clazz) throws T {
+    if (isError(state)) {
+      T exception = (T) toTajoException(state);
+      if (exception.getClass().equals(clazz)) {
+        throw exception;
+      }
+    }
+  }
+
+  /**
+   * It can throw any TajoException if any error occurs.
+   *
+   * @param state
+   * @throws TajoException
+   */
+  public static void throwIfError(ReturnState state) throws TajoException {
+    if (isError(state)) {
+      throw toTajoException(state);
+    }
+  }
+
+  public static DefaultTajoException toTajoExceptionCommon(ReturnState state) {
+    if (state.getReturnCode() == Errors.ResultCode.INTERNAL_ERROR) {
+      return new TajoInternalError(state);
+
+    } else if (EXCEPTIONS.containsKey(state.getReturnCode())) {
+      Object exception;
+      try {
+        Class clazz = EXCEPTIONS.get(state.getReturnCode());
+        Constructor c = clazz.getConstructor(ReturnState.class);
+        exception = c.newInstance(new Object[]{state});
+      } catch (Throwable t) {
+        throw new TajoInternalError(t);
+      }
+
+      if (exception instanceof TajoException) {
+        return (TajoException) exception;
+      } else if (exception instanceof TajoRuntimeException) {
+        return ((TajoRuntimeException) exception);
+      } else {
+        return ((TajoError) exception);
+      }
+
+    } else {
+      throw new TajoInternalError("Unknown exception: [" + state.getReturnCode().name() +"] " + state.getMessage());
+    }
+  }
+
+  public static TajoException toTajoException(ReturnState state) throws TajoRuntimeException, TajoError {
+    DefaultTajoException e = toTajoExceptionCommon(state);
+
+    if (e instanceof TajoException) {
+      return (TajoException) e;
+    } else if (e instanceof TajoRuntimeException) {
+      throw ((TajoRuntimeException) e);
+    } else {
+      throw ((TajoError) e);
+    }
+  }
+
   /**
    * Determine if a Throwable has Tajo's ReturnCode and error message.
    *
@@ -31,7 +162,7 @@ public class ExceptionUtil {
    * @return true if a Throwable has Tajo's ReturnCode and error message.
    */
   public static boolean isExceptionWithResultCode(Throwable t) {
-    return t instanceof TajoExceptionInterface;
+    return t instanceof DefaultTajoException;
   }
 
   /**

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/InternalException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InternalException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InternalException.java
deleted file mode 100644
index af4ee5c..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/exception/InternalException.java
+++ /dev/null
@@ -1,43 +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.exception;
-
-import java.io.IOException;
-
-public class InternalException extends IOException {
-	private static final long serialVersionUID = -262149616685882358L;
-
-	public InternalException() {
-	}
-	
-	public InternalException(String message) {
-		super(message);
-	}
-	
-	public InternalException(String message, Exception t) {
-	  super(message, t);
-	}
-	
-	public InternalException(Throwable t) {
-	  super(t);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/InvalidDataTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidDataTypeException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidDataTypeException.java
index 9f470f2..fd2f0de 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidDataTypeException.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidDataTypeException.java
@@ -18,12 +18,17 @@
 
 package org.apache.tajo.exception;
 
-import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 
 public class InvalidDataTypeException extends TajoRuntimeException {
 
+  public InvalidDataTypeException(ReturnState state) {
+    super(state);
+  }
+
   public InvalidDataTypeException(DataType dataType) {
     super(Errors.ResultCode.INVALID_DATATYPE, dataType.getType().name());
   }

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/InvalidOperationException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidOperationException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidOperationException.java
index f5de632..5e48113 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidOperationException.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidOperationException.java
@@ -23,9 +23,6 @@ import static org.apache.tajo.common.TajoDataTypes.Type;
 public class InvalidOperationException extends RuntimeException {
 	private static final long serialVersionUID = -7689027447969916148L;
 
-	/**
-	 * 
-	 */
 	public InvalidOperationException() {
 	}
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/InvalidTablePropertyException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidTablePropertyException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidTablePropertyException.java
new file mode 100644
index 0000000..1d0615f
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidTablePropertyException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class InvalidTablePropertyException extends TajoException {
+
+  public InvalidTablePropertyException(ReturnState e) {
+    super(e);
+  }
+
+  public InvalidTablePropertyException(String propertyName, String reason) {
+    super(ResultCode.INVALID_TABLE_PROPERTY, propertyName, reason);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/LMDNoMatchedDatatypeException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/LMDNoMatchedDatatypeException.java b/tajo-common/src/main/java/org/apache/tajo/exception/LMDNoMatchedDatatypeException.java
new file mode 100644
index 0000000..9b18740
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/LMDNoMatchedDatatypeException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class LMDNoMatchedDatatypeException extends TajoException {
+
+  @SuppressWarnings("unused")
+  public LMDNoMatchedDatatypeException(ReturnState e) {
+    super(e);
+  }
+
+  public LMDNoMatchedDatatypeException(String dataType) {
+    super(Errors.ResultCode.LMD_NO_MATCHED_DATATYPE, dataType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/MetadataConnectionException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/MetadataConnectionException.java b/tajo-common/src/main/java/org/apache/tajo/exception/MetadataConnectionException.java
new file mode 100644
index 0000000..6a1c6c8
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/MetadataConnectionException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+
+/**
+ * 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());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/MissingTablePropertyException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/MissingTablePropertyException.java b/tajo-common/src/main/java/org/apache/tajo/exception/MissingTablePropertyException.java
new file mode 100644
index 0000000..9c2a868
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/MissingTablePropertyException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class MissingTablePropertyException extends TajoException {
+
+  public MissingTablePropertyException(ReturnState e) {
+    super(e);
+  }
+
+  public MissingTablePropertyException(String propertyName, String table) {
+    super(ResultCode.MISSING_TABLE_PROPERTY, propertyName, table);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/NotImplementedException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/NotImplementedException.java b/tajo-common/src/main/java/org/apache/tajo/exception/NotImplementedException.java
new file mode 100644
index 0000000..822eea1
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/NotImplementedException.java
@@ -0,0 +1,39 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class NotImplementedException extends TajoRuntimeException {
+  private static final long serialVersionUID = -5467580471721530536L;
+
+  public NotImplementedException() {
+    super(Errors.ResultCode.NOT_IMPLEMENTED,
+        Thread.currentThread().getStackTrace()[1].getClassName());
+  }
+
+  public NotImplementedException(ReturnState state) {
+    super(state);
+  }
+
+  public NotImplementedException(String featureName) {
+    super(Errors.ResultCode.NOT_IMPLEMENTED, featureName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/QueryFailedException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/QueryFailedException.java b/tajo-common/src/main/java/org/apache/tajo/exception/QueryFailedException.java
new file mode 100644
index 0000000..97b184e
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/QueryFailedException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class QueryFailedException extends TajoException {
+
+  @SuppressWarnings("unused")
+  public QueryFailedException(ReturnState e) {
+    super(e);
+  }
+
+  public QueryFailedException(String cause) {
+    super(Errors.ResultCode.QUERY_FAILED, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/QueryKilledException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/QueryKilledException.java b/tajo-common/src/main/java/org/apache/tajo/exception/QueryKilledException.java
new file mode 100644
index 0000000..ef11bce
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/QueryKilledException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class QueryKilledException extends TajoException {
+
+  @SuppressWarnings("unused")
+  public QueryKilledException(ReturnState e) {
+    super(e);
+  }
+
+  public QueryKilledException() {
+    super(Errors.ResultCode.QUERY_KILLED);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/QueryNotFoundException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/QueryNotFoundException.java b/tajo-common/src/main/java/org/apache/tajo/exception/QueryNotFoundException.java
new file mode 100644
index 0000000..0ab9737
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/QueryNotFoundException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class QueryNotFoundException extends TajoException {
+
+  public QueryNotFoundException(ReturnState state) {
+    super(state);
+  }
+
+  public QueryNotFoundException(String queryId) {
+    super(Errors.ResultCode.QUERY_NOT_FOUND, queryId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/ReturnStateUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/ReturnStateUtil.java b/tajo-common/src/main/java/org/apache/tajo/exception/ReturnStateUtil.java
index 81554c4..169c166 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/ReturnStateUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/ReturnStateUtil.java
@@ -20,14 +20,7 @@ package org.apache.tajo.exception;
 
 import com.google.common.base.Preconditions;
 import org.apache.tajo.QueryId;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.ErrorMessages;
-import org.apache.tajo.exception.ErrorUtil;
-import org.apache.tajo.exception.ExceptionUtil;
-import org.apache.tajo.exception.TajoExceptionInterface;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListResponse;
 import org.apache.tajo.util.StringUtils;
@@ -45,6 +38,12 @@ public class ReturnStateUtil {
     OK = builder.build();
   }
 
+  /**
+   * Throw a TajoRuntimeException. It is usually used for unexpected exceptions.
+   *
+   * @param state ReturnState
+   * @return True if no error.
+   */
   public static boolean ensureOk(ReturnState state) {
     if (isError(state)) {
       throw new TajoRuntimeException(state);
@@ -65,13 +64,6 @@ public class ReturnStateUtil {
         .build();
   }
 
-  public static ReturnState returnError(ResultCode code) {
-    ReturnState.Builder builder = ReturnState.newBuilder();
-    builder.setReturnCode(code);
-    builder.setMessage(ErrorMessages.getMessage(code));
-    return builder.build();
-  }
-
   public static ReturnState returnError(ResultCode code, String...args) {
     Preconditions.checkNotNull(args);
 
@@ -85,7 +77,7 @@ public class ReturnStateUtil {
     ReturnState.Builder builder = ReturnState.newBuilder();
 
     if (ExceptionUtil.isExceptionWithResultCode(t)) {
-      TajoExceptionInterface tajoException = (TajoExceptionInterface) t;
+      DefaultTajoException tajoException = (DefaultTajoException) t;
       builder.setReturnCode(tajoException.getErrorCode());
       builder.setMessage(tajoException.getMessage());
     } else {
@@ -130,7 +122,7 @@ public class ReturnStateUtil {
   }
 
   public static ReturnState errNoSuchQueryId(QueryId queryId) {
-    return returnError(ResultCode.NO_SUCH_QUERYID, queryId.toString());
+    return returnError(ResultCode.QUERY_NOT_FOUND, queryId.toString());
   }
 
   public static ReturnState errNoData(QueryId queryId) {
@@ -146,7 +138,7 @@ public class ReturnStateUtil {
   }
 
   public static ReturnState errNoSessionVar(String varName) {
-    return returnError(ResultCode.NO_SUCH_QUERYID, varName);
+    return returnError(ResultCode.QUERY_NOT_FOUND, varName);
   }
 
   public static ReturnState errInsufficientPrivilege(String message) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/SQLExceptionUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/SQLExceptionUtil.java b/tajo-common/src/main/java/org/apache/tajo/exception/SQLExceptionUtil.java
index deeb7f9..e326ae2 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/SQLExceptionUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/SQLExceptionUtil.java
@@ -19,9 +19,7 @@
 package org.apache.tajo.exception;
 
 import com.google.common.collect.Maps;
-import org.apache.log4j.spi.ErrorCode;
 import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.exception.ErrorMessages;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 
 import java.sql.SQLException;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/SQLSyntaxError.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/SQLSyntaxError.java b/tajo-common/src/main/java/org/apache/tajo/exception/SQLSyntaxError.java
new file mode 100644
index 0000000..512ba36
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/SQLSyntaxError.java
@@ -0,0 +1,35 @@
+/**
+ * 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.exception;
+
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class SQLSyntaxError extends TajoRuntimeException {
+  private static final long serialVersionUID = 5388279335175632067L;
+
+  public SQLSyntaxError(ReturnState state) {
+    super(state);
+  }
+
+  public SQLSyntaxError(String errorMessage) {
+    super(Errors.ResultCode.SYNTAX_ERROR, errorMessage);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/TajoError.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/TajoError.java b/tajo-common/src/main/java/org/apache/tajo/exception/TajoError.java
index 765ead3..8618633 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/TajoError.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/TajoError.java
@@ -19,13 +19,12 @@
 package org.apache.tajo.exception;
 
 import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 
 /**
  * Unrecoverable errors
  */
-public class TajoError extends Error implements TajoExceptionInterface {
+public class TajoError extends Error implements DefaultTajoException {
   private ResultCode code;
 
   public TajoError(ReturnState state) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java b/tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java
index e0e2ccb..b5d236e 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/TajoException.java
@@ -25,7 +25,7 @@ import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
  * TajoException contains all exceptions with any exact reason.
  * It always have an exact error code and an error message.
  */
-public class TajoException extends Exception implements TajoExceptionInterface {
+public class TajoException extends Exception implements DefaultTajoException {
   private ResultCode code;
 
   public TajoException(ReturnState e) {
@@ -33,7 +33,7 @@ public class TajoException extends Exception implements TajoExceptionInterface {
     this.code = e.getReturnCode();
   }
 
-  public TajoException(TajoRuntimeException e) {
+  public TajoException(DefaultTajoException e) {
     super(e.getMessage());
     this.code = e.getErrorCode();
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/TajoExceptionInterface.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/TajoExceptionInterface.java b/tajo-common/src/main/java/org/apache/tajo/exception/TajoExceptionInterface.java
deleted file mode 100644
index e0444a0..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/exception/TajoExceptionInterface.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.exception;
-
-import org.apache.tajo.error.Errors.ResultCode;
-
-/**
- * Abstracted exception or error interface. TajoException and
- * TajoRuntimeException always have a ResultCode and a message.
- * This interface helps routines access both TajoException and
- * TajoRuntimeException in a common way.
- */
-public interface TajoExceptionInterface {
-  ResultCode getErrorCode();
-
-  String getMessage();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/TajoInternalError.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/TajoInternalError.java b/tajo-common/src/main/java/org/apache/tajo/exception/TajoInternalError.java
index 072636b..4decd21 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/TajoInternalError.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/TajoInternalError.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.exception;
 
 import org.apache.tajo.error.Errors.ResultCode;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 
 /**

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/TajoRuntimeException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/TajoRuntimeException.java b/tajo-common/src/main/java/org/apache/tajo/exception/TajoRuntimeException.java
index 69da2a8..071efa0 100644
--- a/tajo-common/src/main/java/org/apache/tajo/exception/TajoRuntimeException.java
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/TajoRuntimeException.java
@@ -26,7 +26,7 @@ import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
  *
  * @see @{link TajoException}
  */
-public class TajoRuntimeException extends RuntimeException implements TajoExceptionInterface {
+public class TajoRuntimeException extends RuntimeException implements DefaultTajoException {
   private ResultCode code;
 
   public TajoRuntimeException(ReturnState state) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UnavailableTableLocationException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UnavailableTableLocationException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UnavailableTableLocationException.java
new file mode 100644
index 0000000..b0d19e6
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UnavailableTableLocationException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UnavailableTableLocationException extends TajoException {
+
+  public UnavailableTableLocationException(ReturnState e) {
+    super(e);
+  }
+
+  public UnavailableTableLocationException(String uri, String reason) {
+    super(Errors.ResultCode.UNAVAILABLE_TABLE_LOCATION, uri, reason);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedColumnException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedColumnException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedColumnException.java
new file mode 100644
index 0000000..954107b
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedColumnException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exception;
+
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedColumnException extends TajoException {
+	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-common/src/main/java/org/apache/tajo/exception/UndefinedDatabaseException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedDatabaseException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedDatabaseException.java
new file mode 100644
index 0000000..bf04df8
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedDatabaseException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exception;
+
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedDatabaseException extends TajoException {
+	private static final long serialVersionUID = 277182608283894937L;
+
+  public UndefinedDatabaseException(ReturnState state) {
+    super(state);
+  }
+
+	public UndefinedDatabaseException(String dbName) {
+		super(Errors.ResultCode.UNDEFINED_DATABASE, dbName);
+	}
+}


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

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedFunctionException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedFunctionException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedFunctionException.java
new file mode 100644
index 0000000..0b22970
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedFunctionException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedFunctionException extends TajoException {
+	private static final long serialVersionUID = 5062193018697228028L;
+
+  public UndefinedFunctionException(ReturnState state) {
+    super(state);
+  }
+
+  public UndefinedFunctionException(String signature) {
+    super(ResultCode.UNDEFINED_FUNCTION, signature);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedIndexException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedIndexException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedIndexException.java
new file mode 100644
index 0000000..3f4db74
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedIndexException.java
@@ -0,0 +1,38 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedIndexException extends TajoException {
+  private static final long serialVersionUID = 3705839985189534673L;
+
+  public UndefinedIndexException(ReturnState state) {
+    super(state);
+  }
+
+  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-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionException.java
new file mode 100644
index 0000000..c0ec7d5
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedPartitionException extends TajoException {
+
+  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-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionKeyException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionKeyException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionKeyException.java
new file mode 100644
index 0000000..19219ff
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedPartitionKeyException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedPartitionKeyException extends TajoException {
+
+  @SuppressWarnings("unused")
+  public UndefinedPartitionKeyException(ReturnState state) {
+    super(state);
+  }
+
+  public UndefinedPartitionKeyException(String partitionKey) {
+    super(ResultCode.UNDEFINED_PARTITION_KEY, partitionKey);
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTableException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTableException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTableException.java
new file mode 100644
index 0000000..6060142
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTableException.java
@@ -0,0 +1,39 @@
+/**
+ * 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.exception;
+
+
+import org.apache.tajo.error.Errors.ResultCode;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedTableException extends TajoException {
+	private static final long serialVersionUID = 277182608283894937L;
+
+  public UndefinedTableException(ReturnState state) {
+    super(state);
+  }
+
+  public UndefinedTableException(String dbName, String tbName) {
+		super(ResultCode.UNDEFINED_TABLE, dbName + "." + tbName);
+  }
+
+	public UndefinedTableException(String relName) {
+		super(ResultCode.UNDEFINED_TABLE, relName);
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTablespaceException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTablespaceException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTablespaceException.java
new file mode 100644
index 0000000..eef1648
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UndefinedTablespaceException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UndefinedTablespaceException extends TajoException {
+	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-common/src/main/java/org/apache/tajo/exception/UnimplementedException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UnimplementedException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UnimplementedException.java
deleted file mode 100644
index b32079f..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/exception/UnimplementedException.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.exception;
-
-import org.apache.tajo.error.Errors;
-
-public class UnimplementedException extends TajoRuntimeException {
-  private static final long serialVersionUID = -5467580471721530536L;
-
-  public UnimplementedException() {
-    super(Errors.ResultCode.NOT_IMPLEMENTED,
-        Thread.currentThread().getStackTrace()[1].getClassName());
-  }
-
-  public UnimplementedException(String featureName) {
-    super(Errors.ResultCode.NOT_IMPLEMENTED, featureName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataFormatException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataFormatException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataFormatException.java
new file mode 100644
index 0000000..74ee125
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataFormatException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UnknownDataFormatException extends TajoException {
+
+  private static final long serialVersionUID = -2630390595968966164L;
+
+  public UnknownDataFormatException(ReturnState state) {
+    super(state);
+
+  }
+
+  public UnknownDataFormatException(String dataFormat) {
+    super(Errors.ResultCode.UNKNOWN_DATAFORMAT, dataFormat);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataTypeException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataTypeException.java
deleted file mode 100644
index abd9ab5..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/exception/UnknownDataTypeException.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.exception;
-
-public class UnknownDataTypeException extends Exception {
-
-  private static final long serialVersionUID = -2630390595968966164L;
-
-  public UnknownDataTypeException() {
-
-  }
-
-  public UnknownDataTypeException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/exception/UnsupportedDataTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/UnsupportedDataTypeException.java b/tajo-common/src/main/java/org/apache/tajo/exception/UnsupportedDataTypeException.java
new file mode 100644
index 0000000..65ec777
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/UnsupportedDataTypeException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.exception;
+
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class UnsupportedDataTypeException extends TajoException {
+
+  private static final long serialVersionUID = -2630390595968966164L;
+
+  public UnsupportedDataTypeException(ReturnState state) {
+    super(state);
+  }
+
+  public UnsupportedDataTypeException(String dataType) {
+    super(Errors.ResultCode.UNKNOWN_DATAFORMAT, dataType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
index 12570d7..90bb7fa 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/VTuple.java
@@ -24,7 +24,7 @@ import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.Inet4Datum;
 import org.apache.tajo.datum.IntervalDatum;
 import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.NotImplementedException;
 import org.apache.tajo.util.datetime.TimeMeta;
 
 import java.net.InetAddress;
@@ -185,11 +185,11 @@ public class VTuple implements Tuple, Cloneable {
   }
 
   public InetAddress getIPv6(int fieldId) {
-    throw new UnimplementedException("IPv6 is unsupported yet");
+    throw new NotImplementedException("IPv6 is unsupported yet");
   }
 
   public byte[] getIPv6Bytes(int fieldId) {
-    throw new UnimplementedException("IPv6 is unsupported yet");
+    throw new NotImplementedException("IPv6 is unsupported yet");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-common/src/main/proto/errors.proto
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/proto/errors.proto b/tajo-common/src/main/proto/errors.proto
index 0bc0069..4057456 100644
--- a/tajo-common/src/main/proto/errors.proto
+++ b/tajo-common/src/main/proto/errors.proto
@@ -79,9 +79,12 @@ enum ResultCode {
   INVALID_RPC_CALL              = 204; // When invalid RPC call is invoked (e.g., wrong message and absent fields)
 
   // Query Management and Scheduler
-  NO_SUCH_QUERYID               = 301; // No query id in TajoMaster
-  NO_DATA                       = 302; // No data due to query fail or error
-  INCOMPLETE_QUERY              = 303; // It occurs when a client requests something of a completed query.
+  QUERY_FAILED                  = 301; // SQLState: ? - Query failed
+  QUERY_KILLED                  = 302; // SQLState: ? - Query killed
+  QUERY_TIMEOUT                 = 303; // SQLState: ? - Timeout for the query
+  QUERY_NOT_FOUND               = 304; // No such query in TajoMaster
+  NO_DATA                       = 305; // No data due to query fail or error
+  INCOMPLETE_QUERY              = 306; // It occurs when a client requests something of a completed query.
 
   // Session
   INVALID_SESSION               = 401; // Session already was invalid
@@ -154,11 +157,15 @@ enum ResultCode {
   CAT_UPGRADE_REQUIRED                  = 901;  // Migration
   CAT_CANNOT_CONNECT                    = 902;  // Cannot connect metadata server
 
-  // MetaData Connector (MDC)
-  MDC_NO_MATCHED_DATATYPE               = 910;  // No matched data type between Tajo and connector
+  // Linked Metadata (LMD)
+  LMD_NO_MATCHED_DATATYPE               = 910;  // No matched data type between Tajo and connector
 
   // Storage and Data Format
-  UNKNOWN_DATAFORMAT                    = 1001; // Unknown Data Format
+  UNAVAILABLE_TABLE_LOCATION            = 1001; // SQLState: ? - Unavailable table location
+  UNKNOWN_DATAFORMAT                    = 1002; // SQLState: ? - Unknown Data Format
+  UNSUPPORTED_DATATYPE                  = 1003; // SQLState: ? - Unsupported data type
+  INVALID_TABLE_PROPERTY                = 1004; // SQLState: ? - Invalid Table Property
+  MISSING_TABLE_PROPERTY                = 1005; // SQLState: ? - Missing table property
 
 
   CLIENT_CONNECTION_EXCEPTION           = 1101; // SQLState: 08000 - Client connection error

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/engine/function/FailFunction.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FailFunction.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FailFunction.java
new file mode 100644
index 0000000..9dc3ded
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FailFunction.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.function;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.plan.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+@Description(
+    functionName = "fail",
+    description = "Cause some error if the second parameter value is greater than the first parameter",
+    example = "> SELECT fail(1, col1, 'message');\n"
+        + "1",
+    returnType = Type.INT4,
+    paramTypes = {
+        @ParamTypes(paramTypes = {Type.INT4, Type.INT4, Type.TEXT}),
+    }
+)
+public class FailFunction extends GeneralFunction {
+
+  public FailFunction() {
+    super(new Column[]{
+        new Column("trigger", TajoDataTypes.Type.INT4),
+        new Column("input_number", TajoDataTypes.Type.INT4),
+        new Column("message", TajoDataTypes.Type.TEXT)
+    });
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+
+    // to skip the plannin phase
+    if (params.isBlankOrNull(0) && params.isBlankOrNull(1)) {
+      return DatumFactory.createInt4(params.getInt4(0));
+    }
+
+    final int trigger = params.getInt4(0);
+    final int num = params.getInt4(1);
+    final String msg = params.getText(2);
+
+    if (num >= trigger) {
+      throw new TajoInternalError(msg);
+    }
+
+    return DatumFactory.createInt4(params.getInt4(0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 67ae684..d89a404 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -31,6 +31,7 @@ import org.apache.tajo.algebra.CreateIndex.IndexMethodSpec;
 import org.apache.tajo.algebra.LiteralValue.LiteralType;
 import org.apache.tajo.algebra.Sort.SortSpec;
 import org.apache.tajo.engine.parser.SQLParser.*;
+import org.apache.tajo.exception.SQLSyntaxError;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.StringUtils;
 
@@ -62,7 +63,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
       parser.addErrorListener(new SQLErrorListener());
       context = parser.sql();
     } catch (SQLParseError e) {
-      throw new SQLSyntaxError(e);
+      throw new SQLSyntaxError(e.getMessage());
     }
     return visitSql(context);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java
deleted file mode 100644
index 0b50d4b..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLSyntaxError.java
+++ /dev/null
@@ -1,51 +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.engine.parser;
-
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.exception.TajoRuntimeException;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
-
-public class SQLSyntaxError extends TajoRuntimeException {
-  private static final long serialVersionUID = 5388279335175632067L;
-
-  private transient String detailedMessage;
-
-  public SQLSyntaxError(String errorMessage) {
-    super(Errors.ResultCode.SYNTAX_ERROR, errorMessage);
-  }
-
-  public SQLSyntaxError(SQLParseError e) {
-    super(Errors.ResultCode.SYNTAX_ERROR, e.getMessageHeader());
-  }
-
-  @Override
-  public String getMessage() {
-    if (detailedMessage == null) {
-      if (getCause() != null) {
-        detailedMessage = getCause().getMessage();
-      } else {
-        detailedMessage = String.format("ERROR: %s\n", super.getMessage());
-      }
-    }
-    return detailedMessage;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java
index 0983dc6..ec85544 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlanner.java
@@ -21,16 +21,13 @@
  */
 package org.apache.tajo.engine.planner;
 
-import org.apache.tajo.worker.TaskAttemptContext;
-import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.engine.planner.physical.PhysicalExec;
-import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.worker.TaskAttemptContext;
 
 /**
  * This class generates a physical execution plan.
  */
 public interface PhysicalPlanner {
-  PhysicalExec createPlan(TaskAttemptContext context,
-                          LogicalNode logicalPlan)
-      throws InternalException;
+  PhysicalExec createPlan(TaskAttemptContext context, LogicalNode logicalPlan);
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
index 7b1b1d7..5bbf3a9 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
@@ -16,9 +16,6 @@
  * limitations under the License.
  */
 
-/**
- *
- */
 package org.apache.tajo.engine.planner;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -38,7 +35,7 @@ import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.physical.*;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer;
 import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.DistinctAggregationAlgorithm;
 import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage;
@@ -83,8 +80,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
     this.conf = conf;
   }
 
-  public PhysicalExec createPlan(final TaskAttemptContext context, final LogicalNode logicalPlan)
-      throws InternalException {
+  public PhysicalExec createPlan(final TaskAttemptContext context, final LogicalNode logicalPlan) {
 
     PhysicalExec execPlan;
 
@@ -101,7 +97,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
         return execPlan;
       }
     } catch (IOException ioe) {
-      throw new InternalException(ioe);
+      throw new TajoInternalError(ioe);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
index 07445a4..b10dd1b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
@@ -29,9 +29,6 @@ import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.JoinType;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.AmbiguousFunctionException;
-import org.apache.tajo.catalog.exception.CatalogException;
-import org.apache.tajo.catalog.exception.UndefinedFunctionException;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
@@ -40,9 +37,11 @@ import org.apache.tajo.engine.planner.global.builder.DistinctGroupbyBuilder;
 import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteEngine;
 import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanRewriteRuleProvider;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.exception.*;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.exception.NotImplementedException;
+import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.Target;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.*;
@@ -346,32 +345,32 @@ public class GlobalPlanner {
     }
   }
 
-  private AggregationFunctionCallEval createSumFunction(EvalNode[] args) throws CatalogException {
+  private AggregationFunctionCallEval createSumFunction(EvalNode[] args) throws TajoException {
     FunctionDesc functionDesc = null;
     functionDesc = getCatalog().getFunction("sum", CatalogProtos.FunctionType.AGGREGATION,
         args[0].getValueType());
     return new AggregationFunctionCallEval(functionDesc, args);
   }
 
-  private AggregationFunctionCallEval createCountFunction(EvalNode [] args) throws CatalogException {
+  private AggregationFunctionCallEval createCountFunction(EvalNode [] args) throws TajoException {
     FunctionDesc functionDesc = getCatalog().getFunction("count", CatalogProtos.FunctionType.AGGREGATION,
         args[0].getValueType());
     return new AggregationFunctionCallEval(functionDesc, args);
   }
 
-  private AggregationFunctionCallEval createCountRowFunction(EvalNode[] args) throws CatalogException {
+  private AggregationFunctionCallEval createCountRowFunction(EvalNode[] args) throws TajoException {
     FunctionDesc functionDesc = getCatalog().getFunction("count", CatalogProtos.FunctionType.AGGREGATION,
         new TajoDataTypes.DataType[]{});
     return new AggregationFunctionCallEval(functionDesc, args);
   }
 
-  private AggregationFunctionCallEval createMaxFunction(EvalNode [] args) throws CatalogException {
+  private AggregationFunctionCallEval createMaxFunction(EvalNode [] args) throws TajoException {
     FunctionDesc functionDesc = getCatalog().getFunction("max", CatalogProtos.FunctionType.AGGREGATION,
         args[0].getValueType());
     return new AggregationFunctionCallEval(functionDesc, args);
   }
 
-  private AggregationFunctionCallEval createMinFunction(EvalNode [] args) throws CatalogException {
+  private AggregationFunctionCallEval createMinFunction(EvalNode [] args) throws TajoException {
     FunctionDesc functionDesc = getCatalog().getFunction("min", CatalogProtos.FunctionType.AGGREGATION,
         args[0].getValueType());
     return new AggregationFunctionCallEval(functionDesc, args);
@@ -847,7 +846,7 @@ public class GlobalPlanner {
       // Verify supported partition types
       PartitionMethodDesc partitionMethod = currentNode.getPartitionMethod();
       if (partitionMethod.getPartitionType() != CatalogProtos.PartitionType.COLUMN) {
-        throw new UnimplementedException("partition type '" + partitionMethod.getPartitionType().name() + "'");
+        throw new NotImplementedException("partition type '" + partitionMethod.getPartitionType().name() + "'");
       }
 
       if (hasUnionChild(currentNode)) { // if it has union children

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 2ad45ba..20780ec 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -36,10 +36,12 @@ import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
-import org.apache.tajo.engine.parser.SQLSyntaxError;
+import org.apache.tajo.exception.SQLSyntaxError;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.exception.ExceptionUtil;
 import org.apache.tajo.exception.ReturnStateUtil;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoRuntimeException;
 import org.apache.tajo.master.TajoMaster.MasterContext;
 import org.apache.tajo.master.exec.DDLExecutor;
 import org.apache.tajo.master.exec.QueryExecutor;
@@ -305,6 +307,10 @@ public class GlobalEngine extends AbstractService {
 
           TablespaceManager.get(tableDesc.getUri()).get().verifySchemaToWrite(tableDesc, outSchema);
 
+        } catch (TajoException t) {
+          state.addVerification(t);
+        } catch (TajoRuntimeException t) {
+          state.addVerification(t);
         } catch (Throwable t) {
           state.addVerification(SyntaxErrorUtil.makeSyntaxError(t.getMessage()));
         }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
index 27aabfc..3ec09aa 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -38,10 +38,11 @@ import org.apache.tajo.catalog.CatalogServer;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.LocalCatalogWrapper;
-import org.apache.tajo.catalog.exception.DuplicateDatabaseException;
+import org.apache.tajo.exception.DuplicateDatabaseException;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.function.FunctionLoader;
+import org.apache.tajo.exception.DuplicateTablespaceException;
 import org.apache.tajo.function.FunctionSignature;
 import org.apache.tajo.master.rm.TajoResourceManager;
 import org.apache.tajo.metrics.ClusterResourceMetricSet;
@@ -371,7 +372,9 @@ public class TajoMaster extends CompositeService {
     }
   }
 
-  private void checkBaseTBSpaceAndDatabase() throws IOException, DuplicateDatabaseException {
+  private void checkBaseTBSpaceAndDatabase()
+      throws IOException, DuplicateDatabaseException, DuplicateTablespaceException {
+
     if (!catalog.existTablespace(DEFAULT_TABLESPACE_NAME)) {
       catalog.createTablespace(DEFAULT_TABLESPACE_NAME, context.getConf().getVar(ConfVars.WAREHOUSE_DIR));
     } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index 72e52b2..d6ace8f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -33,14 +33,16 @@ import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.exception.QueryNotFoundException;
 import org.apache.tajo.exception.ReturnStateUtil;
+import org.apache.tajo.exception.UnavailableTableLocationException;
+import org.apache.tajo.exception.UndefinedDatabaseException;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.*;
 import org.apache.tajo.ipc.TajoMasterClientProtocol;
@@ -186,7 +188,6 @@ public class TajoMasterClientService extends AbstractService {
           context.getSessionManager().removeVariable(sessionId, unsetVariable);
         }
 
-
         builder.setState(OK);
         builder.setSessionVars(new KeyValueSet(context.getSessionManager().getAllVariables(sessionId)).getProto());
         return builder.build();
@@ -540,9 +541,13 @@ public class TajoMasterClientService extends AbstractService {
 
         QueryId queryId = new QueryId(request.getQueryId());
         NonForwardQueryResultScanner queryResultScanner = session.getNonForwardQueryResultScanner(queryId);
+
         if (queryResultScanner == null) {
+
           QueryInfo queryInfo = context.getQueryJobManager().getFinishedQuery(queryId);
-          Preconditions.checkNotNull(queryInfo, "QueryInfo cannot be NULL.");
+          if (queryInfo == null) {
+            throw new QueryNotFoundException(queryId.toString());
+          }
 
           TableDesc resultTableDesc = queryInfo.getResultDesc();
           Preconditions.checkNotNull(resultTableDesc, "QueryInfo::getResultDesc results in NULL.");
@@ -603,7 +608,7 @@ public class TajoMasterClientService extends AbstractService {
     }
 
     @Override
-    public GetQueryInfoResponse getQueryInfo(RpcController controller, QueryIdRequest request) throws ServiceException {
+    public GetQueryInfoResponse getQueryInfo(RpcController controller, QueryIdRequest request) {
       GetQueryInfoResponse.Builder builder = GetQueryInfoResponse.newBuilder();
 
       try {
@@ -620,9 +625,11 @@ public class TajoMasterClientService extends AbstractService {
           queryInfo = queryInProgress.getQueryInfo();
         }
 
-        if (queryInfo != null) {
-          builder.setQueryInfo(queryInfo.getProto());
+        if (queryInfo == null) {
+          throw new QueryNotFoundException(queryId.toString());
         }
+
+        builder.setQueryInfo(queryInfo.getProto());
         builder.setState(OK);
 
       } catch (Throwable t) {
@@ -697,14 +704,11 @@ public class TajoMasterClientService extends AbstractService {
     @Override
     public ReturnState createDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
       try {
-        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
-        QueryContext queryContext = new QueryContext(conf, session);
+        final Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        final QueryContext queryContext = new QueryContext(conf, session);
 
-        if (context.getGlobalEngine().getDDLExecutor().createDatabase(queryContext, request.getValue(), null, false)) {
-          return OK;
-        } else {
-          return errDuplicateDatabase(request.getValue());
-        }
+        context.getGlobalEngine().getDDLExecutor().createDatabase(queryContext, request.getValue(), null, false);
+        return OK;
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -731,14 +735,11 @@ public class TajoMasterClientService extends AbstractService {
     @Override
     public ReturnState dropDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
       try {
-        Session session = context.getSessionManager().getSession(request.getSessionId().getId());
-        QueryContext queryContext = new QueryContext(conf, session);
+        final Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        final QueryContext queryContext = new QueryContext(conf, session);
 
-        if (context.getGlobalEngine().getDDLExecutor().dropDatabase(queryContext, request.getValue(), false)) {
-          return OK;
-        } else {
-          return errUndefinedDatabase(request.getValue());
-        }
+        context.getGlobalEngine().getDDLExecutor().dropDatabase(queryContext, request.getValue(), false);
+        return OK;
 
       } catch (Throwable t) {
         printStackTraceIfError(LOG, t);
@@ -873,7 +874,7 @@ public class TajoMasterClientService extends AbstractService {
         FileSystem fs = path.getFileSystem(conf);
 
         if (!fs.exists(path)) {
-          throw new IOException("No such a directory: " + path);
+          throw new UnavailableTableLocationException(path.toString(), "no such a directory");
         }
 
         Schema schema = new Schema(request.getSchema());

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/master/exec/CreateTableExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/CreateTableExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/CreateTableExecutor.java
index a43b95e..38f722d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/CreateTableExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/CreateTableExecutor.java
@@ -24,12 +24,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.DuplicateTableException;
-import org.apache.tajo.catalog.exception.UndefinedTablespaceException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.plan.logical.CreateTableNode;
 import org.apache.tajo.plan.util.PlannerUtil;
@@ -110,17 +107,13 @@ public class CreateTableExecutor {
     }
 
     tableSpace.createTable(desc, ifNotExists);
-
-    if (catalog.createTable(desc)) {
-      LOG.info("Table " + desc.getName() + " is created (" + desc.getStats().getNumBytes() + ")");
-      return desc;
-    } else {
-      LOG.info("Table creation " + tableName + " is failed.");
-      throw new TajoInternalError("Cannot create table \"" + tableName + "\"");
-    }
+    catalog.createTable(desc);
+    return desc;
   }
 
-  private TableDesc handlExistence(boolean ifNotExists, String qualifiedName) throws DuplicateTableException {
+  private TableDesc handlExistence(boolean ifNotExists, String qualifiedName)
+      throws DuplicateTableException, UndefinedTableException {
+
     if (ifNotExists) {
       LOG.info("relation \"" + qualifiedName + "\" is already exists.");
       return catalog.getTableDesc(qualifiedName);

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
index 19eba3e..f7e2fe9 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
@@ -27,19 +27,19 @@ import org.apache.tajo.algebra.AlterTableOpType;
 import org.apache.tajo.algebra.AlterTablespaceSetType;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.*;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.PartitionKeyProto;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.storage.FileTablespace;
 import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.Pair;
 
@@ -203,7 +203,8 @@ public class DDLExecutor {
    * Alter a given table
    */
   public static void alterTablespace(final TajoMaster.MasterContext context, final QueryContext queryContext,
-                                     final AlterTablespaceNode alterTablespace) {
+                                     final AlterTablespaceNode alterTablespace)
+      throws UndefinedTablespaceException, InsufficientPrivilegeException {
 
     final CatalogService catalog = context.getCatalog();
     final String spaceName = alterTablespace.getTablespaceName();
@@ -228,7 +229,7 @@ public class DDLExecutor {
 
   // Database Section
   //--------------------------------------------------------------------------
-  public boolean createDatabase(@Nullable QueryContext queryContext, String databaseName,
+  public void createDatabase(@Nullable QueryContext queryContext, String databaseName,
                                 @Nullable String tablespace,
                                 boolean ifNotExists) throws IOException, DuplicateDatabaseException {
 
@@ -244,30 +245,27 @@ public class DDLExecutor {
     if (exists) {
       if (ifNotExists) {
         LOG.info("database \"" + databaseName + "\" is already exists.");
-        return true;
+        return;
       } else {
         throw new DuplicateDatabaseException(databaseName);
       }
     }
 
-    if (catalog.createDatabase(databaseName, tablespaceName)) {
-      String normalized = databaseName;
-      Path databaseDir = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR), normalized);
-      FileSystem fs = databaseDir.getFileSystem(context.getConf());
-      fs.mkdirs(databaseDir);
-    }
-
-    return true;
+    catalog.createDatabase(databaseName, tablespaceName);
+    String normalized = databaseName;
+    Path databaseDir = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR), normalized);
+    FileSystem fs = databaseDir.getFileSystem(context.getConf());
+    fs.mkdirs(databaseDir);
   }
 
-  public boolean dropDatabase(QueryContext queryContext, String databaseName, boolean ifExists)
-      throws UndefinedDatabaseException {
+  public void dropDatabase(QueryContext queryContext, String databaseName, boolean ifExists)
+      throws UndefinedDatabaseException, InsufficientPrivilegeException {
 
     boolean exists = catalog.existDatabase(databaseName);
     if (!exists) {
       if (ifExists) { // DROP DATABASE IF EXISTS
         LOG.info("database \"" + databaseName + "\" does not exists.");
-        return true;
+        return;
       } else { // Otherwise, it causes an exception.
         throw new UndefinedDatabaseException(databaseName);
       }
@@ -277,9 +275,7 @@ public class DDLExecutor {
       throw new RuntimeException("ERROR: Cannot drop the current open database");
     }
 
-    boolean result = catalog.dropDatabase(databaseName);
-    LOG.info("database " + databaseName + " is dropped.");
-    return result;
+    catalog.dropDatabase(databaseName);
   }
 
   //--------------------------------------------------------------------------
@@ -293,8 +289,8 @@ public class DDLExecutor {
    * @param tableName to be dropped
    * @param purge     Remove all data if purge is true.
    */
-  public boolean dropTable(QueryContext queryContext, String tableName, boolean ifExists, boolean purge)
-      throws UndefinedTableException {
+  public void dropTable(QueryContext queryContext, String tableName, boolean ifExists, boolean purge)
+      throws TajoException {
 
     String databaseName;
     String simpleTableName;
@@ -312,7 +308,7 @@ public class DDLExecutor {
     if (!exists) {
       if (ifExists) { // DROP TABLE IF EXISTS
         LOG.info("relation \"" + qualifiedName + "\" is already exists.");
-        return true;
+        return;
       } else { // Otherwise, it causes an exception.
         throw new UndefinedTableException(qualifiedName);
       }
@@ -328,9 +324,7 @@ public class DDLExecutor {
         throw new InternalError(e.getMessage());
       }
     }
-
     LOG.info(String.format("relation \"%s\" is " + (purge ? " purged." : " dropped."), qualifiedName));
-    return true;
   }
 
   /**
@@ -361,15 +355,20 @@ public class DDLExecutor {
         throw new UndefinedTableException(qualifiedName);
       }
 
-      Path warehousePath = new Path(TajoConf.getWarehouseDir(context.getConf()), databaseName);
+      // only file-based tablespace is supported yet.
       TableDesc tableDesc = catalog.getTableDesc(databaseName, simpleTableName);
-      Path tablePath = new Path(tableDesc.getUri());
-      if (tablePath.getParent() == null ||
-          !tablePath.getParent().toUri().getPath().equals(warehousePath.toUri().getPath())) {
-        throw new IOException("Can't truncate external table:" + eachTableName + ", data dir=" + tablePath +
-            ", warehouse dir=" + warehousePath);
+
+      if (tableDesc.isExternal()) {
+        throw new UnsupportedException("table truncation on an external table '" + eachTableName + "'");
+      }
+
+      Tablespace space = TablespaceManager.get(tableDesc.getUri()).get();
+
+      if (space instanceof FileTablespace) {
+        tableDescList.add(tableDesc);
+      } else {
+        throw new UnsupportedException("table truncation on " + space.getName() + " storage");
       }
-      tableDescList.add(tableDesc);
     }
 
     for (TableDesc eachTable : tableDescList) {
@@ -397,8 +396,7 @@ public class DDLExecutor {
    */
   public void alterTable(TajoMaster.MasterContext context, final QueryContext queryContext,
                          final AlterTableNode alterTable)
-      throws IOException, UndefinedTableException, DuplicateTableException, DuplicateColumnException,
-      DuplicatePartitionException, UndefinedPartitionException, UndefinedPartitionKeyException, AmbiguousPartitionDirectoryExistException {
+      throws IOException, TajoException {
 
     final CatalogService catalog = context.getCatalog();
     final String tableName = alterTable.getTableName();
@@ -563,7 +561,7 @@ public class DDLExecutor {
   }
 
   private boolean ensureColumnPartitionKeys(String tableName, String[] columnNames)
-      throws UndefinedPartitionKeyException {
+      throws UndefinedPartitionKeyException, UndefinedTableException {
 
     for(String columnName : columnNames) {
       if (!ensureColumnPartitionKeys(tableName, columnName)) {
@@ -573,7 +571,7 @@ public class DDLExecutor {
     return true;
   }
 
-  private boolean ensureColumnPartitionKeys(String tableName, String columnName) {
+  private boolean ensureColumnPartitionKeys(String tableName, String columnName) throws UndefinedTableException {
     final TableDesc tableDesc = catalog.getTableDesc(tableName);
     if (tableDesc.getPartitionMethod().getExpressionSchema().contains(columnName)) {
       return true;
@@ -582,7 +580,7 @@ public class DDLExecutor {
     }
   }
 
-  private boolean ensureColumnExistance(String tableName, String columnName) {
+  private boolean ensureColumnExistance(String tableName, String columnName) throws UndefinedTableException {
     final TableDesc tableDesc = catalog.getTableDesc(tableName);
     return tableDesc.getSchema().containsByName(columnName);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
index 3f65831..bd5d696 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.*;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.DuplicateIndexException;
+import org.apache.tajo.exception.DuplicateIndexException;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.common.TajoDataTypes;
@@ -37,7 +37,6 @@ import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.planner.physical.EvalExprExec;
 import org.apache.tajo.engine.planner.physical.InsertRowsExec;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.SerializedResultSet;
 import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
 import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse.ResultType;
@@ -61,7 +60,6 @@ import org.apache.tajo.plan.verifier.VerifyException;
 import org.apache.tajo.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.util.ProtoUtil;
-import org.apache.tajo.util.metrics.TajoMetrics;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
index f351143..b09d5fd 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
@@ -45,6 +45,7 @@ import org.apache.tajo.engine.planner.global.ExecutionBlock;
 import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
 import org.apache.tajo.engine.planner.global.ExecutionQueue;
 import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.engine.query.QueryContext;
@@ -473,7 +474,7 @@ public class Query implements EventHandler<QueryEvent> {
         try {
           LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
           space.rollbackTable(rootNode.getChild());
-        } catch (IOException e) {
+        } catch (Throwable e) {
           LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
         }
       }
@@ -522,17 +523,11 @@ public class Query implements EventHandler<QueryEvent> {
           }
 
           // Store partitions to CatalogStore using alter table statement.
-          boolean result = catalog.addPartitions(databaseName, simpleTableName, partitions, true);
-          if (result) {
-            LOG.info(String.format("Complete adding for partition %s", partitions.size()));
-          } else {
-            LOG.info(String.format("Incomplete adding for partition %s", partitions.size()));
-          }
+          catalog.addPartitions(databaseName, simpleTableName, partitions, true);
         } else {
           LOG.info("Can't find partitions for adding.");
         }
 
-
       } catch (Exception e) {
         query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
         return QueryState.QUERY_ERROR;

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
index ac1bab5..c6e2b35 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
@@ -37,6 +37,7 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
 import org.apache.tajo.master.event.*;
@@ -370,7 +371,7 @@ public class QueryMasterTask extends CompositeService {
         LogicalRootNode rootNode = plan.getRootBlock().getRoot();
         try {
           space.rollbackTable(rootNode.getChild());
-        } catch (IOException e) {
+        } catch (Throwable e) {
           LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
         }
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
index fdaa0bc..53f5504 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
@@ -39,19 +39,21 @@ import org.apache.tajo.engine.planner.global.ExecutionBlock;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.planner.global.rewriter.rules.GlobalPlanRewriteUtil;
 import org.apache.tajo.engine.utils.TupleUtil;
-import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoInternalError;
+import org.apache.tajo.exception.UndefinedTableException;
+import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.plan.logical.SortNode.SortPurpose;
 import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage;
 import org.apache.tajo.plan.serder.PlanProto.EnforceProperty;
-import org.apache.tajo.querymaster.Task.IntermediateEntry;
-import org.apache.tajo.plan.logical.SortNode.SortPurpose;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.plan.PlanningException;
-import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.querymaster.Task.IntermediateEntry;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.Pair;
 import org.apache.tajo.unit.StorageUnit;
+import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.util.TajoIdUtils;
 import org.apache.tajo.worker.FetchImpl;
@@ -77,7 +79,7 @@ public class Repartitioner {
   private final static String UNKNOWN_HOST = "unknown";
 
   public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerContext, Stage stage)
-      throws IOException {
+      throws IOException, TajoException {
     ExecutionBlock execBlock = stage.getBlock();
     QueryMasterTask.QueryMasterTaskContext masterContext = stage.getContext();
 
@@ -282,7 +284,7 @@ public class Repartitioner {
                                                        Fragment[] fragments,
                                                        ScanNode[] broadcastScans,
                                                        long[] broadcastStats,
-                                                       Fragment[] broadcastFragments) throws IOException {
+                                                       Fragment[] broadcastFragments) throws IOException, TajoException {
     MasterPlan masterPlan = stage.getMasterPlan();
     ExecutionBlock execBlock = stage.getBlock();
     // The hash map is modeling as follows:
@@ -597,7 +599,7 @@ public class Repartitioner {
     } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
       scheduleRangeShuffledFetches(schedulerContext, masterPlan, stage, channel, maxNum);
     } else {
-      throw new InternalException("Cannot support partition type");
+      throw new TajoInternalError("Cannot support partition type");
     }
   }
 
@@ -639,8 +641,10 @@ public class Repartitioner {
       String storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
       CatalogService catalog = stage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
       LogicalRootNode rootNode = masterPlan.getLogicalPlan().getRootBlock().getRoot();
-      TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
-      if (tableDesc == null) {
+      TableDesc tableDesc = null;
+      try {
+        tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+      } catch (UndefinedTableException e) {
         throw new IOException("Can't get table meta data from catalog: " +
             PlannerUtil.getStoreTableName(masterPlan.getLogicalPlan()));
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
index 9279f64..cf5cdbd 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
@@ -42,6 +42,7 @@ import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.global.ExecutionBlock;
 import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.plan.serder.PlanProto.DistinctGroupbyEnforcer.MultipleAggregationStage;
 import org.apache.tajo.plan.serder.PlanProto.EnforceProperty;
@@ -1015,7 +1016,7 @@ public class Stage implements EventHandler<StageEvent> {
       }
     }
 
-    private static void schedule(Stage stage) throws IOException {
+    private static void schedule(Stage stage) throws IOException, TajoException {
       MasterPlan masterPlan = stage.getMasterPlan();
       ExecutionBlock execBlock = stage.getBlock();
       if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
@@ -1076,7 +1077,7 @@ public class Stage implements EventHandler<StageEvent> {
       }
     }
 
-    private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException {
+    private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException, TajoException {
       ExecutionBlock execBlock = stage.getBlock();
       ScanNode[] scans = execBlock.getScanNodes();
       Preconditions.checkArgument(scans.length == 1, "Must be Scan Query");

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
index 630b1e9..705a29d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
+++ b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
@@ -8,8 +8,12 @@ import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.client.*;
+import org.apache.tajo.client.QueryStatus;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.client.TajoClientUtil;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.jdbc.FetchResultSet;
 import org.apache.tajo.service.ServiceTrackerFactory;
@@ -40,7 +44,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.tajo.exception.ReturnStateUtil.isError;
-import static org.apache.tajo.exception.ReturnStateUtil.isSuccess;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -394,7 +397,7 @@ public class QueryExecutorServlet extends HttpServlet {
       }
     }
 
-    private QueryStatus waitForComplete(QueryId queryid) throws SQLException {
+    private QueryStatus waitForComplete(QueryId queryid) throws TajoException {
       QueryStatus status = null;
 
       while (!stop.get()) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
index b47bf2f..e7e81d4 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
@@ -21,9 +21,8 @@ package org.apache.tajo.worker;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.planner.PhysicalPlanner;
 import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
-import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.engine.planner.physical.PhysicalExec;
-import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.plan.logical.LogicalNode;
 
 import java.io.IOException;
 
@@ -35,8 +34,7 @@ public class TajoQueryEngine {
     this.phyPlanner = new PhysicalPlannerImpl(conf);
   }
   
-  public PhysicalExec createPlan(TaskAttemptContext ctx, LogicalNode plan)
-      throws InternalException {
+  public PhysicalExec createPlan(TaskAttemptContext ctx, LogicalNode plan) {
     return phyPlanner.createPlan(ctx, plan);
   }
   

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/DatabasesResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/DatabasesResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/DatabasesResource.java
index b807198..046e538 100644
--- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/DatabasesResource.java
+++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/DatabasesResource.java
@@ -18,41 +18,27 @@
 
 package org.apache.tajo.ws.rs.resources;
 
-import java.net.URI;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.Application;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.Response.Status;
-import javax.ws.rs.core.UriInfo;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.ws.rs.JerseyResourceDelegate;
-import org.apache.tajo.ws.rs.JerseyResourceDelegateContext;
-import org.apache.tajo.ws.rs.JerseyResourceDelegateContextKey;
-import org.apache.tajo.ws.rs.JerseyResourceDelegateUtil;
-import org.apache.tajo.ws.rs.ResourcesUtil;
+import org.apache.tajo.ws.rs.*;
 import org.apache.tajo.ws.rs.requests.NewDatabaseRequest;
 import org.apache.tajo.ws.rs.responses.DatabaseInfoResponse;
 
+import javax.ws.rs.*;
+import javax.ws.rs.core.*;
+import javax.ws.rs.core.Response.Status;
+import java.net.URI;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Deals with Database Management
  */
@@ -175,19 +161,18 @@ public class DatabasesResource {
       if (request.getDatabaseName() == null || request.getDatabaseName().isEmpty()) {
         return ResourcesUtil.createBadRequestResponse(LOG, "databaseName is null or empty.");
       }
-      
-      boolean databaseCreated =
-          masterContext.getCatalog().createDatabase(request.getDatabaseName(), 
-              TajoConstants.DEFAULT_TABLESPACE_NAME);
-      
-      if (databaseCreated) {
+
+
+      try {
+        masterContext.getCatalog().createDatabase(request.getDatabaseName(),
+                TajoConstants.DEFAULT_TABLESPACE_NAME);
         URI newDatabaseURI = uriInfo.getBaseUriBuilder()
             .path(DatabasesResource.class)
             .path(DatabasesResource.class, "getDatabase")
             .build(request.getDatabaseName());
         return Response.created(newDatabaseURI).build();
-      } else {
-        return ResourcesUtil.createExceptionResponse(LOG, "Failed to create a new database.");
+      } catch (TajoException e) {
+        return ResourcesUtil.createExceptionResponse(LOG, e.getMessage());
       }
     }
   }
@@ -330,12 +315,11 @@ public class DatabasesResource {
       if (!catalogService.existDatabase(databaseName)) {
         return Response.status(Status.NOT_FOUND).build();
       }
-      
-      boolean databaseDropped = catalogService.dropDatabase(databaseName);
-      
-      if (databaseDropped) {
+
+      try {
+        catalogService.dropDatabase(databaseName);
         return Response.ok().build();
-      } else {
+      } catch (TajoException e) {
         return ResourcesUtil.createExceptionResponse(LOG, "Unable to drop a database " + databaseName);
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/TablesResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/TablesResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/TablesResource.java
index 8e19fcb..3dba02f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/TablesResource.java
+++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/TablesResource.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.master.TajoMaster.MasterContext;
 import org.apache.tajo.ws.rs.*;
 
@@ -182,9 +183,14 @@ public class TablesResource {
           !catalogService.existsTable(databaseName, tableName)) {
         return Response.status(Status.NOT_FOUND).build();
       }
-      
-      TableDesc tableDesc = catalogService.getTableDesc(databaseName, tableName);
-      return Response.ok(tableDesc).build();
+
+      TableDesc tableDesc = null;
+      try {
+        tableDesc = catalogService.getTableDesc(databaseName, tableName);
+        return Response.ok(tableDesc).build();
+      } catch (TajoException e) {
+        return ResourcesUtil.createExceptionResponse(LOG, e.getMessage());
+      }
     }
   }
   
@@ -213,7 +219,6 @@ public class TablesResource {
           LOG);
     } catch (Throwable e) {
       LOG.error(e.getMessage(), e);
-      
       response = ResourcesUtil.createExceptionResponse(null, e.getMessage());
     }
     
@@ -245,14 +250,13 @@ public class TablesResource {
       }
       
       String canonicalTableName = CatalogUtil.getCanonicalTableName(databaseName, tableName);
-      boolean tableDropped = 
-          catalogService.dropTable(canonicalTableName);
-      if (tableDropped) {
+
+      try {
+        catalogService.dropTable(canonicalTableName);
         return Response.ok().build();
-      } else {
-        return ResourcesUtil.createExceptionResponse(LOG, "Unable to drop a " + canonicalTableName + " table.");
+      } catch (TajoException e) {
+        return ResourcesUtil.createExceptionResponse(LOG, e.getMessage());
       }
     }
-    
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/903151ea/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
index ec4796f..bcce612 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -29,8 +29,7 @@ import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.exception.UndefinedDatabaseException;
-import org.apache.tajo.catalog.exception.UndefinedTableException;
+import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.cli.tsql.ParsedResult;
 import org.apache.tajo.cli.tsql.SimpleParser;
 import org.apache.tajo.client.TajoClient;
@@ -350,7 +349,7 @@ public class QueryTestCaseBase {
     fail("Cannot catch any planning error from: " + query);
   }
 
-  protected ResultSet executeString(String sql) throws Exception {
+  protected ResultSet executeString(String sql) throws TajoException {
     return client.executeQueryAndGetResult(sql);
   }
 
@@ -632,7 +631,6 @@ public class QueryTestCaseBase {
    * @param result Query result to be compared.
    */
   public final void assertResultSet(String message, ResultSet result, String resultFileName) throws IOException {
-    FileSystem fs = currentQueryPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
     Path resultFile = getResultFile(resultFileName);
     try {
       verifyResultText(message, result, resultFile);
@@ -683,7 +681,7 @@ public class QueryTestCaseBase {
    * Assert that the database does not exists.
    * @param databaseName The database name to be checked. This name is case sensitive.
    */
-  public void assertDatabaseNotExists(String databaseName) throws SQLException {
+  public void assertDatabaseNotExists(String databaseName) {
     assertTrue(!client.existDatabase(databaseName));
   }
 
@@ -693,7 +691,7 @@ public class QueryTestCaseBase {
    * @param tableName The table name to be checked. This name is case sensitive.
    * @throws ServiceException
    */
-  public void assertTableExists(String tableName) throws SQLException {
+  public void assertTableExists(String tableName) {
     assertTrue(client.existTable(tableName));
   }
 
@@ -702,7 +700,7 @@ public class QueryTestCaseBase {
    *
    * @param tableName The table name to be checked. This name is case sensitive.
    */
-  public void assertTableNotExists(String tableName) throws SQLException {
+  public void assertTableNotExists(String tableName) {
     assertTrue(!client.existTable(tableName));
   }
 
@@ -790,7 +788,12 @@ public class QueryTestCaseBase {
     return queryFilePath;
   }
 
-  private Path getResultFile(String fileName) throws IOException {
+  protected String getResultContents(String fileName) throws IOException {
+    Path resultFile = getResultFile(getMethodName() + ".result");
+    return FileUtil.readTextFile(new File(resultFile.toUri()));
+  }
+
+  protected Path getResultFile(String fileName) throws IOException {
     Path resultPath = StorageUtil.concatPath(currentResultPath, fileName);
     FileSystem fs = currentResultPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
     assertTrue(resultPath.toString() + " existence check", fs.exists(resultPath));


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

Posted by hy...@apache.org.
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;