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/09/25 10:58:58 UTC

tajo git commit: TAJO-1890: Clean up debug and test modes and unhandled exceptions.

Repository: tajo
Updated Branches:
  refs/heads/master bf04b56c5 -> dfec6a07f


TAJO-1890: Clean up debug and test modes and unhandled exceptions.

Closes #1890


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

Branch: refs/heads/master
Commit: dfec6a07f6ec4529451b47cd4258e04bbffd1a9a
Parents: bf04b56
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Sep 25 01:56:26 2015 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Sep 25 01:56:26 2015 -0700

----------------------------------------------------------------------
 CHANGES                                         |  3 ++
 .../tajo-catalog-drivers/tajo-hive/pom.xml      |  2 +-
 tajo-catalog/tajo-catalog-server/pom.xml        |  2 +-
 .../org/apache/tajo/catalog/CatalogServer.java  | 10 +++---
 tajo-cli/pom.xml                                |  2 +-
 tajo-client-example/pom.xml                     |  2 +-
 tajo-client/pom.xml                             |  2 +-
 .../apache/tajo/client/CatalogAdminClient.java  |  3 +-
 .../tajo/client/CatalogAdminClientImpl.java     |  3 +-
 .../org/apache/tajo/client/TajoClientImpl.java  |  2 +-
 tajo-cluster-tests/pom.xml                      |  2 +-
 .../org/apache/tajo/TajoTestingCluster.java     |  4 +--
 .../java/org/apache/tajo/TajoConstants.java     |  6 ++++
 .../java/org/apache/tajo/conf/TajoConf.java     |  3 +-
 .../CannotDropCurrentDatabaseException.java     | 33 +++++++++++++++++++
 .../tajo/exception/DividedByZeroException.java  | 33 +++++++++++++++++++
 .../apache/tajo/exception/ErrorMessages.java    |  9 +++---
 .../apache/tajo/exception/ExceptionUtil.java    | 19 +++++++++--
 .../tajo/exception/InvalidSessionException.java | 33 +++++++++++++++++++
 .../InvalidSessionVariableException.java        | 34 ++++++++++++++++++++
 .../tajo/exception/InvalidURLException.java     | 32 ++++++++++++++++++
 .../org/apache/tajo/storage/BufferPool.java     |  4 +--
 .../org/apache/tajo/util/CommonTestingUtil.java |  5 ++-
 tajo-common/src/main/proto/errors.proto         |  4 ++-
 tajo-core-tests/pom.xml                         |  4 +--
 .../engine/planner/physical/TestSortExec.java   |  2 +-
 .../tajo/engine/query/TestHBaseTable.java       |  3 +-
 .../tajo/worker/TestNodeResourceManager.java    |  2 +-
 .../tajo/worker/TestNodeStatusUpdater.java      |  2 +-
 .../apache/tajo/worker/TestTaskExecutor.java    |  2 +-
 .../org/apache/tajo/worker/TestTaskManager.java |  7 ++--
 .../org/apache/tajo/master/GlobalEngine.java    |  4 +--
 .../java/org/apache/tajo/master/TajoMaster.java |  3 +-
 .../apache/tajo/master/exec/DDLExecutor.java    |  4 +--
 .../NonForwardQueryResultSystemScanner.java     |  2 +-
 .../tajo/session/InvalidSessionException.java   | 29 -----------------
 .../org/apache/tajo/session/SessionManager.java |  3 +-
 .../apache/tajo/worker/NodeResourceManager.java |  4 ++-
 .../java/org/apache/tajo/worker/TajoWorker.java |  4 +--
 .../tajo/ws/rs/resources/QueryResource.java     |  2 +-
 .../tajo/ws/rs/resources/SessionsResource.java  |  2 +-
 tajo-jdbc/pom.xml                               |  2 +-
 .../function/python/PythonScriptEngine.java     |  3 +-
 .../tajo/plan/function/stream/BufferPool.java   |  3 +-
 tajo-storage/tajo-storage-common/pom.xml        |  2 +-
 tajo-storage/tajo-storage-hbase/pom.xml         |  2 +-
 .../tajo/storage/hbase/HBaseTablespace.java     |  3 +-
 tajo-storage/tajo-storage-hdfs/pom.xml          |  2 +-
 tajo-storage/tajo-storage-jdbc/pom.xml          |  2 +-
 tajo-storage/tajo-storage-pgsql/pom.xml         |  6 ++--
 50 files changed, 262 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index ffcad96..245e3ab 100644
