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 2014/08/11 05:53:12 UTC

[1/5] TAJO-928: Session variables should override query configs in TajoConf.

Repository: tajo
Updated Branches:
  refs/heads/master 0603b49dd -> ddfc3f330


http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
index b59c495..9501556 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -49,39 +49,39 @@ public class TestJoinQuery extends QueryTestCaseBase {
   public TestJoinQuery(String joinOption) {
     super(TajoConstants.DEFAULT_DATABASE_NAME);
 
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname,
-        ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.defaultVal);
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname,
-        ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname,
+        ConfVars.$TEST_BROADCAST_JOIN_ENABLED.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname,
+        ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.defaultVal);
 
     testingCluster.setAllTajoDaemonConfValue(
-        ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD.varname,
-        ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD.defaultVal);
+        ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname,
+        ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.defaultVal);
 
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD.varname,
-        ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD.defaultVal);
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
-        ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname,
+        ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
+        ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.defaultVal);
 
     if (joinOption.indexOf("NoBroadcast") >= 0) {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname, "false");
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname, "-1");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "false");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname, "-1");
     }
 
     if (joinOption.indexOf("Hash") >= 0) {
       testingCluster.setAllTajoDaemonConfValue(
-          ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD.varname, String.valueOf(256 * 1048576));
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD.varname,
+          ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname, String.valueOf(256 * 1048576));
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname,
           String.valueOf(256 * 1048576));
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
           String.valueOf(256 * 1048576));
     }
     if (joinOption.indexOf("Sort") >= 0) {
       testingCluster.setAllTajoDaemonConfValue(
-          ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD.varname, String.valueOf(1));
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD.varname,
+          ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname, String.valueOf(1));
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname,
           String.valueOf(1));
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
           String.valueOf(1));
     }
   }
@@ -98,19 +98,19 @@ public class TestJoinQuery extends QueryTestCaseBase {
 
   @AfterClass
   public static void classTearDown() {
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname,
-        ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.defaultVal);
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname,
-        ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname,
+        ConfVars.$TEST_BROADCAST_JOIN_ENABLED.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname,
+        ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.defaultVal);
 
     testingCluster.setAllTajoDaemonConfValue(
-        ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD.varname,
-        ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD.defaultVal);
+        ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname,
+        ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.defaultVal);
 
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD.varname,
-        ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD.defaultVal);
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
-        ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.varname,
+        ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD.defaultVal);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.varname,
+        ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD.defaultVal);
   }
 
   @Test
@@ -392,8 +392,8 @@ public class TestJoinQuery extends QueryTestCaseBase {
   public final void testLeftOuterJoinWithEmptySubquery1() throws Exception {
     // Empty Null Supplying table
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("id", Type.INT4);
@@ -405,7 +405,7 @@ public class TestJoinQuery extends QueryTestCaseBase {
     TajoTestingCluster.createTable("table12", schema, tableOptions, data, 2);
 
     try {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname, "2");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname, "2");
 
       ResultSet res = executeString("select a.id, b.id from table11 a " +
           "left outer join (" +
@@ -423,8 +423,8 @@ public class TestJoinQuery extends QueryTestCaseBase {
       assertEquals(expected, resultSetToString(res));
       cleanupQuery(res);
     } finally {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname,
-          ConfVars.TESTCASE_MIN_TASK_NUM.defaultVal);
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname,
+          ConfVars.$TEST_MIN_TASK_NUM.defaultVal);
       executeString("DROP TABLE table11 PURGE").close();
       executeString("DROP TABLE table12 PURGE").close();
     }
@@ -434,8 +434,8 @@ public class TestJoinQuery extends QueryTestCaseBase {
   public final void testLeftOuterJoinWithEmptySubquery2() throws Exception {
     //Empty Preserved Row table
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("id", Type.INT4);
@@ -447,7 +447,7 @@ public class TestJoinQuery extends QueryTestCaseBase {
     TajoTestingCluster.createTable("table12", schema, tableOptions, data, 2);
 
     try {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname, "2");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname, "2");
 
       ResultSet res = executeString("select a.id, b.id from " +
           "(select table12.id, table12.name, lineitem.l_shipdate " +
@@ -461,8 +461,8 @@ public class TestJoinQuery extends QueryTestCaseBase {
       assertEquals(expected, resultSetToString(res));
       cleanupQuery(res);
     } finally {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname,
-          ConfVars.TESTCASE_MIN_TASK_NUM.defaultVal);
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname,
+          ConfVars.$TEST_MIN_TASK_NUM.defaultVal);
       executeString("DROP TABLE table11 PURGE");
       executeString("DROP TABLE table12 PURGE");
     }
@@ -923,8 +923,8 @@ public class TestJoinQuery extends QueryTestCaseBase {
 
   private void createOuterJoinTestTable() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("id", Type.INT4);
@@ -1057,8 +1057,8 @@ public class TestJoinQuery extends QueryTestCaseBase {
   @Test
   public void testJoinWithDifferentShuffleKey() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("id", Type.INT4);
@@ -1077,8 +1077,8 @@ public class TestJoinQuery extends QueryTestCaseBase {
     }
     TajoTestingCluster.createTable("large_table", schema, tableOptions, data.toArray(new String[]{}));
 
-    int originConfValue = conf.getIntVar(ConfVars.DIST_QUERY_JOIN_PARTITION_VOLUME);
-    testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_JOIN_PARTITION_VOLUME.varname, "1");
+    int originConfValue = conf.getIntVar(ConfVars.$DIST_QUERY_JOIN_PARTITION_VOLUME);
+    testingCluster.setAllTajoDaemonConfValue(ConfVars.$DIST_QUERY_JOIN_PARTITION_VOLUME.varname, "1");
     ResultSet res = executeString(
        "select count(b.id) " +
            "from (select id, count(*) as cnt from large_table group by id) a " +
@@ -1094,7 +1094,7 @@ public class TestJoinQuery extends QueryTestCaseBase {
 
       assertEquals(expected, resultSetToString(res));
     } finally {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_JOIN_PARTITION_VOLUME.varname, "" + originConfValue);
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$DIST_QUERY_JOIN_PARTITION_VOLUME.varname, "" + originConfValue);
       cleanupQuery(res);
       executeString("DROP TABLE large_table PURGE").close();
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNullValues.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
index f41b0ab..de75ca7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
@@ -53,7 +53,7 @@ public class TestNullValues {
         "3|filled|0.2"
     };
     KeyValueSet opts = new KeyValueSet();
-    opts.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    opts.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
     ResultSet res = TajoTestingCluster
         .run(table, schemas, opts, new String[][]{data},
             "select * from nulltable1 where col3 is null");
@@ -80,7 +80,7 @@ public class TestNullValues {
         "3|filled|"
     };
     KeyValueSet opts = new KeyValueSet();
-    opts.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    opts.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
     ResultSet res = TajoTestingCluster
         .run(table, schemas, opts, new String[][]{data},
             "select * from nulltable2 where col1 is not null");
@@ -115,7 +115,7 @@ public class TestNullValues {
         ",,,43578,19,13,6,3581,2557,1024"
     };
     KeyValueSet opts = new KeyValueSet();
-    opts.put(StorageConstants.CSVFILE_DELIMITER, ",");
+    opts.set(StorageConstants.CSVFILE_DELIMITER, ",");
     ResultSet res = TajoTestingCluster
         .run(table, schemas, opts, new String[][]{data},
             "select * from nulltable3 where col1 is null and col2 is null and col3 is null and col4 = 43578");
@@ -148,8 +148,8 @@ public class TestNullValues {
         ",\\N,,43578"
     };
     KeyValueSet opts = new KeyValueSet();
-    opts.put(StorageConstants.CSVFILE_DELIMITER, ",");
-    opts.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    opts.set(StorageConstants.CSVFILE_DELIMITER, ",");
+    opts.set(StorageConstants.CSVFILE_NULL, "\\\\N");
     ResultSet res = TajoTestingCluster
         .run(table, schemas, opts, new String[][]{data},
             "select * from nulltable4 where col1 is null and col2 is null and col3 is null and col5 is null and col4 = 43578");
@@ -253,8 +253,8 @@ public class TestNullValues {
         "4|d|4.0|\\N"
     };
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     ResultSet res = TajoTestingCluster
         .run(table, schemas, tableOptions, new String[][]{data}, query);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
index 50d274d..5528b21 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
@@ -432,8 +432,8 @@ public class TestSelectQuery extends QueryTestCaseBase {
   @Test
   public final void testNowInMultipleTasks() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("id", Type.INT4);
@@ -442,7 +442,7 @@ public class TestSelectQuery extends QueryTestCaseBase {
     TajoTestingCluster.createTable("table11", schema, tableOptions, data, 2);
 
     try {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname, "2");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname, "2");
 
       ResultSet res = executeString("select concat(substr(to_char(now(),'yyyymmddhh24miss'), 1, 14), 'aaa'), sleep(1) from table11");
 
@@ -474,8 +474,8 @@ public class TestSelectQuery extends QueryTestCaseBase {
       }
       assertEquals(5, numRecords);
     } finally {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname,
-          ConfVars.TESTCASE_MIN_TASK_NUM.defaultVal);
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname,
+          ConfVars.$TEST_MIN_TASK_NUM.defaultVal);
       executeString("DROP TABLE table11 PURGE");
     }
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
index a520e56..c7f6406 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
@@ -181,10 +181,10 @@ public class TestSortQuery extends QueryTestCaseBase {
   @Test
   public final void testSortNullColumn() throws Exception {
     try {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname, "2");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname, "2");
       KeyValueSet tableOptions = new KeyValueSet();
-      tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-      tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+      tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+      tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
       Schema schema = new Schema();
       schema.addColumn("id", Type.INT4);
@@ -214,7 +214,7 @@ public class TestSortQuery extends QueryTestCaseBase {
 
       cleanupQuery(res);
     } finally {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.TESTCASE_MIN_TASK_NUM.varname, "0");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$TEST_MIN_TASK_NUM.varname, "0");
       executeString("DROP TABLE nullsort PURGE;").close();
     }
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
index 7ca94c9..38908db 100644
--- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
+++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
@@ -153,7 +153,7 @@ public class TestResultSet {
           "2014-01-01|01:00:00|2014-01-01 01:00:00"
       };
       KeyValueSet tableOptions = new KeyValueSet();
-      tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+      tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
 
       res = TajoTestingCluster
           .run(table, schemas, tableOptions, new String[][]{data}, query);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
index 4b76cb8..6bbf63b 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
@@ -61,7 +61,7 @@ public class TestExecutionBlockCursor {
     util.startCatalogCluster();
 
     conf = util.getConfiguration();
-    conf.set(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname, "false");
+    conf.set(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "false");
 
     catalog = util.getMiniCatalogCluster().getCatalog();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:!234/warehouse");
@@ -107,9 +107,9 @@ public class TestExecutionBlockCursor {
             "join supplier on s_nationkey = n_nationkey " +
             "join partsupp on s_suppkey = ps_suppkey " +
             "join part on p_partkey = ps_partkey and p_type like '%BRASS' and p_size = 15");
-    LogicalPlan logicalPlan = logicalPlanner.createPlan(LocalTajoTestingUtility.createDummySession(), context);
+    LogicalPlan logicalPlan = logicalPlanner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), context);
     optimizer.optimize(logicalPlan);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan plan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), queryContext, logicalPlan);
     planner.build(plan);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java
