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/03/25 02:36:24 UTC

[09/13] TAJO-353: Add Database support to Tajo. (hyunsik)

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalogConstants.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalogConstants.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalogConstants.java
deleted file mode 100644
index f7b5c19..0000000
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalogConstants.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tajo.catalog;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.store.DerbyStore;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-import java.io.File;
-
-public class TestCatalogConstants extends TestDBStore {
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    conf = new TajoConf();
-    Path testDir = CommonTestingUtil.getTestDir("target/test-data/TestDBSTore");
-    File absolutePath = new File(testDir.toUri());
-    conf.set(CatalogConstants.DEPRECATED_CATALOG_URI, "jdbc:derby:"+absolutePath.getAbsolutePath()+"/db;create=true");
-    LOG.info("derby repository is set to "+conf.get(CatalogConstants.DEPRECATED_CATALOG_URI));
-    store = new DerbyStore(conf);
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    store.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java
deleted file mode 100644
index a6b5b9a..0000000
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestDBStore.java
+++ /dev/null
@@ -1,403 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.catalog;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.partition.PartitionMethodDesc;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.catalog.store.AbstractDBStore;
-import org.apache.tajo.catalog.store.DerbyStore;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-
-import static org.junit.Assert.*;
-
-public class TestDBStore {
-  protected static final Log LOG = LogFactory.getLog(TestDBStore.class);
-  protected static Configuration conf;
-  protected static AbstractDBStore store;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    conf = new TajoConf();
-    Path testDir = CommonTestingUtil.getTestDir("target/test-data/TestDBSTore");
-    File absolutePath = new File(testDir.toUri());
-    conf.set(CatalogConstants.CATALOG_URI, "jdbc:derby:"+absolutePath.getAbsolutePath()+"/db;create=true");
-    LOG.info("derby repository is set to "+conf.get(CatalogConstants.CATALOG_URI));
-    store = new DerbyStore(conf);
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    store.close();
-  }
-
-  @Test
-  public final void testAddAndDeleteTable() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4)
-    .addColumn("name", Type.TEXT)
-    .addColumn("age", Type.INT4)
-    .addColumn("score", Type.FLOAT8);
-    
-    String tableName = "addedtable";
-    Options opts = new Options();
-    opts.put("file.delimiter", ",");
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
-    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
-    assertFalse(store.existTable(tableName));
-    store.addTable(desc.getProto());
-    assertTrue(store.existTable(tableName));
-
-    TableDesc retrieved = new TableDesc(store.getTable(tableName));
-    // Schema order check
-    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
-    store.deleteTable(tableName);
-    assertFalse(store.existTable(tableName));
-  }
-  
-  @Test
-  public final void testGetTable() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("gettable.id", Type.INT4)
-    .addColumn("gettable.name", Type.TEXT)
-    .addColumn("gettable.age", Type.INT4)
-    .addColumn("gettable.score", Type.FLOAT8);
-    
-    String tableName = "gettable";
-    Options opts = new Options();
-    opts.put("file.delimiter", ",");
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
-
-    TableStats stat = new TableStats();
-    stat.setNumRows(957685);
-    stat.setNumBytes(1023234);
-
-    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "gettable"));
-    desc.setStats(stat);
-
-    store.addTable(desc.getProto());
-    TableDesc retrieved = new TableDesc(store.getTable(tableName));
-    assertEquals(",", retrieved.getMeta().getOption("file.delimiter"));
-    assertEquals(desc, retrieved);
-    assertTrue(957685 == desc.getStats().getNumRows());
-    assertTrue(1023234 == desc.getStats().getNumBytes());
-    // Schema order check
-    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
-    store.deleteTable(tableName);
-  }
-  
-  @Test
-  public final void testGetAllTableNames() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4)
-    .addColumn("name", Type.TEXT)
-    .addColumn("age", Type.INT4)
-    .addColumn("score", Type.FLOAT8);
-    
-    int numTables = 5;
-    for (int i = 0; i < numTables; i++) {
-      String tableName = "tableA_" + i;
-      TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
-      TableDesc desc = new TableDesc(tableName, schema, meta,
-          new Path(CommonTestingUtil.getTestDir(), "tableA_" + i));
-      store.addTable(desc.getProto());
-    }
-    
-    assertEquals(numTables, store.getAllTableNames().size());
-  }  
-  
-  @Test
-  public final void testAddAndDeleteIndex() throws Exception {
-    TableDesc table = prepareTable();
-    store.addTable(table.getProto());
-    
-    store.addIndex(TestCatalog.desc1.getProto());
-    assertTrue(store.existIndex(TestCatalog.desc1.getName()));
-    store.delIndex(TestCatalog.desc1.getName());
-    assertFalse(store.existIndex(TestCatalog.desc1.getName()));
-    
-    store.deleteTable(table.getName());
-  }
-  
-  @Test
-  public final void testGetIndex() throws Exception {
-    
-    TableDesc table = prepareTable();
-    store.addTable(table.getProto());
-    
-    store.addIndex(TestCatalog.desc2.getProto());
-    assertEquals(
-        new IndexDesc(TestCatalog.desc2.getProto()),
-        new IndexDesc(store.getIndex(TestCatalog.desc2.getName())));
-    store.delIndex(TestCatalog.desc2.getName());
-    
-    store.deleteTable(table.getName());
-  }
-  
-  @Test
-  public final void testGetIndexByTableAndColumn() throws Exception {
-    
-    TableDesc table = prepareTable();
-    store.addTable(table.getProto());
-    
-    store.addIndex(TestCatalog.desc2.getProto());
-    
-    String tableId = TestCatalog.desc2.getTableId();
-    String columnName = "score";
-    assertEquals(
-        new IndexDesc(TestCatalog.desc2.getProto()),
-        new IndexDesc(store.getIndex(tableId, columnName)));
-    store.delIndex(TestCatalog.desc2.getName());
-    
-    store.deleteTable(table.getName());
-  }
-  
-  @Test
-  public final void testGetAllIndexes() throws Exception {
-    
-    TableDesc table = prepareTable();
-    store.addTable(table.getProto());
-    
-    store.addIndex(TestCatalog.desc1.getProto());
-    store.addIndex(TestCatalog.desc2.getProto());
-        
-    assertEquals(2, 
-        store.getIndexes(TestCatalog.desc2.getTableId()).length);
-    store.delIndex(TestCatalog.desc1.getName());
-    store.delIndex(TestCatalog.desc2.getName());
-    
-    store.deleteTable(table.getName());
-  }
-  
-  public static TableDesc prepareTable() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("indexed.id", Type.INT4)
-    .addColumn("indexed.name", Type.TEXT)
-    .addColumn("indexed.age", Type.INT4)
-    .addColumn("indexed.score", Type.FLOAT8);
-    
-    String tableName = "indexed";
-    
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
-    return new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "indexed"));
-  }
-
-  public static void assertSchemaOrder(Schema s1, Schema s2) {
-    // Schema order check
-    assertEquals(s1.size(),
-        s2.size());
-
-    for (int i = 0; i < s1.size(); i++) {
-      assertEquals(s1.getColumn(i).getSimpleName(),
-          s2.getColumn(i).getSimpleName());
-    }
-  }
-
-  @Test
-  public final void testAddAndDeleteTablePartitionByHash1() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4)
-        .addColumn("name", Type.TEXT)
-        .addColumn("age", Type.INT4)
-        .addColumn("score", Type.FLOAT8);
-
-    String tableName = "addedtable";
-    Options opts = new Options();
-    opts.put("file.delimiter", ",");
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
-
-    PartitionMethodDesc partitionDesc = new PartitionMethodDesc();
-    partitionDesc.setTableId(tableName);
-    partitionDesc.setExpression("id");
-    Schema partSchema = new Schema();
-    partSchema.addColumn("id", Type.INT4);
-    partitionDesc.setExpressionSchema(partSchema);
-    partitionDesc.setPartitionType(CatalogProtos.PartitionType.HASH);
-
-    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
-    desc.setPartitionMethod(partitionDesc);
-    assertFalse(store.existTable(tableName));
-    store.addTable(desc.getProto());
-    assertTrue(store.existTable(tableName));
-
-    TableDesc retrieved = new TableDesc(store.getTable(tableName));
-
-    // Schema order check
-    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
-    store.deleteTable(tableName);
-    assertFalse(store.existTable(tableName));
-  }
-
-  @Test
-  public final void testAddAndDeleteTablePartitionByHash2() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4)
-        .addColumn("name", Type.TEXT)
-        .addColumn("age", Type.INT4)
-        .addColumn("score", Type.FLOAT8);
-
-    String tableName = "addedtable";
-    Options opts = new Options();
-    opts.put("file.delimiter", ",");
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
-
-
-    PartitionMethodDesc partitionDesc = new PartitionMethodDesc();
-    partitionDesc.setTableId(tableName);
-    partitionDesc.setExpression("id");
-    Schema partSchema = new Schema();
-    partSchema.addColumn("id", Type.INT4);
-    partitionDesc.setExpressionSchema(partSchema);
-    partitionDesc.setPartitionType(CatalogProtos.PartitionType.HASH);
-
-    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
-    desc.setPartitionMethod(partitionDesc);
-    assertFalse(store.existTable(tableName));
-    store.addTable(desc.getProto());
-    assertTrue(store.existTable(tableName));
-
-    TableDesc retrieved = new TableDesc(store.getTable(tableName));
-
-    // Schema order check
-    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
-    store.deleteTable(tableName);
-    assertFalse(store.existTable(tableName));
-  }
-
-  @Test
-  public final void testAddAndDeleteTablePartitionByList() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4)
-        .addColumn("name", Type.TEXT)
-        .addColumn("age", Type.INT4)
-        .addColumn("score", Type.FLOAT8);
-
-    String tableName = "addedtable";
-    Options opts = new Options();
-    opts.put("file.delimiter", ",");
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
-
-    PartitionMethodDesc partitionDesc = new PartitionMethodDesc();
-    partitionDesc.setTableId(tableName);
-    partitionDesc.setExpression("id");
-    Schema partSchema = new Schema();
-    partSchema.addColumn("id", Type.INT4);
-    partitionDesc.setExpressionSchema(partSchema);
-    partitionDesc.setPartitionType(CatalogProtos.PartitionType.LIST);
-
-    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
-    desc.setPartitionMethod(partitionDesc);
-    assertFalse(store.existTable(tableName));
-    store.addTable(desc.getProto());
-    assertTrue(store.existTable(tableName));
-
-    TableDesc retrieved = new TableDesc(store.getTable(tableName));
-
-    // Schema order check
-    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
-    store.deleteTable(tableName);
-    assertFalse(store.existTable(tableName));
-  }
-
-  @Test
-  public final void testAddAndDeleteTablePartitionByRange() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4)
-        .addColumn("name", Type.TEXT)
-        .addColumn("age", Type.INT4)
-        .addColumn("score", Type.FLOAT8);
-
-    String tableName = "addedtable";
-    Options opts = new Options();
-    opts.put("file.delimiter", ",");
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
-
-    PartitionMethodDesc partitionDesc = new PartitionMethodDesc();
-    partitionDesc.setTableId(tableName);
-    partitionDesc.setExpression("id");
-    Schema partSchema = new Schema();
-    partSchema.addColumn("id", Type.INT4);
-    partitionDesc.setExpressionSchema(partSchema);
-    partitionDesc.setPartitionType(CatalogProtos.PartitionType.RANGE);
-
-    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
-    desc.setPartitionMethod(partitionDesc);
-    assertFalse(store.existTable(tableName));
-    store.addTable(desc.getProto());
-    assertTrue(store.existTable(tableName));
-
-    TableDesc retrieved = new TableDesc(store.getTable(tableName));
-
-    // Schema order check
-    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
-    store.deleteTable(tableName);
-    assertFalse(store.existTable(tableName));
-  }
-
-  @Test
-  public final void testAddAndDeleteTablePartitionByColumn() throws Exception {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4)
-        .addColumn("name", Type.TEXT)
-        .addColumn("age", Type.INT4)
-        .addColumn("score", Type.FLOAT8);
-
-    String tableName = "addedtable";
-    Options opts = new Options();
-    opts.put("file.delimiter", ",");
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV, opts);
-
-    PartitionMethodDesc partitionDesc = new PartitionMethodDesc();
-    partitionDesc.setTableId(tableName);
-    partitionDesc.setExpression("id");
-    Schema partSchema = new Schema();
-    partSchema.addColumn("id", Type.INT4);
-    partitionDesc.setExpressionSchema(partSchema);
-    partitionDesc.setPartitionType(CatalogProtos.PartitionType.COLUMN);
-
-    TableDesc desc = new TableDesc(tableName, schema, meta, new Path(CommonTestingUtil.getTestDir(), "addedtable"));
-    desc.setPartitionMethod(partitionDesc);
-    assertFalse(store.existTable(tableName));
-    store.addTable(desc.getProto());
-    assertTrue(store.existTable(tableName));
-
-    TableDesc retrieved = new TableDesc(store.getTable(tableName));
-
-    // Schema order check
-    assertSchemaOrder(desc.getSchema(), retrieved.getSchema());
-    store.deleteTable(tableName);
-    assertFalse(store.existTable(tableName));
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java
new file mode 100644
index 0000000..02f195a
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/ConnectDatabaseCommand.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import com.google.protobuf.ServiceException;
+
+public class ConnectDatabaseCommand extends TajoShellCommand {
+
+  public ConnectDatabaseCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\c";
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    if (cmd.length == 1) {
+      context.getOutput().write(String.format("You are now connected to database \"%s\" as user \"%s\".%n",
+          client.getCurrentDatabase(), client.getUserInfo().getUserName()));
+    } else if (cmd.length == 2) {
+      try {
+        if (client.selectDatabase(cmd[1])) {
+          context.setCurrentDatabase(client.getCurrentDatabase());
+          context.getOutput().write(String.format("You are now connected to database \"%s\" as user \"%s\".%n",
+              context.getCurrentDatabase(), client.getUserInfo().getUserName()));
+        }
+      } catch (ServiceException se) {
+        if (se.getMessage() != null) {
+          context.getOutput().write(se.getMessage());
+        } else {
+          context.getOutput().write(String.format("cannot connect the database \"%s\"", cmd[1]));
+        }
+      }
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "connect to new database";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/CopyrightCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/CopyrightCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/CopyrightCommand.java
index 809cd37..07a3b51 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/CopyrightCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/CopyrightCommand.java
@@ -18,13 +18,9 @@
 
 package org.apache.tajo.cli;
 
-import org.apache.tajo.client.TajoClient;
-
-import java.io.PrintWriter;
-
 public class CopyrightCommand extends TajoShellCommand {
-  public CopyrightCommand(TajoClient client, PrintWriter sout) {
-    super(client, sout);
+  public CopyrightCommand(TajoCli.TajoCliContext context) {
+    super(context);
   }
 
   @Override
@@ -34,8 +30,8 @@ public class CopyrightCommand extends TajoShellCommand {
 
   @Override
   public void invoke(String[] cmd) throws Exception {
-    sout.println();
-    sout.println(
+    context.getOutput().println();
+    context.getOutput().println(
         "  Licensed to the Apache Software Foundation (ASF) under one\n" +
             "  or more contributor license agreements.  See the NOTICE file\n" +
             "  distributed with this work for additional information\n" +
@@ -51,7 +47,7 @@ public class CopyrightCommand extends TajoShellCommand {
             "   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" +
             "   See the License for the specific language governing permissions and\n" +
             "   limitations under the License.");
-    sout.println();
+    context.getOutput().println();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/DescFunctionCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/DescFunctionCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/DescFunctionCommand.java
index e9f4964..c20840b 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/DescFunctionCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/DescFunctionCommand.java
@@ -21,14 +21,12 @@ package org.apache.tajo.cli;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.FunctionDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.client.TajoClient;
 
-import java.io.PrintWriter;
 import java.util.*;
 
 public class DescFunctionCommand extends TajoShellCommand {
-  public DescFunctionCommand(TajoClient client, PrintWriter sout) {
-    super(client, sout);
+  public DescFunctionCommand(TajoCli.TajoCliContext context) {
+    super(context);
   }
 
   @Override
@@ -77,20 +75,20 @@ public class DescFunctionCommand extends TajoShellCommand {
 
       int index = 0;
       printLeft(" " + name, columnWidths[index++]);
-      sout.print("|");
+      context.getOutput().print("|");
       printLeft(" " + resultDataType, columnWidths[index++]);
-      sout.print("|");
+      context.getOutput().print("|");
       printLeft(" " + arguments, columnWidths[index++]);
-      sout.print("|");
+      context.getOutput().print("|");
       printLeft(" " + description, columnWidths[index++]);
-      sout.print("|");
+      context.getOutput().print("|");
       printLeft(" " + functionType, columnWidths[index++]);
 
       println();
     }
 
     println();
-    sout.println("(" + functions.size() + ") rows");
+    context.getOutput().println("(" + functions.size() + ") rows");
     println();
 
     if (printDetail && !functions.isEmpty()) {
@@ -105,15 +103,15 @@ public class DescFunctionCommand extends TajoShellCommand {
 
       for (CatalogProtos.FunctionDescProto eachFunction: functionMap.values()) {
         String signature = eachFunction.getReturnType().getType() + " " +
-            CatalogUtil.getCanonicalName(eachFunction.getSignature(), eachFunction.getParameterTypesList());
+            CatalogUtil.getCanonicalSignature(eachFunction.getSignature(), eachFunction.getParameterTypesList());
         String fullDescription = eachFunction.getDescription();
         if(eachFunction.getDetail() != null && !eachFunction.getDetail().isEmpty()) {
           fullDescription += "\n" + eachFunction.getDetail();
         }
 
-        sout.println("Function:    " + signature);
-        sout.println("Description: " + fullDescription);
-        sout.println("Example:\n" + eachFunction.getExample());
+        context.getOutput().println("Function:    " + signature);
+        context.getOutput().println("Description: " + fullDescription);
+        context.getOutput().println("Example:\n" + eachFunction.getExample());
         println();
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/DescTableCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/DescTableCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/DescTableCommand.java
index bdeb173..0907b56 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/DescTableCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/DescTableCommand.java
@@ -21,17 +21,15 @@ package org.apache.tajo.cli;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
-import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.TUtil;
 
-import java.io.PrintWriter;
 import java.util.List;
 import java.util.Map;
 
 public class DescTableCommand extends TajoShellCommand {
-  public DescTableCommand(TajoClient client, PrintWriter sout) {
-    super(client, sout);
+  public DescTableCommand(TajoCli.TajoCliContext context) {
+    super(context);
   }
 
   @Override
@@ -44,17 +42,17 @@ public class DescTableCommand extends TajoShellCommand {
     if (cmd.length == 2) {
       TableDesc desc = client.getTableDesc(cmd[1]);
       if (desc == null) {
-        sout.println("Did not find any relation named \"" + cmd[1] + "\"");
+        context.getOutput().println("Did not find any relation named \"" + cmd[1] + "\"");
       } else {
-        sout.println(toFormattedString(desc));
+        context.getOutput().println(toFormattedString(desc));
       }
     } else if (cmd.length == 1) {
-      List<String> tableList = client.getTableList();
+      List<String> tableList = client.getTableList(null);
       if (tableList.size() == 0) {
-        sout.println("No Relation Found");
+        context.getOutput().println("No Relation Found");
       }
       for (String table : tableList) {
-        sout.println(table);
+        context.getOutput().println(table);
       }
     } else {
       throw new IllegalArgumentException();

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/ExitCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/ExitCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/ExitCommand.java
index dcf8893..1a18841 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/ExitCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/ExitCommand.java
@@ -18,14 +18,10 @@
 
 package org.apache.tajo.cli;
 
-import org.apache.tajo.client.TajoClient;
-
-import java.io.PrintWriter;
-
 public class ExitCommand extends TajoShellCommand {
 
-  public ExitCommand(TajoClient client, PrintWriter sout) {
-    super(client, sout);
+  public ExitCommand(TajoCli.TajoCliContext context) {
+    super(context);
   }
 
   @Override
@@ -35,7 +31,7 @@ public class ExitCommand extends TajoShellCommand {
 
   @Override
   public void invoke(String[] cmd) throws Exception {
-    sout.println("bye!");
+    context.getOutput().println("bye!");
     System.exit(0);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/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 85ba808..a808fc6 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
@@ -18,13 +18,13 @@
 
 package org.apache.tajo.cli;
 
-import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.TajoConstants;
 
 import java.io.PrintWriter;
 
 public class HelpCommand extends TajoShellCommand {
-  public HelpCommand(TajoClient client, PrintWriter sout) {
-    super(client, sout);
+  public HelpCommand(TajoCli.TajoCliContext context) {
+    super(context);
   }
 
   @Override
@@ -34,6 +34,8 @@ public class HelpCommand extends TajoShellCommand {
 
   @Override
   public void invoke(String[] cmd) throws Exception {
+    String docVersion = getDocumentationVersion();
+    PrintWriter sout = context.getOutput();
     sout.println();
 
     sout.println("General");
@@ -45,22 +47,40 @@ public class HelpCommand extends TajoShellCommand {
     sout.println();
 
     sout.println("Informational");
-    sout.println("  \\d         list tables");
-    sout.println("  \\d  NAME   describe table");
-    sout.println("  \\df        list functions");
-    sout.println("  \\df NAME   describe function");
+    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("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://wiki.apache.org/tajo/tsql");
-    sout.println("  Query language    http://wiki.apache.org/tajo/QueryLanguage");
-    sout.println("  Functions         http://wiki.apache.org/tajo/Functions");
-    sout.println("  Backup & restore  http://wiki.apache.org/tajo/BackupAndRestore");
-    sout.println("  Configuration     http://wiki.apache.org/tajo/Configuration");
+    sout.println("  tsql guide        http://tajo.incubator.apache.org/docs/"+ docVersion +"/cli.html");
+    sout.println("  Query language    http://tajo.incubator.apache.org/docs/"+ docVersion +"/sql_language.html");
+    sout.println("  Functions         http://tajo.incubator.apache.org/docs/"+ docVersion +"/functions.html");
+    sout.println("  Backup & restore  http://tajo.incubator.apache.org/docs/"+ docVersion +"/backup_and_restore.html");
+    sout.println("  Configuration     http://tajo.incubator.apache.org/docs/"+ docVersion +"/configuration.html");
     sout.println();
   }
 
+  private String getDocumentationVersion() {
+    int delimiterIdx = TajoConstants.TAJO_VERSION.indexOf("-");
+    if (delimiterIdx > -1) {
+      return TajoConstants.TAJO_VERSION.substring(0, delimiterIdx);
+    } else {
+      return TajoConstants.TAJO_VERSION;
+    }
+  }
+
   @Override
   public String getUsage() {
     return "";

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/InvalidClientSessionException.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/InvalidClientSessionException.java b/tajo-client/src/main/java/org/apache/tajo/cli/InvalidClientSessionException.java
new file mode 100644
index 0000000..5c6c96e
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/InvalidClientSessionException.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import com.google.protobuf.ServiceException;
+
+public class InvalidClientSessionException extends ServiceException {
+  public InvalidClientSessionException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/InvalidStatementException.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/InvalidStatementException.java b/tajo-client/src/main/java/org/apache/tajo/cli/InvalidStatementException.java
new file mode 100644
index 0000000..9782e84
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/InvalidStatementException.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+public class InvalidStatementException extends Exception {
+  public InvalidStatementException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/ListDatabaseCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/ListDatabaseCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/ListDatabaseCommand.java
new file mode 100644
index 0000000..199fed2
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/ListDatabaseCommand.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+public class ListDatabaseCommand extends TajoShellCommand {
+
+  public ListDatabaseCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\l";
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    for (String databaseName : client.getAllDatabaseNames()) {
+      context.getOutput().println(databaseName);
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "list all databases";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/ParsedResult.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/ParsedResult.java b/tajo-client/src/main/java/org/apache/tajo/cli/ParsedResult.java
new file mode 100644
index 0000000..fb89678
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/ParsedResult.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+
+public class ParsedResult {
+  public static enum StatementType {
+    META,
+    STATEMENT
+  }
+
+  private final StatementType type;
+  private final String statement;
+
+  public ParsedResult(StatementType type, String statement) {
+    this.type = type;
+    this.statement = statement;
+  }
+
+  public StatementType getType() {
+    return type;
+  }
+
+  public String getStatement() {
+    return statement.trim();
+  }
+
+  public String toString() {
+    return "(" + type.name() + ") " + statement;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/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
new file mode 100644
index 0000000..8b31d39
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/SetCommand.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import org.apache.tajo.util.StringUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetCommand extends TajoShellCommand {
+
+  public SetCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\set";
+  }
+
+  @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()));
+      }
+    } else if (cmd.length == 3) {
+      Map<String, String> variables = new HashMap<String, String>();
+      variables.put(cmd[1], cmd[2]);
+      client.updateSessionVariables(variables);
+    } else {
+      context.getOutput().println("usage: \\set [[NAME] VALUE]");
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "set session variable or shows all session variables";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/SimpleParser.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/SimpleParser.java b/tajo-client/src/main/java/org/apache/tajo/cli/SimpleParser.java
new file mode 100644
index 0000000..2146df7
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/SimpleParser.java
@@ -0,0 +1,262 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.tajo.cli.ParsedResult.StatementType.META;
+import static org.apache.tajo.cli.ParsedResult.StatementType.STATEMENT;
+
+/**
+ * This is a parser used in tsql to parse multiple SQL lines into SQL statements.
+ * It helps tsql recognizes the termination of each SQL statement and quotation mark (') while
+ * parses multiple separate lines.
+ */
+public class SimpleParser {
+
+  public static enum ParsingState {
+    TOK_START,     // Start State
+    META,          // Meta Command
+    STATEMENT,     // Statement
+    WITHIN_QUOTE,  // Within Quote
+    COMMENT,
+    INVALID,       // Invalid Statement
+    STATEMENT_EOS, // End State (End of Statement)
+    META_EOS       // End State (End of Statement)
+  }
+
+  ParsingState state = START_STATE;
+  int lineNum;
+  StringBuilder appender = new StringBuilder();
+
+  public static final ParsingState START_STATE = ParsingState.TOK_START;
+
+  /**
+   * <h2>State Machine</h2>
+   * All whitespace are ignored in all cases except for
+   *
+   * <pre>
+   * (start) TOK_START --> META ---------------------> META_EOS
+   *                    |
+   *                    |
+   *                    |
+   *                    |-----------> STATEMENT ----------> STMT_EOS
+   *                                  \       ^
+   *                                  \      /
+   *                                  \-> WITHIN_QUOTE
+   *                                        \    ^
+   *                                        \---/
+   * </pre>
+   */
+
+  public static List<ParsedResult> parseScript(String str) throws InvalidStatementException {
+    SimpleParser parser = new SimpleParser();
+    List<ParsedResult> parsedResults = new ArrayList<ParsedResult>();
+    parsedResults.addAll(parser.parseLines(str));
+    parsedResults.addAll(parser.EOF());
+    return parsedResults;
+  }
+
+  public List<ParsedResult> parseLines(String str) throws InvalidStatementException {
+    List<ParsedResult> statements = new ArrayList<ParsedResult>();
+    int lineStartIdx;
+    int idx = 0;
+    char [] chars = str.toCharArray();
+
+    while(idx < str.length()) {
+
+      // initialization for new statement
+      if (state == ParsingState.TOK_START) {
+        lineNum = 0;
+
+        // ignore all whitespace before start
+        if (Character.isWhitespace(chars[idx])) {
+          idx++;
+          continue;
+        }
+      }
+
+      ////////////////////////////
+      // TOK_START --> META
+      ////////////////////////////
+
+      lineStartIdx = idx;
+
+      if (state == ParsingState.TOK_START && chars[idx] == '\\') {
+        state = ParsingState.META;
+
+        ////////////////////////////
+        // META --> TOK_EOS
+        ////////////////////////////
+        while (state != ParsingState.META_EOS && idx < chars.length) {
+          char character = chars[idx++];
+          if (Character.isWhitespace(character)) {
+            // skip
+          } else if (isEndOfMeta(character)) {
+            state = ParsingState.META_EOS;
+          }
+        }
+
+        if (state == ParsingState.META_EOS) {
+          appender.append(str.subSequence(lineStartIdx, idx - 1).toString());
+        } else {
+          appender.append(str.subSequence(lineStartIdx, idx).toString());
+        }
+
+      } else if (isCommentStart(chars[idx])) {
+        idx++;
+        while (!isLineEnd(chars[idx]) && idx < chars.length) {
+          idx++;
+        }
+      /////////////////////////////////
+      //    TOK_START     -> STATEMENT
+      // or TOK_STATEMENT -> STATEMENT
+      ////////////////////////////////
+      } else if (isStatementContinue() || isStatementStart(chars[idx])) {
+        int endIdx = 0;
+        if (!isStatementContinue()) { // TOK_START -> STATEMENT
+          state = ParsingState.STATEMENT;
+        }
+
+        while (!isTerminateState(state) && idx < chars.length) {
+          char character = chars[idx++];
+
+          if (isEndOfStatement(character)) {
+            state = ParsingState.STATEMENT_EOS;
+            endIdx = idx - 1;
+          } else if (state == ParsingState.STATEMENT && character == '\'') { // TOK_STATEMENT -> WITHIN_QUOTE
+            state = ParsingState.WITHIN_QUOTE;
+
+            if (idx < chars.length) {
+              character = chars[idx++];
+            } else {
+              continue;
+            }
+          }
+
+          if (state == ParsingState.WITHIN_QUOTE) {
+            while(idx < chars.length) {
+              ///////////////////////////////
+              // WITHIN_QUOTE --> STATEMENT
+              ///////////////////////////////
+              if (character == '\'') {
+                state = ParsingState.STATEMENT;
+                break;
+              }
+              character = chars[idx++];
+            }
+            if (state == ParsingState.WITHIN_QUOTE && character == '\'') {
+              state = ParsingState.STATEMENT;
+            }
+          }
+        }
+
+        if (state == ParsingState.STATEMENT_EOS) {
+          appender.append(str.subSequence(lineStartIdx, endIdx).toString());
+        } else {
+          appender.append(str.subSequence(lineStartIdx, idx).toString());
+
+          // if it is not within quote and there is no space between lines, add a space.
+          if (state == ParsingState.STATEMENT && (appender.charAt(appender.length() - 1) != ' ')) {
+            appender.append(" ");
+          }
+        }
+      }
+
+      lineNum++;
+      statements.addAll(doProcessEndOfStatement(state == ParsingState.META));
+    }
+
+    return statements;
+  }
+
+  private static boolean isEndOfMeta(char character) {
+    return character == ';' || character == '\n';
+  }
+
+  private static boolean isEndOfStatement(char character) {
+    return character == ';';
+  }
+
+  private boolean isCommentStart(char character) {
+    return state == ParsingState.TOK_START && character == '-';
+  }
+
+  private boolean isLineEnd(char character) {
+    return character == '\n';
+  }
+
+  private boolean isStatementStart(char character) {
+    return state == ParsingState.TOK_START && (Character.isLetterOrDigit(character));
+  }
+
+  private boolean isStatementContinue() {
+    return state == ParsingState.WITHIN_QUOTE || state == ParsingState.STATEMENT;
+  }
+
+  private List<ParsedResult> doProcessEndOfStatement(boolean endOfFile) throws InvalidStatementException {
+    List<ParsedResult> parsedResults = new ArrayList<ParsedResult>();
+    String errorMessage = "";
+    if (endOfFile) {
+      if (state == ParsingState.META) {
+        state = ParsingState.META_EOS;
+      } else if (state == ParsingState.STATEMENT) {
+        state = ParsingState.STATEMENT_EOS;
+      } else if (state == ParsingState.WITHIN_QUOTE) {
+        state = ParsingState.INVALID;
+        errorMessage = "unterminated quoted string at LINE " + lineNum;
+      }
+    }
+
+    if (isTerminateState(state)) {
+      String statement = appender.toString();
+      if (state == ParsingState.META_EOS) {
+        parsedResults.add(new ParsedResult(META, statement));
+        state = ParsingState.TOK_START;
+      } else if (state == ParsingState.STATEMENT_EOS) {
+        parsedResults.add(new ParsedResult(STATEMENT, statement));
+      } else {
+        throw new InvalidStatementException("ERROR: " + errorMessage);
+      }
+
+      // reset all states
+      appender.delete(0, appender.length());
+      state = START_STATE;
+    }
+
+    return parsedResults;
+  }
+
+  public List<ParsedResult> EOF() throws InvalidStatementException {
+    return doProcessEndOfStatement(true);
+  }
+
+  private static boolean isTerminateState(ParsingState state) {
+    return (state == ParsingState.META_EOS || state == ParsingState.STATEMENT_EOS || state == ParsingState.INVALID);
+  }
+
+  public ParsingState getState() {
+    return state;
+  }
+
+  public String toString() {
+    return "[" + state.name() + "]: " + appender.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/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 3026d9c..426c115 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,9 @@
 
 package org.apache.tajo.cli;
 
+import com.google.protobuf.ServiceException;
 import jline.console.ConsoleReader;
-import jline.console.history.FileHistory;
-import jline.console.history.PersistentHistory;
 import org.apache.commons.cli.*;
-import org.apache.commons.lang.StringUtils;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoProtos.QueryState;
@@ -37,29 +35,33 @@ import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.jdbc.TajoResultSet;
 import org.apache.tajo.util.FileUtil;
 
-import java.io.File;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
+import java.io.*;
 import java.lang.reflect.Constructor;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
+import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import static org.apache.tajo.cli.ParsedResult.StatementType.META;
+import static org.apache.tajo.cli.ParsedResult.StatementType.STATEMENT;
+import static org.apache.tajo.cli.SimpleParser.ParsingState;
+
 public class TajoCli {
-  public static final int PRINT_LIMIT = 24;
 
   private final TajoConf conf;
-  private static final Options options;
-
   private TajoClient client;
+  private final TajoCliContext context;
 
+  // Jline and Console related things
   private final ConsoleReader reader;
   private final InputStream sin;
   private final PrintWriter sout;
+  private TajoFileHistory history;
 
-  private final Map<String, TajoShellCommand> commands = new TreeMap<String, TajoShellCommand>();
+  // Current States
+  private String currentDatabase;
 
   private static final Class [] registeredCommands = {
       DescTableCommand.class,
@@ -67,9 +69,16 @@ public class TajoCli {
       HelpCommand.class,
       ExitCommand.class,
       CopyrightCommand.class,
-      VersionCommand.class
+      VersionCommand.class,
+      ConnectDatabaseCommand.class,
+      ListDatabaseCommand.class,
+      SetCommand.class,
+      UnsetCommand.class
   };
+  private final Map<String, TajoShellCommand> commands = new TreeMap<String, TajoShellCommand>();
 
+  public static final int PRINT_LIMIT = 24;
+  private static final Options options;
   private static final String HOME_DIR = System.getProperty("user.home");
   private static final String HISTORY_FILE = ".tajo_history";
 
@@ -79,10 +88,29 @@ public class TajoCli {
     options.addOption("f", "file", true, "execute commands from file, then exit");
     options.addOption("h", "host", true, "Tajo server host");
     options.addOption("p", "port", true, "Tajo server port");
+    options.addOption("help", "help", false, "help");
   }
 
-  public TajoCli(TajoConf c, String [] args,
-                 InputStream in, OutputStream out) throws Exception {
+  public class TajoCliContext {
+
+    public TajoClient getTajoClient() {
+      return client;
+    }
+
+    public void setCurrentDatabase(String databasae) {
+      currentDatabase = databasae;
+    }
+
+    public String getCurrentDatabase() {
+      return currentDatabase;
+    }
+
+    public PrintWriter getOutput() {
+      return sout;
+    }
+  }
+
+  public TajoCli(TajoConf c, String [] args, InputStream in, OutputStream out) throws Exception {
     this.conf = new TajoConf(c);
     this.sin = in;
     this.reader = new ConsoleReader(sin, out);
@@ -91,6 +119,10 @@ public class TajoCli {
     CommandLineParser parser = new PosixParser();
     CommandLine cmd = parser.parse(options, args);
 
+    if (cmd.hasOption("help")) {
+      printUsage();
+    }
+
     String hostName = null;
     Integer port = null;
     if (cmd.hasOption("h")) {
@@ -100,6 +132,11 @@ public class TajoCli {
       port = Integer.parseInt(cmd.getOptionValue("p"));
     }
 
+    String baseDatabase = null;
+    if (cmd.getArgList().size() > 0) {
+      baseDatabase = (String) cmd.getArgList().get(0);
+    }
+
     // if there is no "-h" option,
     if(hostName == null) {
       if (conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != null) {
@@ -121,24 +158,26 @@ public class TajoCli {
       System.exit(-1);
     } else if (hostName != null && port != null) {
       conf.setVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName+":"+port);
-      client = new TajoClient(conf);
+      client = new TajoClient(conf, baseDatabase);
     } else if (hostName == null && port == null) {
-      client = new TajoClient(conf);
+      client = new TajoClient(conf, baseDatabase);
     }
 
+    context = new TajoCliContext();
+    context.setCurrentDatabase(client.getCurrentDatabase());
     initHistory();
     initCommands();
 
     if (cmd.hasOption("c")) {
-      executeStatements(cmd.getOptionValue("c"));
+      executeScript(cmd.getOptionValue("c"));
       sout.flush();
       System.exit(0);
     }
     if (cmd.hasOption("f")) {
       File sqlFile = new File(cmd.getOptionValue("f"));
       if (sqlFile.exists()) {
-        String contents = FileUtil.readTextFile(new File(cmd.getOptionValue("f")));
-        executeStatements(contents);
+        String script = FileUtil.readTextFile(new File(cmd.getOptionValue("f")));
+        executeScript(script);
         sout.flush();
         System.exit(0);
       } else {
@@ -146,13 +185,16 @@ public class TajoCli {
         System.exit(-1);
       }
     }
+
+    addShutdownHook();
   }
 
   private void initHistory() {
     try {
       String historyPath = HOME_DIR + File.separator + HISTORY_FILE;
       if ((new File(HOME_DIR)).exists()) {
-        reader.setHistory(new FileHistory(new File(historyPath)));
+        history = new TajoFileHistory(new File(historyPath));
+        reader.setHistory(history);
       } else {
         System.err.println("ERROR: home directory : '" + HOME_DIR +"' does not exist.");
       }
@@ -165,8 +207,8 @@ public class TajoCli {
     for (Class clazz : registeredCommands) {
       TajoShellCommand cmd = null;
       try {
-         Constructor cons = clazz.getConstructor(new Class[] {TajoClient.class, PrintWriter.class});
-         cmd = (TajoShellCommand) cons.newInstance(client, sout);
+         Constructor cons = clazz.getConstructor(new Class[] {TajoCliContext.class});
+         cmd = (TajoShellCommand) cons.newInstance(context);
       } catch (Exception e) {
         System.err.println(e.getMessage());
         System.exit(-1);
@@ -175,160 +217,114 @@ public class TajoCli {
     }
   }
 
-  public int runShell() throws Exception {
+  private void addShutdownHook() {
+    Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          history.flush();
+        } catch (IOException e) {
+        }
+        client.close();
+      }
+    }));
+  }
+
+  private String updatePrompt(ParsingState state) throws ServiceException {
+    if (state == ParsingState.WITHIN_QUOTE) {
+      return "'";
+    } else if (state == ParsingState.TOK_START) {
+      return context.getCurrentDatabase();
+    } else {
+      return "";
+    }
+  }
 
-    String raw;
+  public int runShell() throws Exception {
     String line;
-    StringBuffer accumulatedLine = new StringBuffer();
-    String prompt = "tajo";
-    String curPrompt = prompt;
-    boolean newStatement = true;
+    String currentPrompt = context.getCurrentDatabase();
     int code = 0;
 
     sout.write("Try \\? for help.\n");
-    while((raw = reader.readLine(curPrompt + "> ")) != null) {
-      // each accumulated line has a space delimiter
-      if (accumulatedLine.length() > 0) {
-        accumulatedLine.append(' ');
-      }
-
-      line = raw.trim();
 
-      if (line.length() == 0) { // if empty line
-        continue;
+    SimpleParser parser = new SimpleParser();
+    while((line = reader.readLine(currentPrompt + "> ")) != null) {
 
-      } else if (line.charAt(0) == '/') { // warning for legacy usage
-        printInvalidCommand(line);
+      if (line.equals("")) {
         continue;
+      }
 
-      } else if (line.charAt(0) == '\\') { // command mode
-        ((PersistentHistory)reader.getHistory()).flush();
-        executeCommand(line);
-
-      } else if (line.endsWith(";") && !line.endsWith("\\;")) {
-
-        // remove a trailing newline
-        line = StringUtils.chomp(line).trim();
-
-        // get a punctuated statement
-        String punctuated = accumulatedLine + line;
-
-        if (!newStatement) {
-          // why do two lines are removed?
-          // First history line indicates an accumulated line.
-          // Second history line is a just typed line.
-          reader.getHistory().removeLast();
-          reader.getHistory().removeLast();
-          reader.getHistory().add(punctuated);
-          ((PersistentHistory)reader.getHistory()).flush();
-        }
-
-        code = executeStatements(punctuated);
-
-        // reset accumulated lines
-        newStatement = true;
-        accumulatedLine = new StringBuffer();
-        curPrompt = prompt;
-
-      } else {
-        line = StringUtils.chomp(raw).trim();
-
-        // accumulate a line
-        accumulatedLine.append(line);
+      List<ParsedResult> parsedResults = parser.parseLines(line);
 
-        // replace the latest line with a accumulated line
-        if (!newStatement) { // if this is not first line, remove one more line.
-          reader.getHistory().removeLast();
-        } else {
-          newStatement = false;
+      if (parsedResults.size() > 0) {
+        for (ParsedResult parsed : parsedResults) {
+          history.addStatement(parsed.getStatement() + (parsed.getType() == STATEMENT ? ";":""));
         }
-        reader.getHistory().removeLast();
-        reader.getHistory().add(accumulatedLine.toString());
-
-        // use an alternative prompt during accumulating lines
-        curPrompt = StringUtils.repeat(" ", prompt.length());
-        continue;
       }
+      executeParsedResults(parsedResults);
+      currentPrompt = updatePrompt(parser.getState());
     }
     return code;
   }
 
-  private void invokeCommand(String [] cmds) {
-    // this command should be moved to GlobalEngine
-    TajoShellCommand invoked;
-    try {
-      invoked = commands.get(cmds[0]);
-      invoked.invoke(cmds);
-    } catch (Throwable t) {
-      sout.println(t.getMessage());
+  private void executeParsedResults(Collection<ParsedResult> parsedResults) throws Exception {
+    for (ParsedResult parsedResult : parsedResults) {
+      if (parsedResult.getType() == META) {
+        executeMetaCommand(parsedResult.getStatement());
+      } else {
+        executeQuery(parsedResult.getStatement());
+      }
     }
   }
 
-  public int executeStatements(String line) throws Exception {
+  public int executeMetaCommand(String line) throws Exception {
+    String [] metaCommands = line.split(";");
+    for (String metaCommand : metaCommands) {
+      String arguments [];
+      arguments = metaCommand.split(" ");
 
-    // TODO - comment handling and multi line queries should be improved
-    // remove comments
-    String filtered = line.replaceAll("--[^\\r\\n]*", "").trim();
+      TajoShellCommand invoked = commands.get(arguments[0]);
+      if (invoked == null) {
+        printInvalidCommand(arguments[0]);
+        return -1;
+      }
 
-    String stripped;
-    for (String statement : filtered.split(";")) {
-      stripped = StringUtils.chomp(statement);
-      if (StringUtils.isBlank(stripped)) {
-        continue;
+      try {
+        invoked.invoke(arguments);
+      } catch (IllegalArgumentException ige) {
+        sout.println(ige.getMessage());
+      } catch (Exception e) {
+        sout.println(e.getMessage());
       }
+    }
 
-      String [] cmds = stripped.split(" ");
-      if (cmds[0].equalsIgnoreCase("exit") || cmds[0].equalsIgnoreCase("quit")) {
-        sout.println("\n\nbye!");
-        sout.flush();
-        ((PersistentHistory)this.reader.getHistory()).flush();
-        System.exit(0);
-      } else if (cmds[0].equalsIgnoreCase("detach") && cmds.length > 1 && cmds[1].equalsIgnoreCase("table")) {
-        // this command should be moved to GlobalEngine
-        invokeCommand(cmds);
-      } else if (cmds[0].equalsIgnoreCase("explain") && cmds.length > 1) {
-        String sql = stripped.substring(8);
-        ClientProtos.ExplainQueryResponse response = client.explainQuery(sql);
-        if (response == null) {
-          sout.println("response is null");
+    return 0;
+  }
+
+  private void executeQuery(String statement) throws ServiceException {
+    ClientProtos.GetQueryStatusResponse response = client.executeQuery(statement);
+    if (response == null) {
+      sout.println("response is null");
+    }
+    else if (response.getResultCode() == ClientProtos.ResultCode.OK) {
+      QueryId queryId = null;
+      try {
+        queryId = new QueryId(response.getQueryId());
+        if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
+          sout.println("OK");
         } else {
-          if (response.hasExplain()) {
-            sout.println(response.getExplain());
-          } else {
-            if (response.hasErrorMessage()) {
-              sout.println(response.getErrorMessage());
-            } else {
-              sout.println("No Explain");
-            }
-          }
+          waitForQueryCompleted(queryId);
         }
-      } else { // submit a query to TajoMaster
-        ClientProtos.GetQueryStatusResponse response = client.executeQuery(stripped);
-        if (response == null) {
-          sout.println("response is null");
-        }
-        else if (response.getResultCode() == ClientProtos.ResultCode.OK) {
-          QueryId queryId = null;
-          try {
-            queryId = new QueryId(response.getQueryId());
-            if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
-              sout.println("OK");
-            } else {
-              waitForQueryCompleted(queryId);
-            }
-          } finally {
-            if(queryId != null) {
-              client.closeQuery(queryId);
-            }
-          }
-        } else {
-          if (response.hasErrorMessage()) {
-            sout.println(response.getErrorMessage());
-          }
+      } finally {
+        if(queryId != null) {
+          client.closeQuery(queryId);
         }
       }
+    } else {
+      if (response.hasErrorMessage()) {
+        sout.println(response.getErrorMessage());
+      }
     }
-    return 0;
   }
 
   private void waitForQueryCompleted(QueryId queryId) {
@@ -453,33 +449,15 @@ public class TajoCli {
     }
   }
 
-  private void printUsage() {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp( "tajo cli [options]", options );
+  public int executeScript(String script) throws Exception {
+    List<ParsedResult> results = SimpleParser.parseScript(script);
+    executeParsedResults(results);
+    return 0;
   }
 
-  public int executeCommand(String line) throws Exception {
-    String [] metaCommands = line.split(";");
-    for (String metaCommand : metaCommands) {
-      String arguments [];
-      arguments = metaCommand.split(" ");
-
-      TajoShellCommand invoked = commands.get(arguments[0]);
-      if (invoked == null) {
-        printInvalidCommand(arguments[0]);
-        return -1;
-      }
-
-      try {
-        invoked.invoke(arguments);
-      } catch (IllegalArgumentException ige) {
-        sout.println(ige.getMessage());
-      } catch (Exception e) {
-        sout.println(e.getMessage());
-      }
-    }
-
-    return 0;
+  private void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp( "tsql [options] [database]", options );
   }
 
   private void printInvalidCommand(String command) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/TajoFileHistory.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/TajoFileHistory.java b/tajo-client/src/main/java/org/apache/tajo/cli/TajoFileHistory.java
new file mode 100644
index 0000000..3257f28
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/TajoFileHistory.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import jline.console.history.FileHistory;
+
+import java.io.File;
+import java.io.IOException;
+
+public class TajoFileHistory extends FileHistory {
+
+  public TajoFileHistory(File file) throws IOException {
+    super(file);
+  }
+
+  public void add(CharSequence item) {
+    // skip add
+  }
+
+  public void addStatement(String item) {
+    internalAdd(item);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/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 2b61068..1ea2893 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
@@ -21,37 +21,33 @@ package org.apache.tajo.cli;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.conf.TajoConf;
 
-import java.io.PrintWriter;
-
 public abstract class TajoShellCommand {
   public abstract String getCommand();
   public abstract void invoke(String [] command) throws Exception;
   public abstract String getUsage();
   public abstract String getDescription();
 
+  protected TajoCli.TajoCliContext context;
   protected TajoClient client;
-  protected PrintWriter sout;
-
   protected int maxColumn;
 
-  public TajoShellCommand(TajoClient client, PrintWriter sout) {
-    this.client = client;
-    this.sout = sout;
-
-    maxColumn = client.getConf().getIntVar(TajoConf.ConfVars.CLI_MAX_COLUMN);
+  public TajoShellCommand(TajoCli.TajoCliContext context) {
+    maxColumn = context.getTajoClient().getConf().getIntVar(TajoConf.ConfVars.CLI_MAX_COLUMN);
+    this.context = context;
+    client = context.getTajoClient();
   }
 
   protected void println() {
-    this.sout.println();
+    context.getOutput().println();
   }
 
   protected void printLeft(String message, int columnWidth) {
     int messageLength = message.length();
 
     if(messageLength >= columnWidth) {
-      sout.print(message.substring(0, columnWidth - 1));
+      context.getOutput().print(message.substring(0, columnWidth - 1));
     } else {
-      sout.print(message);
+      context.getOutput().print(message);
       print(' ', columnWidth - messageLength - 1);
     }
   }
@@ -60,12 +56,12 @@ public abstract class TajoShellCommand {
     int messageLength = message.length();
 
     if(messageLength > columnWidth) {
-      sout.print(message.substring(0, columnWidth - 1));
+      context.getOutput().print(message.substring(0, columnWidth - 1));
     } else {
       int numPadding = (columnWidth - messageLength)/2;
 
       print(' ', numPadding);
-      sout.print(message);
+      context.getOutput().print(message);
       print(' ', numPadding);
     }
     if(warp) {
@@ -79,7 +75,7 @@ public abstract class TajoShellCommand {
 
   protected void print(char c, int count) {
     for(int i = 0; i < count; i++) {
-      sout.print(c);
+      context.getOutput().print(c);
     }
   }
 
@@ -98,7 +94,7 @@ public abstract class TajoShellCommand {
 
     String prefix = "";
     for(int i = 0; i < headers.length; i++) {
-      sout.print(prefix);
+      context.getOutput().print(prefix);
       printLeft(" " + headers[i], columnWidths[i]);
       prefix = "|";
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/UnsetCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/UnsetCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/UnsetCommand.java
new file mode 100644
index 0000000..7b7685f
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/UnsetCommand.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.cli;
+
+import com.google.common.collect.Lists;
+
+public class UnsetCommand extends TajoShellCommand {
+
+  public UnsetCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\unset";
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    if (cmd.length == 2) {
+      client.unsetSessionVariables(Lists.newArrayList(cmd[1]));
+    } else {
+      context.getOutput().println("usage: \\unset NAME");
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "unset a session variable";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/3ba26241/tajo-client/src/main/java/org/apache/tajo/cli/VersionCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/VersionCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/VersionCommand.java
index 744c786..b62c425 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/VersionCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/VersionCommand.java
@@ -19,14 +19,11 @@
 package org.apache.tajo.cli;
 
 import org.apache.tajo.TajoConstants;
-import org.apache.tajo.client.TajoClient;
-
-import java.io.PrintWriter;
 
 public class VersionCommand extends TajoShellCommand {
 
-  public VersionCommand(TajoClient client, PrintWriter sout) {
-    super(client, sout);
+  public VersionCommand(TajoCli.TajoCliContext context) {
+    super(context);
   }
 
   @Override
@@ -36,7 +33,7 @@ public class VersionCommand extends TajoShellCommand {
 
   @Override
   public void invoke(String[] cmd) throws Exception {
-    sout.println(TajoConstants.TAJO_VERSION);
+    context.getOutput().println(TajoConstants.TAJO_VERSION);
   }
 
   @Override