--- a/CHANGES
+++ b/CHANGES
@@ -56,6 +56,9 @@ Release 0.11.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1890: Clean up debug and test modes and unhandled exceptions. 
+    (hyunsik)
+
     TAJO-1860: Refactor Rpc clients to take Connection Parameters. (hyunsik)
  
     TAJO-1868: Allow TablespaceManager::get to return a unregistered 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml b/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
index 4516b27..b2995a8 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
@@ -80,7 +80,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-catalog/tajo-catalog-server/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/pom.xml b/tajo-catalog/tajo-catalog-server/pom.xml
index f00f8d3..cc421a8 100644
--- a/tajo-catalog/tajo-catalog-server/pom.xml
+++ b/tajo-catalog/tajo-catalog-server/pom.xml
@@ -118,7 +118,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 44eec84..50f2c4a 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
@@ -36,6 +36,8 @@ import org.apache.tajo.annotation.ThreadSafe;
 import org.apache.tajo.catalog.CatalogProtocol.*;
 import org.apache.tajo.catalog.dictionary.InfoSchemaMetadataDictionary;
 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.catalog.proto.CatalogProtos.*;
 import org.apache.tajo.catalog.store.CatalogStore;
@@ -58,6 +60,7 @@ import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.net.InetSocketAddress;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.Lock;
@@ -375,11 +378,10 @@ public class CatalogServer extends AbstractService {
           for (AlterTablespaceCommand command : request.getCommandList()) {
             if (command.getType() == AlterTablespaceProto.AlterTablespaceType.LOCATION) {
               try {
-                URI uri = URI.create(command.getLocation());
+                URI uri = new URI(command.getLocation());
                 Preconditions.checkArgument(uri.getScheme() != null);
-              } catch (Exception e) {
-                throw new ServiceException("ALTER TABLESPACE's LOCATION must be a URI form (scheme:///.../), but "
-                    + command.getLocation());
+              } catch (URISyntaxException e) {
+                return returnError(ResultCode.INVALID_URL, command.getLocation());
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-cli/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-cli/pom.xml b/tajo-cli/pom.xml
index 8de3c54..a7a3b91 100644
--- a/tajo-cli/pom.xml
+++ b/tajo-cli/pom.xml
@@ -69,7 +69,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -Dfile.encoding=UTF-8</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-client-example/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-client-example/pom.xml b/tajo-client-example/pom.xml
index e81311d..7db1129 100644
--- a/tajo-client-example/pom.xml
+++ b/tajo-client-example/pom.xml
@@ -68,7 +68,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -Dfile.encoding=UTF-8</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-client/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-client/pom.xml b/tajo-client/pom.xml
index ca4809c..e9f03ed 100644
--- a/tajo-client/pom.xml
+++ b/tajo-client/pom.xml
@@ -68,7 +68,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -Dfile.encoding=UTF-8</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 ddae0fc..0e4c1b8 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
@@ -55,7 +55,8 @@ public interface CatalogAdminClient extends Closeable {
    * @return True if the database is dropped successfully.
    * @throws UndefinedDatabaseException
    */
-  void dropDatabase(final String databaseName) throws UndefinedDatabaseException, InsufficientPrivilegeException;
+  void dropDatabase(final String databaseName)
+      throws UndefinedDatabaseException, InsufficientPrivilegeException, CannotDropCurrentDatabaseException;
 
   List<String> getAllDatabaseNames();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 d98d8cf..97f28a6 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
@@ -88,7 +88,7 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
 
   @Override
   public void dropDatabase(final String databaseName)
-      throws UndefinedDatabaseException, InsufficientPrivilegeException {
+      throws UndefinedDatabaseException, InsufficientPrivilegeException, CannotDropCurrentDatabaseException {
 
     try {
       final BlockingInterface stub = conn.getTMStub();
@@ -96,6 +96,7 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
 
       throwsIfThisError(state, UndefinedDatabaseException.class);
       throwsIfThisError(state, InsufficientPrivilegeException.class);
+      throwsIfThisError(state, CannotDropCurrentDatabaseException.class);
       ensureOk(state);
 
     } catch (ServiceException e) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 36b8e4e..cbc194f 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
@@ -197,7 +197,7 @@ public class TajoClientImpl extends SessionConnection implements TajoClient, Que
   }
 
   public void dropDatabase(final String databaseName) throws UndefinedDatabaseException,
-      InsufficientPrivilegeException {
+      InsufficientPrivilegeException, CannotDropCurrentDatabaseException {
 
     catalogClient.dropDatabase(databaseName);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-cluster-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/pom.xml b/tajo-cluster-tests/pom.xml
index 2d879d9..0aafa27 100644
--- a/tajo-cluster-tests/pom.xml
+++ b/tajo-cluster-tests/pom.xml
@@ -65,7 +65,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java
index 3517cea..a8d4583 100644
--- a/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-cluster-tests/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -113,8 +113,8 @@ public class TajoTestingCluster {
   }
 
   void setTestingFlagProperties() {
-    System.setProperty(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
-    conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
+    System.setProperty(TajoConstants.TEST_KEY, Boolean.TRUE.toString());
+    conf.set(TajoConstants.TEST_KEY, Boolean.TRUE.toString());
   }
 
   void initPropertiesAndConfigs() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
index e6fc89b..f431fad 100644
--- a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
@@ -43,5 +43,11 @@ public class TajoConstants {
   public static final int UNKNOWN_ROW_NUMBER = -1;
   public static final int UNKNOWN_LENGTH = -1;
 
+  public static final String TEST_KEY = "tajo.test.enabled";
+  public static final String DEBUG_KEY = "tajo.debug.enabled";
+
+  public static final boolean IS_TEST_MODE = Boolean.parseBoolean(System.getProperty(TEST_KEY, "false"));
+  public static final boolean IS_DEBUG_MODE = Boolean.parseBoolean(System.getProperty(DEBUG_KEY, "false"));
+
   private TajoConstants() {}
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index d7789f8..f53ada9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -369,7 +369,8 @@ public class TajoConf extends Configuration {
     $TEXT_NULL("tajo.text.null", "\\\\N"),
 
     // Only for Debug and Testing
-    $DEBUG_ENABLED("tajo.debug.enabled", false),
+    $DEBUG_ENABLED(TajoConstants.DEBUG_KEY, false),
+    $TEST_MODE(TajoConstants.TEST_KEY, false),
     $TEST_BROADCAST_JOIN_ENABLED("tajo.dist-query.join.auto-broadcast", true),
     $TEST_JOIN_OPT_ENABLED("tajo.test.plan.join-optimization.enabled", true),
     $TEST_FILTER_PUSHDOWN_ENABLED("tajo.test.plan.filter-pushdown.enabled", true),

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/exception/CannotDropCurrentDatabaseException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/CannotDropCurrentDatabaseException.java b/tajo-common/src/main/java/org/apache/tajo/exception/CannotDropCurrentDatabaseException.java
new file mode 100644
index 0000000..d2ad770
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/CannotDropCurrentDatabaseException.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 CannotDropCurrentDatabaseException extends TajoException {
+
+  public CannotDropCurrentDatabaseException() {
+    super(Errors.ResultCode.CANNOT_DROP_CURRENT_DATABASE);
+  }
+
+  public CannotDropCurrentDatabaseException(ReturnState e) {
+    super(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/exception/DividedByZeroException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/DividedByZeroException.java b/tajo-common/src/main/java/org/apache/tajo/exception/DividedByZeroException.java
new file mode 100644
index 0000000..d7ec278
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/DividedByZeroException.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 DividedByZeroException extends TajoException {
+
+  public DividedByZeroException(ReturnState e) {
+    super(e);
+  }
+
+  public DividedByZeroException(String variableName) {
+    super(Errors.ResultCode.DIVISION_BY_ZERO, variableName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 990178a..1c69756 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
@@ -39,7 +39,6 @@ public class ErrorMessages {
     ADD_MESSAGE(NOT_IMPLEMENTED, "not implemented feature: %s", 1);
     ADD_MESSAGE(FEATURE_NOT_SUPPORTED, "unsupported feature: %s", 1);
     ADD_MESSAGE(INVALID_RPC_CALL, "invalid RPC Call: %s", 1);
-    ADD_MESSAGE(INVALID_VALUE_FOR_CAST, "%s value cannot be casted to %s", 2);
 
     // Query Management and Scheduler
     ADD_MESSAGE(QUERY_FAILED, "query has been failed due to %s", 1);
@@ -53,11 +52,10 @@ public class ErrorMessages {
     ADD_MESSAGE(NO_SUCH_SESSION_VARIABLE, "no such session variable '%s", 1);
     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", 1);
-    ADD_MESSAGE(INVALID_NAME, "Invalid name '%s'");
+    ADD_MESSAGE(CANNOT_DROP_CURRENT_DATABASE, "Cannot drop the current database", 1);
 
     ADD_MESSAGE(UNDEFINED_TABLESPACE, "tablespace '%s' does not exist", 1);
     ADD_MESSAGE(UNDEFINED_DATABASE, "database '%s' does not exist", 1);
@@ -88,6 +86,9 @@ public class ErrorMessages {
     ADD_MESSAGE(AMBIGUOUS_COLUMN, "column name '%s' is ambiguous", 1);
     ADD_MESSAGE(AMBIGUOUS_FUNCTION, "function '%s' is ambiguous", 1);
 
+    ADD_MESSAGE(INVALID_NAME, "Invalid name '%s'");
+    ADD_MESSAGE(INVALID_VALUE_FOR_CAST, "%s value cannot be casted to %s", 2);
+
     ADD_MESSAGE(DIVISION_BY_ZERO, "Division by zero: %s", 1);
 
     ADD_MESSAGE(DATATYPE_MISMATCH,
@@ -104,7 +105,7 @@ public class ErrorMessages {
     ADD_MESSAGE(LMD_NO_MATCHED_DATATYPE, "no matched type for %s", 1);
 
     // Storage and Data Format
-    ADD_MESSAGE(UNAVAILABLE_TABLE_LOCATION, "unavailable table location '%s': %s", 2);
+    ADD_MESSAGE(UNAVAILABLE_TABLE_LOCATION, "unavailable table location '%s', cause: %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);

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 ad6c154..adfb0c4 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
@@ -21,6 +21,7 @@ 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.TajoConstants;
 import org.apache.tajo.error.Errors;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
 
@@ -44,10 +45,21 @@ public class ExceptionUtil {
     // Query Management and Scheduler
     ADD_EXCEPTION(QUERY_NOT_FOUND, QueryNotFoundException.class);
 
+    // Session
+    ADD_EXCEPTION(INVALID_SESSION, InvalidSessionException.class);
+    ADD_EXCEPTION(NO_SUCH_SESSION_VARIABLE, NoSuchSessionVariableException.class);
+    ADD_EXCEPTION(INVALID_SESSION_VARIABLE, InvalidSessionVariableException.class);
+
+    // Data Exception (SQLState Class - 22)
+    ADD_EXCEPTION(DIVISION_BY_ZERO, DividedByZeroException.class);
+    ADD_EXCEPTION(INVALID_URL, InvalidURLException.class);
+    ADD_EXCEPTION(INVALID_VALUE_FOR_CAST, InvalidValueForCastException.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(CANNOT_DROP_CURRENT_DATABASE, CannotDropCurrentDatabaseException.class);
 
     ADD_EXCEPTION(UNDEFINED_TABLESPACE, UndefinedTablespaceException.class);
     ADD_EXCEPTION(UNDEFINED_DATABASE, UndefinedDatabaseException.class);
@@ -75,8 +87,8 @@ public class ExceptionUtil {
     ADD_EXCEPTION(AMBIGUOUS_COLUMN, AmbiguousColumnException.class);
     ADD_EXCEPTION(AMBIGUOUS_FUNCTION, AmbiguousFunctionException.class);
 
+    // Expressions
     ADD_EXCEPTION(DATATYPE_MISMATCH, DataTypeMismatchException.class);
-    ADD_EXCEPTION(DATATYPE_MISMATCH, InvalidValueForCastException.class);
 
     ADD_EXCEPTION(UNAVAILABLE_TABLE_LOCATION, UnavailableTableLocationException.class);
     ADD_EXCEPTION(UNKNOWN_DATAFORMAT, UnknownDataFormatException.class);
@@ -190,7 +202,10 @@ public class ExceptionUtil {
   }
 
   public static void printStackTraceIfError(Log log, Throwable t) {
-    if (System.getProperty("DEBUG") != null || !ExceptionUtil.isManagedException(t)) {
+    // if this runs as an actual cluster instance or a debug mode, it will print all stacktraces.
+    // In other cases (i.e., run as a test mode and not debug mode), it will print stacktraces
+    // if the query is managed mode.
+    if (!TajoConstants.IS_TEST_MODE || TajoConstants.IS_DEBUG_MODE || !ExceptionUtil.isManagedException(t)) {
       ExceptionUtil.printStackTrace(log, t);
     }
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionException.java
new file mode 100644
index 0000000..4724b4e
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionException.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;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class InvalidSessionException extends TajoException {
+  public InvalidSessionException(ReturnState state) {
+    super(state);
+  }
+
+  public InvalidSessionException(String sessionId) {
+    super(Errors.ResultCode.INVALID_SESSION, sessionId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionVariableException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionVariableException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionVariableException.java
new file mode 100644
index 0000000..8351fc5
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidSessionVariableException.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;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState;
+
+public class InvalidSessionVariableException extends TajoException {
+
+  public InvalidSessionVariableException(ReturnState e) {
+    super(e);
+  }
+
+  public InvalidSessionVariableException(String variableName) {
+    super(Errors.ResultCode.INVALID_SESSION_VARIABLE, variableName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/exception/InvalidURLException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/exception/InvalidURLException.java b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidURLException.java
new file mode 100644
index 0000000..4d8388c
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/exception/InvalidURLException.java
@@ -0,0 +1,32 @@
+/*
+ * 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 InvalidURLException extends TajoException {
+  public InvalidURLException(ReturnState e) {
+    super(e);
+  }
+
+  public InvalidURLException(String url) {
+    super(Errors.ResultCode.INVALID_URL, url);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/storage/BufferPool.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/BufferPool.java b/tajo-common/src/main/java/org/apache/tajo/storage/BufferPool.java
index 3120083..403d789 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/BufferPool.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/BufferPool.java
@@ -22,8 +22,8 @@ import io.netty.buffer.*;
 import io.netty.util.ResourceLeakDetector;
 import io.netty.util.internal.PlatformDependent;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.util.CommonTestingUtil;
 
 import java.lang.reflect.Field;
 import java.nio.ByteBuffer;
@@ -44,7 +44,7 @@ public class BufferPool {
     *  Because the TaskRunner thread is newly created
     * */
 
-    if (System.getProperty(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) {
+    if (TajoConstants.IS_TEST_MODE) {
       /* Disable pooling buffers for memory usage  */
       ALLOCATOR = UnpooledByteBufAllocator.DEFAULT;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
index e62cb12..e7d44f6 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
@@ -24,18 +24,17 @@ import org.apache.hadoop.fs.Path;
 import org.apache.tajo.ConfigKey;
 import org.apache.tajo.OverridableConf;
 import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.conf.TajoConf;
 
 import java.io.IOException;
 import java.util.UUID;
 
 public class CommonTestingUtil {
-  public static final String TAJO_TEST_KEY = "tajo.test.enabled";
-  public static final String TAJO_TEST_TRUE = "true";
   private static OverridableConf userSessionVars;
 
   static {
-    System.setProperty(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
+    System.setProperty(TajoConstants.TEST_KEY, Boolean.TRUE.toString());
 
     userSessionVars = new OverridableConf(new TajoConf(), ConfigKey.ConfigType.SESSION);
     for (SessionVars var : SessionVars.values()) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 677f799..7585a83 100644
--- a/tajo-common/src/main/proto/errors.proto
+++ b/tajo-common/src/main/proto/errors.proto
@@ -94,14 +94,16 @@ enum ResultCode {
   // Data Exception (SQLState Class - 22)
   DIVISION_BY_ZERO                      = 451; // SQLState: 22012 - Division by zero
   INVALID_VALUE_FOR_CAST                = 452; // SQLState: 22T01 - Invalid data value for cast
+  INVALID_URL                           = 453; // SQLState: 22T02 - Invalid URL
 
   // Section: Class 42 - Syntax Error or Access Rule Violation
   SYNTAX_ERROR_OR_ACCESS_RULE_VIOLATION = 501; // SQLState: 42000
   SYNTAX_ERROR                          = 502; // SQLState: 42601
   INSUFFICIENT_PRIVILEGE                = 503; // SQLState: 42501
+  CANNOT_DROP_CURRENT_DATABASE          = 504; // SQLState: 42T01
 
   UNDEFINED_TABLESPACE                  = 511; // ?
-  UNDEFINED_DATABASE                    = 512; // SQLState: 42T01
+  UNDEFINED_DATABASE                    = 512; // SQLState: 42T02
   UNDEFINED_SCHEMA                      = 513; // SQLState:
   UNDEFINED_TABLE                       = 514; // SQLState: 42P01
   UNDEFINED_COLUMN                      = 515; // SQLState: 42703

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core-tests/pom.xml b/tajo-core-tests/pom.xml
index 20b7378..6d3d75c 100644
--- a/tajo-core-tests/pom.xml
+++ b/tajo-core-tests/pom.xml
@@ -71,7 +71,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>
@@ -379,7 +379,7 @@
               <useSystemClassLoader>true</useSystemClassLoader>
               <useManifestOnlyJar>true</useManifestOnlyJar>
               <systemProperties>
-                <tajo.test>TRUE</tajo.test>
+                <tajo.test.enabled>true</tajo.test.enabled>
                 <tajo.test.data.dir>test-data${surefire.forkNumber}</tajo.test.data.dir>
               </systemProperties>
             </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
index 2d6ce0e..58c2a98 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
@@ -73,7 +73,7 @@ public class TestSortExec {
   @BeforeClass
   public static void setUp() throws Exception {
     conf = new TajoConf();
-    conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
+    conf.setBoolVar(TajoConf.ConfVars.$TEST_MODE, true);
     util = TpchTestBase.getInstance().getTestingCluster();
     catalog = util.getMaster().getCatalog();
     workDir = CommonTestingUtil.getTestDir(TEST_PATH);

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
index 0d9c49b..5cbac60 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
@@ -41,6 +41,7 @@ import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.TextDatum;
 import org.apache.tajo.error.Errors.ResultCode;
 import org.apache.tajo.exception.TajoException;
+import org.apache.tajo.exception.UnavailableTableLocationException;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.ScanNode;
 import org.apache.tajo.storage.StorageConstants;
@@ -164,7 +165,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
         executeString(sql).close();
         fail("External table should be a existed table.");
       } catch (Throwable e) {
-        assertTrue(e.getMessage().indexOf("External table should be a existed table.") >= 0);
+        assertTrue(e instanceof UnavailableTableLocationException);
       }
     } finally {
       TablespaceManager.addTableSpaceForTest(existing.get());

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java
index 3d0554a..7a7b95a 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java
@@ -59,7 +59,7 @@ public class TestNodeResourceManager {
   @Before
   public void setup() {
     conf = new TajoConf();
-    conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
+    conf.setBoolVar(TajoConf.ConfVars.$TEST_MODE, true);
 
     taskMemory = 512;
     conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4);

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java
index ac4b7dd..c9ac649 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java
@@ -50,7 +50,7 @@ public class TestNodeStatusUpdater {
   @Before
   public void setup() {
     conf = new TajoConf();
-    conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
+    conf.setBoolVar(TajoConf.ConfVars.$TEST_MODE, true);
     conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 2);
     conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM, 2);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java
index 5039c8b..b8244d7 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java
@@ -62,7 +62,7 @@ public class TestTaskExecutor {
   @Before
   public void setup() {
     conf = new TajoConf();
-    conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
+    conf.setBoolVar(TajoConf.ConfVars.$TEST_MODE, true);
     conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 2);
     conf.setIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM, 2);
     dispatcher = new AsyncDispatcher();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskManager.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskManager.java
index c19d8fa..74a3b71 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskManager.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/worker/TestTaskManager.java
@@ -21,10 +21,7 @@ package org.apache.tajo.worker;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.LocalTajoTestingUtility;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.*;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.master.cluster.WorkerConnectionInfo;
 import org.apache.tajo.resource.NodeResource;
@@ -62,7 +59,7 @@ public class TestTaskManager {
   @Before
   public void setup() {
     conf = new TajoConf();
-    conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE);
+    conf.setBoolVar(TajoConf.ConfVars.$TEST_MODE, true);
 
     taskMemory = 512;
     conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4);

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 d5e5dd5..4f6873b 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.algebra.JsonHelper;
 import org.apache.tajo.catalog.CatalogService;
@@ -145,8 +146,7 @@ public class GlobalEngine extends AbstractService {
     newQueryContext.setDefaultSpaceUri(TablespaceManager.getDefault().getUri());
     newQueryContext.setDefaultSpaceRootUri(TablespaceManager.getDefault().getRootUri());
 
-    String tajoTest = System.getProperty(CommonTestingUtil.TAJO_TEST_KEY);
-    if (tajoTest != null && tajoTest.equalsIgnoreCase(CommonTestingUtil.TAJO_TEST_TRUE)) {
+    if (TajoConstants.IS_TEST_MODE) {
       newQueryContext.putAll(CommonTestingUtil.getSessionVarsForTest());
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 922a5eb..30b3b7a 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
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.algebra.AlterTablespace;
 import org.apache.tajo.catalog.CatalogServer;
 import org.apache.tajo.catalog.CatalogService;
@@ -242,7 +243,7 @@ public class TajoMaster extends CompositeService {
   }
 
   private void initWebServer() throws Exception {
-    if (!systemConf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) {
+    if (!systemConf.getBoolVar(ConfVars.$TEST_MODE)) {
       InetSocketAddress address = systemConf.getSocketAddrVar(ConfVars.TAJO_MASTER_INFO_ADDRESS);
       webServer = StaticHttpServer.getInstance(this ,"admin", address.getHostName(), address.getPort(),
           true, null, context.getConf(), null);

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 57d01aa..8d9facb 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
@@ -258,7 +258,7 @@ public class DDLExecutor {
   }
 
   public void dropDatabase(QueryContext queryContext, String databaseName, boolean ifExists)
-      throws UndefinedDatabaseException, InsufficientPrivilegeException {
+      throws UndefinedDatabaseException, InsufficientPrivilegeException, CannotDropCurrentDatabaseException {
 
     boolean exists = catalog.existDatabase(databaseName);
     if (!exists) {
@@ -271,7 +271,7 @@ public class DDLExecutor {
     }
 
     if (queryContext.getCurrentDatabase().equals(databaseName)) {
-      throw new RuntimeException("ERROR: Cannot drop the current open database");
+      throw new CannotDropCurrentDatabaseException();
     }
 
     catalog.dropDatabase(databaseName);

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
index 82bc679..f2abe0e 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
@@ -52,7 +52,7 @@ import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.plan.logical.ScanNode;
 import org.apache.tajo.resource.NodeResource;
 import org.apache.tajo.resource.NodeResources;
-import org.apache.tajo.session.InvalidSessionException;
+import org.apache.tajo.exception.InvalidSessionException;
 import org.apache.tajo.storage.RowStoreUtil;
 import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
 import org.apache.tajo.storage.Tuple;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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
deleted file mode 100644
index 00085e5..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.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.session;
-
-import org.apache.tajo.error.Errors;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.exception.TajoRuntimeException;
-
-public class InvalidSessionException extends TajoException {
-  public InvalidSessionException(String sessionId) {
-    super(Errors.ResultCode.INVALID_SESSION, sessionId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java b/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java
index 38a39ab..8bc6fb8 100644
--- a/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java
@@ -24,9 +24,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.exception.InvalidSessionException;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java
index 0580ebc..e360df7 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.resource.NodeResource;
 import org.apache.tajo.resource.NodeResources;
@@ -61,7 +62,8 @@ public class NodeResourceManager extends AbstractService implements EventHandler
     this.availableResource = NodeResources.clone(totalResource);
     this.dispatcher.register(NodeResourceEvent.EventType.class, this);
     validateConf(tajoConf);
-    this.enableTest = conf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE");
+    this.enableTest = conf.get(TajoConstants.TEST_KEY, Boolean.FALSE.toString())
+        .equalsIgnoreCase(Boolean.TRUE.toString());
     super.serviceInit(conf);
     LOG.info("Initialized NodeResourceManager for " + totalResource);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
index 0071a14..a7188a1 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -196,7 +196,7 @@ public class TajoWorker extends CompositeService {
       addIfService(pullService);
     }
 
-    if (!systemConf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) {
+    if (!systemConf.getBoolVar(ConfVars.$TEST_MODE)) {
       httpPort = initWebServer();
     }
 
@@ -324,7 +324,7 @@ public class TajoWorker extends CompositeService {
     }
     connectToCatalog();
 
-    if (!systemConf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) {
+    if (!systemConf.getBoolVar(ConfVars.$TEST_MODE)) {
       initCleanupService();
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java
index e4cf025..f7d1897 100644
--- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java
+++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/QueryResource.java
@@ -29,7 +29,7 @@ import org.apache.tajo.master.QueryInfo;
 import org.apache.tajo.master.QueryManager;
 import org.apache.tajo.master.TajoMaster.MasterContext;
 import org.apache.tajo.querymaster.QueryJobEvent;
-import org.apache.tajo.session.InvalidSessionException;
+import org.apache.tajo.exception.InvalidSessionException;
 import org.apache.tajo.session.Session;
 import org.apache.tajo.util.TajoIdUtils;
 import org.apache.tajo.ws.rs.*;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/SessionsResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/SessionsResource.java b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/SessionsResource.java
index 52d8fe7..f679f7a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/SessionsResource.java
+++ b/tajo-core/src/main/java/org/apache/tajo/ws/rs/resources/SessionsResource.java
@@ -24,7 +24,7 @@ import org.apache.tajo.TajoConstants;
 import org.apache.tajo.error.Errors.ResultCode;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.session.InvalidSessionException;
+import org.apache.tajo.exception.InvalidSessionException;
 import org.apache.tajo.session.Session;
 import org.apache.tajo.ws.rs.*;
 import org.apache.tajo.ws.rs.requests.NewSessionRequest;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-jdbc/pom.xml b/tajo-jdbc/pom.xml
index 09fabdd..686c14f 100644
--- a/tajo-jdbc/pom.xml
+++ b/tajo-jdbc/pom.xml
@@ -92,7 +92,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>true</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=152m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java
index cb49d49..e71f534 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java
@@ -23,6 +23,7 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
@@ -336,7 +337,7 @@ public class PythonScriptEngine extends TajoScriptEngine {
     try {
       int exitCode = process.waitFor();
 
-      if (systemConf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) {
+      if (systemConf.get(TajoConstants.TEST_KEY, Boolean.FALSE.toString()).equalsIgnoreCase(Boolean.TRUE.toString())) {
         LOG.warn("Process exit code: " + exitCode);
       } else {
         if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java
index 9b65e4b..d39aeb8 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java
@@ -25,6 +25,7 @@ import io.netty.buffer.UnpooledByteBufAllocator;
 import io.netty.util.ResourceLeakDetector;
 import io.netty.util.internal.PlatformDependent;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.util.CommonTestingUtil;
 
@@ -45,7 +46,7 @@ public class BufferPool {
     *  Because the TaskRunner thread is newly created
     * */
 
-    if (System.getProperty(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) {
+    if (TajoConstants.IS_TEST_MODE) {
       /* Disable pooling buffers for memory usage  */
       ALLOCATOR = UnpooledByteBufAllocator.DEFAULT;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-storage/tajo-storage-common/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/pom.xml b/tajo-storage/tajo-storage-common/pom.xml
index 4321a13..106d9f3 100644
--- a/tajo-storage/tajo-storage-common/pom.xml
+++ b/tajo-storage/tajo-storage-common/pom.xml
@@ -73,7 +73,7 @@ limitations under the License.
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>TRUE</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-storage/tajo-storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/pom.xml b/tajo-storage/tajo-storage-hbase/pom.xml
index d499cd4..1adfa10 100644
--- a/tajo-storage/tajo-storage-hbase/pom.xml
+++ b/tajo-storage/tajo-storage-hbase/pom.xml
@@ -76,7 +76,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>TRUE</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/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 91840d4..baf3b8c 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
@@ -172,8 +172,7 @@ public class HBaseTablespace extends Tablespace {
           throw new MissingTablePropertyException(HBaseStorageConstants.META_COLUMNS_KEY, hbaseTableName);
         }
         if (!hAdmin.tableExists(hTableName)) {
-          throw new IOException("HBase table [" + hbaseTableName + "] not exists. " +
-              "External table should be a existed table.");
+          throw new UnavailableTableLocationException(hbaseTableName, "the table does not exist");
         }
         HTableDescriptor hTableDescriptor = hAdmin.getTableDescriptor(hTableName);
         Set<String> tableColumnFamilies = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-storage/tajo-storage-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/pom.xml b/tajo-storage/tajo-storage-hdfs/pom.xml
index c3057d7..325df22 100644
--- a/tajo-storage/tajo-storage-hdfs/pom.xml
+++ b/tajo-storage/tajo-storage-hdfs/pom.xml
@@ -79,7 +79,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>TRUE</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8 -Dderby.storage.pageSize=1024 -Dderby.stream.error.file=/dev/null</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-storage/tajo-storage-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-jdbc/pom.xml b/tajo-storage/tajo-storage-jdbc/pom.xml
index 35000a4..3c3bee3 100644
--- a/tajo-storage/tajo-storage-jdbc/pom.xml
+++ b/tajo-storage/tajo-storage-jdbc/pom.xml
@@ -80,7 +80,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemProperties>
-            <tajo.test>TRUE</tajo.test>
+            <tajo.test.enabled>TRUE</tajo.test.enabled>
           </systemProperties>
           <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
         </configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfec6a07/tajo-storage/tajo-storage-pgsql/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-pgsql/pom.xml b/tajo-storage/tajo-storage-pgsql/pom.xml
index cd5f2cc..f36f350 100644
--- a/tajo-storage/tajo-storage-pgsql/pom.xml
+++ b/tajo-storage/tajo-storage-pgsql/pom.xml
@@ -238,7 +238,7 @@
             <artifactId>maven-surefire-plugin</artifactId>
             <configuration combine.self="override">
               <systemProperties>
-                <tajo.test>TRUE</tajo.test>
+                <tajo.test.enabled>TRUE</tajo.test.enabled>
               </systemProperties>
               <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
             </configuration>
@@ -262,7 +262,7 @@
             <artifactId>maven-surefire-plugin</artifactId>
             <configuration combine.self="override">
               <systemProperties>
-                <tajo.test>TRUE</tajo.test>
+                <tajo.test.enabled>TRUE</tajo.test.enabled>
               </systemProperties>
               <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
             </configuration>
@@ -286,7 +286,7 @@
             <artifactId>maven-surefire-plugin</artifactId>
             <configuration combine.self="override">
               <systemProperties>
-                <tajo.test>TRUE</tajo.test>
+                <tajo.test.enabled>TRUE</tajo.test.enabled>
               </systemProperties>
               <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
             </configuration>