You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2015/02/05 10:03:53 UTC

[3/4] tajo git commit: TAJO-1269: Separate cli from tajo-client.

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
new file mode 100644
index 0000000..f48a5b4
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
@@ -0,0 +1,686 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ServiceException;
+import jline.UnsupportedTerminal;
+import jline.console.ConsoleReader;
+import org.apache.commons.cli.*;
+import org.apache.tajo.*;
+import org.apache.tajo.TajoProtos.QueryState;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.cli.tsql.ParsedResult.StatementType;
+import org.apache.tajo.cli.tsql.SimpleParser.ParsingState;
+import org.apache.tajo.cli.tsql.commands.*;
+import org.apache.tajo.client.*;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.util.FileUtil;
+
+import java.io.*;
+import java.lang.reflect.Constructor;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class TajoCli {
+  public static final String ERROR_PREFIX = "ERROR: ";
+  public static final String KILL_PREFIX = "KILL: ";
+
+  private final TajoConf conf;
+  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;
+
+  // Current States
+  private String currentDatabase;
+
+  private TajoCliOutputFormatter displayFormatter;
+
+  private boolean wasError = false;
+
+  private static final Class [] registeredCommands = {
+      DescTableCommand.class,
+      DescFunctionCommand.class,
+      HelpCommand.class,
+      ExitCommand.class,
+      CopyrightCommand.class,
+      VersionCommand.class,
+      ConnectDatabaseCommand.class,
+      ListDatabaseCommand.class,
+      SetCommand.class,
+      UnsetCommand.class,
+      ExecExternalShellCommand.class,
+      HdfsCommand.class,
+      TajoAdminCommand.class,
+      TajoGetConfCommand.class,
+      TajoHAAdminCommand.class
+  };
+  private final Map<String, TajoShellCommand> commands = new TreeMap<String, TajoShellCommand>();
+
+  protected static final Options options;
+  private static final String HOME_DIR = System.getProperty("user.home");
+  private static final String HISTORY_FILE = ".tajo_history";
+
+  static {
+    options = new Options();
+    options.addOption("c", "command", true, "execute only single command, then exit");
+    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("B", "background", false, "execute as background process");
+    options.addOption("conf", "conf", true, "configuration value");
+    options.addOption("param", "param", true, "parameter value in SQL file");
+    options.addOption("help", "help", false, "help");
+  }
+
+  public class TajoCliContext extends OverridableConf {
+    public TajoCliContext(TajoConf conf) {
+      super(conf, ConfigKey.ConfigType.SESSION);
+    }
+
+    public TajoClient getTajoClient() {
+      return client;
+    }
+
+    public void setCurrentDatabase(String databasae) {
+      currentDatabase = databasae;
+    }
+
+    public String getCurrentDatabase() {
+      return currentDatabase;
+    }
+
+    public PrintWriter getOutput() {
+      return sout;
+    }
+
+    public TajoConf getConf() {
+      return conf;
+    }
+
+    @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 {
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd = parser.parse(options, args);
+
+    this.conf = new TajoConf(c);
+    context = new TajoCliContext(conf);
+    this.sin = in;
+    if (cmd.hasOption("B")) {
+      this.reader = new ConsoleReader(sin, out, new UnsupportedTerminal());
+    } else {
+      this.reader = new ConsoleReader(sin, out);
+    }
+
+    this.reader.setExpandEvents(false);
+    this.sout = new PrintWriter(reader.getOutput());
+    initFormatter();
+
+    if (cmd.hasOption("help")) {
+      printUsage();
+      System.exit(0);
+    }
+
+    String hostName = null;
+    Integer port = null;
+    if (cmd.hasOption("h")) {
+      hostName = cmd.getOptionValue("h");
+    }
+    if (cmd.hasOption("p")) {
+      port = Integer.parseInt(cmd.getOptionValue("p"));
+    }
+
+    String baseDatabase = null;
+    if (cmd.getArgList().size() > 0) {
+      baseDatabase = (String) cmd.getArgList().get(0);
+    }
+
+    if (cmd.getOptionValues("conf") != null) {
+      processConfVarCommand(cmd.getOptionValues("conf"));
+    }
+
+    // if there is no "-h" option,
+    if(hostName == null) {
+      if (conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != null) {
+        // it checks if the client service address is given in configuration and distributed mode.
+        // if so, it sets entryAddr.
+        hostName = conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
+      }
+    }
+    if (port == null) {
+      if (conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != null) {
+        // it checks if the client service address is given in configuration and distributed mode.
+        // if so, it sets entryAddr.
+        port = Integer.parseInt(conf.getVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
+      }
+    }
+
+    if ((hostName == null) ^ (port == null)) {
+      System.err.println(ERROR_PREFIX + "cannot find valid Tajo server address");
+      throw new RuntimeException("cannot find valid Tajo server address");
+    } else if (hostName != null && port != null) {
+      conf.setVar(ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName+":"+port);
+      client = new TajoClientImpl(conf, baseDatabase);
+    } else if (hostName == null && port == null) {
+      client = new TajoClientImpl(conf, baseDatabase);
+    }
+
+    try {
+      context.setCurrentDatabase(client.getCurrentDatabase());
+      initHistory();
+      initCommands();
+
+      if (cmd.getOptionValues("conf") != null) {
+        processSessionVarCommand(cmd.getOptionValues("conf"));
+      }
+
+      if (cmd.hasOption("c")) {
+        displayFormatter.setScriptMode();
+        int exitCode = executeScript(cmd.getOptionValue("c"));
+        sout.flush();
+        System.exit(exitCode);
+      }
+      if (cmd.hasOption("f")) {
+        displayFormatter.setScriptMode();
+        cmd.getOptionValues("");
+        File sqlFile = new File(cmd.getOptionValue("f"));
+        if (sqlFile.exists()) {
+          String script = FileUtil.readTextFile(new File(cmd.getOptionValue("f")));
+          script = replaceParam(script, cmd.getOptionValues("param"));
+          int exitCode = executeScript(script);
+          sout.flush();
+          System.exit(exitCode);
+        } else {
+          System.err.println(ERROR_PREFIX + "No such a file \"" + cmd.getOptionValue("f") + "\"");
+          System.exit(-1);
+        }
+      }
+    } catch (Exception e) {
+      System.err.println(ERROR_PREFIX + "Exception was thrown. Caused by " + e.getMessage());
+      
+      if (client != null) {
+        client.close();
+      }
+      
+      throw e;
+    }
+
+    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 = context.getClass(SessionVars.CLI_FORMATTER_CLASS);
+    if (displayFormatter == null || !displayFormatter.getClass().equals(formatterClass)) {
+      displayFormatter = (TajoCliOutputFormatter)formatterClass.newInstance();
+    }
+    displayFormatter.init(context);
+  }
+
+  public TajoCliContext getContext() {
+    return context;
+  }
+
+  protected static String replaceParam(String script, String[] params) {
+    if (params == null || params.length == 0) {
+      return script;
+    }
+
+    for (String eachParam: params) {
+      String[] tokens = eachParam.split("=");
+      if (tokens.length != 2) {
+        continue;
+      }
+      script = script.replace("${" + tokens[0] + "}", tokens[1]);
+    }
+
+    return script;
+  }
+
+  private void initHistory() {
+    try {
+      String historyPath = HOME_DIR + File.separator + HISTORY_FILE;
+      if ((new File(HOME_DIR)).exists()) {
+        history = new TajoFileHistory(new File(historyPath));
+        history.setAutoTrim(false);
+        history.setIgnoreDuplicates(false);
+        reader.setHistory(history);
+      } else {
+        System.err.println(ERROR_PREFIX + "home directory : '" + HOME_DIR +"' does not exist.");
+      }
+    } catch (Exception e) {
+      System.err.println(ERROR_PREFIX + e.getMessage());
+    }
+  }
+
+  private void initCommands() {
+    for (Class clazz : registeredCommands) {
+      TajoShellCommand cmd = null;
+      try {
+         Constructor cons = clazz.getConstructor(new Class[] {TajoCliContext.class});
+         cmd = (TajoShellCommand) cons.newInstance(context);
+      } catch (Exception e) {
+        System.err.println(e.getMessage());
+        throw new RuntimeException(e.getMessage());
+      }
+      commands.put(cmd.getCommand(), cmd);
+      for (String alias : cmd.getAliases()) {
+        commands.put(alias, cmd);
+      }
+    }
+  }
+
+  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 "";
+    }
+  }
+
+  public int runShell() throws Exception {
+    String line;
+    String currentPrompt = context.getCurrentDatabase();
+    int exitCode;
+    ParsingState latestState = SimpleParser.START_STATE;
+
+    sout.write("Try \\? for help.\n");
+
+    SimpleParser parser = new SimpleParser();
+    
+    try {
+      while((line = reader.readLine(currentPrompt + "> ")) != null) {
+        if (line.equals("")) {
+          continue;
+        }
+        wasError = false;
+        if (line.startsWith("{")) {
+          executeJsonQuery(line);
+        } else {
+          List<ParsedResult> parsedResults = parser.parseLines(line);
+
+          if (latestState != ParsingState.TOK_START && parsedResults.size() > 0) {
+            // Add multi-line statements to history in addition to individual lines.
+            ParsedResult parsed = parsedResults.get(0);
+            history.add(parsed.getHistoryStatement() + (parsed.getType() == StatementType.STATEMENT ? ";" : ""));
+          }
+
+          exitCode = executeParsedResults(parsedResults);
+          latestState = parser.getState();
+          currentPrompt = updatePrompt(latestState);
+
+          // if at least one failed
+          if (exitCode != 0) {
+            return exitCode;
+          }
+        }
+      }
+    } catch (Exception e) {
+      System.err.println(ERROR_PREFIX + "Exception was thrown. Casued by " + e.getMessage());
+      
+      if (client != null) {
+        client.close();
+      }
+      
+      throw e;
+    }
+    return 0;
+  }
+
+  private int executeParsedResults(Collection<ParsedResult> parsedResults) throws Exception {
+    int exitCode;
+    for (ParsedResult parsedResult : parsedResults) {
+      if (parsedResult.getType() == StatementType.META) {
+        exitCode = executeMetaCommand(parsedResult.getStatement());
+      } else {
+        exitCode = executeQuery(parsedResult.getStatement());
+      }
+
+      if (exitCode != 0 && context.getBool(SessionVars.ON_ERROR_STOP)) {
+        return exitCode;
+      }
+    }
+
+    return 0;
+  }
+
+  public int executeMetaCommand(String line) throws Exception {
+    String [] metaCommands = line.split(";");
+    for (String metaCommand : metaCommands) {
+      String arguments [] = metaCommand.split(" ");
+
+      TajoShellCommand invoked = commands.get(arguments[0]);
+      if (invoked == null) {
+        printInvalidCommand(arguments[0]);
+        wasError = true;
+        return -1;
+      }
+
+      try {
+        invoked.invoke(arguments);
+      } catch (IllegalArgumentException ige) {
+        displayFormatter.printErrorMessage(sout, ige);
+        wasError = true;
+        return -1;
+      } catch (Exception e) {
+        displayFormatter.printErrorMessage(sout, e);
+        wasError = true;
+        return -1;
+      } finally {
+        context.getOutput().flush();
+      }
+
+      if (wasError && context.getBool(SessionVars.ON_ERROR_STOP)) {
+        break;
+      }
+    }
+
+    return 0;
+  }
+
+  private void executeJsonQuery(String json) throws ServiceException, IOException {
+
+    long startTime = System.currentTimeMillis();
+    ClientProtos.SubmitQueryResponse response = client.executeQueryWithJson(json);
+    if (response == null) {
+      displayFormatter.printErrorMessage(sout, "response is null");
+      wasError = true;
+    } else if (response.getResultCode() == ClientProtos.ResultCode.OK) {
+      if (response.getIsForwarded()) {
+        QueryId queryId = new QueryId(response.getQueryId());
+        waitForQueryCompleted(queryId);
+      } else {
+        if (!response.hasTableDesc() && !response.hasResultSet()) {
+          displayFormatter.printMessage(sout, "OK");
+          wasError = true;
+        } else {
+          localQueryCompleted(response, startTime);
+        }
+      }
+    } else {
+      if (response.hasErrorMessage()) {
+        displayFormatter.printErrorMessage(sout, response.getErrorMessage());
+        wasError = true;
+      }
+    }
+  }
+
+  private int executeQuery(String statement) throws ServiceException, IOException {
+
+    long startTime = System.currentTimeMillis();
+    ClientProtos.SubmitQueryResponse response = null;
+    try{
+      response = client.executeQuery(statement);
+    } catch (ServiceException e){
+      displayFormatter.printErrorMessage(sout, e.getMessage());
+      wasError = true;
+    } catch(Throwable te){
+      displayFormatter.printErrorMessage(sout, te);
+      wasError = true;
+    }
+
+    if (response == null) {
+      displayFormatter.printErrorMessage(sout, "response is null");
+      wasError = true;
+    } else if (response.getResultCode() == ClientProtos.ResultCode.OK) {
+      if (response.getIsForwarded()) {
+        QueryId queryId = new QueryId(response.getQueryId());
+        waitForQueryCompleted(queryId);
+      } else {
+        if (!response.hasTableDesc() && !response.hasResultSet()) {
+          displayFormatter.printMessage(sout, "OK");
+        } else {
+          localQueryCompleted(response, startTime);
+        }
+      }
+    } else {
+      if (response.hasErrorMessage()) {
+        displayFormatter.printErrorMessage(sout, response.getErrorMessage());
+        wasError = true;
+      }
+    }
+
+    return wasError ? -1 : 0;
+  }
+
+  private void localQueryCompleted(ClientProtos.SubmitQueryResponse response, long startTime) {
+    ResultSet res = null;
+    try {
+      QueryId queryId = new QueryId(response.getQueryId());
+      float responseTime = ((float)(System.currentTimeMillis() - startTime) / 1000.0f);
+      TableDesc desc = new TableDesc(response.getTableDesc());
+
+      // 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)) {
+        displayFormatter.printResult(sout, sin, desc, responseTime, res);
+      } else {
+        res = TajoClientUtil.createResultSet(conf, client, response);
+        displayFormatter.printResult(sout, sin, desc, responseTime, res);
+      }
+    } catch (Throwable t) {
+      displayFormatter.printErrorMessage(sout, t);
+      wasError = true;
+    } finally {
+      if (res != null) {
+        try {
+          res.close();
+        } catch (SQLException e) {
+        }
+      }
+    }
+  }
+
+  private void waitForQueryCompleted(QueryId queryId) {
+    // if query is empty string
+    if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
+      return;
+    }
+
+    // query execute
+    ResultSet res = null;
+    QueryStatus status = null;
+    try {
+
+      int initRetries = 0;
+      int progressRetries = 0;
+      while (true) {
+        // TODO - configurable
+        status = client.getQueryStatus(queryId);
+        if(TajoClientUtil.isQueryWaitingForSchedule(status.getState())) {
+          Thread.sleep(Math.min(20 * initRetries, 1000));
+          initRetries++;
+          continue;
+        }
+
+        if (TajoClientUtil.isQueryRunning(status.getState()) || status.getState() == QueryState.QUERY_SUCCEEDED) {
+          displayFormatter.printProgress(sout, status);
+        }
+
+        if (TajoClientUtil.isQueryComplete(status.getState()) && status.getState() != QueryState.QUERY_KILL_WAIT) {
+          break;
+        } else {
+          Thread.sleep(Math.min(200 * progressRetries, 1000));
+          progressRetries += 2;
+        }
+      }
+
+      if (status.getState() == QueryState.QUERY_ERROR || status.getState() == QueryState.QUERY_FAILED) {
+        displayFormatter.printErrorMessage(sout, status);
+        wasError = true;
+      } else if (status.getState() == QueryState.QUERY_KILLED) {
+        displayFormatter.printKilledMessage(sout, queryId);
+        wasError = true;
+      } else {
+        if (status.getState() == QueryState.QUERY_SUCCEEDED) {
+          float responseTime = ((float)(status.getFinishTime() - status.getSubmitTime()) / 1000.0f);
+          ClientProtos.GetQueryResultResponse response = client.getResultResponse(queryId);
+          if (status.hasResult()) {
+            res = TajoClientUtil.createResultSet(conf, client, queryId, response);
+            TableDesc desc = new TableDesc(response.getTableDesc());
+            displayFormatter.printResult(sout, sin, desc, responseTime, res);
+          } else {
+            TableDesc desc = new TableDesc(response.getTableDesc());
+            displayFormatter.printResult(sout, sin, desc, responseTime, res);
+          }
+        }
+      }
+    } catch (Throwable t) {
+      displayFormatter.printErrorMessage(sout, t);
+      wasError = true;
+    } finally {
+      if (res != null) {
+        try {
+          res.close();
+        } catch (SQLException e) {
+        }
+      } else {
+        if (status != null && status.getQueryId() != null) {
+          client.closeQuery(status.getQueryId());
+        }
+      }
+    }
+  }
+
+  public int executeScript(String script) throws Exception {
+    wasError = false;
+    List<ParsedResult> results = SimpleParser.parseScript(script);
+    return executeParsedResults(results);
+  }
+
+  private void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("tsql [options] [database]", options);
+  }
+
+  private void printInvalidCommand(String command) {
+    sout.println("Invalid command " + command + ". Try \\? for help.");
+  }
+
+  @VisibleForTesting
+  public void close() {
+    //for testcase
+    if (client != null) {
+      client.close();
+    }
+
+    if (reader != null) {
+      reader.shutdown();
+    }
+  }
+
+  public static void main(String [] args) throws Exception {
+    TajoConf conf = new TajoConf();
+    TajoCli shell = new TajoCli(conf, args, System.in, System.out);
+    System.out.println();
+    System.exit(shell.runShell());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java
new file mode 100644
index 0000000..13d9cb1
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql;
+
+import org.apache.tajo.QueryId;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.client.QueryStatus;
+
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.sql.ResultSet;
+
+public interface TajoCliOutputFormatter {
+  /**
+   * Initialize formatter
+   * @param context
+   */
+  public void init(TajoCli.TajoCliContext context);
+
+  /**
+   * print query result to console
+   * @param sout
+   * @param sin
+   * @param tableDesc
+   * @param responseTime
+   * @param res
+   * @throws Exception
+   */
+  public void printResult(PrintWriter sout, InputStream sin, TableDesc tableDesc,
+                          float responseTime, ResultSet res) throws Exception;
+
+  /**
+   * print no result message
+   * @param sout
+   */
+  public void printNoResult(PrintWriter sout);
+
+  /**
+   * print simple message
+   * @param sout
+   * @param message
+   */
+  public void printMessage(PrintWriter sout, String message);
+
+  /**
+   * print query progress message
+   * @param sout
+   * @param status
+   */
+  public void printProgress(PrintWriter sout, QueryStatus status);
+
+  /**
+   * print error message
+   * @param sout
+   * @param t
+   */
+  public void printErrorMessage(PrintWriter sout, Throwable t);
+
+  /**
+   * print error message
+   * @param sout
+   * @param message
+   */
+  public void printErrorMessage(PrintWriter sout, String message);
+
+  /**
+   * print error message
+   * @param sout
+   * @param queryId
+   */
+  public void printKilledMessage(PrintWriter sout, QueryId queryId);
+
+  /**
+   * print query status error message
+   * @param sout
+   * @param status
+   */
+  void printErrorMessage(PrintWriter sout, QueryStatus status);
+
+  void setScriptMode();
+}

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

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

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.java
new file mode 100644
index 0000000..24276a2
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.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.tsql.commands;
+
+import org.apache.tajo.cli.tsql.TajoCli;
+
+public class CopyrightCommand extends TajoShellCommand {
+  public CopyrightCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\copyright";
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    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" +
+            "  regarding copyright ownership.  The ASF licenses this file\n" +
+            "  to you under the Apache License, Version 2.0 (the\n" +
+            "  \"License\"); you may not use this file except in compliance\n" +
+            "  with the License.  You may obtain a copy of the License at\n" +
+            "\n" +
+            "       http://www.apache.org/licenses/LICENSE-2.0\n" +
+            "\n" +
+            "   Unless required by applicable law or agreed to in writing, software\n" +
+            "   distributed under the License is distributed on an \"AS IS\" BASIS,\n" +
+            "   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.");
+    context.getOutput().println();
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "show Apache License 2.0";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java
new file mode 100644
index 0000000..295d326
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.function.FunctionUtil;
+import org.apache.tajo.cli.tsql.TajoCli;
+
+import java.util.*;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class DescFunctionCommand extends TajoShellCommand {
+  public DescFunctionCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\df";
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    boolean printDetail = false;
+    String functionName = "";
+    if(cmd.length == 0) {
+      throw new IllegalArgumentException();
+    }
+
+    if (cmd.length == 2) {
+      printDetail = true;
+      functionName = cmd[1];
+    }
+
+    List<CatalogProtos.FunctionDescProto> functions =
+        new ArrayList<CatalogProtos.FunctionDescProto>(client.getFunctions(functionName));
+
+    Collections.sort(functions, new Comparator<CatalogProtos.FunctionDescProto>() {
+      @Override
+      public int compare(CatalogProtos.FunctionDescProto f1, CatalogProtos.FunctionDescProto f2) {
+        int nameCompared = f1.getSignature().getName().compareTo(f2.getSignature().getName());
+        if (nameCompared != 0) {
+          return nameCompared;
+        } else {
+          return f1.getSignature().getReturnType().getType().compareTo(f2.getSignature().getReturnType().getType());
+        }
+      }
+    });
+
+    String[] headers = new String[]{"Name", "Result type", "Argument types", "Description", "Type"};
+    float[] columnWidthRates = new float[]{0.15f, 0.15f, 0.2f, 0.4f, 0.1f};
+    int[] columnWidths = printHeader(headers, columnWidthRates);
+
+    for(CatalogProtos.FunctionDescProto eachFunction: functions) {
+      String name = eachFunction.getSignature().getName();
+      String resultDataType = eachFunction.getSignature().getReturnType().getType().toString();
+      String arguments = FunctionUtil.buildParamTypeString(
+          eachFunction.getSignature().getParameterTypesList().toArray(
+              new DataType[eachFunction.getSignature().getParameterTypesCount()]));
+      String functionType = eachFunction.getSignature().getType().toString();
+      String description = eachFunction.getSupplement().getShortDescription();
+
+      int index = 0;
+      printLeft(" " + name, columnWidths[index++]);
+      context.getOutput().print("|");
+      printLeft(" " + resultDataType, columnWidths[index++]);
+      context.getOutput().print("|");
+      printLeft(" " + arguments, columnWidths[index++]);
+      context.getOutput().print("|");
+      printLeft(" " + description, columnWidths[index++]);
+      context.getOutput().print("|");
+      printLeft(" " + functionType, columnWidths[index++]);
+
+      println();
+    }
+
+    println();
+    context.getOutput().println("(" + functions.size() + ") rows");
+    println();
+
+    if (printDetail && !functions.isEmpty()) {
+      Map<String, CatalogProtos.FunctionDescProto> functionMap =
+          new HashMap<String, CatalogProtos.FunctionDescProto>();
+
+      for (CatalogProtos.FunctionDescProto eachFunction: functions) {
+        if (!functionMap.containsKey(eachFunction.getSupplement().getShortDescription())) {
+          functionMap.put(eachFunction.getSupplement().getShortDescription(), eachFunction);
+        }
+      }
+
+      for (CatalogProtos.FunctionDescProto eachFunction: functionMap.values()) {
+        String signature = eachFunction.getSignature().getReturnType().getType() + " " +
+            FunctionUtil.buildSimpleFunctionSignature(eachFunction.getSignature().getName(),
+                eachFunction.getSignature().getParameterTypesList());
+        String fullDescription = eachFunction.getSupplement().getShortDescription();
+        if(eachFunction.getSupplement().getDetail() != null && !eachFunction.getSupplement().getDetail().isEmpty()) {
+          fullDescription += "\n" + eachFunction.getSupplement().getDetail();
+        }
+
+        context.getOutput().println("Function:    " + signature);
+        context.getOutput().println("Description: " + fullDescription);
+        context.getOutput().println("Example:\n" + eachFunction.getSupplement().getExample());
+        println();
+      }
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "[function_name]";
+  }
+
+  @Override
+  public String getDescription() {
+    return "show function description";
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
new file mode 100644
index 0000000..b7d9334
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.commons.lang.CharUtils;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.partition.PartitionMethodDesc;
+import org.apache.tajo.cli.tsql.TajoCli;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.TUtil;
+
+import java.util.List;
+import java.util.Map;
+
+public class DescTableCommand extends TajoShellCommand {
+  public DescTableCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\d";
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    if (cmd.length == 2) {
+      String tableName = cmd[1];
+      tableName = tableName.replace("\"", "");
+      TableDesc desc = client.getTableDesc(tableName);
+      if (desc == null) {
+        context.getOutput().println("Did not find any relation named \"" + tableName + "\"");
+      } else {
+        context.getOutput().println(toFormattedString(desc));
+      }
+    } else if (cmd.length == 1) {
+      List<String> tableList = client.getTableList(null);
+      if (tableList.size() == 0) {
+        context.getOutput().println("No Relation Found");
+      }
+      for (String table : tableList) {
+        context.getOutput().println(table);
+      }
+    } else {
+      throw new IllegalArgumentException();
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "[table_name]";
+  }
+
+  @Override
+  public String getDescription() {
+    return "show table description";
+  }
+
+  protected String toFormattedString(TableDesc desc) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("\ntable name: ").append(desc.getName()).append("\n");
+    sb.append("table path: ").append(desc.getPath()).append("\n");
+    sb.append("store type: ").append(CatalogUtil.getStoreTypeString(desc.getMeta().getStoreType())).append("\n");
+    if (desc.getStats() != null) {
+
+      long row = desc.getStats().getNumRows();
+      String rowText = row == TajoConstants.UNKNOWN_ROW_NUMBER ? "unknown" : row + "";
+      sb.append("number of rows: ").append(rowText).append("\n");
+      sb.append("volume: ").append(
+          FileUtil.humanReadableByteCount(desc.getStats().getNumBytes(),
+              true)).append("\n");
+    }
+    sb.append("Options: \n");
+    for(Map.Entry<String, String> entry : desc.getMeta().toMap().entrySet()){
+
+      /*
+      *  Checks whether the character is ASCII 7 bit printable.
+      *  For example, a printable unicode '\u007c' become the character ‘|’.
+      *
+      *  Control-chars : ctrl-a(\u0001), tab(\u0009) ..
+      *  Printable-chars : '|'(\u007c), ','(\u002c) ..
+      * */
+
+      String value = entry.getValue();
+      String unescaped = StringEscapeUtils.unescapeJava(value);
+      if (unescaped.length() == 1 && CharUtils.isAsciiPrintable(unescaped.charAt(0))) {
+        value = unescaped;
+      }
+      sb.append("\t").append("'").append(entry.getKey()).append("'").append("=")
+          .append("'").append(value).append("'").append("\n");
+    }
+    sb.append("\n");
+    sb.append("schema: \n");
+
+    for(int i = 0; i < desc.getSchema().size(); i++) {
+      Column col = desc.getSchema().getColumn(i);
+      sb.append(col.getSimpleName()).append("\t").append(col.getDataType().getType());
+      if (col.getDataType().hasLength()) {
+        sb.append("(").append(col.getDataType().getLength()).append(")");
+      }
+      sb.append("\n");
+    }
+
+    sb.append("\n");
+    if (desc.getPartitionMethod() != null) {
+      PartitionMethodDesc partition = desc.getPartitionMethod();
+      sb.append("Partitions: \n");
+
+      sb.append("type:").append(partition.getPartitionType().name()).append("\n");
+
+      sb.append("columns:").append(":");
+      sb.append(TUtil.arrayToString(partition.getExpressionSchema().toArray()));
+    }
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java
new file mode 100644
index 0000000..ac97959
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.cli.tsql.TajoCli;
+
+import java.io.*;
+import java.util.concurrent.CountDownLatch;
+
+public class ExecExternalShellCommand extends TajoShellCommand {
+  public ExecExternalShellCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\!";
+  }
+
+  @Override
+  public void invoke(String[] command) throws Exception {
+    StringBuilder shellCommand = new StringBuilder();
+    String prefix = "";
+    for(int i = 1; i < command.length; i++) {
+      shellCommand.append(prefix).append(command[i]);
+      prefix = " ";
+    }
+
+    String builtCommand = shellCommand.toString();
+    if (command.length < 2) {
+      throw new IOException("ERROR: '" + builtCommand + "' is an invalid command.");
+    }
+
+    String[] execCommand = new String[3];
+    execCommand[0] = "/bin/bash";
+    execCommand[1] = "-c";
+    execCommand[2] = builtCommand;
+
+    PrintWriter sout = context.getOutput();
+
+    CountDownLatch latch = new CountDownLatch(2);
+    Process process = Runtime.getRuntime().exec(execCommand);
+    try {
+      InputStreamConsoleWriter inWriter = new InputStreamConsoleWriter(process.getInputStream(), sout, "", latch);
+      InputStreamConsoleWriter errWriter = new InputStreamConsoleWriter(process.getErrorStream(), sout, "ERROR: ", latch);
+
+      inWriter.start();
+      errWriter.start();
+
+      int processResult = process.waitFor();
+      latch.await();
+      if (processResult != 0) {
+        throw new IOException("ERROR: Failed with exit code = " + processResult);
+      }
+    } finally {
+      org.apache.commons.io.IOUtils.closeQuietly(process.getInputStream());
+      org.apache.commons.io.IOUtils.closeQuietly(process.getOutputStream());
+      org.apache.commons.io.IOUtils.closeQuietly(process.getErrorStream());
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "<command> [params]";
+  }
+
+  @Override
+  public String getDescription() {
+    return "executes external shell command in TAJO shell";
+  }
+
+  static class InputStreamConsoleWriter extends Thread {
+    private InputStream in;
+    private PrintWriter writer;
+    private String prefix;
+    private CountDownLatch latch;
+
+    public InputStreamConsoleWriter(InputStream in, PrintWriter writer, String prefix, CountDownLatch latch) {
+      this.in = in;
+      this.writer = writer;
+      this.prefix = prefix;
+      this.latch = latch;
+    }
+
+    @Override
+    public void run() {
+      BufferedReader reader = null;
+      try {
+        reader = new BufferedReader(new InputStreamReader(in));
+        String line;
+        while ((line = reader.readLine()) != null) {
+          writer.println(prefix + line);
+          writer.flush();
+        }
+      } catch (Exception e) {
+        writer.println("ERROR: " + e.getMessage());
+      } finally {
+        if (reader != null) {
+          try {
+            reader.close();
+          } catch (IOException e) {
+          }
+        }
+        latch.countDown();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.java
new file mode 100644
index 0000000..d882b2f
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.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.tsql.commands;
+
+import org.apache.tajo.cli.tsql.TajoCli;
+
+public class ExitCommand extends TajoShellCommand {
+
+  // Sharing the exit command string publicly to filter it out from the command history.
+  public static final String COMMAND_STRING = "\\q";
+
+  public ExitCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return COMMAND_STRING;
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    context.getOutput().println("bye!");
+    System.exit(0);
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "quit";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java
new file mode 100644
index 0000000..8f57b74
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.hadoop.fs.FsShell;
+import org.apache.tajo.cli.tsql.TajoCli;
+
+public class HdfsCommand extends TajoShellCommand {
+  private FsShell fsShell;
+
+  public HdfsCommand(TajoCli.TajoCliContext context) {
+    super(context);
+    fsShell = new FsShell(context.getConf());
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\dfs";
+  }
+
+  @Override
+  public void invoke(String[] command) throws Exception {
+    try {
+      String[] dfsCommands = new String[command.length - 1];
+      System.arraycopy(command, 1, dfsCommands, 0, dfsCommands.length);
+
+      fsShell.run(dfsCommands);
+    } catch (Exception e) {
+      context.getOutput().println("ERROR: " + e.getMessage());
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "<hdfs command> [options]";
+  }
+
+  @Override
+  public String getDescription() {
+    return "executes a dfs command in TAJO shell ";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java
new file mode 100644
index 0000000..ce56d12
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.cli.tsql.TajoCli;
+import org.apache.tajo.util.VersionInfo;
+
+import java.io.PrintWriter;
+
+public class HelpCommand extends TajoShellCommand {
+  private String targetDocVersion = "";
+
+  public HelpCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\?";
+  }
+
+  @Override
+  public String [] getAliases() {
+    return new String [] {"\\help"};
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    if(targetDocVersion.equalsIgnoreCase("")) {
+      targetDocVersion = getDocumentationVersion();
+    }
+
+    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 + "/tsql.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() {
+    String tajoVersion = "", docVersion = "", docDefaultVersion = "current";
+    String tajoFullVersion = VersionInfo.getVersion();
+
+    int delimiterIdx = tajoFullVersion.indexOf("-");
+    if (delimiterIdx > -1) {
+      tajoVersion =  tajoFullVersion.substring(0, delimiterIdx);
+    } else {
+      tajoVersion = tajoFullVersion;
+    }
+    
+    if(tajoVersion.equalsIgnoreCase("")) {
+      docVersion = docDefaultVersion;
+    } else {
+    	docVersion = tajoVersion;
+    }
+
+    return docVersion;
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "show command lists and their usages";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java
new file mode 100644
index 0000000..534bece
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.cli.tsql.TajoCli;
+
+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/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java
new file mode 100644
index 0000000..21c4be5
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import com.google.protobuf.ServiceException;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.cli.tsql.TajoCli;
+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) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    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);
+  }
+
+  public 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) {
+      showAllSessionVars();
+    } else if (cmd.length == 3) {
+      set(cmd[1], cmd[2]);
+    } 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";
+  }
+
+  @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/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java
new file mode 100644
index 0000000..53f66b0
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.cli.tools.TajoAdmin;
+import org.apache.tajo.cli.tsql.TajoCli;
+
+public class TajoAdminCommand extends TajoShellCommand {
+  private TajoAdmin admin;
+
+  public TajoAdminCommand(TajoCli.TajoCliContext context) {
+    super(context);
+    admin = new TajoAdmin(context.getConf(), context.getOutput(), context.getTajoClient());
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\admin";
+  }
+
+  @Override
+  public void invoke(String[] command) throws Exception {
+    try {
+      String[] dfsCommands = new String[command.length - 1];
+      System.arraycopy(command, 1, dfsCommands, 0, dfsCommands.length);
+
+      admin.runCommand(dfsCommands);
+    } catch (Exception e) {
+      context.getOutput().println("ERROR: " + e.getMessage());
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "<command> [options]";
+  }
+
+  @Override
+  public String getDescription() {
+    return "execute a tajo amdin command.";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java
new file mode 100644
index 0000000..5c7dd0e
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.cli.tsql.TajoCli;
+import org.apache.tajo.cli.tools.TajoGetConf;
+
+public class TajoGetConfCommand extends TajoShellCommand {
+  private TajoGetConf getconf;
+
+  public TajoGetConfCommand(TajoCli.TajoCliContext context) {
+    super(context);
+    getconf = new TajoGetConf(context.getConf(), context.getOutput(), context.getTajoClient());
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\getconf";
+  }
+
+  @Override
+  public void invoke(String[] command) throws Exception {
+    try {
+      String[] getConfCommands = new String[command.length - 1];
+      System.arraycopy(command, 1, getConfCommands, 0, getConfCommands.length);
+
+      getconf.runCommand(getConfCommands);
+    } catch (Exception e) {
+      context.getOutput().println("ERROR: " + e.getMessage());
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "<command> [options]";
+  }
+
+  @Override
+  public String getDescription() {
+    return "execute a tajo getconf command.";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java
new file mode 100644
index 0000000..49dee28
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.cli.tools.TajoHAAdmin;
+import org.apache.tajo.cli.tsql.TajoCli;
+
+public class TajoHAAdminCommand extends TajoShellCommand {
+  private TajoHAAdmin haAdmin;
+
+  public TajoHAAdminCommand(TajoCli.TajoCliContext context) {
+    super(context);
+    haAdmin = new TajoHAAdmin(context.getConf(), context.getOutput(), context.getTajoClient());
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\haadmin";
+  }
+
+  @Override
+  public void invoke(String[] command) throws Exception {
+    try {
+      String[] haAdminCommands = new String[command.length - 1];
+      System.arraycopy(command, 1, haAdminCommands, 0, haAdminCommands.length);
+
+      haAdmin.runCommand(haAdminCommands);
+    } catch (Exception e) {
+      context.getOutput().println("ERROR: " + e.getMessage());
+    }
+  }
+
+  @Override
+  public String getUsage() {
+    return "<command> [options]";
+  }
+
+  @Override
+  public String getDescription() {
+    return "execute a tajo haAdminF command.";
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
new file mode 100644
index 0000000..2ac5854
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.cli.tsql.TajoCli;
+
+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.getConf().getIntVar(TajoConf.ConfVars.$CLI_MAX_COLUMN);
+    this.context = context;
+    client = context.getTajoClient();
+  }
+
+  protected void println() {
+    context.getOutput().println();
+  }
+
+  protected void printLeft(String message, int columnWidth) {
+    int messageLength = message.length();
+
+    if(messageLength >= columnWidth) {
+      context.getOutput().print(message.substring(0, columnWidth - 1));
+    } else {
+      context.getOutput().print(message);
+      print(' ', columnWidth - messageLength - 1);
+    }
+  }
+
+  protected void printCenter(String message, int columnWidth, boolean warp) {
+    int messageLength = message.length();
+
+    if(messageLength > columnWidth) {
+      context.getOutput().print(message.substring(0, columnWidth - 1));
+    } else {
+      int numPadding = (columnWidth - messageLength)/2;
+
+      print(' ', numPadding);
+      context.getOutput().print(message);
+      print(' ', numPadding);
+    }
+    if(warp) {
+      println();
+    }
+  }
+
+  protected void printCenter(String message) {
+    printCenter(message, maxColumn, true);
+  }
+
+  protected void print(char c, int count) {
+    for(int i = 0; i < count; i++) {
+      context.getOutput().print(c);
+    }
+  }
+
+  protected int[] printHeader(String[] headers, float[] columnWidthRates) {
+    int[] columnWidths = new int[columnWidthRates.length];
+
+    int columnWidthSum = 0;
+    for(int i = 0; i < columnWidths.length; i++) {
+      columnWidths[i] = (int)(maxColumn * columnWidthRates[i]);
+      if(i > 0) {
+        columnWidthSum += columnWidths[i - 1];
+      }
+    }
+
+    columnWidths[columnWidths.length - 1] = maxColumn - columnWidthSum;
+
+    String prefix = "";
+    for(int i = 0; i < headers.length; i++) {
+      context.getOutput().print(prefix);
+      printLeft(" " + headers[i], columnWidths[i]);
+      prefix = "|";
+    }
+    println();
+
+    int index = 0;
+    int printPos = columnWidths[index] - 1;
+    for(int i = 0; i < maxColumn; i++) {
+      if(i == printPos) {
+        if(index < columnWidths.length - 1) {
+          print('+', 1);
+          index++;
+          printPos += columnWidths[index];
+        }
+      } else {
+        print('-', 1);
+      }
+    }
+
+    println();
+    return columnWidths;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java
new file mode 100644
index 0000000..b540ca1
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import com.google.common.collect.Lists;
+import org.apache.tajo.cli.tsql.TajoCli;
+
+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/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java
new file mode 100644
index 0000000..9c4aa25
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql.commands;
+
+import org.apache.tajo.cli.tsql.TajoCli;
+import org.apache.tajo.util.VersionInfo;
+
+public class VersionCommand extends TajoShellCommand {
+
+  public VersionCommand(TajoCli.TajoCliContext context) {
+    super(context);
+  }
+
+  @Override
+  public String getCommand() {
+    return "\\version";
+  }
+
+  @Override
+  public void invoke(String[] cmd) throws Exception {
+    context.getOutput().println(VersionInfo.getDisplayVersion());
+  }
+
+  @Override
+  public String getUsage() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "show Tajo version";
+  }
+}