index 0ce7746..b6ac551 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java
@@ -112,9 +112,9 @@ public class TestGlobalPlanner {
 
   private MasterPlan buildPlan(String sql) throws PlanningException, IOException {
     Expr expr = sqlAnalyzer.parse(sql);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(util.getConfiguration()), expr);
     optimizer.optimize(plan);
-    QueryContext context = new QueryContext();
+    QueryContext context = new QueryContext(util.getConfiguration());
     MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), context, plan);
     globalPlanner.build(masterPlan);
     return masterPlan;

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
index e340953..b4be00b 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
@@ -44,7 +44,7 @@ public class TestQueryUnitStatusUpdate extends QueryTestCaseBase {
 
   @BeforeClass
   public static void setUp() throws Exception {
-    conf.set(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname, "false");
+    conf.set(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "false");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
index 5375749..5f8efe7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
@@ -143,12 +143,12 @@ public class TestRangeRetrieverHandler {
 
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employeeMeta, tableDir, Integer.MAX_VALUE);
 
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(),
         new FileFragment[] {frags[0]}, testDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(SORT_QUERY[0]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -267,12 +267,12 @@ public class TestRangeRetrieverHandler {
     FileFragment[] frags = sm.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE);
 
     TaskAttemptContext
-        ctx = new TaskAttemptContext(conf, new QueryContext(),
+        ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(),
         new FileFragment[] {frags[0]}, testDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(SORT_QUERY[1]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/resources/results/TestTajoCli/testHelpSessionVars.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCli/testHelpSessionVars.result b/tajo-core/src/test/resources/results/TestTajoCli/testHelpSessionVars.result
new file mode 100644
index 0000000..c294407
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTajoCli/testHelpSessionVars.result
@@ -0,0 +1,36 @@
+Available Session Variables:
+
+\set SESSION_EXPIRY_TIME [int value] - session expiry time (secs)
+\set CLI_COLUMNS [int value] - Sets the width for the wrapped format
+\set CLI_FORMATTER_CLASS [text value] - Sets the output format class to display results
+\set CLI_NULL_CHAR [text value] - Sets the string to be printed in place of a null value.
+\set CLI_PAGE_ROWS [int value] - Sets the number of rows for paging
+\set CLI_PAGING_ENABLED [true or false] - Enable paging of result display
+\set CLI_DISPLAY_ERROR_TRACE [true or false] - Enable display of error trace
+\set ON_ERROR_STOP [true or false] - tsql will exist if an error occurs.
+\set TZ [text value] - Sets timezone
+\set DATE_ORDER [text value] - date order (default is YMD)
+\set LANG [text value] - Language
+\set LC_ALL [text value] - String sort order
+\set LC_COLLATE [text value] - String sort order
+\set LC_CTYPE [text value] - Character classification (What is a letter? Its upper-case equivalent?)
+\set LC_MESSAGES [text value] - Language of messages
+\set LC_MONETARY [text value] - Formatting of currency amounts
+\set LC_NUMERIC [text value] - Formatting of numbers
+\set LC_TIME [text value] - Formatting of dates and times
+\set BROADCAST_TABLE_SIZE_LIMIT [long value] - limited size (bytes) of broadcast table
+\set JOIN_TASK_INPUT_SIZE [int value] - join task input size (mb) 
+\set SORT_TASK_INPUT_SIZE [int value] - sort task input size (mb)
+\set GROUPBY_TASK_INPUT_SIZE [int value] - group by task input size (mb)
+\set JOIN_PER_SHUFFLE_SIZE [int value] - shuffle output size for join (mb)
+\set GROUPBY_PER_SHUFFLE_SIZE [int value] - shuffle output size for sort (mb)
+\set TABLE_PARTITION_PER_SHUFFLE_SIZE [int value] - shuffle output size for partition table write (mb)
+\set EXTSORT_BUFFER_SIZE [long value] - sort buffer size for external sort (mb)
+\set HASH_JOIN_SIZE_LIMIT [long value] - limited size for hash join (mb)
+\set INNER_HASH_JOIN_SIZE_LIMIT [long value] - limited size for hash inner join (mb)
+\set OUTER_HASH_JOIN_SIZE_LIMIT [long value] - limited size for hash outer join (mb)
+\set HASH_GROUPBY_SIZE_LIMIT [long value] - limited size for hash groupby (mb)
+\set MAX_OUTPUT_FILE_SIZE [int value] - Maximum per-output file size (mb). 0 means infinite.
+\set NULL_CHAR [text value] - null char of text file output
+\set ARITHABORT [true or false] - If true, a running query will be terminated when an overflow or divide-by-zero occurs.
+\set DEBUG_ENABLED [true or false] - (debug only) debug mode enabled
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/resources/results/TestTajoCli/testSelectResultWithNullTrueDeprecated.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCli/testSelectResultWithNullTrueDeprecated.result b/tajo-core/src/test/resources/results/TestTajoCli/testSelectResultWithNullTrueDeprecated.result
new file mode 100644
index 0000000..36ea548
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTajoCli/testSelectResultWithNullTrueDeprecated.result
@@ -0,0 +1,9 @@
+Warning: deprecated to directly use config key in TajoConf.ConfVars. Please execute '\help set'.
+c_custkey,  o_orderkey,  o_orderstatus
+-------------------------------
+1,  1,  O
+2,  2,  O
+3,  3,  F
+4,  testnull,  testnull
+5,  testnull,  testnull
+(5 rows, , 30 B selected)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/resources/results/TestTajoCli/testStopWhenErrorDeprecated.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCli/testStopWhenErrorDeprecated.result b/tajo-core/src/test/resources/results/TestTajoCli/testStopWhenErrorDeprecated.result
new file mode 100644
index 0000000..5c49139
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTajoCli/testStopWhenErrorDeprecated.result
@@ -0,0 +1,6 @@
+Warning: deprecated to directly use config key in TajoConf.ConfVars. Please execute '\help set'.
+?count
+-------------------------------
+5
+(1 rows, , 2 B selected)
+ERROR: relation "default.lineitem2" does not exist

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
index ed6ea34..5b42cbd 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
@@ -30,7 +30,7 @@ public interface Appender extends Closeable {
   void addTuple(Tuple t) throws IOException;
   
   void flush() throws IOException;
-  
+
   void close() throws IOException;
 
   void enableStats();

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
index 8e26ec6..2113794 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -84,7 +84,9 @@ public class CSVFile {
       this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.CSVFILE_DELIMITER,
           StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
       this.columnNum = schema.size();
-      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.CSVFILE_NULL));
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.CSVFILE_NULL,
+          NullDatum.DEFAULT_TEXT));
       if (StringUtils.isEmpty(nullCharacters)) {
         nullChars = NullDatum.get().asTextBytes();
       } else {
@@ -107,8 +109,8 @@ public class CSVFile {
         isShuffle = false;
       }
 
-      String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
-      if(!StringUtils.isEmpty(codecName)){
+      if(this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
+        String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
         codecFactory = new CompressionCodecFactory(conf);
         codec = codecFactory.getCodecByClassName(codecName);
         compressor =  CodecPool.getCompressor(codec);
@@ -262,7 +264,8 @@ public class CSVFile {
       String delim  = meta.getOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
       this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0);
 
-      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.CSVFILE_NULL));
+      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.CSVFILE_NULL,
+          NullDatum.DEFAULT_TEXT));
       if (StringUtils.isEmpty(nullCharacters)) {
         nullChars = NullDatum.get().asTextBytes();
       } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
index 5b2d711..07fa16b 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
@@ -114,18 +114,18 @@ public class StorageUtil extends StorageConstants {
   public static KeyValueSet newPhysicalProperties(CatalogProtos.StoreType type) {
     KeyValueSet options = new KeyValueSet();
     if (CatalogProtos.StoreType.CSV == type) {
-      options.put(CSVFILE_DELIMITER, DEFAULT_FIELD_DELIMITER);
+      options.set(CSVFILE_DELIMITER, DEFAULT_FIELD_DELIMITER);
     } else if (CatalogProtos.StoreType.RCFILE == type) {
-      options.put(RCFILE_SERDE, DEFAULT_BINARY_SERDE);
+      options.set(RCFILE_SERDE, DEFAULT_BINARY_SERDE);
     } else if (CatalogProtos.StoreType.SEQUENCEFILE == type) {
-      options.put(SEQUENCEFILE_SERDE, DEFAULT_TEXT_SERDE);
-      options.put(SEQUENCEFILE_DELIMITER, DEFAULT_FIELD_DELIMITER);
+      options.set(SEQUENCEFILE_SERDE, DEFAULT_TEXT_SERDE);
+      options.set(SEQUENCEFILE_DELIMITER, DEFAULT_FIELD_DELIMITER);
     } else if (type == CatalogProtos.StoreType.PARQUET) {
-      options.put(ParquetOutputFormat.BLOCK_SIZE, PARQUET_DEFAULT_BLOCK_SIZE);
-      options.put(ParquetOutputFormat.PAGE_SIZE, PARQUET_DEFAULT_PAGE_SIZE);
-      options.put(ParquetOutputFormat.COMPRESSION, PARQUET_DEFAULT_COMPRESSION_CODEC_NAME);
-      options.put(ParquetOutputFormat.ENABLE_DICTIONARY, PARQUET_DEFAULT_IS_DICTIONARY_ENABLED);
-      options.put(ParquetOutputFormat.VALIDATION, PARQUET_DEFAULT_IS_VALIDATION_ENABLED);
+      options.set(ParquetOutputFormat.BLOCK_SIZE, PARQUET_DEFAULT_BLOCK_SIZE);
+      options.set(ParquetOutputFormat.PAGE_SIZE, PARQUET_DEFAULT_PAGE_SIZE);
+      options.set(ParquetOutputFormat.COMPRESSION, PARQUET_DEFAULT_COMPRESSION_CODEC_NAME);
+      options.set(ParquetOutputFormat.ENABLE_DICTIONARY, PARQUET_DEFAULT_IS_DICTIONARY_ENABLED);
+      options.set(ParquetOutputFormat.VALIDATION, PARQUET_DEFAULT_IS_VALIDATION_ENABLED);
     }
 
     return options;

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
index 962c63d..c15d20b 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
@@ -31,15 +31,20 @@ import org.apache.tajo.catalog.TableMeta;
 public class AvroUtil {
   public static Schema getAvroSchema(TableMeta meta, Configuration conf)
       throws IOException {
-    String schemaLiteral = meta.getOption(StorageConstants.AVRO_SCHEMA_LITERAL);
-    String schemaUrl = meta.getOption(StorageConstants.AVRO_SCHEMA_URL);
-    if (schemaLiteral == null && schemaUrl == null) {
+
+
+    boolean isSchemaLiteral = meta.containsOption(StorageConstants.AVRO_SCHEMA_LITERAL);
+    boolean isSchemaUrl = meta.containsOption(StorageConstants.AVRO_SCHEMA_URL);
+    if (!isSchemaLiteral && !isSchemaUrl) {
       throw new RuntimeException("No Avro schema for table.");
     }
-    if (schemaLiteral != null) {
-      return new Schema.Parser().parse(schemaLiteral);
+    if (isSchemaLiteral) {
+      String schema = meta.getOption(StorageConstants.AVRO_SCHEMA_LITERAL);
+      return new Schema.Parser().parse(schema);
     }
-    Path schemaPath = new Path(schemaUrl);
+
+    String schemaURL = meta.getOption(StorageConstants.AVRO_SCHEMA_URL);
+    Path schemaPath = new Path(schemaURL);
     FileSystem fs = schemaPath.getFileSystem(conf);
     FSDataInputStream inputStream = fs.open(schemaPath);
     return new Schema.Parser().parse(inputStream);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
index 78498c7..e5507ad 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
@@ -733,8 +733,8 @@ public class RCFile {
         isShuffle = false;
       }
 
-      String codecClassname = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
-      if (!StringUtils.isEmpty(codecClassname)) {
+      if (this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
+        String codecClassname = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
         try {
           Class<? extends CompressionCodec> codecClass = conf.getClassByName(
               codecClassname).asSubclass(CompressionCodec.class);
@@ -745,7 +745,8 @@ public class RCFile {
         }
       }
 
-      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.RCFILE_NULL));
+      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.RCFILE_NULL,
+          NullDatum.DEFAULT_TEXT));
       if (StringUtils.isEmpty(nullCharacters)) {
         nullChars = NullDatum.get().asTextBytes();
       } else {
@@ -1193,7 +1194,8 @@ public class RCFile {
       rowId = new LongWritable();
       readBytes = 0;
 
-      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.RCFILE_NULL));
+      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.RCFILE_NULL,
+          NullDatum.DEFAULT_TEXT));
       if (StringUtils.isEmpty(nullCharacters)) {
         nullChars = NullDatum.get().asTextBytes();
       } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
index 86d902a..f5cef62 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
@@ -97,7 +97,8 @@ public class SequenceFileAppender extends FileAppender {
     this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_DELIMITER,
         StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
     this.columnNum = schema.size();
-    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL));
+    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL,
+        NullDatum.DEFAULT_TEXT));
     if (StringUtils.isEmpty(nullCharacters)) {
       nullChars = NullDatum.get().asTextBytes();
     } else {
@@ -108,8 +109,8 @@ public class SequenceFileAppender extends FileAppender {
       throw new FileNotFoundException(path.toString());
     }
 
-    String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
-    if(!StringUtils.isEmpty(codecName)){
+    if(this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
+      String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
       codecFactory = new CompressionCodecFactory(conf);
       codec = codecFactory.getCodecByClassName(codecName);
     } else {
@@ -119,7 +120,8 @@ public class SequenceFileAppender extends FileAppender {
     }
 
     try {
-      String serdeClass = this.meta.getOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
+      String serdeClass = this.meta.getOption(StorageConstants.SEQUENCEFILE_SERDE,
+          TextSerializerDeserializer.class.getName());
       serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
index 3c39841..b0ef67d 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
@@ -25,7 +25,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.*;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
@@ -35,7 +34,6 @@ import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.BytesUtils;
-import org.apache.tajo.util.ReflectionUtil;
 
 import java.io.IOException;
 
@@ -86,7 +84,8 @@ public class SequenceFileScanner extends FileScanner {
 
     reader = new SequenceFile.Reader(fs, fragment.getPath(), conf);
 
-    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL));
+    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL,
+        NullDatum.DEFAULT_TEXT));
     if (StringUtils.isEmpty(nullCharacters)) {
       nullChars = NullDatum.get().asTextBytes();
     } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
index 5d2f861..c8e19dd 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
@@ -70,7 +70,7 @@ public class RCFileScanner extends FileScannerV2 {
     key = new LongWritable();
     column = new BytesRefArrayWritable();
 
-    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(NULL));
+    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(NULL, NullDatum.DEFAULT_TEXT));
     if (StringUtils.isEmpty(nullCharacters)) {
       nullChars = NullDatum.get().asTextBytes();
     } else {


[3/5] TAJO-928: Session variables should override query configs in TajoConf.

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java b/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
index a67b6c8..cdf552d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.master.session;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.common.ProtoObject;
 
@@ -30,17 +31,21 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.SessionProto;
 public class Session implements SessionConstants, ProtoObject<SessionProto> {
   private final String sessionId;
   private final String userName;
+  private String currentDatabase;
   private final Map<String, String> sessionVariables;
 
   // transient status
   private volatile long lastAccessTime;
-  private volatile String currentDatabase;
 
   public Session(String sessionId, String userName, String databaseName) {
     this.sessionId = sessionId;
     this.userName = userName;
+    this.currentDatabase = databaseName;
     this.lastAccessTime = System.currentTimeMillis();
+
     this.sessionVariables = new HashMap<String, String>();
+    sessionVariables.put(SessionVars.SESSION_ID.keyname(), sessionId);
+    sessionVariables.put(SessionVars.USERNAME.keyname(), userName);
     selectDatabase(databaseName);
   }
 
@@ -71,64 +76,61 @@ public class Session implements SessionConstants, ProtoObject<SessionProto> {
 
   public void setVariable(String name, String value) {
     synchronized (sessionVariables) {
-      sessionVariables.put(name, value);
+      sessionVariables.put(SessionVars.handleDeprecatedName(name), value);
     }
   }
 
   public String getVariable(String name) throws NoSuchSessionVariableException {
     synchronized (sessionVariables) {
       if (sessionVariables.containsKey(name)) {
-        return sessionVariables.get(name);
+        return sessionVariables.get(SessionVars.handleDeprecatedName(name));
       } else {
         throw new NoSuchSessionVariableException(name);
       }
     }
   }
 
-  public String getVariable(String name, String defaultValue) {
-    synchronized (sessionVariables) {
-      if (sessionVariables.containsKey(name)) {
-        return sessionVariables.get(name);
-      } else {
-        return defaultValue;
-      }
-    }
-  }
-
   public void removeVariable(String name) {
     synchronized (sessionVariables) {
-      sessionVariables.remove(name);
+      sessionVariables.remove(SessionVars.handleDeprecatedName(name));
     }
   }
 
   public synchronized Map<String, String> getAllVariables() {
     synchronized (sessionVariables) {
+      sessionVariables.put(SessionVars.SESSION_ID.keyname(), sessionId);
+      sessionVariables.put(SessionVars.USERNAME.keyname(), userName);
+      sessionVariables.put(SessionVars.SESSION_LAST_ACCESS_TIME.keyname(), String.valueOf(lastAccessTime));
+      sessionVariables.put(SessionVars.CURRENT_DATABASE.keyname(), currentDatabase);
       return ImmutableMap.copyOf(sessionVariables);
     }
   }
 
-  public void selectDatabase(String databaseName) {
+  public synchronized void selectDatabase(String databaseName) {
     this.currentDatabase = databaseName;
   }
 
-  public String getCurrentDatabase() {
-    return this.currentDatabase;
+  public synchronized String getCurrentDatabase() {
+    return currentDatabase;
   }
 
   @Override
   public SessionProto getProto() {
     SessionProto.Builder builder = SessionProto.newBuilder();
-    builder.setSessionId(sessionId);
-    builder.setUsername(userName);
-    builder.setCurrentDatabase(currentDatabase);
+    builder.setSessionId(getSessionId());
+    builder.setUsername(getUserName());
+    builder.setCurrentDatabase(getCurrentDatabase());
     builder.setLastAccessTime(lastAccessTime);
     KeyValueSet variables = new KeyValueSet();
-    variables.putAll(this.sessionVariables);
-    builder.setVariables(variables.getProto());
-    return builder.build();
+
+    synchronized (sessionVariables) {
+      variables.putAll(this.sessionVariables);
+      builder.setVariables(variables.getProto());
+      return builder.build();
+    }
   }
 
   public String toString() {
-    return "user=" + userName + ",id=" + sessionId;
+    return "user=" + getUserName() + ",id=" + getSessionId() +",last_atime=" + getLastAccessTime();
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
index 483920f..912f769 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
@@ -40,7 +40,7 @@ public class SessionLivelinessMonitor extends AbstractLivelinessMonitor<String>
     TajoConf systemConf = (TajoConf) conf;
 
     // seconds
-    int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.CLIENT_SESSION_EXPIRY_TIME);
+    int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.$CLIENT_SESSION_EXPIRY_TIME);
     setExpireInterval(expireIntvl);
     setMonitorInterval(expireIntvl / 3);
     super.serviceInit(conf);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
index 195b35e..200892a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
@@ -148,7 +148,7 @@ public class Task {
     this.taskDir = StorageUtil.concatPath(taskRunnerContext.getBaseDir(),
         taskId.getQueryUnitId().getId() + "_" + taskId.getId());
 
-    this.context = new TaskAttemptContext(systemConf, queryContext, taskId,
+    this.context = new TaskAttemptContext(queryContext, taskId,
         request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir);
     this.context.setDataChannel(request.getDataChannel());
     this.context.setEnforcer(request.getEnforcer());
@@ -537,7 +537,7 @@ public class Task {
         FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder();
         for (Fetcher fetcher : fetcherRunners) {
           // TODO store the fetcher histories
-          if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_DEBUG)) {
+          if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) {
             builder.setStartTime(fetcher.getStartTime());
             builder.setFinishTime(fetcher.getFinishTime());
             builder.setFileLength(fetcher.getFileLen());

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
index db4af45..211f953 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java
@@ -50,7 +50,6 @@ import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
  */
 public class TaskAttemptContext {
   private static final Log LOG = LogFactory.getLog(TaskAttemptContext.class);
-  private final TajoConf conf;
   private final Map<String, List<FragmentProto>> fragmentMap = Maps.newHashMap();
 
   private TaskAttemptState state;
@@ -76,10 +75,9 @@ public class TaskAttemptContext {
   /** a output volume for each partition */
   private Map<Integer, Long> partitionOutputVolume;
 
-  public TaskAttemptContext(TajoConf conf, QueryContext queryContext, final QueryUnitAttemptId queryId,
+  public TaskAttemptContext(final QueryContext queryContext, final QueryUnitAttemptId queryId,
                             final FragmentProto[] fragments,
                             final Path workDir) {
-    this.conf = conf;
     this.queryContext = queryContext;
     this.queryId = queryId;
 
@@ -104,15 +102,15 @@ public class TaskAttemptContext {
   }
 
   @VisibleForTesting
-  public TaskAttemptContext(TajoConf conf, QueryContext queryContext, final QueryUnitAttemptId queryId,
+  public TaskAttemptContext(final QueryContext queryContext, final QueryUnitAttemptId queryId,
                             final Fragment [] fragments,  final Path workDir) {
-    this(conf, queryContext, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir);
+    this(queryContext, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir);
   }
 
   public TajoConf getConf() {
-    return this.conf;
+    return queryContext.getConf();
   }
-  
+
   public TaskAttemptState getState() {
     return this.state;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
index c0bee9b..3c40e99 100644
--- a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
@@ -29,6 +29,8 @@
 <%@ page import="org.apache.tajo.worker.TajoWorker" %>
 <%@ page import="java.text.SimpleDateFormat" %>
 <%@ page import="java.util.List" %>
+<%@ page import="java.util.Map" %>
+<%@ page import="org.apache.tajo.SessionVars" %>
 
 <%
   QueryId queryId = TajoIdUtils.parseQueryId(request.getParameter("queryId"));
@@ -93,6 +95,14 @@ for(SubQuery eachSubQuery: subQueries) {
   %>
   </table>
   <p/>
+  <h3>Applied Session Variables</h3>
+  <table width="100%" border="1" class="border_table">
+  <%for(Map.Entry<String,String> entry: query.getPlan().getContext().getAllKeyValus().entrySet()) {
+      if (SessionVars.exists(entry.getKey()) && SessionVars.isPublic(SessionVars.get(entry.getKey()))) {
+          %> <tr><td width="200"><%=entry.getKey()%></td><td><%=entry.getValue()%></td> <%
+      }
+  } %>
+  </table>
   <hr/>
   <h3>Logical Plan</h3>
   <pre style="white-space:pre-wrap;"><%=query.getPlan().getLogicalPlan().toString()%></pre>

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/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 84522e1..271ba70 100644
--- a/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
+++ b/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
@@ -31,6 +31,7 @@ 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.master.session.Session;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TajoIdUtils;
@@ -69,6 +70,10 @@ public class LocalTajoTestingUtility {
     return new Session(UUID.randomUUID().toString(), dummyUserInfo.getUserName(), TajoConstants.DEFAULT_DATABASE_NAME);
   }
 
+  public static QueryContext createDummyContext(TajoConf conf) {
+    return new QueryContext(conf, createDummySession());
+  }
+
   /**
    * for test
    * @return The generated QueryId

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/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 9d15732..8cd1bff 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -214,7 +214,7 @@ public class QueryTestCaseBase {
     } catch (ServiceException e) {
       e.printStackTrace();
     }
-    testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname, "false");
+    testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "false");
   }
 
   protected TajoClient getClient() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java b/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java
index 5ed8821..aacae36 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TpchTestBase.java
@@ -91,7 +91,7 @@ public class TpchTestBase {
   private void setUp() throws Exception {
     util = new LocalTajoTestingUtility();
     KeyValueSet opt = new KeyValueSet();
-    opt.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    opt.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
     util.setup(names, paths, schemas, opt);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/cli/TestDefaultCliOutputFormatter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/TestDefaultCliOutputFormatter.java b/tajo-core/src/test/java/org/apache/tajo/cli/TestDefaultCliOutputFormatter.java
index e13eeef..bd2b4e1 100644
--- a/tajo-core/src/test/java/org/apache/tajo/cli/TestDefaultCliOutputFormatter.java
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/TestDefaultCliOutputFormatter.java
@@ -18,6 +18,9 @@
 
 package org.apache.tajo.cli;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.TpchTestBase;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.common.TajoDataTypes;
@@ -27,10 +30,14 @@ import org.apache.tajo.datum.Int4Datum;
 import org.apache.tajo.datum.TextDatum;
 import org.apache.tajo.jdbc.MetaDataTuple;
 import org.apache.tajo.jdbc.TajoMetaDataResultSet;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
+import java.io.ByteArrayOutputStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.net.URL;
 import java.sql.ResultSet;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -39,6 +46,38 @@ import java.util.List;
 import static org.junit.Assert.assertEquals;
 
 public class TestDefaultCliOutputFormatter {
+  protected static final TpchTestBase testBase;
+  protected static final TajoTestingCluster cluster;
+
+  /** the base path of result directories */
+  protected static final Path resultBasePath;
+  static {
+    testBase = TpchTestBase.getInstance();
+    cluster = testBase.getTestingCluster();
+    URL resultBaseURL = ClassLoader.getSystemResource("results");
+    resultBasePath = new Path(resultBaseURL.toString());
+  }
+
+  private TajoConf conf;
+  private TajoCli tajoCli;
+  private TajoCli.TajoCliContext cliContext;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = cluster.getConfiguration();
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    tajoCli = new TajoCli(conf, new String[]{}, System.in, out);
+    cliContext = tajoCli.getContext();
+  }
+
+  @After
+  public void tearDown() {
+    if (tajoCli != null) {
+      tajoCli.close();
+    }
+  }
+
+
   @Test
   public void testParseErrorMessage() {
     String message = "java.sql.SQLException: ERROR: no such a table: table1";
@@ -65,9 +104,10 @@ public class TestDefaultCliOutputFormatter {
 
   @Test
   public void testPrintResultInsertStatement() throws Exception {
-    TajoConf tajoConf = new TajoConf();
+
+
     DefaultTajoCliOutputFormatter outputFormatter = new DefaultTajoCliOutputFormatter();
-    outputFormatter.init(tajoConf);
+    outputFormatter.init(cliContext);
 
     float responseTime = 10.1f;
     long numBytes = 102;
@@ -89,9 +129,8 @@ public class TestDefaultCliOutputFormatter {
 
   @Test
   public void testPrintResultSelectStatement() throws Exception {
-    TajoConf tajoConf = new TajoConf();
     DefaultTajoCliOutputFormatter outputFormatter = new DefaultTajoCliOutputFormatter();
-    outputFormatter.init(tajoConf);
+    outputFormatter.init(cliContext);
 
     float responseTime = 10.1f;
     long numBytes = 102;

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/cli/TestTajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/TestTajoCli.java b/tajo-core/src/test/java/org/apache/tajo/cli/TestTajoCli.java
index 0631b6e..5795f0c 100644
--- a/tajo-core/src/test/java/org/apache/tajo/cli/TestTajoCli.java
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/TestTajoCli.java
@@ -23,15 +23,16 @@ import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.PosixParser;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ConfigKey;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.TpchTestBase;
 import org.apache.tajo.client.QueryStatus;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.NetUtils;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -41,9 +42,7 @@ import java.io.File;
 import java.io.PrintWriter;
 import java.net.URL;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 public class TestTajoCli {
   protected static final TpchTestBase testBase;
@@ -60,6 +59,7 @@ public class TestTajoCli {
 
   private TajoCli tajoCli;
   private Path currentResultPath;
+  private ByteArrayOutputStream out;
 
   @Rule
   public TestName name = new TestName();
@@ -69,6 +69,12 @@ public class TestTajoCli {
     currentResultPath = new Path(resultBasePath, className);
   }
 
+  @Before
+  public void setUp() throws Exception {
+    out = new ByteArrayOutputStream();
+    tajoCli = new TajoCli(cluster.getConfiguration(), new String[]{}, System.in, out);
+  }
+
   @After
   public void tearDown() {
     if (tajoCli != null) {
@@ -76,6 +82,40 @@ public class TestTajoCli {
     }
   }
 
+  private static void setVar(TajoCli cli, ConfigKey key, String val) throws Exception {
+    cli.executeMetaCommand("\\set " + key.keyname() +" " + val);
+  }
+
+  private static void assertSessionVar(TajoCli cli, String key, String expectedVal) {
+    assertEquals(cli.getContext().getCliSideVar(key), expectedVal);
+  }
+
+  private void assertOutputResult(String actual) throws Exception {
+    assertOutputResult(name.getMethodName() + ".result", actual);
+  }
+
+  private void assertOutputResult(String expectedResultFile, String actual) throws Exception {
+    assertOutputResult(expectedResultFile, actual, null, null);
+  }
+
+  private void assertOutputResult(String expectedResultFile, String actual, String[] paramKeys, String[] paramValues)
+      throws Exception {
+    FileSystem fs = currentResultPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
+    Path resultFile = StorageUtil.concatPath(currentResultPath, expectedResultFile);
+    assertTrue(resultFile.toString() + " existence check", fs.exists(resultFile));
+
+    String expectedResult = FileUtil.readTextFile(new File(resultFile.toUri()));
+
+    if (paramKeys != null) {
+      for (int i = 0; i < paramKeys.length; i++) {
+        if (i < paramValues.length) {
+          expectedResult = expectedResult.replace(paramKeys[i], paramValues[i]);
+        }
+      }
+    }
+    assertEquals(expectedResult.trim(), actual.trim());
+  }
+
   @Test
   public void testParseParam() throws Exception {
     String[] args = new String[]{"-f", "test.sql", "--param", "test1=10", "--param", "test2=20"};
@@ -112,10 +152,9 @@ public class TestTajoCli {
     assertEquals("tajo.executor.join.inner.in-memory-table-num=256", confValues[1]);
 
     TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-
-    tajoCli = new TajoCli(tajoConf, args, System.in, System.out);
-    assertEquals("false", tajoCli.getContext().getConf().get("tajo.cli.print.pause"));
-    assertEquals("256", tajoCli.getContext().getConf().get("tajo.executor.join.inner.in-memory-table-num"));
+    TajoCli testCli = new TajoCli(tajoConf, args, System.in, System.out);
+    assertEquals("false", testCli.getContext().get(SessionVars.CLI_PAGING_ENABLED));
+    assertEquals("256", testCli.getContext().getConf().get("tajo.executor.join.inner.in-memory-table-num"));
   }
 
   @Test
@@ -131,10 +170,7 @@ public class TestTajoCli {
   @Test
   public void testLocalQueryWithoutFrom() throws Exception {
     String sql = "select 'abc', '123'; select substr('123456', 1,3);";
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
     tajoCli.executeScript(sql);
     String consoleResult = new String(out.toByteArray());
 
@@ -151,11 +187,7 @@ public class TestTajoCli {
       databaseName = "TEST_CONNECTION_DATABASE";
     }
     String sql = "create database \"" + databaseName + "\";";
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
 
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
     tajoCli.executeScript(sql);
 
     tajoCli.executeMetaCommand("\\c " + databaseName);
@@ -179,11 +211,7 @@ public class TestTajoCli {
 
     String sql = "create table \"" + tableName + "\" (col1 int4, col2 int4);";
 
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
-
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
     tajoCli.executeScript(sql);
 
     tajoCli.executeMetaCommand("\\d " + tableName);
@@ -211,79 +239,51 @@ public class TestTajoCli {
             "  c_custkey,\n" +
             "  orders.o_orderkey;\n";
 
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
-
-    tajoConf.setVar(ConfVars.CLI_NULL_CHAR, "");
-
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
     tajoCli.executeScript(sql);
 
     String consoleResult = new String(out.toByteArray());
     assertOutputResult(consoleResult);
   }
 
-  @Test
-  public void testSelectResultWithNullTrue() throws Exception {
+  private void verifySelectResultWithNullTrue() throws Exception {
     String sql =
         "select\n" +
-        "  c_custkey,\n" +
-        "  orders.o_orderkey,\n" +
-        "  orders.o_orderstatus \n" +
-        "from\n" +
-        "  orders full outer join customer on c_custkey = o_orderkey\n" +
-        "order by\n" +
-        "  c_custkey,\n" +
-        "  orders.o_orderkey;\n";
+            "  c_custkey,\n" +
+            "  orders.o_orderkey,\n" +
+            "  orders.o_orderstatus \n" +
+            "from\n" +
+            "  orders full outer join customer on c_custkey = o_orderkey\n" +
+            "order by\n" +
+            "  c_custkey,\n" +
+            "  orders.o_orderkey;\n";
 
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
 
-    tajoConf.setVar(ConfVars.CLI_NULL_CHAR, "testnull");
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
+    assertSessionVar(tajoCli, SessionVars.CLI_NULL_CHAR.keyname(), "testnull");
 
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
     tajoCli.executeScript(sql);
 
     String consoleResult = new String(out.toByteArray());
     assertOutputResult(consoleResult);
   }
 
-  private void assertOutputResult(String actual) throws Exception {
-    assertOutputResult(name.getMethodName() + ".result", actual);
-  }
-
-  private void assertOutputResult(String expectedResultFile, String actual) throws Exception {
-    assertOutputResult(expectedResultFile, actual, null, null);
+  @Test
+  public void testSelectResultWithNullTrueDeprecated() throws Exception {
+    setVar(tajoCli, TajoConf.ConfVars.$CLI_NULL_CHAR, "testnull");
+    verifySelectResultWithNullTrue();
   }
 
-  private void assertOutputResult(String expectedResultFile, String actual, String[] paramKeys, String[] paramValues)
-      throws Exception {
-    FileSystem fs = currentResultPath.getFileSystem(testBase.getTestingCluster().getConfiguration());
-    Path resultFile = StorageUtil.concatPath(currentResultPath, expectedResultFile);
-    assertTrue(resultFile.toString() + " existence check", fs.exists(resultFile));
-
-    String expectedResult = FileUtil.readTextFile(new File(resultFile.toUri()));
-
-    if (paramKeys != null) {
-      for (int i = 0; i < paramKeys.length; i++) {
-        if (i < paramValues.length) {
-          expectedResult = expectedResult.replace(paramKeys[i], paramValues[i]);
-        }
-      }
-    }
-    assertEquals(expectedResult, actual);
+  @Test
+  public void testSelectResultWithNullTrue() throws Exception {
+    setVar(tajoCli, SessionVars.CLI_NULL_CHAR, "testnull");
+    verifySelectResultWithNullTrue();
   }
 
-  @Test
-  public void testStopWhenError() throws Exception {
-    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
+  private void verifyStopWhenError() throws Exception {
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
 
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
-    tajoCli.executeMetaCommand("\\set tajo.cli.error.stop true");
+    assertSessionVar(tajoCli, SessionVars.ON_ERROR_STOP.keyname(), "true");
 
     tajoCli.executeScript("select count(*) from lineitem; " +
         "select count(*) from lineitem2; " +
@@ -296,20 +296,20 @@ public class TestTajoCli {
   @Test
   public void testGetConf() throws Exception {
     TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
 
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
     tajoCli.executeMetaCommand("\\getconf tajo.rootdir");
 
     String consoleResult = new String(out.toByteArray());
-    assertEquals(consoleResult, tajoCli.getContext().getConf().getVar(ConfVars.ROOT_DIR) + "\n");
+    assertEquals(consoleResult, tajoCli.getContext().getConf().getVar(TajoConf.ConfVars.ROOT_DIR) + "\n");
   }
 
   @Test
   public void testShowMasters() throws Exception {
     TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
-    tajoConf.setVar(ConfVars.CLI_OUTPUT_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
+    setVar(tajoCli, SessionVars.CLI_FORMATTER_CLASS, TajoCliOutputTestFormatter.class.getName());
 
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     tajoCli = new TajoCli(tajoConf, new String[]{}, System.in, out);
@@ -317,12 +317,30 @@ public class TestTajoCli {
 
     String consoleResult = new String(out.toByteArray());
 
-    String masterAddress = tajoCli.getContext().getConf().getVar(ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS);
+    String masterAddress = tajoCli.getContext().getConf().getVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS);
     String host = masterAddress.split(":")[0];
 
     assertEquals(consoleResult, host + "\n");
   }
 
+  @Test
+  public void testStopWhenErrorDeprecated() throws Exception {
+    tajoCli.executeMetaCommand("\\set tajo.cli.error.stop true");
+    verifyStopWhenError();
+  }
+
+  @Test
+  public void testStopWhenError() throws Exception {
+    tajoCli.executeMetaCommand("\\set ON_ERROR_STOP true");
+    verifyStopWhenError();
+  }
+
+  @Test
+  public void testHelpSessionVars() throws Exception {
+    tajoCli.executeMetaCommand("\\help set");
+    assertOutputResult(new String(out.toByteArray()));
+  }
+
   public static class TajoCliOutputTestFormatter extends DefaultTajoCliOutputFormatter {
     @Override
     protected String getResponseTimeReadable(float responseTime) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/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 732f02c..3d6ed2b 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
@@ -33,7 +33,6 @@ import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.jdbc.TajoResultSet;
 import org.apache.tajo.storage.StorageConstants;
@@ -176,7 +175,8 @@ public class TestTajoClient {
       String key = prefixName + i;
       String val = prefixValue + i;
 
-      assertEquals(i, client.getAllSessionVariables().size());
+      // Basically,
+      assertEquals(i + 4, client.getAllSessionVariables().size());
       assertFalse(client.getAllSessionVariables().containsKey(key));
       assertFalse(client.existSessionVariable(key));
 
@@ -184,7 +184,7 @@ public class TestTajoClient {
       map.put(key, val);
       client.updateSessionVariables(map);
 
-      assertEquals(i + 1, client.getAllSessionVariables().size());
+      assertEquals(i + 5, client.getAllSessionVariables().size());
       assertTrue(client.getAllSessionVariables().containsKey(key));
       assertTrue(client.existSessionVariable(key));
     }
@@ -690,7 +690,7 @@ public class TestTajoClient {
     TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
 
     Map<String, String> variables = new HashMap<String, String>();
-    variables.put(ConfVars.CSVFILE_NULL.varname, "\\\\T");
+    variables.put(SessionVars.NULL_CHAR.keyname(), "\\\\T");
     client.updateSessionVariables(variables);
 
     TajoResultSet res = (TajoResultSet)client.executeQueryAndGetResult(sql);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/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 ad80ddf..7f9436a 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
@@ -32,6 +32,7 @@ import org.apache.tajo.datum.*;
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.engine.utils.SchemaUtil;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.session.Session;
@@ -107,22 +108,24 @@ public class ExprTestBase {
       InvalidStatementException {
 
     Session session = LocalTajoTestingUtility.createDummySession();
+    QueryContext context = new QueryContext(util.getConfiguration(), session);
+
     List<ParsedResult> parsedResults = SimpleParser.parseScript(query);
     if (parsedResults.size() > 1) {
       throw new RuntimeException("this query includes two or more statements.");
     }
     Expr expr = analyzer.parse(parsedResults.get(0).getHistoryStatement());
     VerificationState state = new VerificationState();
-    preLogicalPlanVerifier.verify(session, state, expr);
+    preLogicalPlanVerifier.verify(context, state, expr);
     if (state.getErrorMessages().size() > 0) {
       if (!condition && state.getErrorMessages().size() > 0) {
         throw new PlanningException(state.getErrorMessages().get(0));
       }
       assertFalse(state.getErrorMessages().get(0), true);
     }
-    LogicalPlan plan = planner.createPlan(session, expr, true);
+    LogicalPlan plan = planner.createPlan(context, expr, true);
     optimizer.optimize(plan);
-    annotatedPlanVerifier.verify(session, state, plan);
+    annotatedPlanVerifier.verify(context, state, plan);
 
     if (state.getErrorMessages().size() > 0) {
       assertFalse(state.getErrorMessages().get(0), true);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
index 7bb619d..35f5aa2 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
@@ -40,9 +40,9 @@ import org.apache.tajo.engine.planner.Target;
 import org.apache.tajo.engine.planner.logical.GroupbyNode;
 import org.apache.tajo.engine.planner.logical.NodeType;
 import org.apache.tajo.engine.planner.nameresolver.NameResolvingMode;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.AfterClass;
@@ -67,7 +67,7 @@ public class TestEvalTreeUtil {
   static EvalNode expr3;
   static SQLAnalyzer analyzer;
   static LogicalPlanner planner;
-  static Session session = LocalTajoTestingUtility.createDummySession();
+  static QueryContext defaultContext;
 
   public static class TestSum extends GeneralFunction {
     private Integer x;
@@ -123,6 +123,8 @@ public class TestEvalTreeUtil {
         "select name, score, age from people where test_sum(score * age, 50)", // 2
     };
 
+    defaultContext = LocalTajoTestingUtility.createDummyContext(util.getConfiguration());
+
     expr1 = getRootSelection(QUERIES[0]);
     expr2 = getRootSelection(QUERIES[1]);
     expr3 = getRootSelection(QUERIES[2]);
@@ -137,7 +139,7 @@ public class TestEvalTreeUtil {
     Expr expr = analyzer.parse(query);
     LogicalPlan plan = null;
     try {
-      plan = planner.createPlan(session, expr);
+      plan = planner.createPlan(defaultContext, expr);
     } catch (PlanningException e) {
       e.printStackTrace();
     }
@@ -149,7 +151,7 @@ public class TestEvalTreeUtil {
     Expr block = analyzer.parse(query);
     LogicalPlan plan = null;
     try {
-      plan = planner.createPlan(session, block);
+      plan = planner.createPlan(defaultContext, block);
     } catch (PlanningException e) {
       e.printStackTrace();
     }
@@ -225,7 +227,7 @@ public class TestEvalTreeUtil {
   @Test
   public final void testGetContainExprs() throws CloneNotSupportedException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[1]);
-    LogicalPlan plan = planner.createPlan(session, expr, true);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr, true);
     Target [] targets = plan.getRootBlock().getRawTargets();
     Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4);
     Collection<EvalNode> exprs =
@@ -303,7 +305,7 @@ public class TestEvalTreeUtil {
     assertTrue(7.0d == node.eval(null, null).asFloat8());
 
     Expr expr = analyzer.parse(QUERIES[1]);
-    LogicalPlan plan = planner.createPlan(session, expr, true);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr, true);
     targets = plan.getRootBlock().getRawTargets();
     Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4);
     Collection<EvalNode> exprs =
@@ -343,7 +345,7 @@ public class TestEvalTreeUtil {
   public final void testFindDistinctAggFunctions() throws PlanningException {
     String query = "select sum(score) + max(age) from people";
     Expr expr = analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
     GroupbyNode groupByNode = plan.getRootBlock().getNode(NodeType.GROUP_BY);
     EvalNode [] aggEvals = groupByNode.getAggFunctions();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
index 773382d..82e7f21 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
@@ -29,8 +29,8 @@ import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.engine.function.builtin.SumInt;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.junit.AfterClass;
@@ -48,7 +48,7 @@ public class TestLogicalOptimizer {
   private static SQLAnalyzer sqlAnalyzer;
   private static LogicalPlanner planner;
   private static LogicalOptimizer optimizer;
-  private static Session session = LocalTajoTestingUtility.createDummySession();
+  private static QueryContext defaultContext;
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -101,6 +101,8 @@ public class TestLogicalOptimizer {
     sqlAnalyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
     optimizer = new LogicalOptimizer(util.getConfiguration());
+
+    defaultContext = LocalTajoTestingUtility.createDummyContext(util.getConfiguration());
   }
 
   @AfterClass
@@ -121,7 +123,7 @@ public class TestLogicalOptimizer {
   public final void testProjectionPushWithNaturalJoin() throws PlanningException, CloneNotSupportedException {
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[4]);
-    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalPlan newPlan = planner.createPlan(defaultContext, expr);
     LogicalNode plan = newPlan.getRootBlock().getRoot();
     assertEquals(NodeType.ROOT, plan.getType());
     LogicalRootNode root = (LogicalRootNode) plan;
@@ -148,7 +150,7 @@ public class TestLogicalOptimizer {
   public final void testProjectionPushWithInnerJoin() throws PlanningException {
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[5]);
-    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalPlan newPlan = planner.createPlan(defaultContext, expr);
     optimizer.optimize(newPlan);
   }
   
@@ -156,7 +158,7 @@ public class TestLogicalOptimizer {
   public final void testProjectionPush() throws CloneNotSupportedException, PlanningException {
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[2]);
-    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalPlan newPlan = planner.createPlan(defaultContext, expr);
     LogicalNode plan = newPlan.getRootBlock().getRoot();
     
     assertEquals(NodeType.ROOT, plan.getType());
@@ -178,7 +180,7 @@ public class TestLogicalOptimizer {
   @Test
   public final void testOptimizeWithGroupBy() throws CloneNotSupportedException, PlanningException {
     Expr expr = sqlAnalyzer.parse(QUERIES[3]);
-    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalPlan newPlan = planner.createPlan(defaultContext, expr);
     LogicalNode plan = newPlan.getRootBlock().getRoot();
         
     assertEquals(NodeType.ROOT, plan.getType());
@@ -205,7 +207,7 @@ public class TestLogicalOptimizer {
   public final void testPushable() throws CloneNotSupportedException, PlanningException {
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[0]);
-    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalPlan newPlan = planner.createPlan(defaultContext, expr);
     LogicalNode plan = newPlan.getRootBlock().getRoot();
     
     assertEquals(NodeType.ROOT, plan.getType());
@@ -236,7 +238,7 @@ public class TestLogicalOptimizer {
     
     // Scan Pushable Test
     expr = sqlAnalyzer.parse(QUERIES[1]);
-    newPlan = planner.createPlan(session, expr);
+    newPlan = planner.createPlan(defaultContext, expr);
     plan = newPlan.getRootBlock().getRoot();
     
     assertEquals(NodeType.ROOT, plan.getType());
@@ -258,7 +260,7 @@ public class TestLogicalOptimizer {
   @Test
   public final void testInsertInto() throws CloneNotSupportedException, PlanningException {
     Expr expr = sqlAnalyzer.parse(TestLogicalPlanner.insertStatements[0]);
-    LogicalPlan newPlan = planner.createPlan(session, expr);
+    LogicalPlan newPlan = planner.createPlan(defaultContext, expr);
     optimizer.optimize(newPlan);
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
index 2f1e0f9..934beef 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.engine.planner;
 
-import org.apache.tajo.LocalTajoTestingUtility;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.benchmark.TPCH;
 import org.apache.tajo.catalog.*;
@@ -88,7 +87,7 @@ public class TestLogicalPlan {
 
   @Test
   public final void testQueryBlockGraph() {
-    LogicalPlan plan = new LogicalPlan(LocalTajoTestingUtility.createDummySession().getCurrentDatabase(), planner);
+    LogicalPlan plan = new LogicalPlan(planner);
     LogicalPlan.QueryBlock root = plan.newAndGetBlock(LogicalPlan.ROOT_BLOCK);
     LogicalPlan.QueryBlock new1 = plan.newQueryBlock();
     LogicalPlan.QueryBlock new2 = plan.newQueryBlock();

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 6d0f7e0..73f72be 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -37,6 +37,7 @@ import org.apache.tajo.engine.function.builtin.SumInt;
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.session.Session;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -154,8 +155,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSingleRelation() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(QUERIES[0]);
-    LogicalPlan planNode = planner.createPlan(session, expr);
+    LogicalPlan planNode = planner.createPlan(qc, expr);
     LogicalNode plan = planNode.getRootBlock().getRoot();
     assertEquals(NodeType.ROOT, plan.getType());
     TestLogicalNode.testCloneLogicalNode(plan);
@@ -186,9 +189,11 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testImplicityJoinPlan() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[1]);
-    LogicalPlan planNode = planner.createPlan(session, expr);
+    LogicalPlan planNode = planner.createPlan(qc, expr);
     LogicalNode plan = planNode.getRootBlock().getRoot();
 
     assertEquals(NodeType.ROOT, plan.getType());
@@ -221,7 +226,7 @@ public class TestLogicalPlanner {
 
     // three relations
     expr = sqlAnalyzer.parse(QUERIES[2]);
-    plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     TestLogicalNode.testCloneLogicalNode(plan);
 
@@ -273,9 +278,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testNaturalJoinPlan() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
     // two relations
     Expr context = sqlAnalyzer.parse(JOINS[0]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     assertSchema(expectedJoinSchema, plan.getOutSchema());
 
@@ -304,9 +310,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInnerJoinPlan() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
     // two relations
     Expr expr = sqlAnalyzer.parse(JOINS[1]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     LogicalNode root = plan.getRootBlock().getRoot();
     testJsonSerDerObject(root);
     assertSchema(expectedJoinSchema, root.getOutSchema());
@@ -336,9 +343,11 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testOuterJoinPlan() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     // two relations
     Expr expr = sqlAnalyzer.parse(JOINS[2]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     assertSchema(expectedJoinSchema, plan.getOutSchema());
 
@@ -369,9 +378,11 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testGroupby() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     // without 'having clause'
     Expr context = sqlAnalyzer.parse(QUERIES[7]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot();
 
     assertEquals(NodeType.ROOT, plan.getType());
     LogicalRootNode root = (LogicalRootNode) plan;
@@ -380,7 +391,7 @@ public class TestLogicalPlanner {
 
     // with having clause
     context = sqlAnalyzer.parse(QUERIES[3]);
-    plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    plan = planner.createPlan(qc, context).getRootBlock().getRoot();
     TestLogicalNode.testCloneLogicalNode(plan);
 
     assertEquals(NodeType.ROOT, plan.getType());
@@ -411,7 +422,8 @@ public class TestLogicalPlanner {
   public final void testMultipleJoin() throws IOException, PlanningException {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File("src/test/resources/queries/TestJoinQuery/testTPCHQ2Join.sql")));
-    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr).getRootBlock().getRoot();
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     Schema expected = tpch.getOutSchema("q2");
     assertSchema(expected, plan.getOutSchema());
@@ -448,8 +460,8 @@ public class TestLogicalPlanner {
               qualMap.put(entry.getKey(), Boolean.TRUE);
             }
           } else if (rightType == EvalType.ROW_CONSTANT) {
-            RowConstantEval rightField = (RowConstantEval)qual.getRightExpr();
-            RowConstantEval rightJoinField = (RowConstantEval)entry.getKey().getRightExpr();
+            RowConstantEval rightField = qual.getRightExpr();
+            RowConstantEval rightJoinField = entry.getKey().getRightExpr();
 
             if (leftJoinField.getColumnRef().getQualifiedName().equals(leftField.getColumnRef().getQualifiedName())) {
               assertEquals(rightField.getValues().length, rightJoinField.getValues().length);
@@ -469,8 +481,9 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual1.sql")));
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
 
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     LogicalNode node = plan.getRootBlock().getRoot();
     testJsonSerDerObject(node);
 
@@ -510,8 +523,9 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual2.sql")));
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
 
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalPlan plan = planner.createPlan(qc,expr);
     LogicalNode node = plan.getRootBlock().getRoot();
     testJsonSerDerObject(node);
 
@@ -550,8 +564,9 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual3.sql")));
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
 
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     LogicalNode node = plan.getRootBlock().getRoot();
     testJsonSerDerObject(node);
 
@@ -596,8 +611,9 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual4.sql")));
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
 
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     LogicalNode node = plan.getRootBlock().getRoot();
     testJsonSerDerObject(node);
 
@@ -683,8 +699,11 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testStoreTable() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr context = sqlAnalyzer.parse(QUERIES[8]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+
+    LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot();
     TestLogicalNode.testCloneLogicalNode(plan);
     testJsonSerDerObject(plan);
 
@@ -698,8 +717,11 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testOrderBy() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(QUERIES[4]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     TestLogicalNode.testCloneLogicalNode(plan);
 
@@ -725,8 +747,11 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testLimit() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(QUERIES[12]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     TestLogicalNode.testCloneLogicalNode(plan);
 
@@ -744,8 +769,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSPJPush() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(QUERIES[5]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     TestLogicalNode.testCloneLogicalNode(plan);
 
@@ -764,16 +791,20 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSPJ() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(QUERIES[6]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     TestLogicalNode.testCloneLogicalNode(plan);
   }
 
   @Test
   public final void testJson() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
 	  Expr expr = sqlAnalyzer.parse(QUERIES[9]);
-	  LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+	  LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
 
 	  String json = plan.toJson();
@@ -792,9 +823,11 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testVisitor() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
 
     TestVisitor vis = new TestVisitor();
     plan.postOrder(vis);
@@ -817,8 +850,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testExprNode() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(QUERIES[10]);
-    LogicalPlan rootNode = planner.createPlan(session, expr);
+    LogicalPlan rootNode = planner.createPlan(qc, expr);
     LogicalNode plan = rootNode.getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     assertEquals(NodeType.ROOT, plan.getType());
@@ -837,8 +872,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testAsterisk() throws CloneNotSupportedException, PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(QUERIES[13]);
-    LogicalPlan planNode = planner.createPlan(session, expr);
+    LogicalPlan planNode = planner.createPlan(qc, expr);
     LogicalNode plan = planNode.getRootBlock().getRoot();
     assertEquals(NodeType.ROOT, plan.getType());
     TestLogicalNode.testCloneLogicalNode(plan);
@@ -865,8 +902,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testAlias1() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(ALIAS[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     LogicalRootNode root = (LogicalRootNode) plan;
     testJsonSerDerObject(root);
 
@@ -878,7 +917,7 @@ public class TestLogicalPlanner {
     assertEquals("total", col.getSimpleName());
 
     expr = sqlAnalyzer.parse(ALIAS[1]);
-    plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     root = (LogicalRootNode) plan;
 
     finalSchema = root.getOutSchema();
@@ -891,8 +930,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testAlias2() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(ALIAS[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     LogicalRootNode root = (LogicalRootNode) plan;
     testJsonSerDerObject(root);
 
@@ -910,8 +951,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testCreateTableDef() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(CREATE_TABLE[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     LogicalRootNode root = (LogicalRootNode) plan;
     testJsonSerDerObject(root);
     assertEquals(NodeType.CREATE_TABLE, root.getChild().getType());
@@ -994,8 +1037,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSetPlan() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(setStatements[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     assertEquals(NodeType.ROOT, plan.getType());
     LogicalRootNode root = (LogicalRootNode) plan;
@@ -1013,8 +1058,10 @@ public class TestLogicalPlanner {
 
   @Test
   public void testSetQualifier() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr context = sqlAnalyzer.parse(setQualifiers[0]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     assertEquals(NodeType.ROOT, plan.getType());
     LogicalRootNode root = (LogicalRootNode) plan;
@@ -1023,7 +1070,7 @@ public class TestLogicalPlanner {
     assertEquals(NodeType.SCAN, projectionNode.getChild().getType());
 
     context = sqlAnalyzer.parse(setQualifiers[1]);
-    plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    plan = planner.createPlan(qc, context).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     assertEquals(NodeType.ROOT, plan.getType());
     root = (LogicalRootNode) plan;
@@ -1032,7 +1079,7 @@ public class TestLogicalPlanner {
     assertEquals(NodeType.GROUP_BY, projectionNode.getChild().getType());
 
     context = sqlAnalyzer.parse(setQualifiers[2]);
-    plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    plan = planner.createPlan(qc, context).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     root = (LogicalRootNode) plan;
     assertEquals(NodeType.PROJECTION, root.getChild().getType());
@@ -1063,8 +1110,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto0() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(insertStatements[0]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertFalse(insertNode.isOverwrite());
@@ -1074,8 +1123,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto1() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(insertStatements[1]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertFalse(insertNode.isOverwrite());
@@ -1084,8 +1135,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto2() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(insertStatements[2]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertFalse(insertNode.isOverwrite());
@@ -1097,8 +1150,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto3() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(insertStatements[3]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertFalse(insertNode.isOverwrite());
@@ -1107,8 +1162,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto4() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(insertStatements[4]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertTrue(insertNode.isOverwrite());
@@ -1121,8 +1178,10 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto5() throws PlanningException {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+
     Expr expr = sqlAnalyzer.parse(insertStatements[5]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertTrue(insertNode.isOverwrite());

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
index 82e7818..b370be7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
@@ -31,7 +31,6 @@ import org.apache.tajo.engine.eval.*;
 import org.apache.tajo.engine.function.builtin.SumInt;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleComparator;
 import org.apache.tajo.storage.VTuple;
@@ -52,7 +51,6 @@ public class TestPlannerUtil {
   private static CatalogService catalog;
   private static SQLAnalyzer analyzer;
   private static LogicalPlanner planner;
-  private static Session session = LocalTajoTestingUtility.createDummySession();
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -111,7 +109,8 @@ public class TestPlannerUtil {
   public final void testFindTopNode() throws CloneNotSupportedException, PlanningException {
     // two relations
     Expr expr = analyzer.parse(TestLogicalPlanner.QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(util.getConfiguration()),
+        expr).getRootBlock().getRoot();
 
     assertEquals(NodeType.ROOT, plan.getType());
     LogicalRootNode root = (LogicalRootNode) plan;

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
index ec39609..4beb5c5 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
@@ -66,6 +66,7 @@ public class TestBroadcastJoinPlan {
   private TajoTestingCluster util;
   private CatalogService catalog;
   private SQLAnalyzer analyzer;
+  private QueryContext defaultContext;
   private Path testDir;
 
   private TableDesc smallTable1;
@@ -79,8 +80,8 @@ public class TestBroadcastJoinPlan {
   public void setUp() throws Exception {
     util = new TajoTestingCluster();
     conf = util.getConfiguration();
-    conf.setLongVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD, 500 * 1024);
-    conf.setBoolVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO, true);
+    conf.setLongVar(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD, 500 * 1024);
+    conf.setBoolVar(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED, true);
 
     testDir = CommonTestingUtil.getTestDir(TEST_PATH);
     catalog = util.startCatalogCluster().getCatalog();
@@ -126,6 +127,7 @@ public class TestBroadcastJoinPlan {
     catalog.createTable(largeTable3);
 
     analyzer = new SQLAnalyzer();
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   private TableDesc makeTestData(String tableName, Schema schema, int dataSize) throws Exception {
@@ -183,12 +185,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -244,12 +246,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -308,12 +310,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -336,12 +338,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -386,12 +388,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -427,12 +429,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -483,12 +485,12 @@ public class TestBroadcastJoinPlan {
         LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -537,12 +539,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -620,12 +622,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -703,12 +705,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -762,12 +764,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -815,12 +817,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -907,12 +909,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);
@@ -972,12 +974,12 @@ public class TestBroadcastJoinPlan {
     LogicalPlanner planner = new LogicalPlanner(catalog);
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr = analyzer.parse(query);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
 
     optimizer.optimize(plan);
 
     QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
-    QueryContext queryContext = new QueryContext();
+    QueryContext queryContext = new QueryContext(conf);
     MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
     GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
     globalPlanner.build(masterPlan);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
index d84796a..3fecabd 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
@@ -141,7 +141,8 @@ public class TestBNLJoinExec {
   @Test
   public final void testBNLCrossJoin() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf),
+        expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN);
@@ -152,7 +153,7 @@ public class TestBNLJoinExec {
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testBNLCrossJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -174,7 +175,7 @@ public class TestBNLJoinExec {
   @Test
   public final void testBNLInnerJoin() throws IOException, PlanningException {
     Expr context = analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf),
         context).getRootBlock().getRoot();
 
     FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(), employee.getPath(),
@@ -189,7 +190,7 @@ public class TestBNLJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testBNLInnerJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(),
         merged, workDir);
     ctx.setEnforcer(enforcer);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
index bfc3522..f817776 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
@@ -169,10 +169,10 @@ public class TestBSTIndexExec {
     
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEqual");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
     Expr expr = analyzer.parse(QUERY);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     TmpPlanner phyPlanner = new TmpPlanner(conf, sm);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
index 1ce5b5b..e7aac3c 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
@@ -121,11 +121,11 @@ public class TestExternalSortExec {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
     Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);


[2/5] TAJO-928: Session variables should override query configs in TajoConf.

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
index 1a8a90e..548d43a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
@@ -35,7 +35,6 @@ import org.apache.tajo.engine.planner.logical.JoinNode;
 import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.planner.logical.NodeType;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -61,7 +60,7 @@ public class TestFullOuterHashJoinExec {
   private LogicalPlanner planner;
   private AbstractStorageManager sm;
   private Path testDir;
-  private static Session session = LocalTajoTestingUtility.createDummySession();
+  private QueryContext defaultContext;
 
   private TableDesc dep3;
   private TableDesc job3;
@@ -234,11 +233,10 @@ public class TestFullOuterHashJoinExec {
     phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path);
     catalog.createTable(phone3);
 
-
-
-
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
+
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   @After
@@ -260,7 +258,7 @@ public class TestFullOuterHashJoinExec {
   @Test
   public final void testFullOuterHashJoinExec0() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
@@ -270,7 +268,7 @@ public class TestFullOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuterHashJoinExec0");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -297,7 +295,7 @@ public class TestFullOuterHashJoinExec {
   @Test
   public final void testFullOuterHashJoinExec1() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
@@ -307,7 +305,7 @@ public class TestFullOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuter_HashJoinExec1");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -333,7 +331,7 @@ public class TestFullOuterHashJoinExec {
   @Test
   public final void testFullOuterHashJoinExec2() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[2]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
@@ -343,7 +341,7 @@ public class TestFullOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuterHashJoinExec2");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -370,7 +368,7 @@ public class TestFullOuterHashJoinExec {
   @Test
   public final void testFullOuterHashJoinExec3() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[3]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
@@ -381,7 +379,7 @@ public class TestFullOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestFullOuterHashJoinExec3");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged,
         workDir);
     ctx.setEnforcer(enforcer);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
index 50e5906..1b9f7aa 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
@@ -36,7 +36,6 @@ import org.apache.tajo.engine.planner.logical.JoinNode;
 import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.planner.logical.NodeType;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -62,7 +61,7 @@ public class TestFullOuterMergeJoinExec {
   private LogicalPlanner planner;
   private AbstractStorageManager sm;
   private Path testDir;
-  private static final Session session = LocalTajoTestingUtility.createDummySession();
+  private QueryContext defaultContext;
 
   private TableDesc dep3;
   private TableDesc dep4;
@@ -279,6 +278,8 @@ public class TestFullOuterMergeJoinExec {
 
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
+
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   @After
@@ -304,7 +305,7 @@ public class TestFullOuterMergeJoinExec {
   @Test
   public final void testFullOuterMergeJoin0() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -316,7 +317,7 @@ public class TestFullOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin0");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -341,7 +342,7 @@ public class TestFullOuterMergeJoinExec {
   @Test
   public final void testFullOuterMergeJoin1() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -353,7 +354,7 @@ public class TestFullOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin1");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -378,7 +379,7 @@ public class TestFullOuterMergeJoinExec {
   @Test
   public final void testFullOuterMergeJoin2() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[2]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -390,7 +391,7 @@ public class TestFullOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin2");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -416,7 +417,7 @@ public class TestFullOuterMergeJoinExec {
   @Test
   public final void testFullOuterMergeJoin3() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[3]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -428,7 +429,7 @@ public class TestFullOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin3");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -456,7 +457,7 @@ public class TestFullOuterMergeJoinExec {
   @Test
   public final void testFullOuterMergeJoin4() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[4]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -469,7 +470,7 @@ public class TestFullOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin4");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -495,7 +496,7 @@ public class TestFullOuterMergeJoinExec {
   @Test
   public final void testFullOuterMergeJoin5() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[5]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -508,7 +509,7 @@ public class TestFullOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin5");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
index 794ca79..7055e44 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
@@ -154,11 +154,11 @@ public class TestHashAntiJoinExec {
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashAntiJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
index 0e07536..a8828ab 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.planner.physical;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.algebra.Expr;
@@ -36,7 +37,6 @@ import org.apache.tajo.engine.planner.logical.JoinNode;
 import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.planner.logical.NodeType;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -61,7 +61,7 @@ public class TestHashJoinExec {
   private LogicalPlanner planner;
   private AbstractStorageManager sm;
   private Path testDir;
-  private final Session session = LocalTajoTestingUtility.createDummySession();
+  private QueryContext defaultContext;
 
   private TableDesc employee;
   private TableDesc people;
@@ -126,6 +126,7 @@ public class TestHashJoinExec {
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   @After
@@ -140,8 +141,9 @@ public class TestHashJoinExec {
 
   @Test
   public final void testHashInnerJoin() throws IOException, PlanningException {
+
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
@@ -152,7 +154,7 @@ public class TestHashJoinExec {
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashInnerJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -182,7 +184,7 @@ public class TestHashJoinExec {
   @Test
   public final void testCheckIfInMemoryInnerJoinIsPossible() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
@@ -195,13 +197,12 @@ public class TestHashJoinExec {
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashInnerJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    TajoConf localConf = new TajoConf(conf);
-    localConf.setLongVar(TajoConf.ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD, 100l);
-    PhysicalPlannerImpl phyPlanner = new PhysicalPlannerImpl(localConf, sm);
+    ctx.getQueryContext().setLong(SessionVars.HASH_JOIN_SIZE_LIMIT.keyname(), 100l);
+    PhysicalPlannerImpl phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
index 835260f..6373f23 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
@@ -158,11 +158,11 @@ public class TestHashSemiJoinExec {
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashSemiJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
index bb25875..ef740e3 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
@@ -35,7 +35,6 @@ import org.apache.tajo.engine.planner.logical.JoinNode;
 import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.planner.logical.NodeType;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -62,7 +61,7 @@ public class TestLeftOuterHashJoinExec {
   private LogicalPlanner planner;
   private AbstractStorageManager sm;
   private Path testDir;
-  private Session session = LocalTajoTestingUtility.createDummySession();
+  private QueryContext defaultContext;
 
   private TableDesc dep3;
   private TableDesc job3;
@@ -239,6 +238,7 @@ public class TestLeftOuterHashJoinExec {
 
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   @After
@@ -262,7 +262,7 @@ public class TestLeftOuterHashJoinExec {
   @Test
   public final void testLeftOuterHashJoinExec0() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
@@ -272,7 +272,7 @@ public class TestLeftOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuterHashJoinExec0");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -303,11 +303,11 @@ public class TestLeftOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec1");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
@@ -344,11 +344,11 @@ public class TestLeftOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec2");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[2]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
@@ -386,11 +386,11 @@ public class TestLeftOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec3");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[3]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
@@ -428,11 +428,11 @@ public class TestLeftOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_HashJoinExec4");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[4]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
@@ -444,12 +444,10 @@ public class TestLeftOuterHashJoinExec {
       assertEquals(1, 0);
     }
     else{
-       Tuple tuple;
        int count = 0;
-       int i = 1;
        exec.init();
   
-       while ((tuple = exec.next()) != null) {
+       while (exec.next() != null) {
          //TODO check contents
          count = count + 1;
        }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
index e935f57..2a8d9f0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
@@ -36,7 +36,6 @@ import org.apache.tajo.engine.planner.PlanningException;
 import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -60,7 +59,7 @@ public class TestLeftOuterNLJoinExec {
   private CatalogService catalog;
   private SQLAnalyzer analyzer;
   private LogicalPlanner planner;
-  private static final Session session = LocalTajoTestingUtility.createDummySession();
+  private QueryContext defaultContext;
   private AbstractStorageManager sm;
   private Path testDir;
 
@@ -236,6 +235,8 @@ public class TestLeftOuterNLJoinExec {
 
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
+
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   @After
@@ -261,11 +262,11 @@ public class TestLeftOuterNLJoinExec {
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuterNLJoinExec0");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context =  analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
@@ -303,11 +304,11 @@ public class TestLeftOuterNLJoinExec {
 
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec1");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context =  analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
@@ -347,11 +348,11 @@ public class TestLeftOuterNLJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec2");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context =  analyzer.parse(QUERIES[2]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
@@ -392,11 +393,11 @@ public class TestLeftOuterNLJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec3");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context =  analyzer.parse(QUERIES[3]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
@@ -436,11 +437,11 @@ public class TestLeftOuterNLJoinExec {
     FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestLeftOuter_NLJoinExec4");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context =  analyzer.parse(QUERIES[4]);
-    LogicalNode plan = planner.createPlan(session, context).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
index a4449fa..ed19e4f 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
@@ -154,7 +154,7 @@ public class TestMergeJoinExec {
   @Test
   public final void testMergeInnerJoin() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode root = plan.getRootBlock().getRoot();
 
     JoinNode joinNode = PlannerUtil.findTopNode(root, NodeType.JOIN);
@@ -166,7 +166,7 @@ public class TestMergeJoinExec {
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testMergeInnerJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
index 019929d..1120bbe 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
@@ -152,12 +152,12 @@ public class TestNLJoinExec {
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testNLCrossJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
-    Expr context = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),
-        context).getRootBlock().getRoot();
+    Expr expr = analyzer.parse(QUERIES[0]);
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf),
+        expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
@@ -181,13 +181,12 @@ public class TestNLJoinExec {
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testNLInnerJoin");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan), merged, workDir);
     ctx.setEnforcer(new Enforcer());
-    Expr context =  analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(),
-        context).getRootBlock().getRoot();
-    //LogicalOptimizer.optimize(ctx, plan);
+    Expr expr =  analyzer.parse(QUERIES[1]);
+    LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf),
+        expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index a823d2b..a184a9a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -85,6 +85,7 @@ public class TestPhysicalPlanner {
   private static AbstractStorageManager sm;
   private static Path testDir;
   private static Session session = LocalTajoTestingUtility.createDummySession();
+  private static QueryContext defaultContext;
 
   private static TableDesc employee = null;
   private static TableDesc score = null;
@@ -168,6 +169,7 @@ public class TestPhysicalPlanner {
     planner = new LogicalPlanner(catalog);
     optimizer = new LogicalOptimizer(conf);
 
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
     masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
   }
 
@@ -201,12 +203,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         employee.getPath(), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanPlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode =plan.getRootBlock().getRoot();
     optimizer.optimize(plan);
 
@@ -232,12 +234,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         employee.getPath(), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanWithFilterPlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[16]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode =plan.getRootBlock().getRoot();
     optimizer.optimize(plan);
 
@@ -261,12 +263,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByPlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(QUERIES[7]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
@@ -293,12 +295,12 @@ public class TestPhysicalPlanner {
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(
         "target/test-data/testHashGroupByPlanWithALLField");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[15]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -322,12 +324,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortGroupByPlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[]{frags[0]}, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(QUERIES[7]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -382,14 +384,14 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     ctx.setOutputPath(new Path(workDir, "grouped1"));
 
     Expr context = analyzer.parse(CreateTableAsStmts[0]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
 
@@ -424,14 +426,14 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlanWithRCFile");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     ctx.setOutputPath(new Path(workDir, "grouped2"));
 
     Expr context = analyzer.parse(CreateTableAsStmts[1]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     TableMeta outputMeta = CatalogUtil.newTableMeta(StoreType.RCFILE);
@@ -465,14 +467,14 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     ctx.setOutputPath(new Path(workDir, "grouped3"));
 
     Expr context = analyzer.parse(CreateTableAsStmts[2]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
@@ -483,7 +485,7 @@ public class TestPhysicalPlanner {
   public final void testEnforceForHashBasedColumnPartitionStorePlan() throws IOException, PlanningException {
 
     Expr context = analyzer.parse(CreateTableAsStmts[2]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalRootNode rootNode = (LogicalRootNode) optimizer.optimize(plan);
     CreateTableNode createTableNode = rootNode.getChild();
     Enforcer enforcer = new Enforcer();
@@ -492,7 +494,7 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(enforcer);
@@ -507,7 +509,7 @@ public class TestPhysicalPlanner {
   public final void testEnforceForSortBasedColumnPartitionStorePlan() throws IOException, PlanningException {
 
     Expr context = analyzer.parse(CreateTableAsStmts[2]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalRootNode rootNode = (LogicalRootNode) optimizer.optimize(plan);
     CreateTableNode createTableNode = rootNode.getChild();
     Enforcer enforcer = new Enforcer();
@@ -516,7 +518,7 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(enforcer);
@@ -533,11 +535,11 @@ public class TestPhysicalPlanner {
         Integer.MAX_VALUE);
     QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         id, new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(QUERIES[7]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
 
     int numPartitions = 3;
     Column key1 = new Column("default.score.deptname", Type.TEXT);
@@ -594,11 +596,11 @@ public class TestPhysicalPlanner {
 
     Path workDir = CommonTestingUtil.getTestDir(
         "target/test-data/testPartitionedStorePlanWithEmptyGroupingSet");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         id, new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[14]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
     int numPartitions = 1;
     DataChannel dataChannel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
@@ -648,12 +650,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testAggregationFunction");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(QUERIES[8]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     // Set all aggregation functions to the first phase mode
@@ -679,12 +681,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCountFunction");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(QUERIES[9]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     // Set all aggregation functions to the first phase mode
@@ -707,12 +709,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByWithNullValue");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(QUERIES[11]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -732,12 +734,12 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testUnionPlan");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr  context = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
     LogicalRootNode root = (LogicalRootNode) rootNode;
     UnionNode union = plan.createNode(UnionNode.class);
@@ -760,11 +762,11 @@ public class TestPhysicalPlanner {
   @Test
   public final void testEvalExpr() throws IOException, PlanningException {
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEvalExpr");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] { }, workDir);
     Expr expr = analyzer.parse(QUERIES[12]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
@@ -777,7 +779,7 @@ public class TestPhysicalPlanner {
     assertTrue(7.0d == tuple.get(1).asFloat8());
 
     expr = analyzer.parse(QUERIES[13]);
-    plan = planner.createPlan(session, expr);
+    plan = planner.createPlan(defaultContext, expr);
     rootNode = optimizer.optimize(plan);
 
     phyPlanner = new PhysicalPlannerImpl(conf, sm);
@@ -797,11 +799,11 @@ public class TestPhysicalPlanner {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateIndex");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] {frags[0]}, workDir);
     Expr context = analyzer.parse(createIndexStmt[0]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
@@ -825,12 +827,12 @@ public class TestPhysicalPlanner {
         Integer.MAX_VALUE);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testDuplicateEliminate");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(duplicateElimination[0]);
-    LogicalPlan plan = planner.createPlan(session, expr);
+    LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
@@ -859,12 +861,12 @@ public class TestPhysicalPlanner {
         employee.getPath(), Integer.MAX_VALUE);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testIndexedStoreExec");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(SORT_QUERY[0]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     SortNode sortNode = PlannerUtil.findTopNode(rootNode, NodeType.SORT);
@@ -953,7 +955,7 @@ public class TestPhysicalPlanner {
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortEnforcer");
     Expr context = analyzer.parse(SORT_QUERY[0]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
@@ -961,7 +963,7 @@ public class TestPhysicalPlanner {
 
     Enforcer enforcer = new Enforcer();
     enforcer.enforceSortAlgorithm(sortNode.getPID(), SortAlgorithm.IN_MEMORY_SORT);
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);
@@ -975,7 +977,7 @@ public class TestPhysicalPlanner {
     assertTrue(exec instanceof MemSortExec);
 
     context = analyzer.parse(SORT_QUERY[0]);
-    plan = planner.createPlan(session, context);
+    plan = planner.createPlan(defaultContext, context);
     optimizer.optimize(plan);
     rootNode = plan.getRootBlock().getRoot();
 
@@ -983,7 +985,7 @@ public class TestPhysicalPlanner {
 
     enforcer = new Enforcer();
     enforcer.enforceSortAlgorithm(sortNode.getPID(), SortAlgorithm.MERGE_SORT);
-    ctx = new TaskAttemptContext(conf, new QueryContext(),
+    ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);
@@ -1003,7 +1005,7 @@ public class TestPhysicalPlanner {
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByEnforcer");
     Expr context = analyzer.parse(QUERIES[7]);
-    LogicalPlan plan = planner.createPlan(session, context);
+    LogicalPlan plan = planner.createPlan(defaultContext, context);
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
@@ -1011,7 +1013,7 @@ public class TestPhysicalPlanner {
 
     Enforcer enforcer = new Enforcer();
     enforcer.enforceHashAggregation(groupByNode.getPID());
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);
@@ -1025,7 +1027,7 @@ public class TestPhysicalPlanner {
     assertNotNull(PhysicalPlanUtil.findExecutor(exec, HashAggregateExec.class));
 
     context = analyzer.parse(QUERIES[7]);
-    plan = planner.createPlan(session, context);
+    plan = planner.createPlan(defaultContext, context);
     optimizer.optimize(plan);
     rootNode = plan.getRootBlock().getRoot();
 
@@ -1033,7 +1035,7 @@ public class TestPhysicalPlanner {
 
     enforcer = new Enforcer();
     enforcer.enforceSortAggregation(groupByNode.getPID(), null);
-    ctx = new TaskAttemptContext(conf, new QueryContext(),
+    ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
index ed6cb4e..f649dac 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
@@ -139,11 +139,11 @@ public class TestProgressExternalSortExec {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(),
         Integer.MAX_VALUE);
     Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), expr);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
index de90f70..506555e 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
@@ -36,7 +36,6 @@ import org.apache.tajo.engine.planner.PlanningException;
 import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -62,7 +61,7 @@ public class TestRightOuterHashJoinExec {
   private LogicalPlanner planner;
   private AbstractStorageManager sm;
   private Path testDir;
-  private static Session session = LocalTajoTestingUtility.createDummySession();
+  private QueryContext defaultContext;
 
   private TableDesc dep3;
   private TableDesc job3;
@@ -214,6 +213,7 @@ public class TestRightOuterHashJoinExec {
 
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   @After
@@ -237,11 +237,11 @@ public class TestRightOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestRightOuter_HashJoinExec0");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
@@ -278,11 +278,11 @@ public class TestRightOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestRightOuter_HashJoinExec1");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
@@ -319,11 +319,11 @@ public class TestRightOuterHashJoinExec {
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestRightOuter_HashJoinExec2");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr expr = analyzer.parse(QUERIES[2]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
index d971073..0ce74eb 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
@@ -35,7 +35,6 @@ import org.apache.tajo.engine.planner.logical.JoinNode;
 import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.planner.logical.NodeType;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -61,7 +60,8 @@ public class TestRightOuterMergeJoinExec {
   private LogicalPlanner planner;
   private AbstractStorageManager sm;
   private Path testDir;
-  private static final Session session = LocalTajoTestingUtility.createDummySession();
+  private QueryContext defaultContext;
+
 
   private TableDesc dep3;
   private TableDesc dep4;
@@ -276,10 +276,10 @@ public class TestRightOuterMergeJoinExec {
     phone3 = CatalogUtil.newTableDesc(PHONE3_NAME, phone3Schema, phone3Meta, phone3Path);
     catalog.createTable(phone3);
 
-
-
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
+
+    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
   @After
@@ -305,7 +305,7 @@ public class TestRightOuterMergeJoinExec {
   @Test
   public final void testRightOuterMergeJoin0() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[0]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -315,7 +315,7 @@ public class TestRightOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin0");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -340,7 +340,7 @@ public class TestRightOuterMergeJoinExec {
   @Test
   public final void testRightOuter_MergeJoin1() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[1]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -352,7 +352,7 @@ public class TestRightOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin1");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -376,7 +376,7 @@ public class TestRightOuterMergeJoinExec {
   @Test
   public final void testRightOuterMergeJoin2() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[2]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -388,7 +388,7 @@ public class TestRightOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin2");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -412,7 +412,7 @@ public class TestRightOuterMergeJoinExec {
   @Test
   public final void testRightOuter_MergeJoin3() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[3]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -424,7 +424,7 @@ public class TestRightOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuter_MergeJoin3");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -449,7 +449,7 @@ public class TestRightOuterMergeJoinExec {
   @Test
   public final void testRightOuter_MergeJoin4() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[4]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -462,7 +462,7 @@ public class TestRightOuterMergeJoinExec {
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuter_MergeJoin4");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
@@ -486,7 +486,7 @@ public class TestRightOuterMergeJoinExec {
   @Test
   public final void testRightOuterMergeJoin5() throws IOException, PlanningException {
     Expr expr = analyzer.parse(QUERIES[5]);
-    LogicalNode plan = planner.createPlan(session, expr).getRootBlock().getRoot();
+    LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
     JoinNode joinNode = PlannerUtil.findTopNode(plan, NodeType.JOIN);
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
@@ -499,7 +499,7 @@ public class TestRightOuterMergeJoinExec {
 
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin5");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
index da6fb34..4d4cc3d 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
@@ -110,12 +110,12 @@ public class TestSortExec {
   public final void testNext() throws IOException, PlanningException {
     FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employeeMeta, tablePath, Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestSortExec");
-    TaskAttemptContext ctx = new TaskAttemptContext(conf, new QueryContext(),
+    TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility
         .newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
     ctx.setEnforcer(new Enforcer());
     Expr context = analyzer.parse(QUERIES[0]);
-    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummySession(), context);
+    LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
index fe9f990..2b62c0e 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
@@ -291,8 +291,8 @@ public class TestGroupByQuery extends QueryTestCaseBase {
 
     // case9
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("id", Type.TEXT);
@@ -345,8 +345,8 @@ public class TestGroupByQuery extends QueryTestCaseBase {
   public final void testDistinctAggregationCaseByCase3() throws Exception {
     // first distinct is smaller than second distinct.
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("col1", Type.TEXT);
@@ -375,8 +375,8 @@ public class TestGroupByQuery extends QueryTestCaseBase {
   public final void testDistinctAggregationCaseByCase4() throws Exception {
     // Reproduction case for TAJO-994
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("col1", Type.TEXT);
@@ -562,8 +562,8 @@ public class TestGroupByQuery extends QueryTestCaseBase {
   @Test
   public final void testNumShufflePartition() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("col1", Type.TEXT);
@@ -592,7 +592,7 @@ public class TestGroupByQuery extends QueryTestCaseBase {
     TajoTestingCluster.createTable("testnumshufflepartition", schema, tableOptions, data.toArray(new String[]{}), 3);
 
     try {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_GROUPBY_PARTITION_VOLUME.varname, "2");
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$DIST_QUERY_GROUPBY_PARTITION_VOLUME.varname, "2");
       ResultSet res = executeString(
           "select col1 \n" +
               ",count(distinct col2) as cnt1\n" +
@@ -646,8 +646,8 @@ public class TestGroupByQuery extends QueryTestCaseBase {
       assertEquals(2, partitionIds.size());
       executeString("DROP TABLE testnumshufflepartition PURGE").close();
     } finally {
-      testingCluster.setAllTajoDaemonConfValue(ConfVars.DIST_QUERY_GROUPBY_PARTITION_VOLUME.varname,
-          ConfVars.DIST_QUERY_GROUPBY_PARTITION_VOLUME.defaultVal);
+      testingCluster.setAllTajoDaemonConfValue(ConfVars.$DIST_QUERY_GROUPBY_PARTITION_VOLUME.varname,
+          ConfVars.$DIST_QUERY_GROUPBY_PARTITION_VOLUME.defaultVal);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
index 349ad0c..2625136 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
@@ -48,9 +48,9 @@ import static org.junit.Assert.*;
 public class TestJoinBroadcast extends QueryTestCaseBase {
   public TestJoinBroadcast() throws Exception {
     super(TajoConstants.DEFAULT_DATABASE_NAME);
-    testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO.varname, "true");
+    testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "true");
     testingCluster.setAllTajoDaemonConfValue(
-        TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname, "" + (5 * 1024));
+        TajoConf.ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD.varname, "" + (5 * 1024));
 
     executeDDL("create_lineitem_large_ddl.sql", "lineitem_large");
     executeDDL("create_customer_large_ddl.sql", "customer_large");
@@ -575,8 +575,8 @@ public class TestJoinBroadcast extends QueryTestCaseBase {
   @Test
   public final void testLeftOuterJoinLeftSideSmallTable() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
-    tableOptions.put(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    tableOptions.put(StorageConstants.CSVFILE_NULL, "\\\\N");
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
 
     Schema schema = new Schema();
     schema.addColumn("id", Type.INT4);


[5/5] git commit: TAJO-928: Session variables should override query configs in TajoConf.

Posted by hy...@apache.org.
TAJO-928: Session variables should override query configs in TajoConf.

Closes #98


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

Branch: refs/heads/master
Commit: ddfc3f33039ee15ab0a2d3fe5890b9acb40aec3d
Parents: 0603b49
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Aug 11 12:52:52 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Aug 11 12:52:52 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../java/org/apache/tajo/catalog/TableMeta.java |  14 +-
 .../apache/tajo/catalog/TestKeyValueSet.java    |  83 ++++++-
 .../tajo/catalog/store/HCatalogStore.java       |  28 +--
 .../tajo/catalog/store/TestHCatalogStore.java   |  12 +-
 .../org/apache/tajo/catalog/TestCatalog.java    |  10 +-
 .../tajo/cli/DefaultTajoCliOutputFormatter.java |  16 +-
 .../java/org/apache/tajo/cli/HelpCommand.java   | 100 ++++----
 .../java/org/apache/tajo/cli/SetCommand.java    |  78 +++++-
 .../main/java/org/apache/tajo/cli/TajoCli.java  | 149 ++++++++----
 .../apache/tajo/cli/TajoCliOutputFormatter.java |   5 +-
 .../org/apache/tajo/cli/TajoShellCommand.java   |  10 +-
 .../main/java/org/apache/tajo/ConfigKey.java    |  52 ++++
 .../java/org/apache/tajo/OverridableConf.java   | 235 +++++++++++++++++++
 .../main/java/org/apache/tajo/SessionVars.java  | 211 +++++++++++++++++
 .../java/org/apache/tajo/conf/TajoConf.java     | 199 ++++++++--------
 .../main/java/org/apache/tajo/datum/Datum.java  |   2 +-
 .../java/org/apache/tajo/datum/NullDatum.java   |   1 +
 .../java/org/apache/tajo/util/KeyValueSet.java  | 149 ++++++++++--
 .../tajo/datum/TestArithmeticOperator.java      |   4 +-
 .../tajo/engine/planner/LogicalOptimizer.java   |   9 +-
 .../apache/tajo/engine/planner/LogicalPlan.java |   4 +-
 .../engine/planner/LogicalPlanPreprocessor.java |  18 +-
 .../engine/planner/LogicalPlanVerifier.java     |  12 +-
 .../tajo/engine/planner/LogicalPlanner.java     |  35 ++-
 .../engine/planner/PhysicalPlannerImpl.java     |  44 +++-
 .../engine/planner/PreLogicalPlanVerifier.java  |  21 +-
 .../engine/planner/global/GlobalPlanner.java    |  18 +-
 .../planner/physical/ExternalSortExec.java      |   4 +-
 .../engine/planner/physical/StoreTableExec.java |   6 +-
 .../apache/tajo/engine/query/QueryContext.java  | 144 ++++++------
 .../tajo/engine/query/QueryUnitRequestImpl.java |   3 +-
 .../org/apache/tajo/master/GlobalEngine.java    |  94 ++++----
 .../tajo/master/TajoMasterClientService.java    |  31 ++-
 .../apache/tajo/master/querymaster/Query.java   |   4 +-
 .../tajo/master/querymaster/QueryMaster.java    |  11 +-
 .../querymaster/QueryMasterManagerService.java  |   4 +-
 .../master/querymaster/QueryMasterTask.java     |   4 +-
 .../tajo/master/querymaster/Repartitioner.java  |  16 +-
 .../tajo/master/querymaster/SubQuery.java       |  15 +-
 .../org/apache/tajo/master/session/Session.java |  50 ++--
 .../session/SessionLivelinessMonitor.java       |   2 +-
 .../main/java/org/apache/tajo/worker/Task.java  |   4 +-
 .../apache/tajo/worker/TaskAttemptContext.java  |  12 +-
 .../resources/webapps/worker/querydetail.jsp    |  10 +
 .../apache/tajo/LocalTajoTestingUtility.java    |   5 +
 .../java/org/apache/tajo/QueryTestCaseBase.java |   2 +-
 .../test/java/org/apache/tajo/TpchTestBase.java |   2 +-
 .../tajo/cli/TestDefaultCliOutputFormatter.java |  47 +++-
 .../java/org/apache/tajo/cli/TestTajoCli.java   | 172 ++++++++------
 .../org/apache/tajo/client/TestTajoClient.java  |   8 +-
 .../apache/tajo/engine/eval/ExprTestBase.java   |   9 +-
 .../tajo/engine/eval/TestEvalTreeUtil.java      |  16 +-
 .../engine/planner/TestLogicalOptimizer.java    |  20 +-
 .../tajo/engine/planner/TestLogicalPlan.java    |   3 +-
 .../tajo/engine/planner/TestLogicalPlanner.java | 135 ++++++++---
 .../tajo/engine/planner/TestPlannerUtil.java    |   5 +-
 .../planner/global/TestBroadcastJoinPlan.java   |  62 ++---
 .../planner/physical/TestBNLJoinExec.java       |   9 +-
 .../planner/physical/TestBSTIndexExec.java      |   4 +-
 .../planner/physical/TestExternalSortExec.java  |   4 +-
 .../physical/TestFullOuterHashJoinExec.java     |  24 +-
 .../physical/TestFullOuterMergeJoinExec.java    |  29 +--
 .../planner/physical/TestHashAntiJoinExec.java  |   4 +-
 .../planner/physical/TestHashJoinExec.java      |  19 +-
 .../planner/physical/TestHashSemiJoinExec.java  |   4 +-
 .../physical/TestLeftOuterHashJoinExec.java     |  28 +--
 .../physical/TestLeftOuterNLJoinExec.java       |  25 +-
 .../planner/physical/TestMergeJoinExec.java     |   4 +-
 .../engine/planner/physical/TestNLJoinExec.java |  17 +-
 .../planner/physical/TestPhysicalPlanner.java   | 100 ++++----
 .../physical/TestProgressExternalSortExec.java  |   4 +-
 .../physical/TestRightOuterHashJoinExec.java    |  16 +-
 .../physical/TestRightOuterMergeJoinExec.java   |  32 +--
 .../engine/planner/physical/TestSortExec.java   |   4 +-
 .../tajo/engine/query/TestGroupByQuery.java     |  22 +-
 .../tajo/engine/query/TestJoinBroadcast.java    |   8 +-
 .../apache/tajo/engine/query/TestJoinQuery.java |  90 +++----
 .../tajo/engine/query/TestNullValues.java       |  14 +-
 .../tajo/engine/query/TestSelectQuery.java      |  10 +-
 .../apache/tajo/engine/query/TestSortQuery.java |   8 +-
 .../org/apache/tajo/jdbc/TestResultSet.java     |   2 +-
 .../tajo/master/TestExecutionBlockCursor.java   |   6 +-
 .../apache/tajo/master/TestGlobalPlanner.java   |   4 +-
 .../querymaster/TestQueryUnitStatusUpdate.java  |   2 +-
 .../tajo/worker/TestRangeRetrieverHandler.java  |   8 +-
 .../TestTajoCli/testHelpSessionVars.result      |  36 +++
 ...estSelectResultWithNullTrueDeprecated.result |   9 +
 .../testStopWhenErrorDeprecated.result          |   6 +
 .../java/org/apache/tajo/storage/Appender.java  |   2 +-
 .../java/org/apache/tajo/storage/CSVFile.java   |  11 +-
 .../org/apache/tajo/storage/StorageUtil.java    |  18 +-
 .../org/apache/tajo/storage/avro/AvroUtil.java  |  17 +-
 .../org/apache/tajo/storage/rcfile/RCFile.java  |  10 +-
 .../sequencefile/SequenceFileAppender.java      |  10 +-
 .../sequencefile/SequenceFileScanner.java       |   5 +-
 .../apache/tajo/storage/v2/RCFileScanner.java   |   2 +-
 97 files changed, 2057 insertions(+), 992 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index deb41a0..182d07d 100644
--- a/CHANGES
+++ b/CHANGES
@@ -29,6 +29,9 @@ Release 0.9.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-928: Session variables should override query configs in TajoConf.
+    (hyunsik)
+
     TAJO-989: Cleanup of child blocks after parent execution block is complete
     (jinho)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
index 6015fc9..2d95e6b 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
@@ -78,7 +78,19 @@ public class TableMeta implements ProtoObject<CatalogProtos.TableProto>, GsonObj
 
   public void putOption(String key, String val) {
     maybeInitBuilder();
-    options.put(key, val);
+    options.set(key, val);
+  }
+
+  public boolean containsOption(String key) {
+    TableProtoOrBuilder p = viaProto ? proto : builder;
+    if (options != null) {
+      return this.options.containsKey(key);
+    }
+    if (!p.hasParams()) {
+      return false;
+    }
+    this.options = new KeyValueSet(p.getParams());
+    return options.containsKey(key);
   }
 
   public String getOption(String key) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestKeyValueSet.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestKeyValueSet.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestKeyValueSet.java
index 39d5fbe..b317ba4 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestKeyValueSet.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestKeyValueSet.java
@@ -24,23 +24,70 @@ import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 public class TestKeyValueSet {
+
 	@Test
-	public final void testPutAndGet() {
+	public final void testPutAndGetString() {
 		KeyValueSet opts = new KeyValueSet();
-		opts.put("name", "abc");
-		opts.put("delimiter", ",");
+		opts.set("k1", "v1");
+		opts.set("k2", "v2");
 		
-		assertEquals(",", opts.get("delimiter"));
-		assertEquals("abc", opts.get("name"));
+		assertEquals("v1", opts.get("k1"));
+		assertEquals("v2", opts.get("k2"));
+    assertEquals("default", opts.get("k3", "default"));
 	}
 
+  @Test
+  public final void testPutAndGetBool() {
+    KeyValueSet opts = new KeyValueSet();
+    opts.setBool("k1", true);
+    opts.setBool("k2", false);
+
+    assertEquals(true, opts.getBool("k1"));
+    assertEquals(false, opts.getBool("k2"));
+    assertEquals(true, opts.getBool("k3", true));
+  }
+
+  @Test
+  public final void testPutAndGetInt() {
+    KeyValueSet opts = new KeyValueSet();
+    opts.setInt("k1", 1980);
+    opts.setInt("k2", 401);
+
+    assertEquals(1980, opts.getInt("k1"));
+    assertEquals(401, opts.getInt("k2"));
+    assertEquals(2020, opts.getInt("k3", 2020));
+  }
+
+  @Test
+  public final void testPutAndGetLong() {
+    KeyValueSet opts = new KeyValueSet();
+    opts.setLong("k1", 1980);
+    opts.setLong("k2", 401);
+
+    assertEquals(1980, opts.getLong("k1"));
+    assertEquals(401, opts.getLong("k2"));
+    assertEquals(2020, opts.getLong("k3", 2020l));
+  }
+
+  @Test
+  public final void testPutAndGetFloat() {
+    KeyValueSet opts = new KeyValueSet();
+    opts.setFloat("k1", 1980.4f);
+    opts.setFloat("k2", 401.150f);
+
+    assertTrue(1980.4f == opts.getFloat("k1"));
+    assertTrue(401.150f == opts.getFloat("k2"));
+    assertTrue(3.14f == opts.getFloat("k3", 3.14f));
+  }
+
 	@Test
 	public final void testGetProto() {		
 		KeyValueSet opts = new KeyValueSet();
-		opts.put("name", "abc");
-		opts.put("delimiter", ",");
+		opts.set("name", "abc");
+		opts.set("delimiter", ",");
 		
 		PrimitiveProtos.KeyValueSetProto proto = opts.getProto();
 		KeyValueSet opts2 = new KeyValueSet(proto);
@@ -49,16 +96,26 @@ public class TestKeyValueSet {
 	}
 	
 	@Test
-	public final void testDelete() {
+	public final void testRemove() {
 		KeyValueSet opts = new KeyValueSet();
-		opts.put("name", "abc");
-		opts.put("delimiter", ",");
+		opts.set("name", "abc");
+		opts.set("delimiter", ",");
 		
 		assertEquals("abc", opts.get("name"));
-		assertEquals("abc", opts.delete("name"));
-		assertNull(opts.get("name"));
+		assertEquals("abc", opts.remove("name"));
+    try {
+		  opts.get("name");
+      assertTrue(false);
+    } catch (IllegalArgumentException iae) {
+      assertTrue(true);
+    }
 		
 		KeyValueSet opts2 = new KeyValueSet(opts.getProto());
-		assertNull(opts2.get("name"));
+    try {
+      opts2.get("name");
+      assertTrue(false);
+    } catch (IllegalArgumentException iae) {
+      assertTrue(true);
+    }
 	}
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
index 6f48348..61bdccd 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/main/java/org/apache/tajo/catalog/store/HCatalogStore.java
@@ -166,7 +166,7 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
       stats = new TableStats();
       options = new KeyValueSet();
       options.putAll(table.getParameters());
-      options.delete("EXTERNAL");
+      options.remove("EXTERNAL");
 
       Properties properties = table.getMetadata();
       if (properties != null) {
@@ -186,31 +186,31 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
         } else {
           nullFormat = "\\N";
         }
-        options.delete(serdeConstants.SERIALIZATION_NULL_FORMAT);
+        options.remove(serdeConstants.SERIALIZATION_NULL_FORMAT);
 
         // set file output format
         String fileOutputformat = properties.getProperty(hive_metastoreConstants.FILE_OUTPUT_FORMAT);
         storeType = CatalogUtil.getStoreType(HCatalogUtil.getStoreType(fileOutputformat));
 
         if (storeType.equals(CatalogProtos.StoreType.CSV)) {
-          options.put(StorageConstants.CSVFILE_DELIMITER, StringEscapeUtils.escapeJava(fieldDelimiter));
-          options.put(StorageConstants.CSVFILE_NULL, StringEscapeUtils.escapeJava(nullFormat));
+          options.set(StorageConstants.CSVFILE_DELIMITER, StringEscapeUtils.escapeJava(fieldDelimiter));
+          options.set(StorageConstants.CSVFILE_NULL, StringEscapeUtils.escapeJava(nullFormat));
         } else if (storeType.equals(CatalogProtos.StoreType.RCFILE)) {
-          options.put(StorageConstants.RCFILE_NULL, StringEscapeUtils.escapeJava(nullFormat));
+          options.set(StorageConstants.RCFILE_NULL, StringEscapeUtils.escapeJava(nullFormat));
           String serde = properties.getProperty(serdeConstants.SERIALIZATION_LIB);
           if (LazyBinaryColumnarSerDe.class.getName().equals(serde)) {
-            options.put(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
+            options.set(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
           } else if (ColumnarSerDe.class.getName().equals(serde)) {
-            options.put(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
+            options.set(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
           }
         } else if (storeType.equals(CatalogProtos.StoreType.SEQUENCEFILE) ) {
-          options.put(StorageConstants.SEQUENCEFILE_DELIMITER, StringEscapeUtils.escapeJava(fieldDelimiter));
-          options.put(StorageConstants.SEQUENCEFILE_NULL, StringEscapeUtils.escapeJava(nullFormat));
+          options.set(StorageConstants.SEQUENCEFILE_DELIMITER, StringEscapeUtils.escapeJava(fieldDelimiter));
+          options.set(StorageConstants.SEQUENCEFILE_NULL, StringEscapeUtils.escapeJava(nullFormat));
           String serde = properties.getProperty(serdeConstants.SERIALIZATION_LIB);
           if (LazyBinarySerDe.class.getName().equals(serde)) {
-            options.put(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
+            options.set(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
           } else if (LazySimpleSerDe.class.getName().equals(serde)) {
-            options.put(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
+            options.set(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
           }
         }
 
@@ -481,7 +481,7 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
               org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe.class.getName());
         }
 
-        if (tableDesc.getMeta().getOption(StorageConstants.RCFILE_NULL) != null) {
+        if (tableDesc.getMeta().getOptions().containsKey(StorageConstants.RCFILE_NULL)) {
           table.putToParameters(serdeConstants.SERIALIZATION_NULL_FORMAT,
               StringEscapeUtils.unescapeJava(tableDesc.getMeta().getOption(StorageConstants.RCFILE_NULL)));
         }
@@ -504,7 +504,7 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
             StringEscapeUtils.unescapeJava(fieldDelimiter));
         table.getParameters().remove(StorageConstants.CSVFILE_DELIMITER);
 
-        if (tableDesc.getMeta().getOption(StorageConstants.CSVFILE_NULL) != null) {
+        if (tableDesc.getMeta().containsOption(StorageConstants.CSVFILE_NULL)) {
           table.putToParameters(serdeConstants.SERIALIZATION_NULL_FORMAT,
               StringEscapeUtils.unescapeJava(tableDesc.getMeta().getOption(StorageConstants.CSVFILE_NULL)));
           table.getParameters().remove(StorageConstants.CSVFILE_NULL);
@@ -534,7 +534,7 @@ public class HCatalogStore extends CatalogConstants implements CatalogStore {
           sd.getSerdeInfo().setSerializationLib(org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe.class.getName());
         }
 
-        if (tableDesc.getMeta().getOption(StorageConstants.SEQUENCEFILE_NULL) != null) {
+        if (tableDesc.getMeta().containsOption(StorageConstants.SEQUENCEFILE_NULL)) {
           table.putToParameters(serdeConstants.SERIALIZATION_NULL_FORMAT,
               StringEscapeUtils.unescapeJava(tableDesc.getMeta().getOption(StorageConstants.SEQUENCEFILE_NULL)));
           table.getParameters().remove(StorageConstants.SEQUENCEFILE_NULL);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
index a507b08..f225424 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/src/test/java/org/apache/tajo/catalog/store/TestHCatalogStore.java
@@ -117,7 +117,7 @@ public class TestHCatalogStore {
   @Test
   public void testTableUsingRCFileWithBinarySerde() throws Exception {
     KeyValueSet options = new KeyValueSet();
-    options.put(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
+    options.set(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
     TableMeta meta = new TableMeta(CatalogProtos.StoreType.RCFILE, options);
 
     org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
@@ -146,7 +146,7 @@ public class TestHCatalogStore {
   @Test
   public void testTableUsingRCFileWithTextSerde() throws Exception {
     KeyValueSet options = new KeyValueSet();
-    options.put(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
+    options.set(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
     TableMeta meta = new TableMeta(CatalogProtos.StoreType.RCFILE, options);
 
     org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
@@ -174,8 +174,8 @@ public class TestHCatalogStore {
   @Test
   public void testTableWithNullValue() throws Exception {
     KeyValueSet options = new KeyValueSet();
-    options.put(StorageConstants.CSVFILE_DELIMITER, StringEscapeUtils.escapeJava("\u0002"));
-    options.put(StorageConstants.CSVFILE_NULL, StringEscapeUtils.escapeJava("\u0003"));
+    options.set(StorageConstants.CSVFILE_DELIMITER, StringEscapeUtils.escapeJava("\u0002"));
+    options.set(StorageConstants.CSVFILE_NULL, StringEscapeUtils.escapeJava("\u0003"));
     TableMeta meta = new TableMeta(CatalogProtos.StoreType.CSV, options);
 
     org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
@@ -315,7 +315,7 @@ public class TestHCatalogStore {
   @Test
   public void testTableUsingSequenceFileWithBinarySerde() throws Exception {
     KeyValueSet options = new KeyValueSet();
-    options.put(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
+    options.set(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE);
     TableMeta meta = new TableMeta(CatalogProtos.StoreType.SEQUENCEFILE, options);
 
     org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();
@@ -344,7 +344,7 @@ public class TestHCatalogStore {
   @Test
   public void testTableUsingSequenceFileWithTextSerde() throws Exception {
     KeyValueSet options = new KeyValueSet();
-    options.put(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
+    options.set(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE);
     TableMeta meta = new TableMeta(CatalogProtos.StoreType.SEQUENCEFILE, options);
 
     org.apache.tajo.catalog.Schema schema = new org.apache.tajo.catalog.Schema();

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
index 4039ff2..654736d 100644
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
+++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java
@@ -565,7 +565,7 @@ public class TestCatalog {
 
     String tableName = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "addedtable");
     KeyValueSet opts = new KeyValueSet();
-    opts.put("file.delimiter", ",");
+    opts.set("file.delimiter", ",");
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
 
 
@@ -605,7 +605,7 @@ public class TestCatalog {
 
     String tableName = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "addedtable");
     KeyValueSet opts = new KeyValueSet();
-    opts.put("file.delimiter", ",");
+    opts.set("file.delimiter", ",");
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
 
     Schema partSchema = new Schema();
@@ -643,7 +643,7 @@ public class TestCatalog {
 
     String tableName = CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "addedtable");
     KeyValueSet opts = new KeyValueSet();
-    opts.put("file.delimiter", ",");
+    opts.set("file.delimiter", ",");
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
 
     Schema partSchema = new Schema();
@@ -680,7 +680,7 @@ public class TestCatalog {
 
     String tableName = CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "addedtable");
     KeyValueSet opts = new KeyValueSet();
-    opts.put("file.delimiter", ",");
+    opts.set("file.delimiter", ",");
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
 
     Schema partSchema = new Schema();
@@ -717,7 +717,7 @@ public class TestCatalog {
 
     String tableName = CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "addedtable");
     KeyValueSet opts = new KeyValueSet();
-    opts.put("file.delimiter", ",");
+    opts.set("file.delimiter", ",");
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
 
     Schema partSchema = new Schema();

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-client/src/main/java/org/apache/tajo/cli/DefaultTajoCliOutputFormatter.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/DefaultTajoCliOutputFormatter.java b/tajo-client/src/main/java/org/apache/tajo/cli/DefaultTajoCliOutputFormatter.java
index 62736b5..543ac8c 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/DefaultTajoCliOutputFormatter.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/DefaultTajoCliOutputFormatter.java
@@ -20,11 +20,10 @@ package org.apache.tajo.cli;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.client.QueryStatus;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.util.FileUtil;
 
 import java.io.InputStream;
@@ -33,20 +32,17 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 
 public class DefaultTajoCliOutputFormatter implements TajoCliOutputFormatter {
-  private TajoConf tajoConf;
   private int printPauseRecords;
   private boolean printPause;
   private boolean printErrorTrace;
   private String nullChar;
 
   @Override
-  public void init(TajoConf tajoConf) {
-    this.tajoConf = tajoConf;
-
-    this.printPause = tajoConf.getBoolVar(TajoConf.ConfVars.CLI_PRINT_PAUSE);
-    this.printPauseRecords = tajoConf.getIntVar(TajoConf.ConfVars.CLI_PRINT_PAUSE_NUM_RECORDS);
-    this.printErrorTrace = tajoConf.getBoolVar(TajoConf.ConfVars.CLI_PRINT_ERROR_TRACE);
-    this.nullChar = tajoConf.getVar(ConfVars.CLI_NULL_CHAR);
+  public void init(TajoCli.TajoCliContext context) {
+    this.printPause = context.getBool(SessionVars.CLI_PAGING_ENABLED);
+    this.printPauseRecords = context.getInt(SessionVars.CLI_PAGE_ROWS);
+    this.printErrorTrace = context.getBool(SessionVars.CLI_DISPLAY_ERROR_TRACE);
+    this.nullChar = context.get(SessionVars.CLI_NULL_CHAR);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
index 8d908ec..52f13b0 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
@@ -40,53 +40,69 @@ public class HelpCommand extends TajoShellCommand {
   }
 
   @Override
+  public String [] getAliases() {
+    return new String [] {"\\help"};
+  }
+
+  @Override
   public void invoke(String[] cmd) throws Exception {
     if(targetDocVersion.equalsIgnoreCase("")) {
       targetDocVersion = getDocumentationVersion();
     }
 
-    PrintWriter sout = context.getOutput();
-    sout.println();
-
-    sout.println("General");
-    sout.println("  \\copyright  show Apache License 2.0");
-    sout.println("  \\version    show Tajo version");
-    sout.println("  \\?          show help");
-    sout.println("  \\q          quit tsql");
-    sout.println();
-    sout.println();
-
-    sout.println("Informational");
-    sout.println("  \\l           list databases");
-    sout.println("  \\c           show current database");
-    sout.println("  \\c [DBNAME]  connect to new database");
-    sout.println("  \\d           list tables");
-    sout.println("  \\d [TBNAME]  describe table");
-    sout.println("  \\df          list functions");
-    sout.println("  \\df NAME     describe function");
-    sout.println();
-    sout.println();
-
-    sout.println("Tool");
-    sout.println("  \\!           execute a linux shell command");
-    sout.println("  \\dfs         execute a dfs command");
-    sout.println("  \\admin       execute tajo admin command");
-    sout.println();
-    sout.println();
-
-    sout.println("Variables");
-    sout.println("  \\set [[NAME] [VALUE]  set session variable or list session variables");
-    sout.println("  \\unset NAME           unset session variable");
-    sout.println();
-    sout.println();
-
-    sout.println("Documentations");
-    sout.println("  tsql guide        http://tajo.apache.org/docs/"+ targetDocVersion +"/cli.html");
-    sout.println("  Query language    http://tajo.apache.org/docs/"+ targetDocVersion +"/sql_language.html");
-    sout.println("  Functions         http://tajo.apache.org/docs/"+ targetDocVersion +"/functions.html");
-    sout.println("  Backup & restore  http://tajo.apache.org/docs/"+ targetDocVersion +"/backup_and_restore.html");
-    sout.println("  Configuration     http://tajo.apache.org/docs/"+ targetDocVersion +"/configuration.html");
-    sout.println();
+    if (cmd.length == 1) {
+      PrintWriter sout = context.getOutput();
+      sout.println();
+
+      sout.println("General");
+      sout.println("  \\copyright    show Apache License 2.0");
+      sout.println("  \\version      show Tajo version");
+      sout.println("  \\?            show help");
+      sout.println("  \\? [COMMAND]  show help of a given command");
+      sout.println("  \\help         alias of \\?");
+      sout.println("  \\q            quit tsql");
+      sout.println();
+      sout.println();
+
+      sout.println("Informational");
+      sout.println("  \\l           list databases");
+      sout.println("  \\c           show current database");
+      sout.println("  \\c [DBNAME]  connect to new database");
+      sout.println("  \\d           list tables");
+      sout.println("  \\d [TBNAME]  describe table");
+      sout.println("  \\df          list functions");
+      sout.println("  \\df NAME     describe function");
+      sout.println();
+      sout.println();
+
+      sout.println("Tool");
+      sout.println("  \\!           execute a linux shell command");
+      sout.println("  \\dfs         execute a dfs command");
+      sout.println("  \\admin       execute tajo admin command");
+      sout.println();
+      sout.println();
+
+      sout.println("Variables");
+      sout.println("  \\set [[NAME] [VALUE]  set session variable or list session variables");
+      sout.println("  \\unset NAME           unset session variable");
+      sout.println();
+      sout.println();
+
+      sout.println("Documentations");
+      sout.println("  tsql guide        http://tajo.apache.org/docs/" + targetDocVersion + "/cli.html");
+      sout.println("  Query language    http://tajo.apache.org/docs/" + targetDocVersion + "/sql_language.html");
+      sout.println("  Functions         http://tajo.apache.org/docs/" + targetDocVersion + "/functions.html");
+      sout.println("  Backup & restore  http://tajo.apache.org/docs/" + targetDocVersion + "/backup_and_restore.html");
+      sout.println("  Configuration     http://tajo.apache.org/docs/" + targetDocVersion + "/configuration.html");
+      sout.println();
+    } else if (cmd.length == 2) {
+      String slashCommand = "\\" + cmd[1];
+      if (context.getCommands().containsKey(slashCommand)) {
+        context.getCommands().get(slashCommand).printHelp();
+      } else {
+        context.getOutput().println("Command not found: " + cmd[1]);
+      }
+    }
   }
 
   private String getDocumentationVersion() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-client/src/main/java/org/apache/tajo/cli/SetCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/SetCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/SetCommand.java
index c694507..c957a19 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/SetCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/SetCommand.java
@@ -18,11 +18,15 @@
 
 package org.apache.tajo.cli;
 
+import com.google.protobuf.ServiceException;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.util.StringUtils;
 
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.tajo.SessionVars.VariableMode;
+
 public class SetCommand extends TajoShellCommand {
 
   public SetCommand(TajoCli.TajoCliContext context) {
@@ -34,17 +38,46 @@ public class SetCommand extends TajoShellCommand {
     return "\\set";
   }
 
+  private void showAllSessionVars() throws ServiceException {
+    for (Map.Entry<String, String> entry: client.getAllSessionVariables().entrySet()) {
+      context.getOutput().println(StringUtils.quote(entry.getKey()) + "=" + StringUtils.quote(entry.getValue()));
+    }
+  }
+
+  private void updateSessionVariable(String key, String val) throws ServiceException {
+    Map<String, String> variables = new HashMap<String, String>();
+    variables.put(key, val);
+    client.updateSessionVariables(variables);
+  }
+
+  void set(String key, String val) throws ServiceException {
+    SessionVars sessionVar = null;
+
+    if (SessionVars.exists(key)) { // if the variable is one of the session variables
+      sessionVar = SessionVars.get(key);
+
+      // is it cli-side variable?
+      if (sessionVar.getMode() == VariableMode.CLI_SIDE_VAR) {
+        context.setCliSideVar(key, val);
+      } else {
+        updateSessionVariable(key, val);
+      }
+
+      if (SessionVars.isDeprecated(key)) {
+        context.getOutput().println("Warning: deprecated to directly use config key in TajoConf.ConfVars. " +
+            "Please execute '\\help set'.");
+      }
+    } else {
+      updateSessionVariable(key, val);
+    }
+  }
+
   @Override
   public void invoke(String[] cmd) throws Exception {
     if (cmd.length == 1) {
-      for (Map.Entry<String, String> entry: client.getAllSessionVariables().entrySet()) {
-        context.getOutput().println(StringUtils.quote(entry.getKey()) + "=" + StringUtils.quote(entry.getValue()));
-      }
+      showAllSessionVars();
     } else if (cmd.length == 3) {
-      Map<String, String> variables = new HashMap<String, String>();
-      variables.put(cmd[1], cmd[2]);
-      client.updateSessionVariables(variables);
-      context.setVariable(cmd[1], cmd[2]);
+      set(cmd[1], cmd[2]);
     } else {
       context.getOutput().println("usage: \\set [[NAME] VALUE]");
     }
@@ -59,4 +92,35 @@ public class SetCommand extends TajoShellCommand {
   public String getDescription() {
     return "set session variable or shows all session variables";
   }
+
+  @Override
+  public void printHelp() {
+    context.getOutput().println("\nAvailable Session Variables:\n");
+    for (SessionVars var : SessionVars.values()) {
+
+      if (var.getMode() == VariableMode.DEFAULT ||
+          var.getMode() == VariableMode.CLI_SIDE_VAR ||
+          var.getMode() == VariableMode.FROM_SHELL_ENV) {
+
+        context.getOutput().println("\\set " + var.keyname() + " " + getDisplayType(var.getVarType()) + " - " + var
+            .getDescription());
+      }
+    }
+  }
+
+  public static String getDisplayType(Class<?> clazz) {
+    if (clazz == String.class) {
+      return "[text value]";
+    } else if (clazz == Integer.class) {
+      return "[int value]";
+    } else if (clazz == Long.class) {
+      return "[long value]";
+    } else if (clazz == Float.class) {
+      return "[real value]";
+    } else if (clazz == Boolean.class) {
+      return "[true or false]";
+    } else {
+      return clazz.getSimpleName();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java b/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
index c20e44b..81427ab 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
@@ -18,11 +18,12 @@
 
 package org.apache.tajo.cli;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.protobuf.ServiceException;
 import jline.console.ConsoleReader;
 import org.apache.commons.cli.*;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.*;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryStatus;
@@ -62,7 +63,7 @@ public class TajoCli {
   // Current States
   private String currentDatabase;
 
-  private TajoCliOutputFormatter outputFormatter;
+  private TajoCliOutputFormatter displayFormatter;
 
   private boolean wasError = false;
 
@@ -99,7 +100,10 @@ public class TajoCli {
     options.addOption("help", "help", false, "help");
   }
 
-  public class TajoCliContext {
+  public class TajoCliContext extends OverridableConf {
+    public TajoCliContext(TajoConf conf) {
+      super(conf, ConfigKey.ConfigType.SESSION);
+    }
 
     public TajoClient getTajoClient() {
       return client;
@@ -121,18 +125,50 @@ public class TajoCli {
       return conf;
     }
 
-    public void setVariable(String key, String value) {
-      conf.set(key, value);
-      try {
-        initFormatter();
-      } catch (Exception e) {
-        System.err.println(ERROR_PREFIX + e.getMessage());
+    @VisibleForTesting
+    public String getCliSideVar(String key) {
+      if (SessionVars.exists(key)) {
+        ConfigKey configKey = SessionVars.get(key);
+        return get(configKey);
+      } else {
+        return get(key);
+      }
+    }
+
+    public void setCliSideVar(String key, String value) {
+      Preconditions.checkNotNull(key);
+      Preconditions.checkNotNull(value);
+
+      boolean shouldReloadFormatter = false;
+
+      if (SessionVars.exists(key)) {
+        SessionVars configKey = SessionVars.get(key);
+        put(configKey, value);
+        shouldReloadFormatter = configKey.getMode() == SessionVars.VariableMode.CLI_SIDE_VAR;
+      } else {
+        set(key, value);
+
+        // It is hard to recognize it is a client side variable. So, we always reload formatter.
+        shouldReloadFormatter = true;
       }
+
+      if (shouldReloadFormatter) {
+        try {
+          initFormatter();
+        } catch (Exception e) {
+          System.err.println(ERROR_PREFIX + e.getMessage());
+        }
+      }
+    }
+
+    public Map<String, TajoShellCommand> getCommands() {
+      return commands;
     }
   }
 
   public TajoCli(TajoConf c, String [] args, InputStream in, OutputStream out) throws Exception {
     this.conf = new TajoConf(c);
+    context = new TajoCliContext(conf);
     this.sin = in;
     this.reader = new ConsoleReader(sin, out);
     this.reader.setExpandEvents(false);
@@ -161,13 +197,7 @@ public class TajoCli {
     }
 
     if (cmd.getOptionValues("conf") != null) {
-      for (String eachParam: cmd.getOptionValues("conf")) {
-        String[] tokens = eachParam.split("=");
-        if (tokens.length != 2) {
-          continue;
-        }
-        conf.set(tokens[0], tokens[1]);
-      }
+      processConfVarCommand(cmd.getOptionValues("conf"));
     }
 
     // if there is no "-h" option,
@@ -196,19 +226,22 @@ public class TajoCli {
       client = new TajoClient(conf, baseDatabase);
     }
 
-    context = new TajoCliContext();
     context.setCurrentDatabase(client.getCurrentDatabase());
     initHistory();
     initCommands();
 
+    if (cmd.getOptionValues("conf") != null) {
+      processSessionVarCommand(cmd.getOptionValues("conf"));
+    }
+
     if (cmd.hasOption("c")) {
-      outputFormatter.setScirptMode();
+      displayFormatter.setScirptMode();
       int exitCode = executeScript(cmd.getOptionValue("c"));
       sout.flush();
       System.exit(exitCode);
     }
     if (cmd.hasOption("f")) {
-      outputFormatter.setScirptMode();
+      displayFormatter.setScirptMode();
       cmd.getOptionValues("");
       File sqlFile = new File(cmd.getOptionValue("f"));
       if (sqlFile.exists()) {
@@ -226,13 +259,38 @@ public class TajoCli {
     addShutdownHook();
   }
 
+  private void processConfVarCommand(String[] confCommands) throws ServiceException {
+    for (String eachParam: confCommands) {
+      String[] tokens = eachParam.split("=");
+      if (tokens.length != 2) {
+        continue;
+      }
+
+      if (!SessionVars.exists(tokens[0])) {
+        conf.set(tokens[0], tokens[1]);
+      }
+    }
+  }
+
+  private void processSessionVarCommand(String[] confCommands) throws ServiceException {
+    for (String eachParam: confCommands) {
+      String[] tokens = eachParam.split("=");
+      if (tokens.length != 2) {
+        continue;
+      }
+
+      if (SessionVars.exists(tokens[0])) {
+        ((SetCommand)commands.get("\\set")).set(tokens[0], tokens[1]);
+      }
+    }
+  }
+
   private void initFormatter() throws Exception {
-    Class formatterClass = conf.getClass(ConfVars.CLI_OUTPUT_FORMATTER_CLASS.varname,
-        DefaultTajoCliOutputFormatter.class);
-    if (outputFormatter == null || !outputFormatter.getClass().equals(formatterClass)) {
-      outputFormatter = (TajoCliOutputFormatter)formatterClass.newInstance();
+    Class formatterClass = context.getClass(SessionVars.CLI_FORMATTER_CLASS);
+    if (displayFormatter == null || !displayFormatter.getClass().equals(formatterClass)) {
+      displayFormatter = (TajoCliOutputFormatter)formatterClass.newInstance();
     }
-    outputFormatter.init(conf);
+    displayFormatter.init(context);
   }
 
   public TajoCliContext getContext() {
@@ -280,6 +338,9 @@ public class TajoCli {
         throw new RuntimeException(e.getMessage());
       }
       commands.put(cmd.getCommand(), cmd);
+      for (String alias : cmd.getAliases()) {
+        commands.put(alias, cmd);
+      }
     }
   }
 
@@ -332,7 +393,7 @@ public class TajoCli {
         exitCode = executeParsedResults(parsedResults);
         currentPrompt = updatePrompt(parser.getState());
 
-        if (exitCode != 0 && context.getConf().getBoolVar(ConfVars.CLI_ERROR_STOP)) {
+        if (exitCode != 0 && context.getBool(SessionVars.ON_ERROR_STOP)) {
           return exitCode;
         }
       }
@@ -372,18 +433,18 @@ public class TajoCli {
       try {
         invoked.invoke(arguments);
       } catch (IllegalArgumentException ige) {
-        outputFormatter.printErrorMessage(sout, ige);
+        displayFormatter.printErrorMessage(sout, ige);
         wasError = true;
         return -1;
       } catch (Exception e) {
-        outputFormatter.printErrorMessage(sout, e);
+        displayFormatter.printErrorMessage(sout, e);
         wasError = true;
         return -1;
       } finally {
         context.getOutput().flush();
       }
 
-      if (wasError && context.getConf().getBoolVar(ConfVars.CLI_ERROR_STOP)) {
+      if (wasError && context.getBool(SessionVars.ON_ERROR_STOP)) {
         break;
       }
     }
@@ -395,7 +456,7 @@ public class TajoCli {
     long startTime = System.currentTimeMillis();
     ClientProtos.SubmitQueryResponse response = client.executeQueryWithJson(json);
     if (response == null) {
-      outputFormatter.printErrorMessage(sout, "response is null");
+      displayFormatter.printErrorMessage(sout, "response is null");
       wasError = true;
     } else if (response.getResultCode() == ClientProtos.ResultCode.OK) {
       if (response.getIsForwarded()) {
@@ -403,7 +464,7 @@ public class TajoCli {
         waitForQueryCompleted(queryId);
       } else {
         if (!response.hasTableDesc() && !response.hasResultSet()) {
-          outputFormatter.printMessage(sout, "OK");
+          displayFormatter.printMessage(sout, "OK");
           wasError = true;
         } else {
           localQueryCompleted(response, startTime);
@@ -411,7 +472,7 @@ public class TajoCli {
       }
     } else {
       if (response.hasErrorMessage()) {
-        outputFormatter.printErrorMessage(sout, response.getErrorMessage());
+        displayFormatter.printErrorMessage(sout, response.getErrorMessage());
         wasError = true;
       }
     }
@@ -421,7 +482,7 @@ public class TajoCli {
     long startTime = System.currentTimeMillis();
     ClientProtos.SubmitQueryResponse response = client.executeQuery(statement);
     if (response == null) {
-      outputFormatter.printErrorMessage(sout, "response is null");
+      displayFormatter.printErrorMessage(sout, "response is null");
       wasError = true;
     } else if (response.getResultCode() == ClientProtos.ResultCode.OK) {
       if (response.getIsForwarded()) {
@@ -429,14 +490,14 @@ public class TajoCli {
         waitForQueryCompleted(queryId);
       } else {
         if (!response.hasTableDesc() && !response.hasResultSet()) {
-          outputFormatter.printMessage(sout, "OK");
+          displayFormatter.printMessage(sout, "OK");
         } else {
           localQueryCompleted(response, startTime);
         }
       }
     } else {
       if (response.hasErrorMessage()) {
-        outputFormatter.printErrorMessage(sout, response.getErrorMessage());
+        displayFormatter.printErrorMessage(sout, response.getErrorMessage());
         wasError = true;
       }
     }
@@ -454,13 +515,13 @@ public class TajoCli {
       // non-forwarded INSERT INTO query does not have any query id.
       // In this case, it just returns succeeded query information without printing the query results.
       if (response.getMaxRowNum() < 0 && queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
-        outputFormatter.printResult(sout, sin, desc, responseTime, res);
+        displayFormatter.printResult(sout, sin, desc, responseTime, res);
       } else {
         res = TajoClient.createResultSet(client, response);
-        outputFormatter.printResult(sout, sin, desc, responseTime, res);
+        displayFormatter.printResult(sout, sin, desc, responseTime, res);
       }
     } catch (Throwable t) {
-      outputFormatter.printErrorMessage(sout, t);
+      displayFormatter.printErrorMessage(sout, t);
       wasError = true;
     } finally {
       if (res != null) {
@@ -495,7 +556,7 @@ public class TajoCli {
         }
 
         if (status.getState() == QueryState.QUERY_RUNNING || status.getState() == QueryState.QUERY_SUCCEEDED) {
-          outputFormatter.printProgress(sout, status);
+          displayFormatter.printProgress(sout, status);
         }
 
         if (status.getState() != QueryState.QUERY_RUNNING &&
@@ -509,10 +570,10 @@ public class TajoCli {
       }
 
       if (status.getState() == QueryState.QUERY_ERROR || status.getState() == QueryState.QUERY_FAILED) {
-        outputFormatter.printErrorMessage(sout, status);
+        displayFormatter.printErrorMessage(sout, status);
         wasError = true;
       } else if (status.getState() == QueryState.QUERY_KILLED) {
-        outputFormatter.printKilledMessage(sout, queryId);
+        displayFormatter.printKilledMessage(sout, queryId);
         wasError = true;
       } else {
         if (status.getState() == QueryState.QUERY_SUCCEEDED) {
@@ -521,15 +582,15 @@ public class TajoCli {
           if (status.hasResult()) {
             res = TajoClient.createResultSet(client, queryId, response);
             TableDesc desc = new TableDesc(response.getTableDesc());
-            outputFormatter.printResult(sout, sin, desc, responseTime, res);
+            displayFormatter.printResult(sout, sin, desc, responseTime, res);
           } else {
             TableDesc desc = new TableDesc(response.getTableDesc());
-            outputFormatter.printResult(sout, sin, desc, responseTime, res);
+            displayFormatter.printResult(sout, sin, desc, responseTime, res);
           }
         }
       }
     } catch (Throwable t) {
-      outputFormatter.printErrorMessage(sout, t);
+      displayFormatter.printErrorMessage(sout, t);
       wasError = true;
     } finally {
       if (res != null) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-client/src/main/java/org/apache/tajo/cli/TajoCliOutputFormatter.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/TajoCliOutputFormatter.java b/tajo-client/src/main/java/org/apache/tajo/cli/TajoCliOutputFormatter.java
index 0e91669..0ad89f2 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/TajoCliOutputFormatter.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/TajoCliOutputFormatter.java
@@ -21,7 +21,6 @@ package org.apache.tajo.cli;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryStatus;
-import org.apache.tajo.conf.TajoConf;
 
 import java.io.InputStream;
 import java.io.PrintWriter;
@@ -30,9 +29,9 @@ import java.sql.ResultSet;
 public interface TajoCliOutputFormatter {
   /**
    * Initialize formatter
-   * @param tajoConf
+   * @param context
    */
-  public void init(TajoConf tajoConf);
+  public void init(TajoCli.TajoCliContext context);
 
   /**
    * print query result to console

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-client/src/main/java/org/apache/tajo/cli/TajoShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/TajoShellCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/TajoShellCommand.java
index 1ea2893..39f5377 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/TajoShellCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/TajoShellCommand.java
@@ -23,16 +23,24 @@ import org.apache.tajo.conf.TajoConf;
 
 public abstract class TajoShellCommand {
   public abstract String getCommand();
+  public String [] getAliases() {
+    return new String[] {};
+  }
   public abstract void invoke(String [] command) throws Exception;
   public abstract String getUsage();
   public abstract String getDescription();
+  public void printHelp() {
+    context.getOutput().print(getCommand());
+    context.getOutput().print(" - ");
+    context.getOutput().println(getDescription());
+  }
 
   protected TajoCli.TajoCliContext context;
   protected TajoClient client;
   protected int maxColumn;
 
   public TajoShellCommand(TajoCli.TajoCliContext context) {
-    maxColumn = context.getTajoClient().getConf().getIntVar(TajoConf.ConfVars.CLI_MAX_COLUMN);
+    maxColumn = context.getTajoClient().getConf().getIntVar(TajoConf.ConfVars.$CLI_MAX_COLUMN);
     this.context = context;
     client = context.getTajoClient();
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-common/src/main/java/org/apache/tajo/ConfigKey.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/ConfigKey.java b/tajo-common/src/main/java/org/apache/tajo/ConfigKey.java
new file mode 100644
index 0000000..b9d51ec
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/ConfigKey.java
@@ -0,0 +1,52 @@
+/**
+ * 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;
+
+public interface ConfigKey {
+
+  // Client can set or change variables of this mode.
+  public static final int DEFAULT_MODE = 0;
+  // This is similar to DEFAULT mode. In addition, it tries to get values from shell env. variables.
+  public static final int FROM_SHELL_ENV_MODE = 1;
+  // only TajoMaster is able to set and change variables of this mode.
+  public static final int SERVER_SIDE_VAR_MODE = 2;
+  // This type variable will be used only in cli side.
+  public static final int CLI_SIDE_VAR_MODE = 3;
+
+  public static enum ConfigType {
+    SYSTEM(""),
+    SESSION("$"),
+    QUERY("@"),
+    CLI("+");
+
+    private String prefix;
+
+    ConfigType(String prefix) {
+      this.prefix = prefix;
+    }
+
+    public String getPrefix() {
+      return prefix;
+    }
+  }
+
+  public String keyname();
+
+  public ConfigType type();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java b/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
new file mode 100644
index 0000000..220bd43
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/OverridableConf.java
@@ -0,0 +1,235 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.util.KeyValueSet;
+
+import static org.apache.tajo.ConfigKey.ConfigType;
+import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
+
+/**
+ * OverridableConf provides a consolidated config system. Tajo basically uses TajoConf, which is a extended class of
+ * Hadoop's Configuration system, However, TajoConf is only used for sharing static system configs, such as binding
+ * address of master and workers, system directories, other system parameters.
+ *
+ * For modifiable or instant configs, we use OverridableConf, which is a set of key-value pairs.
+ * OverridableConf provides more strong-typed way to set configs and its behavior is more clear than Configuration
+ * system.
+ *
+ * By default, OverridableConf recognizes following config types.
+ *
+ * <ul>
+ *    <li>System Config - it comes from Hadoop's Configuration class. by tajo-site, catalog-site,
+ *    catalog-default and TajoConf.</li>
+ *    <li>Session variables - they are instantly configured by users.
+ *    Each client session has it own set of session variables.</li>
+ * </ul>
+ *
+ * System configs and session variables can set the same config in the same time. System configs are usually used to set
+ * default configs, and session variables is user-specified configs. So, session variables can override system configs.
+ */
+public class OverridableConf extends KeyValueSet {
+  private static final Log LOG = LogFactory.getLog(OverridableConf.class);
+  private ConfigType [] configTypes;
+  private TajoConf conf;
+
+  public OverridableConf(final TajoConf conf, ConfigType...configTypes) {
+    this.conf = conf;
+    this.configTypes = configTypes;
+  }
+
+  public OverridableConf(final TajoConf conf, KeyValueSetProto proto, ConfigType...configTypes) {
+    super(proto);
+    this.conf = conf;
+    this.configTypes = configTypes;
+  }
+
+  public void setConf(TajoConf conf) {
+    this.conf = conf;
+  }
+
+  public TajoConf getConf() {
+    return conf;
+  }
+
+  public void setBool(ConfigKey key, boolean val) {
+    setBool(key.keyname(), val);
+  }
+
+  public boolean getBool(ConfigKey key, Boolean defaultVal) {
+    assertRegisteredEnum(key);
+
+    if (key.type() != ConfigType.SESSION && key.type() != ConfigType.SYSTEM) {
+      return getBool(key.keyname(), defaultVal);
+    } else {
+      switch (key.type()) {
+      case QUERY:
+        return getBool(key.keyname());
+      case SESSION:
+        return getBool(key.keyname(), conf.getBoolVar(((SessionVars) key).getConfVars()));
+      case SYSTEM:
+        return conf.getBoolVar((TajoConf.ConfVars) key);
+      default:
+        throw new IllegalStateException("key does not belong to Session and System config sets");
+      }
+    }
+  }
+
+  public boolean getBool(ConfigKey key) {
+    return getBool(key, null);
+  }
+
+  public int getInt(ConfigKey key, Integer defaultVal) {
+    assertRegisteredEnum(key);
+
+    if (key.type() != ConfigType.SESSION && key.type() != ConfigType.SYSTEM) {
+      return getInt(key.keyname(), defaultVal);
+    } else {
+      switch (key.type()) {
+      case SESSION:
+        return getInt(key.keyname(), conf.getIntVar(((SessionVars) key).getConfVars()));
+      case SYSTEM:
+        return conf.getIntVar((TajoConf.ConfVars) key);
+      default:
+        throw new IllegalStateException("key does not belong to Session and System config sets");
+      }
+    }
+  }
+
+  public int getInt(ConfigKey key) {
+    return getInt(key, null);
+  }
+
+  public long getLong(ConfigKey key, Long defaultVal) {
+    assertRegisteredEnum(key);
+
+    if (key.type() != ConfigType.SESSION && key.type() != ConfigType.SYSTEM) {
+      return getLong(key.keyname(), defaultVal);
+    } else {
+      switch (key.type()) {
+      case SESSION:
+        return getLong(key.keyname(), conf.getLongVar(((SessionVars) key).getConfVars()));
+      case SYSTEM:
+        return conf.getLongVar((TajoConf.ConfVars) key);
+      default:
+        throw new IllegalStateException("key does not belong to Session and System config sets");
+      }
+    }
+  }
+
+  public long getLong(ConfigKey key) {
+    return getLong(key, null);
+  }
+
+  public float getFloat(ConfigKey key, Float defaultVal) {
+    assertRegisteredEnum(key);
+
+    if (key.type() != ConfigType.SESSION && key.type() != ConfigType.SYSTEM) {
+      return getFloat(key.keyname(), defaultVal);
+    } else {
+      switch (key.type()) {
+      case SESSION:
+        return getFloat(key.keyname(), conf.getFloatVar(((SessionVars) key).getConfVars()));
+      case SYSTEM:
+        return conf.getFloatVar((TajoConf.ConfVars) key);
+      default:
+        throw new IllegalStateException("key does not belong to Session and System config sets");
+      }
+    }
+  }
+
+  public float getFloat(ConfigKey key) {
+    return getLong(key, null);
+  }
+
+  public void put(ConfigKey key, String val) {
+    set(key.keyname(), val);
+  }
+
+  private void assertRegisteredEnum(ConfigKey key) {
+    boolean registered = false;
+
+    for (ConfigType c : configTypes) {
+      registered = key.type() == c;
+    }
+
+    registered |= key.type() == ConfigType.SESSION || key.type() != ConfigType.SYSTEM;
+
+    Preconditions.checkArgument(registered, key.keyname() + " (" + key.type() + ") is not allowed in " +
+      getClass().getSimpleName());
+  }
+
+  public String get(ConfigKey key, String defaultVal) {
+    assertRegisteredEnum(key);
+
+    if (key.type() != ConfigType.SESSION && key.type() != ConfigType.SYSTEM) {
+      return get(key.keyname(), defaultVal);
+    } else {
+      switch (key.type()) {
+      case SESSION:
+        return get(key.keyname(), conf.getVar(((SessionVars) key).getConfVars()));
+      case SYSTEM:
+        return conf.getVar((TajoConf.ConfVars) key);
+      default:
+        throw new IllegalStateException("key does not belong to Session and System config sets");
+      }
+    }
+  }
+
+  public String get(ConfigKey key) {
+    return get(key, null);
+  }
+
+  public Class<?> getClass(ConfigKey key) {
+    assertRegisteredEnum(key);
+
+    String className = getTrimmed(key);
+    try {
+      return Class.forName(className);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public String getTrimmed(ConfigKey key) {
+    String value = get(key);
+
+    if (null == value) {
+      return null;
+    } else {
+      return value.trim();
+    }
+  }
+
+  public boolean containsKey(ConfigKey key) {
+    return containsKey(key.keyname());
+  }
+
+  public boolean equalKey(ConfigKey key, String another) {
+    if (containsKey(key)) {
+      return get(key).equals(another);
+    } else {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/SessionVars.java b/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
new file mode 100644
index 0000000..5503aaa
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/SessionVars.java
@@ -0,0 +1,211 @@
+/**
+ * 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;
+
+import com.google.common.collect.Maps;
+
+import java.util.Map;
+
+import static org.apache.tajo.SessionVars.VariableMode.*;
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+
+public enum SessionVars implements ConfigKey {
+
+  // Common Suffix Naming Rules:
+  //
+  // * LIMIT   - We use the suffix 'LIMIT' if the variable is threshold. So, if some value is greater or less than
+  //             the variable with suffix 'LIMIT', some action will be different from before.
+  // * SIZE    - The suffix 'SIZE' means a data volume like bytes or mega bytes.
+  //             It should be used for user's desired volume.
+  // * ENABLED - The suffix 'ENABLED' means a true or false value. If it is true, it will enable some feature.
+  //             Otherwise, the feature will be turned off.
+
+
+  //-------------------------------------------------------------------------------
+  // Server Side Only Variables
+  //-------------------------------------------------------------------------------
+  SESSION_ID(ConfVars.$EMPTY, "session variable", SERVER_SIDE_VAR),
+  SESSION_LAST_ACCESS_TIME(ConfVars.$EMPTY, "last access time", SERVER_SIDE_VAR),
+
+  USERNAME(ConfVars.USERNAME, "username", SERVER_SIDE_VAR),
+  CLIENT_HOST(ConfVars.$EMPTY, "client hostname", SERVER_SIDE_VAR),
+
+  CURRENT_DATABASE(ConfVars.$EMPTY, "current database", SERVER_SIDE_VAR),
+
+  //-------------------------------------------------------------------------------
+  // Client Side Variables
+  //-------------------------------------------------------------------------------
+
+  // Client --------------------------------------------------------
+  SESSION_EXPIRY_TIME(ConfVars.$CLIENT_SESSION_EXPIRY_TIME, "session expiry time (secs)", DEFAULT),
+
+  // Command line interface and its behavior --------------------------------
+  CLI_COLUMNS(ConfVars.$CLI_MAX_COLUMN, "Sets the width for the wrapped format", CLI_SIDE_VAR),
+  CLI_FORMATTER_CLASS(ConfVars.$CLI_OUTPUT_FORMATTER_CLASS, "Sets the output format class to display results",
+      CLI_SIDE_VAR),
+  CLI_NULL_CHAR(ConfVars.$CLI_NULL_CHAR, "Sets the string to be printed in place of a null value.", CLI_SIDE_VAR),
+
+  CLI_PAGE_ROWS(ConfVars.$CLI_PRINT_PAUSE_NUM_RECORDS, "Sets the number of rows for paging", CLI_SIDE_VAR),
+  CLI_PAGING_ENABLED(ConfVars.$CLI_PRINT_PAUSE, "Enable paging of result display", CLI_SIDE_VAR),
+  CLI_DISPLAY_ERROR_TRACE(ConfVars.$CLI_PRINT_ERROR_TRACE, "Enable display of error trace", CLI_SIDE_VAR),
+
+  ON_ERROR_STOP(ConfVars.$CLI_ERROR_STOP, "tsql will exist if an error occurs.", CLI_SIDE_VAR),
+
+  // Timezone & Date ----------------------------------------------------------
+  TZ(ConfVars.$TIMEZONE, "Sets timezone", FROM_SHELL_ENV),
+  DATE_ORDER(ConfVars.$DATE_ORDER, "date order (default is YMD)", FROM_SHELL_ENV),
+
+  // Locales and Character set ------------------------------------------------
+  // TODO - they are reserved variables, and we should support them.
+  LANG(ConfVars.$EMPTY, "Language", FROM_SHELL_ENV),
+  LC_ALL(ConfVars.$EMPTY, "String sort order", FROM_SHELL_ENV),
+  LC_COLLATE(ConfVars.$EMPTY, "String sort order", FROM_SHELL_ENV),
+  LC_CTYPE(ConfVars.$EMPTY, "Character classification (What is a letter? Its upper-case equivalent?)", FROM_SHELL_ENV),
+  LC_MESSAGES(ConfVars.$EMPTY, "Language of messages", FROM_SHELL_ENV),
+  LC_MONETARY(ConfVars.$EMPTY, "Formatting of currency amounts", FROM_SHELL_ENV),
+  LC_NUMERIC(ConfVars.$EMPTY, "Formatting of numbers", FROM_SHELL_ENV),
+  LC_TIME(ConfVars.$EMPTY, "Formatting of dates and times", FROM_SHELL_ENV),
+
+
+  // Query and Optimization ---------------------------------------------------
+
+  // for distributed query strategies
+  BROADCAST_TABLE_SIZE_LIMIT(ConfVars.$DIST_QUERY_BROADCAST_JOIN_THRESHOLD, "limited size (bytes) of broadcast table",
+      DEFAULT),
+
+  JOIN_TASK_INPUT_SIZE(ConfVars.$DIST_QUERY_JOIN_TASK_VOLUME, "join task input size (mb) ", DEFAULT),
+  SORT_TASK_INPUT_SIZE(ConfVars.$DIST_QUERY_SORT_TASK_VOLUME, "sort task input size (mb)", DEFAULT),
+  GROUPBY_TASK_INPUT_SIZE(ConfVars.$DIST_QUERY_GROUPBY_TASK_VOLUME, "group by task input size (mb)", DEFAULT),
+
+  JOIN_PER_SHUFFLE_SIZE(ConfVars.$DIST_QUERY_JOIN_PARTITION_VOLUME, "shuffle output size for join (mb)", DEFAULT),
+  GROUPBY_PER_SHUFFLE_SIZE(ConfVars.$DIST_QUERY_GROUPBY_PARTITION_VOLUME, "shuffle output size for sort (mb)", DEFAULT),
+  TABLE_PARTITION_PER_SHUFFLE_SIZE(ConfVars.$DIST_QUERY_TABLE_PARTITION_VOLUME,
+      "shuffle output size for partition table write (mb)", DEFAULT),
+
+  // for physical Executors
+  EXTSORT_BUFFER_SIZE(ConfVars.$EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE, "sort buffer size for external sort (mb)", DEFAULT),
+  HASH_JOIN_SIZE_LIMIT(ConfVars.$EXECUTOR_HASH_JOIN_SIZE_THRESHOLD, "limited size for hash join (mb)", DEFAULT),
+  INNER_HASH_JOIN_SIZE_LIMIT(ConfVars.$EXECUTOR_INNER_HASH_JOIN_SIZE_THRESHOLD,
+      "limited size for hash inner join (mb)", DEFAULT),
+  OUTER_HASH_JOIN_SIZE_LIMIT(ConfVars.$EXECUTOR_OUTER_HASH_JOIN_SIZE_THRESHOLD, "limited size for hash outer join (mb)",
+      DEFAULT),
+  HASH_GROUPBY_SIZE_LIMIT(ConfVars.$EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD, "limited size for hash groupby (mb)",
+      DEFAULT),
+  MAX_OUTPUT_FILE_SIZE(ConfVars.$MAX_OUTPUT_FILE_SIZE, "Maximum per-output file size (mb). 0 means infinite.", DEFAULT),
+
+  NULL_CHAR(ConfVars.$CSVFILE_NULL, "null char of text file output", DEFAULT),
+
+  // Behavior Control ---------------------------------------------------------
+  ARITHABORT(ConfVars.$BEHAVIOR_ARITHMETIC_ABORT,
+      "If true, a running query will be terminated when an overflow or divide-by-zero occurs.", DEFAULT),
+
+  //-------------------------------------------------------------------------------
+  // Only for Unit Testing
+  //-------------------------------------------------------------------------------
+  DEBUG_ENABLED(ConfVars.$DEBUG_ENABLED, "(debug only) debug mode enabled", DEFAULT),
+  TEST_BROADCAST_JOIN_ENABLED(ConfVars.$TEST_BROADCAST_JOIN_ENABLED, "(test only) broadcast enabled", TEST_VAR),
+  TEST_JOIN_OPT_ENABLED(ConfVars.$TEST_JOIN_OPT_ENABLED, "(test only) join optimization enabled", TEST_VAR),
+  TEST_FILTER_PUSHDOWN_ENABLED(ConfVars.$TEST_FILTER_PUSHDOWN_ENABLED, "filter push down enabled", TEST_VAR),
+  TEST_MIN_TASK_NUM(ConfVars.$TEST_MIN_TASK_NUM, "(test only) min task num", TEST_VAR),
+  ;
+
+  public static Map<String, SessionVars> SESSION_VARS = Maps.newHashMap();
+  public static Map<String, SessionVars> DEPRECATED_SESSION_VARS = Maps.newHashMap();
+
+  static {
+    for (SessionVars var : SessionVars.values()) {
+      SESSION_VARS.put(var.keyname(), var);
+      DEPRECATED_SESSION_VARS.put(var.getConfVars().keyname(), var);
+    }
+  }
+
+  private final ConfVars key;
+  private final String description;
+  private final VariableMode mode;
+
+  public static enum VariableMode {
+    DEFAULT,         // Client can set or change variables of this mode..
+    FROM_SHELL_ENV,  // This is similar to DEFAULT mode. In addition, it tries to get values from shell env. variables.
+    SERVER_SIDE_VAR, // only TajoMaster is able to set and change variables of this mode.
+    CLI_SIDE_VAR,    // This type variable is used in CLI.
+    TEST_VAR         // Only used for unit tests
+  }
+
+  SessionVars(ConfVars key, String description, VariableMode mode) {
+    this.key = key;
+    this.description = description;
+    this.mode = mode;
+  }
+
+  public String keyname() {
+    return name();
+  }
+
+  public ConfigType type() {
+    return ConfigType.SESSION;
+  }
+
+  public ConfVars getConfVars() {
+    return key;
+  }
+
+  public Class<?> getVarType() {
+    return key.valClass;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public VariableMode getMode() {
+    return mode;
+  }
+
+  public static boolean exists(String keyname) {
+    return SESSION_VARS.containsKey(keyname) || DEPRECATED_SESSION_VARS.containsKey(keyname);
+  }
+
+  public static boolean isDeprecated(String keyname) {
+    return DEPRECATED_SESSION_VARS.containsKey(keyname);
+  }
+
+  public static boolean isPublic(SessionVars var) {
+    return var.getMode() != SERVER_SIDE_VAR;
+  }
+
+  public static SessionVars get(String keyname) {
+    if (SESSION_VARS.containsKey(keyname)) {
+      return SESSION_VARS.get(keyname);
+    } else if (DEPRECATED_SESSION_VARS.containsKey(keyname)) {
+      return DEPRECATED_SESSION_VARS.get(keyname);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * rename deprecated name to current name if the name is deprecated.
+   *
+   * @param keyname session variable name
+   * @return The current session variable name
+   */
+  public static String handleDeprecatedName(String keyname) {
+    return SessionVars.exists(keyname) ? SessionVars.get(keyname).keyname() : keyname;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/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 b75530b..2504c23 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
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ConfigKey;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.util.TUtil;
@@ -81,7 +82,7 @@ public class TajoConf extends Configuration {
     try {
       if (CURRENT_TIMEZONE == null) {
         TajoConf tajoConf = new TajoConf();
-        CURRENT_TIMEZONE = TimeZone.getTimeZone(tajoConf.getVar(ConfVars.TAJO_TIMEZONE));
+        CURRENT_TIMEZONE = TimeZone.getTimeZone(tajoConf.getVar(ConfVars.$TIMEZONE));
       }
       return CURRENT_TIMEZONE;
     } finally {
@@ -105,7 +106,7 @@ public class TajoConf extends Configuration {
     try {
       if (DATE_ORDER < 0) {
         TajoConf tajoConf = new TajoConf();
-        String dateOrder = tajoConf.getVar(ConfVars.TAJO_DATE_ORDER);
+        String dateOrder = tajoConf.getVar(ConfVars.$DATE_ORDER);
         if ("YMD".equals(dateOrder)) {
           DATE_ORDER = DateTimeConstants.DATEORDER_YMD;
         } else if ("DMY".equals(dateOrder)) {
@@ -133,11 +134,13 @@ public class TajoConf extends Configuration {
     }
   }
 
-  public static enum ConfVars {
+  public static enum ConfVars implements ConfigKey {
 
-    //////////////////////////////////
+    ///////////////////////////////////////////////////////////////////////////////////////
     // Tajo System Configuration
-    //////////////////////////////////
+    //
+    // They are all static configs which are not changed or not overwritten at all.
+    ///////////////////////////////////////////////////////////////////////////////////////
 
     // a username for a running Tajo cluster
     ROOT_DIR("tajo.rootdir", "file:///tmp/tajo-${user.name}/"),
@@ -196,23 +199,19 @@ public class TajoConf extends Configuration {
     // Catalog
     CATALOG_ADDRESS("tajo.catalog.client-rpc.address", "localhost:26005"),
 
-    //////////////////////////////////
-    // for Yarn Resource Manager
-    //////////////////////////////////
+
+    // for Yarn Resource Manager ----------------------------------------------
+
     /** how many launching TaskRunners in parallel */
     YARN_RM_QUERY_MASTER_MEMORY_MB("tajo.querymaster.memory-mb", 512),
     YARN_RM_QUERY_MASTER_DISKS("tajo.yarn-rm.querymaster.disks", 1),
     YARN_RM_TASKRUNNER_LAUNCH_PARALLEL_NUM("tajo.yarn-rm.parallel-task-runner-launcher-num", 16),
     YARN_RM_WORKER_NUMBER_PER_NODE("tajo.yarn-rm.max-worker-num-per-node", 8),
 
-    //////////////////////////////////
     // Query Configuration
-    //////////////////////////////////
     QUERY_SESSION_TIMEOUT("tajo.query.session.timeout-sec", 60),
 
-    //////////////////////////////////
-    // Shuffle Configuration
-    //////////////////////////////////
+    // Shuffle Configuration --------------------------------------------------
     PULLSERVER_PORT("tajo.pullserver.port", 0),
     SHUFFLE_SSL_ENABLED_KEY("tajo.pullserver.ssl.enabled", false),
     SHUFFLE_FILE_FORMAT("tajo.shuffle.file-format", "RAW"),
@@ -221,61 +220,28 @@ public class TajoConf extends Configuration {
     SHUFFLE_FETCHER_READ_TIMEOUT("tajo.shuffle.fetcher.read.timeout-sec", 120),
     SHUFFLE_FETCHER_READ_RETRY_MAX_NUM("tajo.shuffle.fetcher.read.retry.max-num", 20),
 
-    //////////////////////////////////
-    // Storage Configuration
-    //////////////////////////////////
+
+    // Storage Configuration --------------------------------------------------
     ROWFILE_SYNC_INTERVAL("rowfile.sync.interval", 100),
     MINIMUM_SPLIT_SIZE("tajo.min.split.size", (long) 1),
     // for RCFile
     HIVEUSEEXPLICITRCFILEHEADER("tajo.exec.rcfile.use.explicit.header", true),
 
-    // for Storage Manager v2
+    // for Storage Manager ----------------------------------------------------
     STORAGE_MANAGER_VERSION_2("tajo.storage-manager.v2", false),
     STORAGE_MANAGER_DISK_SCHEDULER_MAX_READ_BYTES_PER_SLOT("tajo.storage-manager.max-read-bytes", 8 * 1024 * 1024),
     STORAGE_MANAGER_DISK_SCHEDULER_REPORT_INTERVAL("tajo.storage-manager.disk-scheduler.report-interval", 60 * 1000),
     STORAGE_MANAGER_CONCURRENCY_PER_DISK("tajo.storage-manager.disk-scheduler.per-disk-concurrency", 2),
 
-    //////////////////////////////////////////
-    // Distributed Query Execution Parameters
-    //////////////////////////////////////////
-    DIST_QUERY_BROADCAST_JOIN_AUTO("tajo.dist-query.join.auto-broadcast", true),
-    DIST_QUERY_BROADCAST_JOIN_THRESHOLD("tajo.dist-query.join.broadcast.threshold-bytes", (long)5 * 1048576),
-
-    DIST_QUERY_JOIN_TASK_VOLUME("tajo.dist-query.join.task-volume-mb", 128),
-    DIST_QUERY_SORT_TASK_VOLUME("tajo.dist-query.sort.task-volume-mb", 128),
-    DIST_QUERY_GROUPBY_TASK_VOLUME("tajo.dist-query.groupby.task-volume-mb", 128),
-
-    DIST_QUERY_JOIN_PARTITION_VOLUME("tajo.dist-query.join.partition-volume-mb", 128),
-    DIST_QUERY_SORT_PARTITION_VOLUME("tajo.dist-query.sort.partition-volume-mb", 256),
-    DIST_QUERY_GROUPBY_PARTITION_VOLUME("tajo.dist-query.groupby.partition-volume-mb", 256),
-
-    DIST_QUERY_TABLE_PARTITION_VOLUME("tajo.dist-query.table-partition.task-volume-mb", 256),
-
-    //////////////////////////////////
-    // Physical Executors
-    //////////////////////////////////
-    EXECUTOR_EXTERNAL_SORT_THREAD_NUM("tajo.executor.external-sort.thread-num", 1),
-    EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE("tajo.executor.external-sort.buffer-mb", 200L),
-    EXECUTOR_EXTERNAL_SORT_FANOUT("tajo.executor.external-sort.fanout-num", 8),
-
-    EXECUTOR_INNER_JOIN_INMEMORY_HASH_TABLE_SIZE("tajo.executor.join.inner.in-memory-table-num", (long)1000000),
-    EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD("tajo.executor.join.inner.in-memory-hash-threshold-bytes",
-        (long)256 * 1048576),
-    EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD("tajo.executor.join.outer.in-memory-hash-threshold-bytes",
-        (long)256 * 1048576),
-    EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD("tajo.executor.groupby.in-memory-hash-threshold-bytes",
-        (long)256 * 1048576),
 
-    //////////////////////////////////
-    // RPC
-    //////////////////////////////////
+    // RPC --------------------------------------------------------------------
     RPC_POOL_MAX_IDLE("tajo.rpc.pool.idle.max", 10),
 
-    //Internal RPC Client
+    //  Internal RPC Client
     INTERNAL_RPC_CLIENT_WORKER_THREAD_NUM("tajo.internal.rpc.client.worker-thread-num",
         Runtime.getRuntime().availableProcessors() * 2),
 
-    //Internal RPC Server
+    // Internal RPC Server
     MASTER_RPC_SERVER_WORKER_THREAD_NUM("tajo.master.rpc.server.worker-thread-num",
         Runtime.getRuntime().availableProcessors() * 2),
     QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM("tajo.querymaster.rpc.server.worker-thread-num",
@@ -296,58 +262,92 @@ public class TajoConf extends Configuration {
     WORKER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM("tajo.worker.service.rpc.server.worker-thread-num",
         Runtime.getRuntime().availableProcessors() * 1),
 
-    //////////////////////////////////
-    // The Below is reserved
-    //////////////////////////////////
-
-    // GeoIP
-    GEOIP_DATA("tajo.function.geoip-database-location", ""),
-
-    //////////////////////////////////
-    // Task Configuration
+    // Task Configuration -----------------------------------------------------
     TASK_DEFAULT_MEMORY("tajo.task.memory-slot-mb.default", 512),
     TASK_DEFAULT_DISK("tajo.task.disk-slot.default", 0.5f),
     TASK_DEFAULT_SIZE("tajo.task.size-mb", 128),
-    //////////////////////////////////
 
-    //////////////////////////////////
-    // User Session Configuration
-    //////////////////////////////////
-    CLIENT_SESSION_EXPIRY_TIME("tajo.client.session.expiry-time-sec", 3600), // default time is one hour.
+    // Query and Optimization -------------------------------------------------
+    EXECUTOR_EXTERNAL_SORT_THREAD_NUM("tajo.executor.external-sort.thread-num", 1),
+    EXECUTOR_EXTERNAL_SORT_FANOUT("tajo.executor.external-sort.fanout-num", 8),
+
+    EXECUTOR_INNER_JOIN_INMEMORY_HASH_TABLE_SIZE("tajo.executor.join.inner.in-memory-table-num", (long)1000000),
 
-    // Metrics
+    // Metrics ----------------------------------------------------------------
     METRICS_PROPERTY_FILENAME("tajo.metrics.property.file", "tajo-metrics.properties"),
 
-    //CLI
-    CLI_MAX_COLUMN("tajo.cli.max_columns", 120),
-    CLI_PRINT_PAUSE_NUM_RECORDS("tajo.cli.print.pause.num.records", 100),
-    CLI_PRINT_PAUSE("tajo.cli.print.pause", true),
-    CLI_PRINT_ERROR_TRACE("tajo.cli.print.error.trace", true),
-    CLI_OUTPUT_FORMATTER_CLASS("tajo.cli.output.formatter", "org.apache.tajo.cli.DefaultTajoCliOutputFormatter"),
-    CLI_NULL_CHAR("tajo.cli.nullchar", ""),
-    CLI_ERROR_STOP("tajo.cli.error.stop", false),
+    // Misc -------------------------------------------------------------------
 
-    //TIME & DATE
-    TAJO_TIMEZONE("tajo.timezone", System.getProperty("user.timezone")),
-    TAJO_DATE_ORDER("tajo.date.order", "YMD"),
+    // Geo IP
+    GEOIP_DATA("tajo.function.geoip-database-location", ""),
 
-    //PLANNER
-    PLANNER_USE_FILTER_PUSHDOWN("tajo.planner.use.filter.pushdown", true),
+    /////////////////////////////////////////////////////////////////////////////////
+    // User Session Configuration
+    //
+    // All session variables begin with dollor($) sign. They are default configs
+    // for session variables. Do not directly use the following configs. Instead,
+    // please use QueryContext in order to access session variables.
+    //
+    // Also, users can change the default values of session variables in tajo-site.xml.
+    /////////////////////////////////////////////////////////////////////////////////
+
+
+    $EMPTY("tajo._", ""),
+
+    // Query and Optimization ---------------------------------------------------
+
+    // for distributed query strategies
+    $DIST_QUERY_BROADCAST_JOIN_THRESHOLD("tajo.dist-query.join.broadcast.threshold-bytes", (long)5 * 1048576),
+
+    $DIST_QUERY_JOIN_TASK_VOLUME("tajo.dist-query.join.task-volume-mb", 128),
+    $DIST_QUERY_SORT_TASK_VOLUME("tajo.dist-query.sort.task-volume-mb", 128),
+    $DIST_QUERY_GROUPBY_TASK_VOLUME("tajo.dist-query.groupby.task-volume-mb", 128),
+
+    $DIST_QUERY_JOIN_PARTITION_VOLUME("tajo.dist-query.join.partition-volume-mb", 128),
+    $DIST_QUERY_GROUPBY_PARTITION_VOLUME("tajo.dist-query.groupby.partition-volume-mb", 256),
+    $DIST_QUERY_TABLE_PARTITION_VOLUME("tajo.dist-query.table-partition.task-volume-mb", 256),
+
+    // for physical Executors
+    $EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE("tajo.executor.external-sort.buffer-mb", 200L),
+    $EXECUTOR_HASH_JOIN_SIZE_THRESHOLD("tajo.executor.join.common.in-memory-hash-threshold-bytes",
+        (long)256 * 1048576),
+    $EXECUTOR_INNER_HASH_JOIN_SIZE_THRESHOLD("tajo.executor.join.inner.in-memory-hash-threshold-bytes",
+        (long)256 * 1048576),
+    $EXECUTOR_OUTER_HASH_JOIN_SIZE_THRESHOLD("tajo.executor.join.outer.in-memory-hash-threshold-bytes",
+        (long)256 * 1048576),
+    $EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD("tajo.executor.groupby.in-memory-hash-threshold-bytes",
+        (long)256 * 1048576),
+    $MAX_OUTPUT_FILE_SIZE("tajo.query.max-outfile-size-mb", 0), // zero means infinite
 
-    // FILE FORMAT
-    CSVFILE_NULL("tajo.csvfile.null", "\\\\N"),
 
-    //OPTIMIZER
-    OPTIMIZER_JOIN_ENABLE("tajo.optimizer.join.enable", true),
+    // Client -----------------------------------------------------------------
+    $CLIENT_SESSION_EXPIRY_TIME("tajo.client.session.expiry-time-sec", 3600), // default time is one hour.
 
-    // DEBUG OPTION
-    TAJO_DEBUG("tajo.debug", false),
+    // Command line interface and its behavior --------------------------------
+    $CLI_MAX_COLUMN("tajo.cli.max_columns", 120),
+    $CLI_NULL_CHAR("tajo.cli.nullchar", ""),
+    $CLI_PRINT_PAUSE_NUM_RECORDS("tajo.cli.print.pause.num.records", 100),
+    $CLI_PRINT_PAUSE("tajo.cli.print.pause", true),
+    $CLI_PRINT_ERROR_TRACE("tajo.cli.print.error.trace", true),
+    $CLI_OUTPUT_FORMATTER_CLASS("tajo.cli.output.formatter", "org.apache.tajo.cli.DefaultTajoCliOutputFormatter"),
+    $CLI_ERROR_STOP("tajo.cli.error.stop", false),
 
-    // ONLY FOR TESTCASE
-    TESTCASE_MIN_TASK_NUM("tajo.testcase.min.task.num", -1),
+    // Timezone & Date ----------------------------------------------------------
+    $TIMEZONE("tajo.timezone", System.getProperty("user.timezone")),
+    $DATE_ORDER("tajo.date.order", "YMD"),
 
-    // behavior control
-    BEHAVIOR_ARITHMETIC_ABORT("tajo.behavior.arithmetic-abort", false);
+    // FILE FORMAT
+    $CSVFILE_NULL("tajo.csvfile.null", "\\\\N"),
+
+    // Only for Debug and Testing
+    $DEBUG_ENABLED("tajo.debug.enabled", 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),
+    $TEST_MIN_TASK_NUM("tajo.test.min-task-num", -1),
+
+    // Behavior Control ---------------------------------------------------------
+    $BEHAVIOR_ARITHMETIC_ABORT("tajo.behavior.arithmetic-abort", false);
     ;
 
     public final String varname;
@@ -429,6 +429,16 @@ public class TajoConf extends Configuration {
       String typeString() { return name().toUpperCase();}
       abstract void checkType(String value) throws Exception;
     }
+
+    @Override
+    public String keyname() {
+      return varname;
+    }
+
+    @Override
+    public ConfigType type() {
+      return ConfigType.SYSTEM;
+    }
   }
 
   public static int getIntVar(Configuration conf, ConfVars var) {
@@ -450,8 +460,12 @@ public class TajoConf extends Configuration {
   }
 
   public static long getLongVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Long.class);
-    return conf.getLong(var.varname, var.defaultLongVal);
+    assert (var.valClass == Long.class || var.valClass == Integer.class);
+    if (var.valClass == Integer.class) {
+      return conf.getInt(var.varname, var.defaultIntVal);
+    } else {
+      return conf.getLong(var.varname, var.defaultLongVal);
+    }
   }
 
   public static long getLongVar(Configuration conf, ConfVars var, long defaultVal) {
@@ -516,7 +530,6 @@ public class TajoConf extends Configuration {
   }
 
   public static String getVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == String.class);
     return conf.get(var.varname, var.defaultVal);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
index 19acafc..874004b 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
@@ -114,7 +114,7 @@ public abstract class Datum implements Comparable<Datum>, GsonObject {
   }
 
   protected static void initAbortWhenDivideByZero(TajoConf tajoConf) {
-    abortWhenDivideByZero = tajoConf.getBoolVar(ConfVars.BEHAVIOR_ARITHMETIC_ABORT);
+    abortWhenDivideByZero = tajoConf.getBoolVar(ConfVars.$BEHAVIOR_ARITHMETIC_ABORT);
   }
 
   public abstract int size();

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
index a4f79d7..532e7cd 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
@@ -25,6 +25,7 @@ import static org.apache.tajo.common.TajoDataTypes.Type;
 
 public class NullDatum extends Datum {
   private static NullDatum instance;
+  public static final String DEFAULT_TEXT = "";
   private static final byte [] EMPTY_BYTES = new byte[0];
   private static final DataType NULL_DATA_TYPE;
 


[4/5] TAJO-928: Session variables should override query configs in TajoConf.

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java b/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
index 4d1cee1..0c3db6d 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
@@ -19,10 +19,13 @@
 package org.apache.tajo.util;
 
 import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.OverridableConf;
 import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.json.CommonGsonHelper;
 import org.apache.tajo.json.GsonObject;
+import sun.misc.FloatingDecimal;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -32,6 +35,9 @@ import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto;
 import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
 
 public class KeyValueSet implements ProtoObject<KeyValueSetProto>, Cloneable, GsonObject {
+  public static final String TRUE_STR = "true";
+  public static final String FALSE_STR = "false";
+
 	private KeyValueSetProto.Builder builder = KeyValueSetProto.newBuilder();
 	
 	@Expose private Map<String,String> keyVals;
@@ -63,40 +69,133 @@ public class KeyValueSet implements ProtoObject<KeyValueSetProto>, Cloneable, Gs
   public int size() {
     return keyVals.size();
   }
-	
-	public void put(String key, String val) {
-		this.keyVals.put(key, val);
-	}
 
   public void putAll(Map<String, String> keyValues) {
     if (keyValues != null) {
       this.keyVals.putAll(keyValues);
     }
   }
-	
-	public void putAll(KeyValueSet keyValueSet) {
+
+  public void putAll(KeyValueSet keyValueSet) {
     if (keyValueSet != null) {
-	    this.keyVals.putAll(keyValueSet.keyVals);
+      this.keyVals.putAll(keyValueSet.keyVals);
     }
-	}
-	
-	public String get(String key) {
-		return this.keyVals.get(key);
-	}
-	
-	public String get(String key, String defaultVal) {
-	  if(keyVals.containsKey(key))
-	    return keyVals.get(key);
-	  else {
-	    return defaultVal;
-	  }
-	}
-	
-	public Map<String,String> getAllKeyValus() {
-	  return keyVals;
-	}
+  }
+
+  public Map<String,String> getAllKeyValus() {
+    return keyVals;
+  }
+
+  public boolean containsKey(String key) {
+    return this.keyVals.containsKey(key);
+  }
+
+  public void set(String key, String val) {
+    Preconditions.checkNotNull(key);
+    Preconditions.checkNotNull(val);
+
+    this.keyVals.put(key, val);
+  }
+
+  public String get(String key, String defaultVal) {
+    if(keyVals.containsKey(key)) {
+      return keyVals.get(key);
+    } else if (defaultVal != null) {
+      return defaultVal;
+    } else {
+      throw new IllegalArgumentException("No such a config key: "  + key);
+    }
+  }
+
+  public String get(String key) {
+    return get(key, null);
+  }
+
+  public void setBool(String key, boolean val) {
+    set(key, val ? TRUE_STR : FALSE_STR);
+  }
+
+  public boolean getBool(String key, Boolean defaultVal) {
+    if (containsKey(key)) {
+      String strVal = get(key, null);
+      return strVal != null ? strVal.equalsIgnoreCase(TRUE_STR) : false;
+    } else if (defaultVal != null) {
+      return defaultVal;
+    } else {
+      return false;
+    }
+  }
+
+  public boolean getBool(String key) {
+    return getBool(key, null);
+  }
+
+  public void setInt(String key, int val) {
+    set(key, String.valueOf(val));
+  }
+
+  public int getInt(String key, Integer defaultVal) {
+    if (containsKey(key)) {
+      String strVal = get(key, null);
+      return Integer.parseInt(strVal);
+    } else if (defaultVal != null) {
+      return defaultVal;
+    } else {
+      throw new IllegalArgumentException("No such a config key: "  + key);
+    }
+  }
+
+  public int getInt(String key) {
+    return getInt(key, null);
+  }
+
+  public void setLong(String key, long val) {
+    set(key, String.valueOf(val));
+  }
+
+  public long getLong(String key, Long defaultVal) {
+    if (containsKey(key)) {
+      String strVal = get(key, null);
+      return Long.parseLong(strVal);
+    } else if (defaultVal != null) {
+      return defaultVal;
+    } else {
+      throw new IllegalArgumentException("No such a config key: "  + key);
+    }
+  }
+
+  public long getLong(String key) {
+    return getLong(key, null);
+  }
+
+  public void setFloat(String key, float val) {
+    set(key, String.valueOf(val));
+  }
+
+  public float getFloat(String key, Float defaultVal) {
+    if (containsKey(key)) {
+      String strVal = get(key, null);
+      try {
+        sun.misc.FloatingDecimal fd = FloatingDecimal.readJavaFormatString(strVal);
+        if (Float.MAX_VALUE < fd.doubleValue()) {
+          throw new IllegalStateException("Parsed value is overflow in float type");
+        }
+        return fd.floatValue();
+      } catch (NumberFormatException nfe) {
+        throw new IllegalArgumentException("No such a config key: "  + key);
+      }
+    } else if (defaultVal != null) {
+      return defaultVal.floatValue();
+    } else {
+      throw new IllegalArgumentException("No such a config key: "  + key);
+    }
+  }
+
+  public float getFloat(String key) {
+    return getFloat(key, null);
+  }
 	
-	public String delete(String key) {
+	public String remove(String key) {
 		return keyVals.remove(key);
 	}
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-common/src/test/java/org/apache/tajo/datum/TestArithmeticOperator.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestArithmeticOperator.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestArithmeticOperator.java
index adf80b0..42623bd 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestArithmeticOperator.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestArithmeticOperator.java
@@ -52,9 +52,9 @@ public class TestArithmeticOperator {
   public void setUp() {
     TajoConf tajoConf = new TajoConf();
     if ("Zero_Exception".equals(option)) {
-      tajoConf.setBoolVar(ConfVars.BEHAVIOR_ARITHMETIC_ABORT, true);
+      tajoConf.setBoolVar(ConfVars.$BEHAVIOR_ARITHMETIC_ABORT, true);
     } else {
-      tajoConf.setBoolVar(ConfVars.BEHAVIOR_ARITHMETIC_ABORT, false);
+      tajoConf.setBoolVar(ConfVars.$BEHAVIOR_ARITHMETIC_ABORT, false);
     }
     Datum.initAbortWhenDivideByZero(tajoConf);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java
index 3ffeeb0..a43cc1a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalOptimizer.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.JoinType;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
@@ -35,7 +36,7 @@ import org.apache.tajo.engine.planner.logical.join.GreedyHeuristicJoinOrderAlgor
 import org.apache.tajo.engine.planner.logical.join.JoinGraph;
 import org.apache.tajo.engine.planner.logical.join.JoinOrderAlgorithm;
 import org.apache.tajo.engine.planner.rewrite.*;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.engine.query.QueryContext;
 
 import java.util.LinkedHashSet;
 import java.util.Set;
@@ -57,7 +58,7 @@ public class LogicalOptimizer {
 
   public LogicalOptimizer(TajoConf systemConf) {
     rulesBeforeJoinOpt = new BasicQueryRewriteEngine();
-    if (systemConf.getBoolVar(ConfVars.PLANNER_USE_FILTER_PUSHDOWN)) {
+    if (systemConf.getBoolVar(ConfVars.$TEST_FILTER_PUSHDOWN_ENABLED)) {
       rulesBeforeJoinOpt.addRewriteRule(new FilterPushDownRule());
     }
 
@@ -84,13 +85,13 @@ public class LogicalOptimizer {
     return optimize(null, plan);
   }
 
-  public LogicalNode optimize(Session session, LogicalPlan plan) throws PlanningException {
+  public LogicalNode optimize(QueryContext context, LogicalPlan plan) throws PlanningException {
     rulesBeforeJoinOpt.rewrite(plan);
 
     DirectedGraphCursor<String, BlockEdge> blockCursor =
         new DirectedGraphCursor<String, BlockEdge>(plan.getQueryBlockGraph(), plan.getRootBlock().getName());
 
-    if (session == null || "true".equals(session.getVariable(ConfVars.OPTIMIZER_JOIN_ENABLE.varname, "true"))) {
+    if (context == null || context.getBool(SessionVars.TEST_JOIN_OPT_ENABLED)) {
       // default is true
       while (blockCursor.hasNext()) {
         optimizeJoinOrder(plan, blockCursor.nextBlock());

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
index 86bacef..ee65b2b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
@@ -67,10 +67,8 @@ public class LogicalPlan {
   LogicalPlanner planner;
 
   private boolean isExplain;
-  private final String currentDatabase;
 
-  public LogicalPlan(String currentDatabase, LogicalPlanner planner) {
-    this.currentDatabase = currentDatabase;
+  public LogicalPlan(LogicalPlanner planner) {
     this.planner = planner;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
index 84fe6c2..6ee0ff8 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java
@@ -26,10 +26,10 @@ import org.apache.tajo.engine.eval.FieldEval;
 import org.apache.tajo.engine.exception.NoSuchColumnException;
 import org.apache.tajo.engine.planner.LogicalPlan.QueryBlock;
 import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.engine.planner.nameresolver.NameResolvingMode;
 import org.apache.tajo.engine.planner.nameresolver.NameResolver;
+import org.apache.tajo.engine.planner.nameresolver.NameResolvingMode;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.engine.utils.SchemaUtil;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.util.TUtil;
 
 import java.util.*;
@@ -42,18 +42,18 @@ public class LogicalPlanPreprocessor extends BaseAlgebraVisitor<LogicalPlanPrepr
   private ExprAnnotator annotator;
 
   public static class PreprocessContext {
-    public Session session;
+    public QueryContext queryContext;
     public LogicalPlan plan;
     public LogicalPlan.QueryBlock currentBlock;
 
-    public PreprocessContext(Session session, LogicalPlan plan, LogicalPlan.QueryBlock currentBlock) {
-      this.session = session;
+    public PreprocessContext(QueryContext queryContext, LogicalPlan plan, LogicalPlan.QueryBlock currentBlock) {
+      this.queryContext = queryContext;
       this.plan = plan;
       this.currentBlock = currentBlock;
     }
 
     public PreprocessContext(PreprocessContext context, LogicalPlan.QueryBlock currentBlock) {
-      this.session = context.session;
+      this.queryContext = context.queryContext;
       this.plan = context.plan;
       this.currentBlock = currentBlock;
     }
@@ -104,7 +104,7 @@ public class LogicalPlanPreprocessor extends BaseAlgebraVisitor<LogicalPlanPrepr
       if (CatalogUtil.isFQTableName(asteriskExpr.getQualifier())) {
         qualifier = asteriskExpr.getQualifier();
       } else {
-        qualifier = CatalogUtil.buildFQName(ctx.session.getCurrentDatabase(), asteriskExpr.getQualifier());
+        qualifier = CatalogUtil.buildFQName(ctx.queryContext.getCurrentDatabase(), asteriskExpr.getQualifier());
       }
 
       relationOp = block.getRelation(qualifier);
@@ -359,7 +359,7 @@ public class LogicalPlanPreprocessor extends BaseAlgebraVisitor<LogicalPlanPrepr
     if (CatalogUtil.isFQTableName(expr.getName())) {
       actualRelationName = relation.getName();
     } else {
-      actualRelationName = CatalogUtil.buildFQName(ctx.session.getCurrentDatabase(), relation.getName());
+      actualRelationName = CatalogUtil.buildFQName(ctx.queryContext.getCurrentDatabase(), relation.getName());
     }
 
     TableDesc desc = catalog.getTableDesc(actualRelationName);
@@ -388,7 +388,7 @@ public class LogicalPlanPreprocessor extends BaseAlgebraVisitor<LogicalPlanPrepr
 
     // a table subquery should be dealt as a relation.
     TableSubQueryNode node = ctx.plan.createNode(TableSubQueryNode.class);
-    node.init(CatalogUtil.buildFQName(ctx.session.getCurrentDatabase(), expr.getName()), child);
+    node.init(CatalogUtil.buildFQName(ctx.queryContext.getCurrentDatabase(), expr.getName()), child);
     ctx.currentBlock.addRelation(node);
     return node;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java
index bb8192f..6512ae0 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVerifier.java
@@ -24,7 +24,7 @@ import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.engine.query.QueryContext;
 
 import java.util.Stack;
 
@@ -38,17 +38,17 @@ public class LogicalPlanVerifier extends BasicLogicalPlanVisitor<LogicalPlanVeri
   }
 
   public static class Context {
-    Session session;
+    QueryContext queryContext;
     VerificationState state;
 
-    public Context(Session session, VerificationState state) {
-      this.session = session;
+    public Context(QueryContext queryContext, VerificationState state) {
+      this.queryContext = this.queryContext;
       this.state = state;
     }
   }
 
-  public VerificationState verify(Session session, VerificationState state, LogicalPlan plan) throws PlanningException {
-    Context context = new Context(session, state);
+  public VerificationState verify(QueryContext queryContext, VerificationState state, LogicalPlan plan) throws PlanningException {
+    Context context = new Context(queryContext, state);
     visit(context, plan, plan.getRootBlock());
     return context.state;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
index a4820cb..35df11f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
@@ -42,17 +42,16 @@ import org.apache.tajo.engine.planner.LogicalPlan.QueryBlock;
 import org.apache.tajo.engine.planner.logical.*;
 import org.apache.tajo.engine.planner.nameresolver.NameResolvingMode;
 import org.apache.tajo.engine.planner.rewrite.ProjectionPushDownRule;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.engine.utils.SchemaUtil;
-import org.apache.tajo.master.session.Session;
 import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TUtil;
 
 import java.util.*;
 
 import static org.apache.tajo.algebra.CreateTable.PartitionType;
-
 import static org.apache.tajo.engine.planner.ExprNormalizer.ExprNormalizedResult;
 import static org.apache.tajo.engine.planner.LogicalPlan.BlockType;
 import static org.apache.tajo.engine.planner.LogicalPlanPreprocessor.PreprocessContext;
@@ -75,7 +74,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
   }
 
   public static class PlanContext {
-    Session session;
+    QueryContext queryContext;
     LogicalPlan plan;
 
     // transient data for each query block
@@ -83,15 +82,15 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
 
     boolean debugOrUnitTests;
 
-    public PlanContext(Session session, LogicalPlan plan, QueryBlock block, boolean debugOrUnitTests) {
-      this.session = session;
+    public PlanContext(QueryContext context, LogicalPlan plan, QueryBlock block, boolean debugOrUnitTests) {
+      this.queryContext = context;
       this.plan = plan;
       this.queryBlock = block;
       this.debugOrUnitTests = debugOrUnitTests;
     }
 
     public PlanContext(PlanContext context, QueryBlock block) {
-      this.session = context.session;
+      this.queryContext = context.queryContext;
       this.plan = context.plan;
       this.queryBlock = block;
       this.debugOrUnitTests = context.debugOrUnitTests;
@@ -109,21 +108,21 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
    * @param expr A relational algebraic expression for a query.
    * @return A logical plan
    */
-  public LogicalPlan createPlan(Session session, Expr expr) throws PlanningException {
-    return createPlan(session, expr, false);
+  public LogicalPlan createPlan(QueryContext context, Expr expr) throws PlanningException {
+    return createPlan(context, expr, false);
   }
 
   @VisibleForTesting
-  public LogicalPlan createPlan(Session session, Expr expr, boolean debug) throws PlanningException {
+  public LogicalPlan createPlan(QueryContext queryContext, Expr expr, boolean debug) throws PlanningException {
 
-    LogicalPlan plan = new LogicalPlan(session.getCurrentDatabase(), this);
+    LogicalPlan plan = new LogicalPlan(this);
 
     QueryBlock rootBlock = plan.newAndGetBlock(LogicalPlan.ROOT_BLOCK);
-    PreprocessContext preProcessorCtx = new PreprocessContext(session, plan, rootBlock);
+    PreprocessContext preProcessorCtx = new PreprocessContext(queryContext, plan, rootBlock);
     preprocessor.visit(preProcessorCtx, new Stack<Expr>(), expr);
     plan.resetGeneratedId();
 
-    PlanContext context = new PlanContext(session, plan, plan.getRootBlock(), debug);
+    PlanContext context = new PlanContext(queryContext, plan, plan.getRootBlock(), debug);
     LogicalNode topMostNode = this.visit(context, new Stack<Expr>(), expr);
 
     // Add Root Node
@@ -1424,7 +1423,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       databaseName = CatalogUtil.extractQualifier(expr.getTableName());
       tableName = CatalogUtil.extractSimpleName(expr.getTableName());
     } else {
-      databaseName = context.session.getCurrentDatabase();
+      databaseName = context.queryContext.getCurrentDatabase();
       tableName = expr.getTableName();
     }
     TableDesc desc = catalog.getTableDesc(databaseName, tableName);
@@ -1624,7 +1623,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
 
     if (CatalogUtil.isFQTableName(parentTableName) == false) {
       parentTableName =
-	CatalogUtil.buildFQName(context.session.getCurrentDatabase(),
+	CatalogUtil.buildFQName(context.queryContext.getCurrentDatabase(),
 				parentTableName);
     }
     TableDesc parentTableDesc = catalog.getTableDesc(parentTableName);
@@ -1657,7 +1656,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       createTableNode.setTableName(expr.getTableName());
     } else {
       createTableNode.setTableName(
-          CatalogUtil.buildFQName(context.session.getCurrentDatabase(), expr.getTableName()));
+          CatalogUtil.buildFQName(context.queryContext.getCurrentDatabase(), expr.getTableName()));
     }
     // This is CREATE TABLE <tablename> LIKE <parentTable>
     if(expr.getLikeParentTableName() != null)
@@ -1753,7 +1752,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       CreateTable.ColumnPartition partition = (CreateTable.ColumnPartition) expr;
       String partitionExpression = Joiner.on(',').join(partition.getColumns());
 
-      partitionMethodDesc = new PartitionMethodDesc(context.session.getCurrentDatabase(), tableName,
+      partitionMethodDesc = new PartitionMethodDesc(context.queryContext.getCurrentDatabase(), tableName,
           CatalogProtos.PartitionType.COLUMN, partitionExpression, convertColumnsToSchema(partition.getColumns()));
     } else {
       throw new PlanningException(String.format("Not supported PartitonType: %s", expr.getPartitionType()));
@@ -1816,7 +1815,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     if (CatalogUtil.isFQTableName(dropTable.getTableName())) {
       qualified = dropTable.getTableName();
     } else {
-      qualified = CatalogUtil.buildFQName(context.session.getCurrentDatabase(), dropTable.getTableName());
+      qualified = CatalogUtil.buildFQName(context.queryContext.getCurrentDatabase(), dropTable.getTableName());
     }
     dropTableNode.init(qualified, dropTable.isIfExists(), dropTable.isPurge());
     return dropTableNode;

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
index 6678e46..9f533e2 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
@@ -28,6 +28,7 @@ import com.google.common.collect.ObjectArrays;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.SortSpec;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -37,6 +38,7 @@ import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.logical.*;
 import org.apache.tajo.engine.planner.physical.*;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer;
@@ -57,7 +59,6 @@ import java.util.Stack;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
 import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType;
-import static org.apache.tajo.conf.TajoConf.ConfVars;
 import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm;
 import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
 import static org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType;
@@ -68,7 +69,6 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.SortEnforce;
 public class PhysicalPlannerImpl implements PhysicalPlanner {
   private static final Log LOG = LogFactory.getLog(PhysicalPlannerImpl.class);
   private static final int UNGENERATED_PID = -1;
-  private final long INNER_JOIN_INMEMORY_HASH_THRESHOLD;
 
   protected final TajoConf conf;
   protected final AbstractStorageManager sm;
@@ -76,8 +76,6 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
   public PhysicalPlannerImpl(final TajoConf conf, final AbstractStorageManager sm) {
     this.conf = conf;
     this.sm = sm;
-
-    this.INNER_JOIN_INMEMORY_HASH_THRESHOLD = conf.getLongVar(ConfVars.EXECUTOR_INNER_JOIN_INMEMORY_HASH_THRESHOLD);
   }
 
   public PhysicalExec createPlan(final TaskAttemptContext context, final LogicalNode logicalPlan)
@@ -258,7 +256,16 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
       throws IOException {
     String [] lineage = PlannerUtil.getRelationLineage(node);
     long volume = estimateSizeRecursive(context, lineage);
-    boolean inMemoryInnerJoinFlag = volume <= INNER_JOIN_INMEMORY_HASH_THRESHOLD;
+    boolean inMemoryInnerJoinFlag = false;
+
+    QueryContext queryContext = context.getQueryContext();
+
+    if (queryContext.containsKey(SessionVars.INNER_HASH_JOIN_SIZE_LIMIT)) {
+      inMemoryInnerJoinFlag = volume <= context.getQueryContext().getLong(SessionVars.INNER_HASH_JOIN_SIZE_LIMIT);
+    } else {
+      inMemoryInnerJoinFlag = volume <= context.getQueryContext().getLong(SessionVars.HASH_JOIN_SIZE_LIMIT);
+    }
+
     LOG.info(String.format("[%s] the volume of %s relations (%s) is %s and is %sfit to main maemory.",
         context.getTaskId().toString(),
         (left ? "Left" : "Right"),
@@ -470,8 +477,17 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
                                                    PhysicalExec leftExec, PhysicalExec rightExec) throws IOException {
     String [] rightLineage = PlannerUtil.getRelationLineage(plan.getRightChild());
     long rightTableVolume = estimateSizeRecursive(context, rightLineage);
+    boolean hashJoin;
 
-    if (rightTableVolume < conf.getLongVar(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD)) {
+    QueryContext queryContext = context.getQueryContext();
+
+    if (queryContext.containsKey(SessionVars.OUTER_HASH_JOIN_SIZE_LIMIT)) {
+      hashJoin = rightTableVolume <  queryContext.getLong(SessionVars.OUTER_HASH_JOIN_SIZE_LIMIT);
+    } else {
+      hashJoin = rightTableVolume <  queryContext.getLong(SessionVars.HASH_JOIN_SIZE_LIMIT);
+    }
+
+    if (hashJoin) {
       // we can implement left outer join using hash join, using the right operand as the build relation
       LOG.info("Left Outer Join (" + plan.getPID() +") chooses [Hash Join].");
       return new HashLeftOuterJoinExec(context, plan, leftExec, rightExec);
@@ -488,8 +504,18 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
     //if the left operand is small enough => implement it as a left outer hash join with exchanged operators (note:
     // blocking, but merge join is blocking as well)
     String [] outerLineage4 = PlannerUtil.getRelationLineage(plan.getLeftChild());
-    long outerSize = estimateSizeRecursive(context, outerLineage4);
-    if (outerSize < conf.getLongVar(ConfVars.EXECUTOR_OUTER_JOIN_INMEMORY_HASH_THRESHOLD)){
+    long leftTableVolume = estimateSizeRecursive(context, outerLineage4);
+    boolean hashJoin;
+
+    QueryContext queryContext = context.getQueryContext();
+
+    if (queryContext.containsKey(SessionVars.OUTER_HASH_JOIN_SIZE_LIMIT)) {
+      hashJoin = leftTableVolume <  queryContext.getLong(SessionVars.OUTER_HASH_JOIN_SIZE_LIMIT);
+    } else {
+      hashJoin = leftTableVolume <  queryContext.getLong(SessionVars.HASH_JOIN_SIZE_LIMIT);
+    }
+
+    if (hashJoin){
       LOG.info("Right Outer Join (" + plan.getPID() +") chooses [Hash Join].");
       return new HashLeftOuterJoinExec(context, plan, rightExec, leftExec);
     } else {
@@ -971,7 +997,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
 
     String [] outerLineage = PlannerUtil.getRelationLineage(groupbyNode.getChild());
     long estimatedSize = estimateSizeRecursive(context, outerLineage);
-    final long threshold = conf.getLongVar(ConfVars.EXECUTOR_GROUPBY_INMEMORY_HASH_THRESHOLD);
+    final long threshold = context.getQueryContext().getLong(SessionVars.HASH_GROUPBY_SIZE_LIMIT);
 
     // if the relation size is less than the threshold,
     // the hash aggregation will be used.

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java
index 2d6c095..75dcc18 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PreLogicalPlanVerifier.java
@@ -24,7 +24,7 @@ import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.util.TUtil;
 
 import java.util.Set;
@@ -38,17 +38,17 @@ public class PreLogicalPlanVerifier extends BaseAlgebraVisitor <PreLogicalPlanVe
   }
 
   public static class Context {
-    Session session;
+    QueryContext queryContext;
     VerificationState state;
 
-    public Context(Session session, VerificationState state) {
-      this.session = session;
+    public Context(QueryContext queryContext, VerificationState state) {
+      this.queryContext = queryContext;
       this.state = state;
     }
   }
 
-  public VerificationState verify(Session session, VerificationState state, Expr expr) throws PlanningException {
-    Context context = new Context(session, state);
+  public VerificationState verify(QueryContext queryContext, VerificationState state, Expr expr) throws PlanningException {
+    Context context = new Context(queryContext, state);
     visit(context, new Stack<Expr>(), expr);
     return context.state;
   }
@@ -127,7 +127,7 @@ public class PreLogicalPlanVerifier extends BaseAlgebraVisitor <PreLogicalPlanVe
     if (CatalogUtil.isFQTableName(tableName)) {
       qualifiedName = tableName;
     } else {
-      qualifiedName = CatalogUtil.buildFQName(context.session.getCurrentDatabase(), tableName);
+      qualifiedName = CatalogUtil.buildFQName(context.queryContext.getCurrentDatabase(), tableName);
     }
 
     if (!catalog.existsTable(qualifiedName)) {
@@ -143,7 +143,10 @@ public class PreLogicalPlanVerifier extends BaseAlgebraVisitor <PreLogicalPlanVe
     if (CatalogUtil.isFQTableName(tableName)) {
       qualifiedName = tableName;
     } else {
-      qualifiedName = CatalogUtil.buildFQName(context.session.getCurrentDatabase(), tableName);
+      qualifiedName = CatalogUtil.buildFQName(context.queryContext.getCurrentDatabase(), tableName);
+    }
+    if(qualifiedName == null) {
+      System.out.println("A");
     }
     if (catalog.existsTable(qualifiedName)) {
       context.state.addVerification(String.format("relation \"%s\" already exists", qualifiedName));
@@ -246,7 +249,7 @@ public class PreLogicalPlanVerifier extends BaseAlgebraVisitor <PreLogicalPlanVe
         if (expr.hasTableName()) {
           String qualifiedName = expr.getTableName();
           if (TajoConstants.EMPTY_STRING.equals(CatalogUtil.extractQualifier(expr.getTableName()))) {
-            qualifiedName = CatalogUtil.buildFQName(context.session.getCurrentDatabase(),
+            qualifiedName = CatalogUtil.buildFQName(context.queryContext.getCurrentDatabase(),
                 expr.getTableName());
           }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
index 2daf799..432589b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.JoinType;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
@@ -120,9 +121,8 @@ public class GlobalPlanner {
     LogicalNode inputPlan = PlannerUtil.clone(masterPlan.getLogicalPlan(),
         masterPlan.getLogicalPlan().getRootBlock().getRoot());
 
-    boolean autoBroadcast = conf.getBoolVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO);
-    if (autoBroadcast) {
-
+    boolean broadcastEnabled = masterPlan.getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED);
+    if (broadcastEnabled) {
       // pre-visit the master plan in order to find tables to be broadcasted
       // this visiting does not make any execution block and change plan.
       BroadcastJoinMarkCandidateVisitor markCandidateVisitor = new BroadcastJoinMarkCandidateVisitor();
@@ -268,11 +268,11 @@ public class GlobalPlanner {
     MasterPlan masterPlan = context.plan;
     ExecutionBlock currentBlock;
 
-    boolean autoBroadcast = conf.getBoolVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO);
-    long broadcastThreshold = conf.getLongVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD);
+    boolean broadcastEnabled = context.getPlan().getContext().getBool(SessionVars.TEST_BROADCAST_JOIN_ENABLED);
+    long broadcastTableSizeLimit = context.getPlan().getContext().getLong(SessionVars.BROADCAST_TABLE_SIZE_LIMIT);
 
     // to check when the tajo.dist-query.join.broadcast.auto property is true
-    if (autoBroadcast && joinNode.isCandidateBroadcast()) {
+    if (broadcastEnabled && joinNode.isCandidateBroadcast()) {
       LogicalNode leftNode = joinNode.getLeftChild();
       LogicalNode rightNode = joinNode.getRightChild();
 
@@ -293,7 +293,7 @@ public class GlobalPlanner {
       // Checking Left Side of Join
       if (ScanNode.isScanNode(leftNode)) {
         ScanNode scanNode = (ScanNode)leftNode;
-        if (joinNode.getJoinType() == JoinType.LEFT_OUTER || getTableVolume(scanNode) >= broadcastThreshold) {
+        if (joinNode.getJoinType() == JoinType.LEFT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) {
           numLargeTables++;
         } else {
           leftBroadcast = true;
@@ -306,7 +306,7 @@ public class GlobalPlanner {
       // Checking Right Side OF Join
       if (ScanNode.isScanNode(rightNode)) {
         ScanNode scanNode = (ScanNode)rightNode;
-        if (joinNode.getJoinType() == JoinType.RIGHT_OUTER || getTableVolume(scanNode) >= broadcastThreshold) {
+        if (joinNode.getJoinType() == JoinType.RIGHT_OUTER || getTableVolume(scanNode) >= broadcastTableSizeLimit) {
           numLargeTables++;
         } else {
           rightBroadcast = true;
@@ -331,7 +331,7 @@ public class GlobalPlanner {
         }
         JoinNode broadcastJoinNode = (JoinNode)eachNode;
         ScanNode scanNode = broadcastJoinNode.getRightChild();
-        if (getTableVolume(scanNode) < broadcastThreshold) {
+        if (getTableVolume(scanNode) < broadcastTableSizeLimit) {
           broadcastTargetScanNodes.add(scanNode);
           blockJoinNode = broadcastJoinNode;
           LOG.info("The table " + scanNode.getCanonicalName() + " ("

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
index f714758..31cb3b6 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
@@ -38,6 +39,7 @@ import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.Scanner;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.worker.TaskAttemptContext;
@@ -111,7 +113,7 @@ public class ExternalSortExec extends SortExec {
       throw new PhysicalPlanningException(ConfVars.EXECUTOR_EXTERNAL_SORT_FANOUT.varname + " cannot be lower than 2");
     }
     // TODO - sort buffer and core num should be changed to use the allocated container resource.
-    this.sortBufferBytesNum = context.getConf().getLongVar(ConfVars.EXECUTOR_EXTERNAL_SORT_BUFFER_SIZE) * 1048576L;
+    this.sortBufferBytesNum = context.getQueryContext().getLong(SessionVars.EXTSORT_BUFFER_SIZE) * StorageUnit.MB;
     this.allocatedCoreNum = context.getConf().getIntVar(ConfVars.EXECUTOR_EXTERNAL_SORT_THREAD_NUM);
     this.executorService = Executors.newFixedThreadPool(this.allocatedCoreNum);
     this.inMemoryTable = new ArrayList<Tuple>(100000);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
index b1d0400..e73cc2f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
@@ -18,9 +18,9 @@
 
 package org.apache.tajo.engine.planner.physical;
 
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.planner.logical.InsertNode;
 import org.apache.tajo.engine.planner.logical.PersistentStoreNode;
 import org.apache.tajo.storage.Appender;
@@ -59,7 +59,7 @@ public class StoreTableExec extends UnaryPhysicalExec {
       appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta,
           createTableNode.getTableSchema(), context.getOutputPath());
     } else {
-      String nullChar = context.getQueryContext().get(ConfVars.CSVFILE_NULL.varname, ConfVars.CSVFILE_NULL.defaultVal);
+      String nullChar = context.getQueryContext().get(SessionVars.NULL_CHAR);
       meta.putOption(StorageConstants.CSVFILE_NULL, nullChar);
       appender = StorageManagerFactory.getStorageManager(context.getConf()).getAppender(meta, outSchema,
           context.getOutputPath());
@@ -77,7 +77,7 @@ public class StoreTableExec extends UnaryPhysicalExec {
     while((tuple = child.next()) != null) {
       appender.addTuple(tuple);
     }
-        
+
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
index 79d6cb3..f4160e4 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
@@ -19,94 +19,90 @@
 package org.apache.tajo.engine.query;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.ConfigKey;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.master.session.Session;
 
 import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
 
-public class QueryContext extends KeyValueSet {
-  public static final String COMMAND_TYPE = "tajo.query.command";
-
-  public static final String STAGING_DIR = "tajo.query.staging_dir";
-
-  public static final String USER_NAME = "tajo.query.username";
-
-  public static final String OUTPUT_TABLE_NAME = "tajo.query.output.table";
-  public static final String OUTPUT_TABLE_PATH = "tajo.query.output.path";
-  public static final String OUTPUT_PARTITIONS = "tajo.query.output.partitions";
-  public static final String OUTPUT_OVERWRITE = "tajo.query.output.overwrite";
-  public static final String OUTPUT_AS_DIRECTORY = "tajo.query.output.asdirectory";
+/**
+ * QueryContent is a overridable config, and it provides a set of various configs for a query instance.
+ */
+public class QueryContext extends OverridableConf {
+  public static enum QueryVars implements ConfigKey {
+    COMMAND_TYPE,
+    STAGING_DIR,
+    OUTPUT_TABLE_NAME,
+    OUTPUT_TABLE_PATH,
+    OUTPUT_PARTITIONS,
+    OUTPUT_OVERWRITE,
+    OUTPUT_AS_DIRECTORY,
+    OUTPUT_PER_FILE_SIZE,
+    ;
 
-  public static final String TRUE_VALUE = "1";
-  public static final String FALSE_VALUE = "0";
+    QueryVars() {
+    }
 
-  public QueryContext() {}
+    @Override
+    public String keyname() {
+      return name().toLowerCase();
+    }
 
-  public QueryContext(KeyValueSetProto proto) {
-    super(proto);
+    @Override
+    public ConfigType type() {
+      return ConfigType.QUERY;
+    }
   }
 
-  public void put(TajoConf.ConfVars key, String value) {
-    put(key.varname, value);
+  public QueryContext(TajoConf conf) {
+    super(conf, ConfigKey.ConfigType.QUERY);
   }
 
-  public String get(TajoConf.ConfVars key) {
-    return get(key.varname);
+  public QueryContext(TajoConf conf, Session session) {
+    super(conf);
+    putAll(session.getAllVariables());
   }
 
-  public String get(String key) {
-    return super.get(key);
+  public QueryContext(TajoConf conf, KeyValueSetProto proto) {
+    super(conf, proto, ConfigKey.ConfigType.QUERY);
   }
 
-  public void setBool(String key, boolean val) {
-    put(key, val ? TRUE_VALUE : FALSE_VALUE);
-  }
+  //-----------------------------------------------------------------------------------------------
+  // Query Config Specified Section
+  //-----------------------------------------------------------------------------------------------
 
-  public boolean getBool(String key) {
-    String strVal = get(key);
-    return strVal != null ? strVal.equals(TRUE_VALUE) : false;
+  public String getCurrentDatabase() {
+    return get(SessionVars.CURRENT_DATABASE);
   }
 
   public void setUser(String username) {
-    put(USER_NAME, username);
+    put(SessionVars.USERNAME, username);
   }
 
   public String getUser() {
-    return get(USER_NAME);
+    return get(SessionVars.USERNAME);
   }
 
   public void setStagingDir(Path path) {
-    put(STAGING_DIR, path.toUri().toString());
+    put(QueryVars.STAGING_DIR, path.toUri().toString());
   }
 
   public Path getStagingDir() {
-    String strVal = get(STAGING_DIR);
+    String strVal = get(QueryVars.STAGING_DIR);
     return strVal != null ? new Path(strVal) : null;
   }
 
   /**
-   * The fact that QueryContext has an output table means this query has a target table.
-   * In other words, this query is 'CREATE TABLE' or 'INSERT (OVERWRITE) INTO <table name>' statement.
-   * This config is not set if a query has INSERT (OVERWRITE) INTO LOCATION '/path/..'.
-   */
-  public boolean hasOutputTable() {
-    return get(OUTPUT_TABLE_NAME) != null;
-  }
-
-  /**
    * Set a target table name
    *
    * @param tableName The target table name
    */
   public void setOutputTable(String tableName) {
-    put(OUTPUT_TABLE_NAME, tableName);
-  }
-
-  public String getOutputTable() {
-    String strVal = get(OUTPUT_TABLE_NAME);
-    return strVal != null ? strVal : null;
+    put(QueryVars.OUTPUT_TABLE_NAME, tableName);
   }
 
   /**
@@ -116,52 +112,64 @@ public class QueryContext extends KeyValueSet {
    * @return
    */
   public boolean hasOutputPath() {
-    return get(OUTPUT_TABLE_PATH) != null;
+    return containsKey(QueryVars.OUTPUT_TABLE_PATH);
   }
 
   public void setOutputPath(Path path) {
-    put(OUTPUT_TABLE_PATH, path.toUri().toString());
+    put(QueryVars.OUTPUT_TABLE_PATH, path.toUri().toString());
   }
 
   public Path getOutputPath() {
-    String strVal = get(OUTPUT_TABLE_PATH);
+    String strVal = get(QueryVars.OUTPUT_TABLE_PATH);
     return strVal != null ? new Path(strVal) : null;
   }
 
   public boolean hasPartition() {
-    return get(OUTPUT_PARTITIONS) != null;
+    return containsKey(QueryVars.OUTPUT_PARTITIONS);
   }
 
   public void setPartitionMethod(PartitionMethodDesc partitionMethodDesc) {
-    put(OUTPUT_PARTITIONS, partitionMethodDesc != null ? partitionMethodDesc.toJson() : null);
+    put(QueryVars.OUTPUT_PARTITIONS, partitionMethodDesc != null ? partitionMethodDesc.toJson() : null);
   }
 
   public PartitionMethodDesc getPartitionMethod() {
-    return PartitionMethodDesc.fromJson(get(OUTPUT_PARTITIONS));
+    return PartitionMethodDesc.fromJson(get(QueryVars.OUTPUT_PARTITIONS));
   }
 
   public void setOutputOverwrite() {
-    setBool(OUTPUT_OVERWRITE, true);
+    setBool(QueryVars.OUTPUT_OVERWRITE, true);
   }
 
   public boolean isOutputOverwrite() {
-    return getBool(OUTPUT_OVERWRITE);
+    return getBool(QueryVars.OUTPUT_OVERWRITE);
   }
 
   public void setFileOutput() {
-    setBool(OUTPUT_AS_DIRECTORY, true);
+    setBool(QueryVars.OUTPUT_AS_DIRECTORY, true);
+  }
+
+  public boolean containsKey(ConfigKey key) {
+    return containsKey(key.keyname());
+  }
+
+  public boolean equalKey(ConfigKey key, String another) {
+    if (containsKey(key)) {
+      return get(key).equals(another);
+    } else {
+      return false;
+    }
   }
 
-  public boolean isFileOutput() {
-    return getBool(OUTPUT_AS_DIRECTORY);
+  public boolean isCommandType(NodeType commandType) {
+    return equalKey(QueryVars.COMMAND_TYPE, commandType.name());
   }
 
   public void setCommandType(NodeType nodeType) {
-    put(COMMAND_TYPE, nodeType.name());
+    put(QueryVars.COMMAND_TYPE, nodeType.name());
   }
 
   public NodeType getCommandType() {
-    String strVal = get(COMMAND_TYPE);
+    String strVal = get(QueryVars.COMMAND_TYPE);
     return strVal != null ? NodeType.valueOf(strVal) : null;
   }
 
@@ -170,7 +178,7 @@ public class QueryContext extends KeyValueSet {
   }
 
   public boolean isCreateTable() {
-    return getCommandType() == NodeType.CREATE_TABLE;
+    return isCommandType(NodeType.CREATE_TABLE);
   }
 
   public void setInsert() {
@@ -178,14 +186,6 @@ public class QueryContext extends KeyValueSet {
   }
 
   public boolean isInsert() {
-    return getCommandType() == NodeType.INSERT;
-  }
-
-  public void setHiveQueryMode() {
-    setBool("hive.query.mode", true);
-  }
-
-  public boolean isHiveQueryMode() {
-    return getBool("hive.query.mode");
+    return isCommandType(NodeType.INSERT);
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
index f1af2ff..56df48d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.engine.query;
 
 import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
@@ -193,7 +194,7 @@ public class QueryUnitRequestImpl implements QueryUnitRequest {
     if (!p.hasQueryContext()) {
       return null;
     }
-    this.queryContext = new QueryContext(p.getQueryContext());
+    this.queryContext = new QueryContext(new TajoConf(), p.getQueryContext());
     return this.queryContext;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/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 73f3cf5..37a56ba 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.hadoop.util.StringUtils;
 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.AlterTablespaceSetType;
 import org.apache.tajo.algebra.Expr;
@@ -116,40 +117,25 @@ public class GlobalEngine extends AbstractService {
 
   public SubmitQueryResponse executeQuery(Session session, String query, boolean isJson) {
     LOG.info("Query: " + query);
-    QueryContext queryContext = new QueryContext();
-    queryContext.putAll(session.getAllVariables());
+    QueryContext queryContext = new QueryContext(context.getConf(), session);
     Expr planningContext;
 
     try {
       if (isJson) {
         planningContext = buildExpressionFromJson(query);
       } else {
-        // setting environment variables
-        String [] cmds = query.split(" ");
-        if(cmds != null) {
-          if(cmds[0].equalsIgnoreCase("set")) {
-            String[] params = cmds[1].split("=");
-            context.getConf().set(params[0], params[1]);
-            SubmitQueryResponse.Builder responseBuilder = SubmitQueryResponse.newBuilder();
-            responseBuilder.setUserName(context.getConf().getVar(TajoConf.ConfVars.USERNAME));
-            responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
-            responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
-            return responseBuilder.build();
-          }
-        }
-
         planningContext = buildExpressionFromSql(queryContext, query);
       }
 
       String jsonExpr = planningContext.toJson();
-      LogicalPlan plan = createLogicalPlan(session, planningContext);
+      LogicalPlan plan = createLogicalPlan(queryContext, planningContext);
       SubmitQueryResponse response = executeQueryInternal(queryContext, session, plan, query, jsonExpr);
       return response;
     } catch (Throwable t) {
       context.getSystemMetrics().counter("Query", "errorQuery").inc();
       LOG.error("\nStack Trace:\n" + StringUtils.stringifyException(t));
       SubmitQueryResponse.Builder responseBuilder = SubmitQueryResponse.newBuilder();
-      responseBuilder.setUserName(context.getConf().getVar(TajoConf.ConfVars.USERNAME));
+      responseBuilder.setUserName(queryContext.get(SessionVars.USERNAME));
       responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
       responseBuilder.setIsForwarded(true);
       responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
@@ -183,11 +169,11 @@ public class GlobalEngine extends AbstractService {
 
     SubmitQueryResponse.Builder responseBuilder = SubmitQueryResponse.newBuilder();
     responseBuilder.setIsForwarded(false);
-    responseBuilder.setUserName(context.getConf().getVar(TajoConf.ConfVars.USERNAME));
+    responseBuilder.setUserName(queryContext.get(SessionVars.USERNAME));
 
     if (PlannerUtil.checkIfDDLPlan(rootNode)) {
       context.getSystemMetrics().counter("Query", "numDDLQuery").inc();
-      updateQuery(session, rootNode.getChild());
+      updateQuery(queryContext, rootNode.getChild());
       responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
       responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
 
@@ -310,7 +296,7 @@ public class GlobalEngine extends AbstractService {
     }
 
     TaskAttemptContext taskAttemptContext =
-        new TaskAttemptContext(context.getConf(), queryContext, null, (CatalogProtos.FragmentProto[]) null, stagingDir);
+        new TaskAttemptContext(queryContext, null, (CatalogProtos.FragmentProto[]) null, stagingDir);
     taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
 
     EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
@@ -390,7 +376,7 @@ public class GlobalEngine extends AbstractService {
   }
 
 
-  public QueryId updateQuery(Session session, String sql, boolean isJson) throws IOException, SQLException, PlanningException {
+  public QueryId updateQuery(QueryContext queryContext, String sql, boolean isJson) throws IOException, SQLException, PlanningException {
     try {
       LOG.info("SQL: " + sql);
 
@@ -402,13 +388,13 @@ public class GlobalEngine extends AbstractService {
         expr = analyzer.parse(sql);
       }
 
-      LogicalPlan plan = createLogicalPlan(session, expr);
+      LogicalPlan plan = createLogicalPlan(queryContext, expr);
       LogicalRootNode rootNode = plan.getRootBlock().getRoot();
 
       if (!PlannerUtil.checkIfDDLPlan(rootNode)) {
         throw new SQLException("This is not update query:\n" + sql);
       } else {
-        updateQuery(session, rootNode.getChild());
+        updateQuery(queryContext, rootNode.getChild());
         return QueryIdFactory.NULL_QUERY_ID;
       }
     } catch (Exception e) {
@@ -417,46 +403,46 @@ public class GlobalEngine extends AbstractService {
     }
   }
 
-  private boolean updateQuery(Session session, LogicalNode root) throws IOException {
+  private boolean updateQuery(QueryContext queryContext, LogicalNode root) throws IOException {
 
     switch (root.getType()) {
       case CREATE_DATABASE:
         CreateDatabaseNode createDatabase = (CreateDatabaseNode) root;
-        createDatabase(session, createDatabase.getDatabaseName(), null, createDatabase.isIfNotExists());
+        createDatabase(queryContext, createDatabase.getDatabaseName(), null, createDatabase.isIfNotExists());
         return true;
       case DROP_DATABASE:
         DropDatabaseNode dropDatabaseNode = (DropDatabaseNode) root;
-        dropDatabase(session, dropDatabaseNode.getDatabaseName(), dropDatabaseNode.isIfExists());
+        dropDatabase(queryContext, dropDatabaseNode.getDatabaseName(), dropDatabaseNode.isIfExists());
         return true;
       case CREATE_TABLE:
         CreateTableNode createTable = (CreateTableNode) root;
-        createTable(session, createTable, createTable.isIfNotExists());
+        createTable(queryContext, createTable, createTable.isIfNotExists());
         return true;
       case DROP_TABLE:
         DropTableNode dropTable = (DropTableNode) root;
-        dropTable(session, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge());
+        dropTable(queryContext, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge());
         return true;
       case ALTER_TABLESPACE:
         AlterTablespaceNode alterTablespace = (AlterTablespaceNode) root;
-        alterTablespace(session, alterTablespace);
+        alterTablespace(queryContext, alterTablespace);
         return true;
       case ALTER_TABLE:
         AlterTableNode alterTable = (AlterTableNode) root;
-        alterTable(session,alterTable);
+        alterTable(queryContext,alterTable);
         return true;
       case TRUNCATE_TABLE:
         TruncateTableNode truncateTable = (TruncateTableNode) root;
-        truncateTable(session, truncateTable);
+        truncateTable(queryContext, truncateTable);
         return true;
       default:
         throw new InternalError("updateQuery cannot handle such query: \n" + root.toJson());
     }
   }
 
-  private LogicalPlan createLogicalPlan(Session session, Expr expression) throws PlanningException {
+  private LogicalPlan createLogicalPlan(QueryContext queryContext, Expr expression) throws PlanningException {
 
     VerificationState state = new VerificationState();
-    preVerifier.verify(session, state, expression);
+    preVerifier.verify(queryContext, state, expression);
     if (!state.verified()) {
       StringBuilder sb = new StringBuilder();
       for (String error : state.getErrorMessages()) {
@@ -465,19 +451,19 @@ public class GlobalEngine extends AbstractService {
       throw new VerifyException(sb.toString());
     }
 
-    LogicalPlan plan = planner.createPlan(session, expression);
+    LogicalPlan plan = planner.createPlan(queryContext, expression);
     if (LOG.isDebugEnabled()) {
       LOG.debug("=============================================");
       LOG.debug("Non Optimized Query: \n" + plan.toString());
       LOG.debug("=============================================");
     }
     LOG.info("Non Optimized Query: \n" + plan.toString());
-    optimizer.optimize(session, plan);
+    optimizer.optimize(queryContext, plan);
     LOG.info("=============================================");
     LOG.info("Optimized Query: \n" + plan.toString());
     LOG.info("=============================================");
 
-    annotatedPlanVerifier.verify(session, state, plan);
+    annotatedPlanVerifier.verify(queryContext, state, plan);
 
     if (!state.verified()) {
       StringBuilder sb = new StringBuilder();
@@ -493,7 +479,7 @@ public class GlobalEngine extends AbstractService {
   /**
    * Alter a given table
    */
-  public void alterTablespace(final Session session, final AlterTablespaceNode alterTablespace) {
+  public void alterTablespace(final QueryContext queryContext, final AlterTablespaceNode alterTablespace) {
 
     final CatalogService catalog = context.getCatalog();
     final String spaceName = alterTablespace.getTablespaceName();
@@ -517,7 +503,7 @@ public class GlobalEngine extends AbstractService {
   /**
    * Alter a given table
    */
-  public void alterTable(final Session session, final AlterTableNode alterTable) throws IOException {
+  public void alterTable(final QueryContext queryContext, final AlterTableNode alterTable) throws IOException {
 
     final CatalogService catalog = context.getCatalog();
     final String tableName = alterTable.getTableName();
@@ -529,7 +515,7 @@ public class GlobalEngine extends AbstractService {
       databaseName = split[0];
       simpleTableName = split[1];
     } else {
-      databaseName = session.getCurrentDatabase();
+      databaseName = queryContext.getCurrentDatabase();
       simpleTableName = tableName;
     }
     final String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
@@ -572,7 +558,8 @@ public class GlobalEngine extends AbstractService {
         if (existColumnName(qualifiedName, alterTable.getNewColumnName())) {
           throw new ColumnNameAlreadyExistException(alterTable.getNewColumnName());
         }
-        catalog.alterTable(CatalogUtil.renameColumn(qualifiedName, alterTable.getColumnName(), alterTable.getNewColumnName(), AlterTableType.RENAME_COLUMN));
+        catalog.alterTable(CatalogUtil.renameColumn(qualifiedName, alterTable.getColumnName(),
+            alterTable.getNewColumnName(), AlterTableType.RENAME_COLUMN));
         break;
       case ADD_COLUMN:
         if (existColumnName(qualifiedName, alterTable.getAddNewColumn().getSimpleName())) {
@@ -588,7 +575,8 @@ public class GlobalEngine extends AbstractService {
   /**
    * Truncate table a given table
    */
-  public void truncateTable(final Session session, final TruncateTableNode truncateTableNode) throws IOException {
+  public void truncateTable(final QueryContext queryContext, final TruncateTableNode truncateTableNode)
+      throws IOException {
     List<String> tableNames = truncateTableNode.getTableNames();
     final CatalogService catalog = context.getCatalog();
 
@@ -602,7 +590,7 @@ public class GlobalEngine extends AbstractService {
         databaseName = split[0];
         simpleTableName = split[1];
       } else {
-        databaseName = session.getCurrentDatabase();
+        databaseName = queryContext.getCurrentDatabase();
         simpleTableName = eachTableName;
       }
       final String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
@@ -641,7 +629,7 @@ public class GlobalEngine extends AbstractService {
     return tableDesc.getSchema().containsByName(columnName) ? true : false;
   }
 
-  private TableDesc createTable(Session session, CreateTableNode createTable, boolean ifNotExists) throws IOException {
+  private TableDesc createTable(QueryContext queryContext, CreateTableNode createTable, boolean ifNotExists) throws IOException {
     TableMeta meta;
 
     if (createTable.hasOptions()) {
@@ -659,7 +647,7 @@ public class GlobalEngine extends AbstractService {
         databaseName = CatalogUtil.extractQualifier(createTable.getTableName());
         tableName = CatalogUtil.extractSimpleName(createTable.getTableName());
       } else {
-        databaseName = session.getCurrentDatabase();
+        databaseName = queryContext.getCurrentDatabase();
         tableName = createTable.getTableName();
       }
 
@@ -668,11 +656,11 @@ public class GlobalEngine extends AbstractService {
       createTable.setPath(tablePath);
     }
 
-    return createTableOnPath(session, createTable.getTableName(), createTable.getTableSchema(),
+    return createTableOnPath(queryContext, createTable.getTableName(), createTable.getTableSchema(),
         meta, createTable.getPath(), createTable.isExternal(), createTable.getPartitionMethod(), ifNotExists);
   }
 
-  public TableDesc createTableOnPath(Session session, String tableName, Schema schema, TableMeta meta,
+  public TableDesc createTableOnPath(QueryContext queryContext, String tableName, Schema schema, TableMeta meta,
                                      Path path, boolean isExternal, PartitionMethodDesc partitionDesc,
                                      boolean ifNotExists)
       throws IOException {
@@ -683,7 +671,7 @@ public class GlobalEngine extends AbstractService {
       databaseName = splitted[0];
       simpleTableName = splitted[1];
     } else {
-      databaseName = session.getCurrentDatabase();
+      databaseName = queryContext.getCurrentDatabase();
       simpleTableName = tableName;
     }
     String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);
@@ -736,7 +724,7 @@ public class GlobalEngine extends AbstractService {
     }
   }
 
-  public boolean createDatabase(@Nullable Session session, String databaseName,
+  public boolean createDatabase(@Nullable QueryContext queryContext, String databaseName,
                                 @Nullable String tablespace,
                                 boolean ifNotExists) throws IOException {
 
@@ -768,7 +756,7 @@ public class GlobalEngine extends AbstractService {
     return true;
   }
 
-  public boolean dropDatabase(Session session, String databaseName, boolean ifExists) {
+  public boolean dropDatabase(QueryContext queryContext, String databaseName, boolean ifExists) {
 
     boolean exists = catalog.existDatabase(databaseName);
     if(!exists) {
@@ -780,7 +768,7 @@ public class GlobalEngine extends AbstractService {
       }
     }
 
-    if (session.getCurrentDatabase().equals(databaseName)) {
+    if (queryContext.getCurrentDatabase().equals(databaseName)) {
       throw new RuntimeException("ERROR: Cannot drop the current open database");
     }
 
@@ -795,7 +783,7 @@ public class GlobalEngine extends AbstractService {
    * @param tableName to be dropped
    * @param purge Remove all data if purge is true.
    */
-  public boolean dropTable(Session session, String tableName, boolean ifExists, boolean purge) {
+  public boolean dropTable(QueryContext queryContext, String tableName, boolean ifExists, boolean purge) {
     CatalogService catalog = context.getCatalog();
 
     String databaseName;
@@ -805,7 +793,7 @@ public class GlobalEngine extends AbstractService {
       databaseName = splitted[0];
       simpleTableName = splitted[1];
     } else {
-      databaseName = session.getCurrentDatabase();
+      databaseName = queryContext.getCurrentDatabase();
       simpleTableName = tableName;
     }
     String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index 97f59ef..7d80a88 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -34,6 +34,7 @@ import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.*;
 import org.apache.tajo.ipc.TajoMasterClientProtocol;
@@ -193,7 +194,8 @@ public class TajoMasterClientService extends AbstractService {
     }
 
     @Override
-    public BoolProto existSessionVariable(RpcController controller, SessionedStringProto request) throws ServiceException {
+    public BoolProto existSessionVariable(RpcController controller, SessionedStringProto request)
+        throws ServiceException {
       try {
         String value = context.getSessionManager().getVariable(request.getSessionId().getId(), request.getValue());
         if (value != null) {
@@ -278,9 +280,16 @@ public class TajoMasterClientService extends AbstractService {
 
       try {
         Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        QueryContext queryContext = new QueryContext(conf, session);
+        if (queryContext.getCurrentDatabase() == null) {
+          for (Map.Entry<String,String> e : queryContext.getAllKeyValus().entrySet()) {
+            System.out.println(e.getKey() + "=" + e.getValue());
+          }
+        }
+
         UpdateQueryResponse.Builder builder = UpdateQueryResponse.newBuilder();
         try {
-          context.getGlobalEngine().updateQuery(session, request.getQuery(), request.getIsJson());
+          context.getGlobalEngine().updateQuery(queryContext, request.getQuery(), request.getIsJson());
           builder.setResultCode(ResultCode.OK);
           return builder.build();
         } catch (Exception e) {
@@ -539,7 +548,9 @@ public class TajoMasterClientService extends AbstractService {
     public BoolProto createDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
       try {
         Session session = context.getSessionManager().getSession(request.getSessionId().getId());
-        if (context.getGlobalEngine().createDatabase(session, request.getValue(), null, false)) {
+        QueryContext queryContext = new QueryContext(conf, session);
+
+        if (context.getGlobalEngine().createDatabase(queryContext, request.getValue(), null, false)) {
           return BOOL_TRUE;
         } else {
           return BOOL_FALSE;
@@ -567,8 +578,9 @@ public class TajoMasterClientService extends AbstractService {
     public BoolProto dropDatabase(RpcController controller, SessionedStringProto request) throws ServiceException {
       try {
         Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        QueryContext queryContext = new QueryContext(conf, session);
 
-        if (context.getGlobalEngine().dropDatabase(session, request.getValue(), false)) {
+        if (context.getGlobalEngine().dropDatabase(queryContext, request.getValue(), false)) {
           return BOOL_TRUE;
         } else {
           return BOOL_FALSE;
@@ -605,6 +617,10 @@ public class TajoMasterClientService extends AbstractService {
           tableName = request.getValue();
         }
 
+        if (databaseName == null) {
+          System.out.println("A");
+        }
+
         if (catalog.existsTable(databaseName, tableName)) {
           return BOOL_TRUE;
         } else {
@@ -672,6 +688,7 @@ public class TajoMasterClientService extends AbstractService {
         throws ServiceException {
       try {
         Session session = context.getSessionManager().getSession(request.getSessionId().getId());
+        QueryContext queryContext = new QueryContext(conf, session);
 
         Path path = new Path(request.getPath());
         FileSystem fs = path.getFileSystem(conf);
@@ -689,7 +706,7 @@ public class TajoMasterClientService extends AbstractService {
 
         TableDesc desc;
         try {
-          desc = context.getGlobalEngine().createTableOnPath(session, request.getName(), schema,
+          desc = context.getGlobalEngine().createTableOnPath(queryContext, request.getName(), schema,
               meta, path, true, partitionDesc, false);
         } catch (Exception e) {
           return TableResponse.newBuilder()
@@ -715,7 +732,9 @@ public class TajoMasterClientService extends AbstractService {
     public BoolProto dropTable(RpcController controller, DropTableRequest dropTable) throws ServiceException {
       try {
         Session session = context.getSessionManager().getSession(dropTable.getSessionId().getId());
-        context.getGlobalEngine().dropTable(session, dropTable.getName(), false, dropTable.getPurge());
+        QueryContext queryContext = new QueryContext(conf, session);
+
+        context.getGlobalEngine().dropTable(queryContext, dropTable.getName(), false, dropTable.getPurge());
         return BOOL_TRUE;
       } catch (Throwable t) {
         throw new ServiceException(t);

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 8bb3dde..8111ef6 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.state.*;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.CatalogService;
@@ -38,7 +39,6 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.global.ExecutionBlock;
 import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
@@ -636,7 +636,7 @@ public class Query implements EventHandler<QueryEvent> {
         SubQuery lastStage = query.getSubQuery(finalExecBlockId);
         TableMeta meta = lastStage.getTableMeta();
 
-        String nullChar = queryContext.get(ConfVars.CSVFILE_NULL.varname, ConfVars.CSVFILE_NULL.defaultVal);
+        String nullChar = queryContext.get(SessionVars.NULL_CHAR);
         meta.putOption(StorageConstants.CSVFILE_NULL, nullChar);
 
         TableStats stats = lastStage.getResultStats();

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
index 25af82f..aed69b2 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -29,10 +29,12 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.ipc.TajoMasterProtocol;
 import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.master.TajoAsyncDispatcher;
@@ -83,6 +85,8 @@ public class QueryMaster extends CompositeService implements EventHandler {
 
   private QueryMasterContext queryMasterContext;
 
+  private QueryContext queryContext;
+
   private QueryHeartbeatThread queryHeartbeatThread;
 
   private FinishedQueryMasterTaskCleanThread finishedQueryMasterTaskCleanThread;
@@ -362,12 +366,9 @@ public class QueryMaster extends CompositeService implements EventHandler {
 
         try {
           queryMasterTask.stop();
-          //if (!systemConf.get(CommonTestingUtil.TAJO_TEST, "FALSE").equalsIgnoreCase("TRUE")
-         //     && !workerContext.isYarnContainerMode()) {
-          if (!getContext().getConf().getBoolVar(TajoConf.ConfVars.TAJO_DEBUG)) {
+          if (!queryContext.getBool(SessionVars.DEBUG_ENABLED)) {
             cleanup(queryId);
           }
-          //}
         } catch (Exception e) {
           LOG.error(e.getMessage(), e);
         }
@@ -408,6 +409,8 @@ public class QueryMaster extends CompositeService implements EventHandler {
         queryMasterTask.start();
       }
 
+      queryContext = event.getQueryContext();
+
       synchronized(queryMasterTasks) {
         queryMasterTasks.put(event.getQueryId(), queryMasterTask);
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
index f52d143..ec975d8 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
@@ -204,7 +204,6 @@ public class QueryMasterManagerService extends CompositeService
     try {
       QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
           new QueryId(report.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
-      // queryMaster terminated by internal error before task has not done
       if (queryMasterTask != null) {
         queryMasterTask.getEventHandler().handle(new TaskCompletionEvent(report));
       }
@@ -239,7 +238,8 @@ public class QueryMasterManagerService extends CompositeService
       LOG.info("Receive executeQuery request:" + queryId);
       queryMaster.handle(new QueryStartEvent(queryId,
           new Session(request.getSession()),
-          new QueryContext(request.getQueryContext()), request.getExprInJson().getValue(),
+          new QueryContext(workerContext.getQueryMaster().getContext().getConf(),
+              request.getQueryContext()), request.getExprInJson().getValue(),
           request.getLogicalPlanJson().getValue()));
       done.run(TajoWorker.TRUE_PROTO);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index 071e5d4..5885a1d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -328,8 +328,8 @@ public class QueryMasterTask extends CompositeService {
       LogicalPlanner planner = new LogicalPlanner(catalog);
       LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
       Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
-      LogicalPlan plan = planner.createPlan(session, expr);
-      optimizer.optimize(session, plan);
+      LogicalPlan plan = planner.createPlan(queryContext, expr);
+      optimizer.optimize(queryContext, plan);
 
       GlobalEngine.DistributedQueryHookManager hookManager = new GlobalEngine.DistributedQueryHookManager();
       hookManager.addHook(new GlobalEngine.InsertHook());

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
index fa1ed4c..940170c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -25,12 +25,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.JoinType;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.StatisticsUtil;
 import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.planner.PlannerUtil;
 import org.apache.tajo.engine.planner.PlanningException;
 import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
@@ -50,6 +50,7 @@ import org.apache.tajo.storage.RowStoreUtil;
 import org.apache.tajo.storage.TupleRange;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.Pair;
+import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.util.TajoIdUtils;
 import org.apache.tajo.worker.FetchImpl;
@@ -373,8 +374,7 @@ public class Repartitioner {
     // Getting the desire number of join tasks according to the volumn
     // of a larger table
     int largerIdx = stats[0] >= stats[1] ? 0 : 1;
-    int desireJoinTaskVolumn = subQuery.getContext().getConf().
-        getIntVar(ConfVars.DIST_QUERY_JOIN_TASK_VOLUME);
+    int desireJoinTaskVolumn = subQuery.getMasterPlan().getContext().getInt(SessionVars.JOIN_TASK_INPUT_SIZE);
 
     // calculate the number of tasks according to the data size
     int mb = (int) Math.ceil((double) stats[largerIdx] / 1048576);
@@ -858,17 +858,17 @@ public class Repartitioner {
 
   // Scattered hash shuffle hashes the key columns and groups the hash keys associated with
   // the same hash key. Then, if the volume of a group is larger
-  // than DIST_QUERY_TABLE_PARTITION_VOLUME, it divides the group into more than two sub groups
-  // according to DIST_QUERY_TABLE_PARTITION_VOLUME (default size = 256MB).
+  // than $DIST_QUERY_TABLE_PARTITION_VOLUME, it divides the group into more than two sub groups
+  // according to $DIST_QUERY_TABLE_PARTITION_VOLUME (default size = 256MB).
   // As a result, each group size always becomes the less than or equal
-  // to DIST_QUERY_TABLE_PARTITION_VOLUME. Finally, each subgroup is assigned to a query unit.
+  // to $DIST_QUERY_TABLE_PARTITION_VOLUME. Finally, each subgroup is assigned to a query unit.
   // It is usually used for writing partitioned tables.
   public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext schedulerContext,
        SubQuery subQuery, Map<ExecutionBlockId, List<IntermediateEntry>> intermediates,
        String tableName) {
     int i = 0;
-    long splitVolume = ((long) 1048576) * subQuery.getContext().getConf().
-        getIntVar(ConfVars.DIST_QUERY_TABLE_PARTITION_VOLUME); // in bytes
+    long splitVolume = StorageUnit.MB *
+        subQuery.getMasterPlan().getContext().getLong(SessionVars.TABLE_PARTITION_PER_SHUFFLE_SIZE);
 
     long sumNumBytes = 0L;
     Map<Integer, List<FetchImpl>> fetches = new HashMap<Integer, List<FetchImpl>>();

http://git-wip-us.apache.org/repos/asf/tajo/blob/ddfc3f33/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 17efa21..b6fe9da 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -32,10 +32,7 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.*;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.*;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
@@ -742,7 +739,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
         LOG.info(subQuery.getId() + ", Bigger Table's volume is approximately " + mb + " MB");
 
         int taskNum = (int) Math.ceil((double) mb /
-            conf.getIntVar(ConfVars.DIST_QUERY_JOIN_PARTITION_VOLUME));
+            conf.getIntVar(ConfVars.$DIST_QUERY_JOIN_PARTITION_VOLUME));
 
         int totalMem = getClusterTotalMemory(subQuery);
         LOG.info(subQuery.getId() + ", Total memory of cluster is " + totalMem + " MB");
@@ -750,8 +747,8 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
         // determine the number of task
         taskNum = Math.min(taskNum, slots);
 
-        if (conf.getIntVar(ConfVars.TESTCASE_MIN_TASK_NUM) > 0) {
-          taskNum = conf.getIntVar(ConfVars.TESTCASE_MIN_TASK_NUM);
+        if (conf.getIntVar(ConfVars.$TEST_MIN_TASK_NUM) > 0) {
+          taskNum = conf.getIntVar(ConfVars.$TEST_MIN_TASK_NUM);
           LOG.warn("!!!!! TESTCASE MODE !!!!!");
         }
 
@@ -795,7 +792,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
           LOG.info(subQuery.getId() + ", Table's volume is approximately " + mb + " MB");
           // determine the number of task
           int taskNumBySize = (int) Math.ceil((double) mb /
-              conf.getIntVar(ConfVars.DIST_QUERY_GROUPBY_PARTITION_VOLUME));
+              conf.getIntVar(ConfVars.$DIST_QUERY_GROUPBY_PARTITION_VOLUME));
 
           int totalMem = getClusterTotalMemory(subQuery);
 
@@ -1110,7 +1107,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
     stopScheduler();
     releaseContainers();
 
-    if (!getContext().getConf().getBoolVar(TajoConf.ConfVars.TAJO_DEBUG)) {
+    if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) {
       List<ExecutionBlock> childs = getMasterPlan().getChilds(getId());
       List<TajoIdProtos.ExecutionBlockIdProto> ebIds = Lists.newArrayList();
       for (ExecutionBlock executionBlock :  childs){