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/04 03:31:27 UTC

[2/4] tajo git commit: TAJO-1699: Tajo Java Client version 2.

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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 89e0a66..19eba3e 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
@@ -33,6 +33,7 @@ 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.master.TajoMaster;
 import org.apache.tajo.plan.LogicalPlan;
@@ -69,8 +70,10 @@ public class DDLExecutor {
   public CreateTableExecutor getCreateTableExecutor() {
     return createTableExecutor;
   }
+  
+  public boolean execute(QueryContext queryContext, LogicalPlan plan)
+      throws IOException, TajoException {
 
-  public boolean execute(QueryContext queryContext, LogicalPlan plan) throws IOException {
     LogicalNode root = ((LogicalRootNode) plan.getRootBlock().getRoot()).getChild();
 
     switch (root.getType()) {
@@ -124,7 +127,9 @@ public class DDLExecutor {
     }
   }
 
-  public void createIndex(final QueryContext queryContext, final CreateIndexNode createIndexNode) {
+  public void createIndex(final QueryContext queryContext, final CreateIndexNode createIndexNode)
+      throws DuplicateIndexException {
+
     String databaseName, simpleIndexName, qualifiedIndexName;
     if (CatalogUtil.isFQTableName(createIndexNode.getIndexName())) {
       String[] splits = CatalogUtil.splitFQTableName(createIndexNode.getIndexName());
@@ -159,7 +164,9 @@ public class DDLExecutor {
     }
   }
 
-  public void dropIndex(final QueryContext queryContext, final DropIndexNode dropIndexNode) {
+  public void dropIndex(final QueryContext queryContext, final DropIndexNode dropIndexNode)
+      throws UndefinedIndexException {
+
     String databaseName, simpleIndexName;
     if (CatalogUtil.isFQTableName(dropIndexNode.getIndexName())) {
       String[] splits = CatalogUtil.splitFQTableName(dropIndexNode.getIndexName());
@@ -223,7 +230,7 @@ public class DDLExecutor {
   //--------------------------------------------------------------------------
   public boolean createDatabase(@Nullable QueryContext queryContext, String databaseName,
                                 @Nullable String tablespace,
-                                boolean ifNotExists) throws IOException {
+                                boolean ifNotExists) throws IOException, DuplicateDatabaseException {
 
     String tablespaceName;
     if (tablespace == null) {
@@ -253,7 +260,9 @@ public class DDLExecutor {
     return true;
   }
 
-  public boolean dropDatabase(QueryContext queryContext, String databaseName, boolean ifExists) {
+  public boolean dropDatabase(QueryContext queryContext, String databaseName, boolean ifExists)
+      throws UndefinedDatabaseException {
+
     boolean exists = catalog.existDatabase(databaseName);
     if (!exists) {
       if (ifExists) { // DROP DATABASE IF EXISTS
@@ -284,7 +293,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) {
+  public boolean dropTable(QueryContext queryContext, String tableName, boolean ifExists, boolean purge)
+      throws UndefinedTableException {
 
     String databaseName;
     String simpleTableName;
@@ -327,7 +337,8 @@ public class DDLExecutor {
    * Truncate table a given table
    */
   public void truncateTable(final QueryContext queryContext, final TruncateTableNode truncateTableNode)
-      throws IOException {
+      throws IOException, UndefinedTableException {
+
     List<String> tableNames = truncateTableNode.getTableNames();
     final CatalogService catalog = context.getCatalog();
 
@@ -385,7 +396,10 @@ public class DDLExecutor {
    * @throws IOException
    */
   public void alterTable(TajoMaster.MasterContext context, final QueryContext queryContext,
-                         final AlterTableNode alterTable) throws IOException {
+                         final AlterTableNode alterTable)
+      throws IOException, UndefinedTableException, DuplicateTableException, DuplicateColumnException,
+      DuplicatePartitionException, UndefinedPartitionException, UndefinedPartitionKeyException, AmbiguousPartitionDirectoryExistException {
+
     final CatalogService catalog = context.getCatalog();
     final String tableName = alterTable.getTableName();
 
@@ -548,8 +562,10 @@ public class DDLExecutor {
     }
   }
 
-  private boolean ensureColumnPartitionKeys(String tableName, String[] columnNames) {
-    for (String columnName : columnNames) {
+  private boolean ensureColumnPartitionKeys(String tableName, String[] columnNames)
+      throws UndefinedPartitionKeyException {
+
+    for(String columnName : columnNames) {
       if (!ensureColumnPartitionKeys(tableName, columnName)) {
         throw new UndefinedPartitionKeyException(columnName);
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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 e7fc4d2..3f65831 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
@@ -529,7 +529,8 @@ public class QueryExecutor {
   }
 
   private void checkIndexExistence(final QueryContext queryContext, final CreateIndexNode createIndexNode)
-      throws IOException {
+      throws DuplicateIndexException {
+
     String databaseName, simpleIndexName, qualifiedIndexName;
     if (CatalogUtil.isFQTableName(createIndexNode.getIndexName())) {
       String[] splits = CatalogUtil.splitFQTableName(createIndexNode.getIndexName());

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java b/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java
index 54c65bf..5aedffb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java
+++ b/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java
@@ -18,8 +18,11 @@
 
 package org.apache.tajo.session;
 
-public class InvalidSessionException extends Exception {
+import org.apache.tajo.error.Errors;
+import org.apache.tajo.exception.TajoRuntimeException;
+
+public class InvalidSessionException extends TajoRuntimeException {
   public InvalidSessionException(String sessionId) {
-    super("Invalid session id \"" + sessionId + "\"");
+    super(Errors.ResultCode.INVALID_SESSION, sessionId);
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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 24534b0..630b1e9 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
@@ -360,7 +360,7 @@ public class QueryExecutorServlet extends HttpServlet {
         if (queryId != null) {
           try {
             tajoClient.closeQuery(queryId);
-          } catch (SQLException e) {
+          } catch (Throwable e) {
             LOG.warn(e);
           }
         }

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java b/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
index eae9e8c..f084138 100644
--- a/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
+++ b/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
@@ -19,7 +19,6 @@
 package org.apache.tajo;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -31,12 +30,12 @@ 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.proto.CatalogProtos;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.client.TajoClient;
 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.session.Session;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.FileUtil;
@@ -47,7 +46,6 @@ import java.io.File;
 import java.io.IOException;
 import java.net.URL;
 import java.sql.ResultSet;
-import java.sql.SQLException;
 import java.util.UUID;
 
 public class LocalTajoTestingUtility {
@@ -140,7 +138,7 @@ public class LocalTajoTestingUtility {
     return util;
   }
 
-  public ResultSet execute(String query) throws IOException, SQLException {
+  public ResultSet execute(String query) throws TajoException {
     return client.executeQueryAndGetResult(query);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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 b5e464b..ec4796f 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -29,6 +29,8 @@ 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.cli.tsql.ParsedResult;
 import org.apache.tajo.cli.tsql.SimpleParser;
 import org.apache.tajo.client.TajoClient;
@@ -199,7 +201,7 @@ public class QueryTestCaseBase {
   }
 
   @AfterClass
-  public static void tearDownClass() throws SQLException {
+  public static void tearDownClass() throws Exception {
     for (String tableName : createdTableGlobalSet) {
       client.updateQuery("DROP TABLE IF EXISTS " + CatalogUtil.denormalizeIdentifier(tableName));
     }
@@ -704,17 +706,17 @@ public class QueryTestCaseBase {
     assertTrue(!client.existTable(tableName));
   }
 
-  public void assertColumnExists(String tableName,String columnName) throws ServiceException, SQLException {
+  public void assertColumnExists(String tableName,String columnName) throws UndefinedTableException {
     TableDesc tableDesc = getTableDesc(tableName);
     assertTrue(tableDesc.getSchema().containsByName(columnName));
   }
 
-  private TableDesc getTableDesc(String tableName) throws ServiceException, SQLException {
+  private TableDesc getTableDesc(String tableName) throws UndefinedTableException {
     return client.getTableDesc(tableName);
   }
 
   public void assertTablePropertyEquals(String tableName, String key, String expectedValue)
-      throws ServiceException, SQLException {
+      throws UndefinedTableException {
 
     TableDesc tableDesc = getTableDesc(tableName);
     assertEquals(expectedValue, tableDesc.getMeta().getOption(key));

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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 d89bca1..0ba0d76 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
@@ -35,6 +35,9 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.error.Errors;
+import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoRuntimeException;
+import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.QueryHistoryProto;
 import org.apache.tajo.ipc.ClientProtos.QueryInfoProto;
@@ -83,7 +86,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCreateAndDropDatabases() throws SQLException {
+  public final void testCreateAndDropDatabases() throws TajoException {
     int currentNum = client.getAllDatabaseNames().size();
 
     String prefix = CatalogUtil.normalizeIdentifier("testCreateDatabase_");
@@ -114,7 +117,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCurrentDatabase() throws IOException, SQLException, InterruptedException {
+  public final void testCurrentDatabase() throws IOException, TajoException, InterruptedException {
     int currentNum = client.getAllDatabaseNames().size();
     assertEquals(TajoConstants.DEFAULT_DATABASE_NAME, client.getCurrentDatabase());
 
@@ -131,7 +134,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testSelectDatabaseToInvalidOne() throws IOException, SQLException, InterruptedException {
+  public final void testSelectDatabaseToInvalidOne() throws IOException, TajoException, InterruptedException {
     int currentNum = client.getAllDatabaseNames().size();
     assertFalse(client.existDatabase("invaliddatabase"));
 
@@ -146,7 +149,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testDropCurrentDatabase() throws IOException, SQLException, InterruptedException {
+  public final void testDropCurrentDatabase() throws IOException, TajoException, InterruptedException {
     int currentNum = client.getAllDatabaseNames().size();
     String databaseName = CatalogUtil.normalizeIdentifier("testdropcurrentdatabase");
     assertTrue(client.createDatabase(databaseName));
@@ -166,7 +169,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testSessionVariables() throws IOException, SQLException, InterruptedException {
+  public final void testSessionVariables() throws IOException, TajoException, InterruptedException {
     String prefixName = "key_";
     String prefixValue = "val_";
 
@@ -212,7 +215,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testKillQuery() throws IOException, SQLException, InterruptedException {
+  public final void testKillQuery() throws IOException, TajoException, InterruptedException {
     ClientProtos.SubmitQueryResponse res = client.executeQuery("select sleep(1) from lineitem");
     Thread.sleep(1000);
     QueryId queryId = new QueryId(res.getQueryId());
@@ -221,7 +224,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testUpdateQuery() throws IOException, SQLException {
+  public final void testUpdateQuery() throws IOException, TajoException {
     final String tableName = CatalogUtil.normalizeIdentifier("testUpdateQuery");
     Path tablePath = writeTmpTable(tableName);
 
@@ -236,7 +239,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCreateAndDropExternalTable() throws IOException, SQLException {
+  public final void testCreateAndDropExternalTable() throws IOException, TajoException {
     final String tableName = "testCreateAndDropExternalTable";
     Path tablePath = writeTmpTable(tableName);
     LOG.error("Full path:" + tablePath.toUri().getRawPath());
@@ -254,7 +257,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCreateAndPurgeExternalTable() throws IOException, SQLException {
+  public final void testCreateAndPurgeExternalTable() throws IOException, TajoException {
     final String tableName = "testCreateAndPurgeExternalTable";
     Path tablePath = writeTmpTable(tableName);
     LOG.error("Full path:" + tablePath.toUri().getRawPath());
@@ -272,7 +275,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCreateAndDropExternalTableByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndDropExternalTableByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropExternalTableByExecuteQuery");
 
@@ -292,7 +295,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCreateAndPurgeExternalTableByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndPurgeExternalTableByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndPurgeExternalTableByExecuteQuery");
 
@@ -312,7 +315,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCreateAndDropTableByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndDropTableByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropTableByExecuteQuery");
 
@@ -333,7 +336,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testCreateAndPurgeTableByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndPurgeTableByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndPurgeTableByExecuteQuery");
 
@@ -354,7 +357,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testDDLByExecuteQuery() throws IOException, SQLException {
+  public final void testDDLByExecuteQuery() throws IOException, TajoException {
     final String tableName = CatalogUtil.normalizeIdentifier("testDDLByExecuteQuery");
     Path tablePath = writeTmpTable(tableName);
 
@@ -367,7 +370,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testGetTableList() throws IOException, SQLException {
+  public final void testGetTableList() throws IOException, TajoException {
     String tableName1 = "GetTableList1".toLowerCase();
     String tableName2 = "GetTableList2".toLowerCase();
 
@@ -387,7 +390,7 @@ public class TestTajoClient {
   Log LOG = LogFactory.getLog(TestTajoClient.class);
 
   @Test
-  public final void testGetTableDesc() throws IOException, SQLException {
+  public final void testGetTableDesc() throws IOException, TajoException {
     final String tableName1 = CatalogUtil.normalizeIdentifier("table3");
     Path tablePath = writeTmpTable(tableName1);
     LOG.error("Full path:" + tablePath.toUri().getRawPath());
@@ -407,7 +410,7 @@ public class TestTajoClient {
   }
 
   //@Test
-  public final void testCreateAndDropTablePartitionedHash1ByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndDropTablePartitionedHash1ByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = "testCreateAndDropTablePartitionedHash1ByExecuteQuery";
 
@@ -430,7 +433,7 @@ public class TestTajoClient {
   }
 
   //@Test
-  public final void testCreateAndPurgeTablePartitionedHash1ByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndPurgeTablePartitionedHash1ByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = "testCreateAndPurgeTablePartitionedHash1ByExecuteQuery";
 
@@ -453,7 +456,7 @@ public class TestTajoClient {
   }
 
   //@Test
-  public final void testCreateAndDropTablePartitionedHash2ByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndDropTablePartitionedHash2ByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = "testCreateAndDropTablePartitionedHash2ByExecuteQuery";
 
@@ -476,7 +479,7 @@ public class TestTajoClient {
   }
 
   //@Test
-  public final void testCreateAndDropTablePartitionedListByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndDropTablePartitionedListByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = "testCreateAndDropTablePartitionedListByExecuteQuery";
 
@@ -500,7 +503,7 @@ public class TestTajoClient {
   }
 
   //@Test
-  public final void testCreateAndDropTablePartitionedRangeByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndDropTablePartitionedRangeByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = "testCreateAndDropTablePartitionedRangeByExecuteQuery";
 
@@ -525,7 +528,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testFailCreateTablePartitionedOtherExceptColumn() throws IOException, SQLException {
+  public final void testFailCreateTablePartitionedOtherExceptColumn() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = "testFailCreateTablePartitionedOtherExceptColumn";
 
@@ -540,8 +543,7 @@ public class TestTajoClient {
     try {
       client.updateQuery(rangeSql);
       fail();
-    } catch (SQLException se) {
-      assertEquals(Errors.ResultCode.FEATURE_NOT_SUPPORTED.getNumber(), se.getErrorCode());
+    } catch (UnsupportedException se) {
     }
  
     String listSql = "create table " + tableName + " (deptname text, score int4)";
@@ -552,8 +554,7 @@ public class TestTajoClient {
     try {
       assertFalse(client.updateQuery(listSql));
       fail();
-    } catch (SQLException se) {
-      assertEquals(Errors.ResultCode.FEATURE_NOT_SUPPORTED.getNumber(), se.getErrorCode());
+    } catch (UnsupportedException se) {
     }
 
     String hashSql = "create table " + tableName + " (deptname text, score int4)";
@@ -563,13 +564,12 @@ public class TestTajoClient {
     try {
       assertFalse(client.updateQuery(hashSql));
       fail();
-    } catch (SQLException se) {
-      assertEquals(Errors.ResultCode.FEATURE_NOT_SUPPORTED.getNumber(), se.getErrorCode());
+    } catch (UnsupportedException se) {
     }
   }
 
   @Test
-  public final void testCreateAndDropTablePartitionedColumnByExecuteQuery() throws IOException, SQLException {
+  public final void testCreateAndDropTablePartitionedColumnByExecuteQuery() throws IOException, TajoException {
     TajoConf conf = cluster.getConfiguration();
     final String tableName = CatalogUtil.normalizeIdentifier("testCreateAndDropTablePartitionedColumnByExecuteQuery");
 
@@ -591,7 +591,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testGetFunctions() throws IOException, SQLException {
+  public final void testGetFunctions() throws IOException, TajoException {
     Collection<FunctionDesc> catalogFunctions = cluster.getMaster().getCatalog().getFunctions();
     String functionName = "sum";
     int numFunctions = 0;
@@ -612,7 +612,7 @@ public class TestTajoClient {
   }
 
   @Test
-  public final void testGetFinishedQueryList() throws IOException, SQLException {
+  public final void testGetFinishedQueryList() throws SQLException, TajoException {
     final String tableName = CatalogUtil.normalizeIdentifier("testGetFinishedQueryList");
     String sql = "create table " + tableName + " (deptname text, score int4)";
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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
index d43f61c..b745caa 100644
--- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClientFailures.java
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClientFailures.java
@@ -21,7 +21,11 @@ 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;
@@ -48,18 +52,18 @@ public class TestTajoClientFailures {
     client.close();
   }
 
-  @Test
-  public final void testCreateDatabase() throws SQLException {
+  @Test(expected = DuplicateDatabaseException.class)
+  public final void testCreateDatabase() throws TajoException {
     assertFalse(client.createDatabase("default")); // duplicate database
   }
 
-  @Test
-  public final void testDropDatabase() throws SQLException {
+  @Test(expected = UndefinedDatabaseException.class)
+  public final void testDropDatabase() throws TajoException {
     assertFalse(client.dropDatabase("unknown-database")); // unknown database
   }
 
-  @Test
-  public final void testDropTable() throws SQLException {
+  @Test(expected = UndefinedTableException.class)
+  public final void testDropTable() throws UndefinedTableException {
     assertFalse(client.dropTable("unknown-table")); // unknown table
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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
new file mode 100644
index 0000000..99b7c15
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/client/v2/TestTajoClientV2.java
@@ -0,0 +1,236 @@
+/**
+ * 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.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.service.ServiceTracker;
+import org.apache.tajo.service.ServiceTrackerFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.junit.Assert.*;
+
+public class TestTajoClientV2 extends QueryTestCaseBase {
+  private static TajoClient clientv2;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = testingCluster.getConfiguration();
+
+    clientv2 = new TajoClient(new ServiceDiscovery() {
+      ServiceTracker tracker = ServiceTrackerFactory.get(conf);
+      @Override
+      public InetSocketAddress clientAddress() {
+        return tracker.getClientServiceAddress();
+      }
+    });
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    clientv2.close();
+  }
+
+  @Test
+  public void testExecuteUpdate() throws TajoException {
+    clientv2.executeUpdate("create database tajoclientv2");
+    clientv2.selectDB("tajoclientv2");
+    clientv2.selectDB("default");
+    clientv2.executeUpdate("drop database tajoclientv2");
+
+    try {
+      clientv2.selectDB("tajoclientv2");
+      fail();
+    } catch (UndefinedDatabaseException e) {
+    }
+  }
+
+  @Test
+  public void testExecuteQueryType1() throws TajoException, IOException, SQLException {
+    ResultSet res = null;
+    try {
+      res = clientv2.executeQuery("select * from lineitem");
+      assertResultSet(res);
+    } finally {
+      if (res != null) {
+        res.close();
+      }
+    }
+  }
+
+  @Test
+  public void testExecuteQueryType2() throws TajoException, IOException, SQLException {
+    ResultSet res = null;
+    try {
+      res = clientv2.executeQuery("select * from lineitem where l_orderkey > 2");
+      assertResultSet(res);
+    } finally {
+      if (res != null) {
+        res.close();
+      }
+    }
+  }
+
+  @Test
+  public void testExecuteQueryType3() throws TajoException, IOException, SQLException {
+    ResultSet res = null;
+    try {
+      clientv2.executeUpdate("create database client_v2_type3");
+      clientv2.selectDB("client_v2_type3");
+      clientv2.executeUpdate("create table t1 (c1 int)");
+      clientv2.executeUpdate("create table t2 (c2 int)");
+
+      // why we shouldn't use join directly on virtual tables? Currently, join on virtual tables is not supported.
+      res = clientv2.executeQuery("select db_id from information_schema.databases where db_name = 'client_v2_type3'");
+      assertTrue(res.next());
+      int dbId = res.getInt(1);
+      res.close();
+
+      res = clientv2.executeQuery(
+          "select table_name from information_schema.tables where db_id = " + dbId + " order by table_name");
+      assertResultSet(res);
+    } finally {
+      if (res != null) {
+        res.close();
+      }
+
+      clientv2.executeUpdate("drop database IF EXISTS client_v2_types3");
+    }
+  }
+
+  @Test
+  public void testExecuteQueryAsync() throws TajoException, IOException, SQLException, ExecutionException,
+      InterruptedException {
+    QueryFuture future = clientv2.executeQueryAsync("select * from lineitem where l_orderkey > 0");
+
+    ResultSet result = future.get();
+    assertResultSet(result);
+
+    assertTrue(future.isDone());
+    assertEquals(QueryState.COMPLETED, future.state());
+    assertTrue(future.isSuccessful());
+    assertFalse(future.isFailed());
+    assertFalse(future.isKilled());
+    assertTrue(1.0f == future.progress());
+    assertEquals("default", future.queue());
+
+    assertTrue(future.submitTime() > 0);
+    assertTrue(future.startTime() > 0);
+    assertTrue(future.finishTime() > 0);
+
+    result.close();
+  }
+
+  @Test(timeout = 10 * 1000)
+  public void testExecuteQueryAsyncWithListener() throws TajoException, IOException, SQLException, ExecutionException,
+      InterruptedException {
+    QueryFuture future = clientv2.executeQueryAsync(
+        "select l_orderkey, sleep(1) from lineitem where l_orderkey > 3");
+
+    final AtomicBoolean success = new AtomicBoolean(false);
+    final List<ResultSet> resultContainer = Lists.newArrayList();
+
+    future.addListener(new FutureListener<QueryFuture>() {
+      @Override
+      public void processingCompleted(QueryFuture future) {
+        try {
+          ResultSet result = future.get();
+          resultContainer.add(result); // for better error handling, it should be verified outside this future.
+
+          assertTrue(future.isDone());
+          assertEquals(QueryState.COMPLETED, future.state());
+          assertTrue(future.isSuccessful());
+          assertFalse(future.isFailed());
+          assertFalse(future.isKilled());
+          assertTrue(1.0f == future.progress());
+          assertEquals("default", future.queue());
+
+          assertTrue(future.submitTime() > 0);
+          assertTrue(future.startTime() > 0);
+          assertTrue(future.finishTime() > 0);
+
+          success.set(true);
+
+        } catch (Throwable t) {
+          throw new RuntimeException(t);
+        }
+      }
+    });
+
+    while(!future.isDone()) {
+      Thread.sleep(100);
+    }
+
+    assertTrue(success.get());
+    assertResultSet(resultContainer.get(0));
+    resultContainer.get(0).close();
+  }
+
+  @Test(timeout = 10 * 1000)
+  public void testQueryFutureKill() throws TajoException, ExecutionException, InterruptedException, SQLException {
+    QueryFuture future = clientv2.executeQueryAsync("select sleep(1) from lineitem where l_orderkey > 4");
+
+    assertTrue(future.isOk());
+    assertFalse(future.isDone());
+    assertFalse(future.isSuccessful());
+    assertFalse(future.isFailed());
+    assertFalse(future.isKilled());
+
+    future.kill();
+    while(!future.isDone()) {
+      Thread.sleep(100);
+    }
+
+    assertTrue(future.isOk());
+    assertTrue(future.isDone());
+    assertFalse(future.isSuccessful());
+    assertFalse(future.isFailed());
+    assertTrue(future.isKilled());
+  }
+
+
+  @Test(expected = DuplicateDatabaseException.class)
+  public void testErrorOnExecuteUpdate() throws TajoException, IOException, SQLException {
+    clientv2.executeUpdate("create database default");
+  }
+
+  @Test(expected = UndefinedTableException.class)
+  public void testErrorOnExecuteQuery() throws TajoException, IOException, SQLException {
+    clientv2.executeQuery("select * from unknown_table");
+  }
+
+  @Test(expected = UndefinedTableException.class)
+  public void testErrorOnExecuteQueryAsync() throws TajoException {
+    clientv2.executeQueryAsync("select * from unknown_table");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java b/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java
index d86081a..da59e8a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/codegen/TestEvalCodeGenerator.java
@@ -23,10 +23,9 @@ import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.engine.eval.ExprTestBase;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
-import java.io.IOException;
-
 public class TestEvalCodeGenerator extends ExprTestBase {
   private static Schema schema;
   static {
@@ -44,7 +43,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testArithmetic() throws IOException {
+  public void testArithmetic() throws TajoException {
     testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 1+1;", new String [] {"2"});
     testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col2 from table1;", new String [] {"3"});
     testEval(schema, "table1", "0,1,2,3,4.5,5.5", "select col1 + col3 from table1;", new String [] {"4"});
@@ -53,7 +52,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testGetField() throws IOException {
+  public void testGetField() throws TajoException {
     testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col1 from table1;", new String [] {"1"});
     testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col2 from table1;", new String [] {"2"});
     testEval(schema, "table1", "0,1,2,3,4.5,5.5,F6", "select col3 from table1;", new String [] {"3"});
@@ -64,7 +63,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testNullHandling() throws IOException {
+  public void testNullHandling() throws TajoException {
     schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -104,7 +103,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testComparison() throws IOException {
+  public void testComparison() throws TajoException {
     Schema inetSchema = new Schema();
     inetSchema.addColumn("addr1", TajoDataTypes.Type.INET4);
     inetSchema.addColumn("addr2", TajoDataTypes.Type.INET4);
@@ -160,7 +159,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testBetweenAsymmetric() throws IOException {
+  public void testBetweenAsymmetric() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", TajoDataTypes.Type.INT4);
     schema.addColumn("col2", TajoDataTypes.Type.INT4);
@@ -194,7 +193,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testBetweenSymmetric() throws IOException {
+  public void testBetweenSymmetric() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", TajoDataTypes.Type.INT4);
     schema.addColumn("col2", TajoDataTypes.Type.INT4);
@@ -229,7 +228,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testUnary() throws IOException {
+  public void testUnary() throws TajoException {
     schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -266,7 +265,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testAndOr() throws IOException {
+  public void testAndOr() throws TajoException {
     testSimpleEval("select true or (false or false) or false;", new String[] {"t"});
 
     testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select true and true;", new String [] {"t"});
@@ -289,7 +288,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testFunction() throws IOException {
+  public void testFunction() throws TajoException {
     testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('abc');", new String [] {"ABC"});
     testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('bbc');", new String [] {"BBC"});
     testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select upper('chs');", new String [] {"CHS"});
@@ -298,7 +297,7 @@ public class TestEvalCodeGenerator extends ExprTestBase {
   }
 
   @Test
-  public void testStringConcat() throws IOException {
+  public void testStringConcat() throws TajoException {
     testSimpleEval("select length('123456') as col1 ", new String[]{"6"});
 
     testEval(schema, "table1", "0,1,2,3,4.5,6.5", "select 'abc' || 'bbc'", new String [] {"abcbbc"});

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index abd0973..6fe1510 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -37,6 +37,7 @@ import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.TajoInternalError;
 import org.apache.tajo.function.FunctionSignature;
 import org.apache.tajo.master.exec.QueryExecutor;
 import org.apache.tajo.plan.*;
@@ -44,6 +45,7 @@ import org.apache.tajo.plan.expr.EvalContext;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.plan.serder.EvalNodeDeserializer;
 import org.apache.tajo.plan.serder.EvalNodeSerializer;
+import org.apache.tajo.plan.serder.PlanProto;
 import org.apache.tajo.plan.verifier.LogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.VerificationState;
@@ -57,7 +59,6 @@ import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.apache.tajo.plan.serder.PlanProto;
 
 import java.io.IOException;
 import java.util.List;
@@ -176,46 +177,46 @@ public class ExprTestBase {
     return targets;
   }
 
-  public void testSimpleEval(String query, String [] expected) throws IOException {
+  public void testSimpleEval(String query, String [] expected) throws TajoException {
     testEval(null, null, null, query, expected);
   }
 
-  public void testSimpleEval(OverridableConf context, String query, String [] expected) throws IOException {
+  public void testSimpleEval(OverridableConf context, String query, String [] expected) throws TajoException {
     testEval(context, null, null, null, query, expected);
   }
 
   public void testSimpleEval(String query, String [] expected, boolean successOrFail)
-      throws IOException {
+      throws TajoException, IOException {
 
     testEval(null, null, null, null, query, expected, ',', successOrFail);
   }
 
   public void testSimpleEval(OverridableConf context, String query, String [] expected, boolean successOrFail)
-      throws IOException {
+      throws TajoException, IOException {
     testEval(context, null, null, null, query, expected, ',', successOrFail);
   }
 
   public void testEval(Schema schema, String tableName, String csvTuple, String query, String [] expected)
-      throws IOException {
+      throws TajoException {
     testEval(null, schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple, query,
         expected, ',', true);
   }
 
   public void testEval(OverridableConf context, Schema schema, String tableName, String csvTuple, String query,
                        String [] expected)
-      throws IOException {
+      throws TajoException {
     testEval(context, schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple,
         query, expected, ',', true);
   }
 
   public void testEval(Schema schema, String tableName, String csvTuple, String query,
-                       String [] expected, char delimiter, boolean condition) throws IOException {
+                       String [] expected, char delimiter, boolean condition) throws TajoException {
     testEval(null, schema, tableName != null ? CatalogUtil.normalizeIdentifier(tableName) : null, csvTuple,
         query, expected, delimiter, condition);
   }
 
   public void testEval(OverridableConf context, Schema schema, String tableName, String csvTuple, String query,
-                       String [] expected, char delimiter, boolean condition) throws IOException {
+                       String [] expected, char delimiter, boolean condition) throws TajoException {
     QueryContext queryContext;
     if (context == null) {
       queryContext = LocalTajoTestingUtility.createDummyContext(conf);
@@ -262,8 +263,12 @@ public class ExprTestBase {
           vtuple.put(i, lazyTuple.get(i));
         }
       }
-      cat.createTable(new TableDesc(qualifiedTableName, inputSchema,"TEXT",
-          new KeyValueSet(), CommonTestingUtil.getTestDir().toUri()));
+      try {
+        cat.createTable(new TableDesc(qualifiedTableName, inputSchema,"TEXT",
+            new KeyValueSet(), CommonTestingUtil.getTestDir().toUri()));
+      } catch (IOException e) {
+        throw new TajoInternalError(e);
+      }
     }
 
     Target [] targets;
@@ -309,6 +314,8 @@ public class ExprTestBase {
         }
         assertEquals(query, expected[i], outTupleAsChars);
       }
+    } catch (IOException e) {
+      throw new TajoInternalError(e);
     } catch (InvalidStatementException e) {
       assertFalse(e.getMessage(), true);
     } catch (TajoException e) {
@@ -317,7 +324,7 @@ public class ExprTestBase {
       if (!condition) {
         assertEquals(expected[0], e.getMessage());
       } else {
-        assertFalse(e.getMessage(), true);
+        throw e;
       }
     } finally {
       if (schema != null) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
index cde370d..9f9d294 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java
@@ -19,15 +19,14 @@
 package org.apache.tajo.engine.eval;
 
 import org.apache.tajo.exception.InvalidOperationException;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
-import java.io.IOException;
-
 import static org.junit.Assert.fail;
 
 public class TestIntervalType extends ExprTestBase {
   @Test
-  public void testIntervalPostgresqlCase() throws IOException {
+  public void testIntervalPostgresqlCase() throws TajoException {
 
     // http://www.postgresql.org/docs/8.2/static/functions-datetime.html
     testSimpleEval("select date '2001-09-28' + 7", new String[]{"2001-10-05"});

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
index 94d5e71..6c42c3e 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestPredicates.java
@@ -21,6 +21,7 @@ package org.apache.tajo.engine.eval;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -36,7 +37,7 @@ public class TestPredicates extends ExprTestBase {
   //////////////////////////////////////////////////////////////////
 
   @Test
-  public void testAnd() throws IOException {
+  public void testAnd() throws TajoException {
     testSimpleEval("select true;", new String[] {"t"});
 
     testSimpleEval("select true and true;", new String[] {"t"});
@@ -46,7 +47,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testOr() throws IOException {
+  public void testOr() throws TajoException {
     testSimpleEval("select true or true;", new String[] {"t"});
     testSimpleEval("select true or false;", new String[] {"t"});
     testSimpleEval("select false or true;", new String[] {"t"});
@@ -54,7 +55,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testLogicalOperatorPrecedence() throws IOException {
+  public void testLogicalOperatorPrecedence() throws TajoException {
     testSimpleEval("select true or (false or false) or false;", new String[] {"t"});
     testSimpleEval("select false or (true or false) or false;", new String[] {"t"});
     testSimpleEval("select false or (false or true) or false;", new String[] {"t"});
@@ -77,7 +78,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testNot() throws IOException {
+  public void testNot() throws TajoException {
 
     testSimpleEval("select true;", new String[] {"t"});
     testSimpleEval("select not true;", new String[] {"f"});
@@ -99,7 +100,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testParenthesizedValues() throws IOException {
+  public void testParenthesizedValues() throws TajoException {
     testSimpleEval("select ((true));", new String[] {"t"});
     testSimpleEval("select ((((true))));", new String[] {"t"});
     testSimpleEval("select not(not(not(false)));", new String[] {"t"});
@@ -110,7 +111,7 @@ public class TestPredicates extends ExprTestBase {
   //////////////////////////////////////////////////////////////////
 
   @Test
-  public void testComparisonEqual() throws IOException {
+  public void testComparisonEqual() throws TajoException {
 
 
     Schema schema = new Schema();
@@ -142,7 +143,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testComparisonNotEqual() throws IOException {
+  public void testComparisonNotEqual() throws TajoException {
     Schema schema1 = new Schema();
     schema1.addColumn("col1", INT4);
     schema1.addColumn("col2", INT4);
@@ -159,7 +160,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testComparisonLessThan() throws IOException {
+  public void testComparisonLessThan() throws TajoException {
     Schema schema1 = new Schema();
     schema1.addColumn("col1", INT4);
     schema1.addColumn("col2", INT4);
@@ -176,7 +177,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testComparisonLessThanEqual() throws IOException {
+  public void testComparisonLessThanEqual() throws TajoException {
     Schema schema1 = new Schema();
     schema1.addColumn("col1", INT4);
     schema1.addColumn("col2", INT4);
@@ -193,7 +194,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testComparisonGreaterThan() throws IOException {
+  public void testComparisonGreaterThan() throws TajoException {
     Schema schema1 = new Schema();
     schema1.addColumn("col1", INT4);
     schema1.addColumn("col2", INT4);
@@ -210,7 +211,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testComparisonGreaterThanEqual() throws IOException {
+  public void testComparisonGreaterThanEqual() throws TajoException {
     Schema schema1 = new Schema();
     schema1.addColumn("col1", INT4);
     schema1.addColumn("col2", INT4);
@@ -231,7 +232,7 @@ public class TestPredicates extends ExprTestBase {
   //////////////////////////////////////////////////////////////////
 
   @Test
-  public void testBetween() throws IOException {
+  public void testBetween() throws TajoException {
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TEXT);
     schema2.addColumn("col2", TEXT);
@@ -255,7 +256,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testBetween2() throws IOException { // for TAJO-249
+  public void testBetween2() throws TajoException { // for TAJO-249
     Schema schema3 = new Schema();
     schema3.addColumn("date_a", INT4);
     schema3.addColumn("date_b", INT4);
@@ -294,7 +295,7 @@ public class TestPredicates extends ExprTestBase {
   //////////////////////////////////////////////////////////////////
 
   @Test
-  public void testInPredicateWithConstant() throws IOException {
+  public void testInPredicateWithConstant() throws TajoException {
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TEXT);
     schema2.addColumn("col2", TEXT);
@@ -319,7 +320,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testInPredicateWithSimpleExprs() throws IOException {
+  public void testInPredicateWithSimpleExprs() throws TajoException {
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TEXT);
     schema2.addColumn("col2", INT4);
@@ -343,7 +344,7 @@ public class TestPredicates extends ExprTestBase {
   //////////////////////////////////////////////////////////////////
 
   @Test
-  public void testIsNullPredicate() throws IOException {
+  public void testIsNullPredicate() throws TajoException {
     Schema schema1 = new Schema();
     schema1.addColumn("col1", INT4);
     schema1.addColumn("col2", INT4);
@@ -354,7 +355,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testIsNullPredicateWithFunction() throws IOException {
+  public void testIsNullPredicateWithFunction() throws TajoException {
     Schema schema2 = new Schema();
     schema2.addColumn("col1", TEXT);
     schema2.addColumn("col2", TEXT);
@@ -370,7 +371,7 @@ public class TestPredicates extends ExprTestBase {
   //////////////////////////////////////////////////////////////////
 
   @Test
-  public void testBooleanTest() throws IOException {
+  public void testBooleanTest() throws TajoException {
     testSimpleEval("select 1 < 3 is true", new String [] {"t"});
     testSimpleEval("select 1 < 3 is not true", new String [] {"f"});
     testSimpleEval("select 1 < 3 is false", new String [] {"f"});
@@ -393,7 +394,7 @@ public class TestPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testBooleanTestOnTable() throws IOException {
+  public void testBooleanTestOnTable() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", BOOLEAN);
     schema.addColumn("col2", BOOLEAN);

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
index fc74339..1b599a6 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestSQLDateTimeTypes.java
@@ -18,14 +18,13 @@
 
 package org.apache.tajo.engine.eval;
 
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
-import java.io.IOException;
-
 public class TestSQLDateTimeTypes extends ExprTestBase {
 
   @Test
-  public void testTimestamp() throws IOException {
+  public void testTimestamp() throws TajoException {
     testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37';", new String[]{"1970-01-17 10:09:37"});
     testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.5';", new String[]{"1970-01-17 10:09:37.5"});
     testSimpleEval("select TIMESTAMP '1970-01-17 10:09:37.01';", new String[]{"1970-01-17 10:09:37.01"});
@@ -33,18 +32,18 @@ public class TestSQLDateTimeTypes extends ExprTestBase {
   }
 
   @Test
-  public void testToTimestamp() throws IOException {
+  public void testToTimestamp() throws TajoException {
     testSimpleEval("select to_char(TIMESTAMP '1970-01-17 10:09:37', 'YYYY-MM-DD HH24:MI:SS');",
         new String[]{"1970-01-17 10:09:37"});
   }
 
   @Test
-  public void testTimeLiteral() throws IOException {
+  public void testTimeLiteral() throws TajoException {
     testSimpleEval("select TIME '10:09:37';", new String[]{"10:09:37"});
   }
 
   @Test
-  public void testDateLiteral() throws IOException {
+  public void testDateLiteral() throws TajoException {
     testSimpleEval("select DATE '1970-01-17';", new String[]{"1970-01-17"});
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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 8d81ae8..684f0f2 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
@@ -26,10 +26,10 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.TimestampDatum;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.junit.Test;
 
-import java.io.IOException;
 import java.util.TimeZone;
 
 import static org.apache.tajo.common.TajoDataTypes.Type.*;
@@ -38,7 +38,7 @@ import static org.junit.Assert.fail;
 public class TestSQLExpression extends ExprTestBase {
 
   @Test
-  public void testQuotedIdentifiers() throws IOException {
+  public void testQuotedIdentifiers() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("컬럼1", TEXT);
     schema.addColumn("컬럼2", TEXT);
@@ -50,7 +50,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testNoSuchFunction() throws IOException {
+  public void testNoSuchFunction() {
     try {
       testSimpleEval("select test123('abc') col1 ", new String[]{"abc"});
       fail("This test should throw UndefinedFunctionException");
@@ -62,7 +62,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testSQLStandardCast() throws IOException {
+  public void testSQLStandardCast() throws TajoException {
     testSimpleEval("select cast (1 as char)", new String[] {"1"});
     testSimpleEval("select cast (119 as char)", new String[] {"1"});
 
@@ -92,7 +92,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testExplicitCast() throws IOException {
+  public void testExplicitCast() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", INT1);
     schema.addColumn("col1", INT2);
@@ -172,7 +172,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testImplicitCastForInt1() throws IOException {
+  public void testImplicitCastForInt1() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -274,7 +274,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testImplicitCastForInt2() throws IOException {
+  public void testImplicitCastForInt2() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -376,7 +376,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testImplicitCastForInt4() throws IOException {
+  public void testImplicitCastForInt4() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -479,7 +479,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testImplicitCastForInt8() throws IOException {
+  public void testImplicitCastForInt8() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -586,7 +586,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testImplicitCastForFloat4() throws IOException {
+  public void testImplicitCastForFloat4() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -705,7 +705,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testImplicitCastForFloat8() throws IOException {
+  public void testImplicitCastForFloat8() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -825,7 +825,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testSigned() throws IOException {
+  public void testSigned() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col0", TajoDataTypes.Type.INT1);
     schema.addColumn("col1", TajoDataTypes.Type.INT2);
@@ -853,7 +853,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testCastWithNestedFunction() throws IOException {
+  public void testCastWithNestedFunction() throws TajoException {
     QueryContext context = new QueryContext(getConf());
     context.put(SessionVars.TIMEZONE, "GMT-6");
     TimeZone tz = TimeZone.getTimeZone("GMT-6");
@@ -865,7 +865,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testCastFromTable() throws IOException {
+  public void testCastFromTable() throws TajoException {
     QueryContext queryContext = new QueryContext(getConf());
     queryContext.put(SessionVars.TIMEZONE, "GMT-6");
     TimeZone tz = TimeZone.getTimeZone("GMT-6");
@@ -898,7 +898,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testBooleanLiteral() throws IOException {
+  public void testBooleanLiteral() throws TajoException {
     testSimpleEval("select true", new String[] {"t"});
     testSimpleEval("select false", new String[]{"f"});
 
@@ -909,7 +909,7 @@ public class TestSQLExpression extends ExprTestBase {
   }
 
   @Test
-  public void testNullComparisons() throws IOException {
+  public void testNullComparisons() throws TajoException {
     testSimpleEval("select (1 > null) is null", new String[] {"t"});
 
     testSimpleEval("select null is null", new String[] {"t"});

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/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 1d35139..7e63bc1 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
@@ -23,15 +23,14 @@ import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.exception.UndefinedFunctionException;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.engine.eval.ExprTestBase;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
-import java.io.IOException;
-
 import static org.junit.Assert.fail;
 
 public class TestConditionalExpressions extends ExprTestBase {
   @Test
-  public void testCaseWhens1() throws IOException {
+  public void testCaseWhens1() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", TajoDataTypes.Type.INT1);
     schema.addColumn("col2", TajoDataTypes.Type.INT2);
@@ -58,7 +57,7 @@ public class TestConditionalExpressions extends ExprTestBase {
   }
 
   @Test
-  public void testCaseWhensWithNullReturn() throws IOException {
+  public void testCaseWhensWithNullReturn() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", TajoDataTypes.Type.TEXT);
     schema.addColumn("col2", TajoDataTypes.Type.TEXT);
@@ -72,7 +71,7 @@ public class TestConditionalExpressions extends ExprTestBase {
   }
 
   @Test
-  public void testCaseWhensWithCommonExpression() throws IOException {
+  public void testCaseWhensWithCommonExpression() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", TajoDataTypes.Type.INT4);
     schema.addColumn("col2", TajoDataTypes.Type.INT4);
@@ -110,7 +109,7 @@ public class TestConditionalExpressions extends ExprTestBase {
   }
 
   @Test
-  public void testCaseWhensWithCommonExpressionAndNull() throws IOException {
+  public void testCaseWhensWithCommonExpressionAndNull() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", TajoDataTypes.Type.INT4);
     schema.addColumn("col2", TajoDataTypes.Type.INT4);

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
index 25a10fd..9dd8653 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java
@@ -25,21 +25,20 @@ import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.TimestampDatum;
 import org.apache.tajo.engine.eval.ExprTestBase;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.apache.tajo.util.datetime.TimeMeta;
 import org.junit.Test;
 
-import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.TimeZone;
 
 import static org.apache.tajo.common.TajoDataTypes.Type.*;
-import static org.junit.Assert.assertEquals;
 
 public class TestDateTimeFunctions extends ExprTestBase {
   @Test
-  public void testToTimestamp() throws IOException {
+  public void testToTimestamp() throws TajoException {
     long expectedTimestamp = System.currentTimeMillis();
     TimestampDatum expected = DatumFactory.createTimestmpDatumWithUnixTime((int)(expectedTimestamp/ 1000));
 
@@ -100,7 +99,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testToChar() throws IOException {
+  public void testToChar() throws TajoException {
     long expectedTimestamp = System.currentTimeMillis();
     TimeMeta tm = new TimeMeta();
     DateTimeUtil.toJulianTimeMeta(DateTimeUtil.javaTimeToJulianTime(expectedTimestamp), tm);
@@ -116,7 +115,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testExtract() throws IOException {
+  public void testExtract() throws TajoException {
     TimeZone GMT = TimeZone.getTimeZone("GMT");
     TimeZone PST = TimeZone.getTimeZone("PST");
 
@@ -232,7 +231,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testDatePart() throws IOException {
+  public void testDatePart() throws TajoException {
     TimeZone GMT = TimeZone.getTimeZone("GMT");
     TimeZone PST = TimeZone.getTimeZone("PST");
 
@@ -345,7 +344,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testUtcUsecTo() throws IOException {
+  public void testUtcUsecTo() throws TajoException {
      testSimpleEval("select utc_usec_to('day' ,1274259481071200);", new String[]{1274227200000000L+""});
      testSimpleEval("select utc_usec_to('hour' ,1274259481071200);", new String[]{1274256000000000L+""});
      testSimpleEval("select utc_usec_to('month' ,1274259481071200);", new String[]{1272672000000000L+""});
@@ -354,7 +353,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testToDate() throws IOException {
+  public void testToDate() throws TajoException {
     testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD')", new String[]{"2014-01-04"});
     testSimpleEval("select to_date('2014-01-04', 'YYYY-MM-DD') + interval '1 day'",
         new String[]{"2014-01-05 00:00:00"});
@@ -394,7 +393,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testAddDays() throws IOException {
+  public void testAddDays() throws TajoException {
     testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT2);",
         new String[]{"2014-01-04 00:00:00"});
     testSimpleEval("SELECT add_days(date '2013-12-30', 5::INT4);",
@@ -425,7 +424,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testDateTimeNow() throws IOException {
+  public void testDateTimeNow() throws TajoException {
     TimeZone originalTimezone = TimeZone.getDefault();
     TimeZone.setDefault(TimeZone.getTimeZone("GMT-6"));
 
@@ -454,7 +453,7 @@ public class TestDateTimeFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testTimeValueKeyword() throws IOException {
+  public void testTimeValueKeyword() throws TajoException {
     TimeZone originTimeZone = TimeZone.getDefault();
     TimeZone.setDefault(TimeZone.getTimeZone("GMT-6"));
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/function/TestJsonFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestJsonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestJsonFunctions.java
index 89f0439..57248a8 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestJsonFunctions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestJsonFunctions.java
@@ -20,15 +20,14 @@ package org.apache.tajo.engine.function;
 
 
 import org.apache.tajo.engine.eval.ExprTestBase;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
-import java.io.IOException;
-
 public class TestJsonFunctions extends ExprTestBase {
   static final String JSON_DOCUMENT = "{\"map\" : {\"name\" : \"tajo\"}, \"array\" : [1,2,3]}";
 
   @Test
-  public void testJsonExtractPathText() throws IOException {
+  public void testJsonExtractPathText() throws TajoException {
     testSimpleEval("select json_extract_path_text('" + JSON_DOCUMENT + "', '$.map.name') ", new String[]{"tajo"});
     testSimpleEval("select json_extract_path_text('" + JSON_DOCUMENT + "', '$.array[1]') ", new String[]{"2"});
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
index 78509f7..4b0303f 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java
@@ -20,15 +20,14 @@ package org.apache.tajo.engine.function;
 
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.engine.eval.ExprTestBase;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
-import java.io.IOException;
-
 import static org.apache.tajo.common.TajoDataTypes.Type.*;
 
 public class TestMathFunctions extends ExprTestBase {
   @Test
-  public void testRound() throws IOException {
+  public void testRound() throws TajoException {
     testSimpleEval("select round(5.1) as col1 ", new String[]{"5"});
     testSimpleEval("select round(5.5) as col1 ", new String[]{"6"});
     testSimpleEval("select round(5.6) as col1 ", new String[]{"6"});
@@ -57,7 +56,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testFloor() throws IOException {
+  public void testFloor() throws TajoException {
     testSimpleEval("select floor(5.1) as col1 ", new String[]{"5"});
     testSimpleEval("select floor(5.5) as col1 ", new String[]{"5"});
     testSimpleEval("select floor(5.6) as col1 ", new String[]{"5"});
@@ -75,7 +74,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testCeil() throws IOException {
+  public void testCeil() throws TajoException {
     testSimpleEval("select ceil(5.0) as col1 ", new String[]{"5"});
     testSimpleEval("select ceil(5.1) as col1 ", new String[]{"6"});
     testSimpleEval("select ceil(5.5) as col1 ", new String[]{"6"});
@@ -94,7 +93,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testCeiling() throws IOException {
+  public void testCeiling() throws TajoException {
     testSimpleEval("select ceiling(5.0) as col1 ", new String[]{"5"});
     testSimpleEval("select ceiling(5.1) as col1 ", new String[]{"6"});
     testSimpleEval("select ceiling(5.5) as col1 ", new String[]{"6"});
@@ -113,7 +112,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testSin() throws IOException {
+  public void testSin() throws TajoException {
     testSimpleEval("select sin(0.0) as col1 ", new String[]{"0.0"});
     testSimpleEval("select sin(0.7) as col1 ", new String[]{"0.644217687237691"});
     testSimpleEval("select sin(1.2) as col1 ", new String[]{"0.9320390859672263"});
@@ -130,7 +129,7 @@ public class TestMathFunctions extends ExprTestBase {
 
 
   @Test
-  public void testCos() throws IOException {
+  public void testCos() throws TajoException {
     testSimpleEval("select cos(0.0) as col1 ", new String[]{"1.0"});
     testSimpleEval("select cos(0.7) as col1 ", new String[]{"0.7648421872844885"});
     testSimpleEval("select cos(1.2) as col1 ", new String[]{"0.3623577544766736"});
@@ -146,7 +145,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testTan() throws IOException {
+  public void testTan() throws TajoException {
     testSimpleEval("select tan(0.0) as col1 ", new String[]{"0.0"});
     testSimpleEval("select tan(0.3) as col1 ", new String[]{"0.30933624960962325"});
     testSimpleEval("select tan(0.8) as col1 ", new String[]{"1.0296385570503641"});
@@ -162,7 +161,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testAsin() throws IOException {
+  public void testAsin() throws TajoException {
     testSimpleEval("select asin(0.0) as col1 ", new String[]{"0.0"});
     testSimpleEval("select asin(0.3) as col1 ", new String[]{"0.3046926540153975"});
     testSimpleEval("select asin(0.8) as col1 ", new String[]{"0.9272952180016123"});
@@ -178,7 +177,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testAcos() throws IOException {
+  public void testAcos() throws TajoException {
     testSimpleEval("select acos(0.0) as col1 ", new String[]{"1.5707963267948966"});
     testSimpleEval("select acos(0.3) as col1 ", new String[]{"1.2661036727794992"});
     testSimpleEval("select acos(0.8) as col1 ", new String[]{"0.6435011087932843"});
@@ -194,7 +193,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testAtan() throws IOException {
+  public void testAtan() throws TajoException {
     testSimpleEval("select atan(0.0) as col1 ", new String[]{"0.0"});
     testSimpleEval("select atan(0.8) as col1 ", new String[]{"0.6747409422235527"});
     testSimpleEval("select atan(1.2) as col1 ", new String[]{"0.8760580505981934"});
@@ -210,7 +209,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testAtan2() throws IOException {
+  public void testAtan2() throws TajoException {
     testSimpleEval("select atan2(0.8, 0.0) as col1 ", new String[]{"1.5707963267948966"});
     testSimpleEval("select atan2(0.8, 1.1) as col1 ", new String[]{"0.628796286415433"});
     testSimpleEval("select atan2(2.7, 0.3) as col1 ", new String[]{"1.460139105621001"});
@@ -227,7 +226,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testMod() throws IOException {
+  public void testMod() throws TajoException {
     testSimpleEval("select mod(9,4) as col1 ", new String[]{"1"});
     testSimpleEval("select mod(200000000001,200000000000) as col1 ", new String[]{"1"});
     testSimpleEval("select mod(200000000000,2) as col1 ", new String[]{"0"});
@@ -243,7 +242,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testDiv() throws IOException {
+  public void testDiv() throws TajoException {
     testSimpleEval("select div(9,4) as col1 ", new String[]{"2"});
     testSimpleEval("select div(200000000001,200000000000) as col1 ", new String[]{"1"});
     testSimpleEval("select div(200000000000,2) as col1 ", new String[]{"100000000000"});
@@ -259,7 +258,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testSign() throws IOException {
+  public void testSign() throws TajoException {
     testSimpleEval("select sign(2) as col1 ", new String[]{"1.0"});
     testSimpleEval("select sign(2.345) as col1 ", new String[]{"1.0"});
     testSimpleEval("select sign(0.3) as col1 ", new String[]{"1.0"});
@@ -284,7 +283,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testSqrt() throws IOException {
+  public void testSqrt() throws TajoException {
     testSimpleEval("select sqrt(27.0) as col1 ", new String[]{"5.196152422706632"});
     testSimpleEval("select sqrt(64.0) as col1 ", new String[]{"8.0"});
     testSimpleEval("select sqrt(8.0) as col1 ", new String[]{"2.8284271247461903"});
@@ -309,7 +308,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testExp() throws IOException {
+  public void testExp() throws TajoException {
     testSimpleEval("select exp(1.0) as col1 ", new String[]{String.valueOf(Math.exp(1.0d))});
     testSimpleEval("select exp(1.1) as col1 ", new String[]{String.valueOf(Math.exp(1.1d))});
     testSimpleEval("select exp(1.2) as col1 ", new String[]{String.valueOf(Math.exp(1.2d))});
@@ -330,7 +329,7 @@ public class TestMathFunctions extends ExprTestBase {
 
 
   @Test
-  public void testAbs() throws IOException {
+  public void testAbs() throws TajoException {
     testSimpleEval("select abs(9) as col1 ", new String[]{"9"});
     testSimpleEval("select abs(-9) as col1 ", new String[]{"9"});
     testSimpleEval("select abs(200000000000) as col1 ", new String[]{"200000000000"});
@@ -351,7 +350,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testCbrt() throws IOException {
+  public void testCbrt() throws TajoException {
     testSimpleEval("select cbrt(27.0) as col1 ", new String[]{"3.0"});
     testSimpleEval("select cbrt(64.0) as col1 ", new String[]{"4.0"});
     testSimpleEval("select cbrt(8.0) as col1 ", new String[]{"2.0"});
@@ -373,7 +372,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testDegrees() throws IOException {
+  public void testDegrees() throws TajoException {
     testSimpleEval("select degrees(0.0) as col1 ", new String[]{String.valueOf(Math.toDegrees(0.0))});
     testSimpleEval("select degrees(0.8) as col1 ", new String[]{String.valueOf(Math.toDegrees(0.8))});
     testSimpleEval("select degrees(2.7) as col1 ", new String[]{String.valueOf(Math.toDegrees(2.7))});
@@ -393,7 +392,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testPow() throws IOException {
+  public void testPow() throws TajoException {
     testSimpleEval("select pow(9,3) as col1 ", new String[]{String.valueOf(Math.pow(9, 3))});
     testSimpleEval("select pow(1.0,3) as col1 ", new String[]{String.valueOf(Math.pow(1.0, 3))});
     testSimpleEval("select pow(20.1,3.1) as col1 ", new String[]{String.valueOf(Math.pow(20.1, 3.1))});
@@ -414,7 +413,7 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testRadians() throws IOException {
+  public void testRadians() throws TajoException {
     testSimpleEval("select radians(0.0) as col1 ", new String[]{String.valueOf(Math.toRadians(0.0))});
     testSimpleEval("select radians(0.8) as col1 ", new String[]{String.valueOf(Math.toRadians(0.8))});
     testSimpleEval("select radians(2.7) as col1 ", new String[]{String.valueOf(Math.toRadians(2.7))});
@@ -434,12 +433,12 @@ public class TestMathFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testPi() throws IOException {
+  public void testPi() throws TajoException {
     testSimpleEval("select pi() as col1 ", new String[]{String.valueOf(Math.PI)});
   }
 
   @Test
-  public void testRoundWithSpecifiedPrecision() throws IOException {
+  public void testRoundWithSpecifiedPrecision() throws TajoException {
     // TODO - in order to make this test possible, testSimpleEval should take session variables. Now, we disable it.
     // divide zero
 //    try {

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java
index 8aae26d..50a8f35 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPatternMatchingPredicates.java
@@ -20,16 +20,15 @@ package org.apache.tajo.engine.function;
 
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.engine.eval.ExprTestBase;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
-import java.io.IOException;
-
 import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
 
 public class TestPatternMatchingPredicates extends ExprTestBase {
 
   @Test
-  public void testLike() throws IOException {
+  public void testLike() throws TajoException {
     Schema schema = new Schema();
     schema.addColumn("col1", TEXT);
 
@@ -58,7 +57,7 @@ public class TestPatternMatchingPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testILike() throws IOException {
+  public void testILike() throws TajoException {
     testSimpleEval("select 'abc' ilike '%c'", new String[]{"t"});
     testSimpleEval("select 'abc' ilike 'a%'", new String[]{"t"});
     testSimpleEval("select 'abc' ilike '_bc'", new String[]{"t"});
@@ -80,7 +79,7 @@ public class TestPatternMatchingPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testSimilarToLike() throws IOException {
+  public void testSimilarToLike() throws TajoException {
     testSimpleEval("select 'abc' similar to '%c'", new String[]{"t"});
     testSimpleEval("select 'abc' similar to 'a%'", new String[]{"t"});
     testSimpleEval("select 'abc' similar to '_bc'", new String[]{"t"});
@@ -104,7 +103,7 @@ public class TestPatternMatchingPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testRegexWithSimilarOperator() throws IOException {
+  public void testRegexWithSimilarOperator() throws TajoException {
     testSimpleEval("select 'abc' ~ '.*c'", new String[]{"t"});
     testSimpleEval("select 'abc' ~ '.*c$'", new String[]{"t"});
     testSimpleEval("select 'aaabc' ~ '([a-z]){3}bc'", new String[]{"t"});
@@ -121,7 +120,7 @@ public class TestPatternMatchingPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testRegexp() throws IOException {
+  public void testRegexp() throws TajoException {
     testSimpleEval("select 'abc' regexp '.*c'", new String[]{"t"});
     testSimpleEval("select 'abc' regexp '.*c$'", new String[]{"t"});
 
@@ -130,7 +129,7 @@ public class TestPatternMatchingPredicates extends ExprTestBase {
   }
 
   @Test
-  public void testRLike() throws IOException {
+  public void testRLike() throws TajoException {
     testSimpleEval("select 'abc' rlike '.*c'", new String[]{"t"});
     testSimpleEval("select 'abc' rlike '.*c$'", new String[]{"t"});
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/4253f1b6/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java
index 6f73d01..e94f869 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.engine.function;
 
 import org.apache.tajo.engine.eval.ExprTestBase;
+import org.apache.tajo.exception.TajoException;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -26,7 +27,7 @@ import java.io.IOException;
 public class TestPythonFunctions extends ExprTestBase {
 
   @Test
-  public void testFunctions() throws IOException {
+  public void testFunctions() throws TajoException {
     testSimpleEval("select return_one()", new String[]{"1"});
     testSimpleEval("select helloworld()", new String[]{"Hello, World"});
     testSimpleEval("select concat_py('1')", new String[]{"11"});
@@ -37,7 +38,7 @@ public class TestPythonFunctions extends ExprTestBase {
   }
 
   @Test
-  public void testNestedFunctions() throws IOException {
+  public void testNestedFunctions() throws TajoException {
     testSimpleEval("select add_py(3, return_one())", new String[]{"4"});
     testSimpleEval("select concat_py(helloworld())", new String[]{"Hello, WorldHello, World"});
   }