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:51 UTC

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

Repository: tajo
Updated Branches:
  refs/heads/master 3f336f500 -> 42d79cf5b


http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java
deleted file mode 100644
index a90baf3..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCliOutputFormatter.java
+++ /dev/null
@@ -1,98 +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.cli.tsql;
-
-import org.apache.tajo.QueryId;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.cli.tsql.TajoCli;
-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-client/src/main/java/org/apache/tajo/cli/tsql/TajoFileHistory.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoFileHistory.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoFileHistory.java
deleted file mode 100644
index c780a77..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoFileHistory.java
+++ /dev/null
@@ -1,41 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java
deleted file mode 100644
index ae644bd..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ConnectDatabaseCommand.java
+++ /dev/null
@@ -1,72 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.java
deleted file mode 100644
index 24276a2..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/CopyrightCommand.java
+++ /dev/null
@@ -1,65 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java
deleted file mode 100644
index 0a7f79b..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescFunctionCommand.java
+++ /dev/null
@@ -1,136 +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.cli.tsql.commands;
-
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.cli.tsql.TajoCli;
-import org.apache.tajo.function.FunctionUtil;
-
-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-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
deleted file mode 100644
index b7d9334..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
+++ /dev/null
@@ -1,137 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java
deleted file mode 100644
index ac97959..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExecExternalShellCommand.java
+++ /dev/null
@@ -1,124 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.java
deleted file mode 100644
index d882b2f..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ExitCommand.java
+++ /dev/null
@@ -1,52 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java
deleted file mode 100644
index 8f57b74..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/HdfsCommand.java
+++ /dev/null
@@ -1,58 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java
deleted file mode 100644
index ce56d12..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/HelpCommand.java
+++ /dev/null
@@ -1,133 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java
deleted file mode 100644
index 534bece..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/ListDatabaseCommand.java
+++ /dev/null
@@ -1,50 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java
deleted file mode 100644
index 21c4be5..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/SetCommand.java
+++ /dev/null
@@ -1,127 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java
deleted file mode 100644
index 53f66b0..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoAdminCommand.java
+++ /dev/null
@@ -1,58 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java
deleted file mode 100644
index 5c7dd0e..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoGetConfCommand.java
+++ /dev/null
@@ -1,58 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java
deleted file mode 100644
index 9010ccb..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoHAAdminCommand.java
+++ /dev/null
@@ -1,58 +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.cli.tsql.commands;
-
-import org.apache.tajo.cli.tsql.TajoCli;
-import org.apache.tajo.cli.tools.TajoHAAdmin;
-
-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-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
deleted file mode 100644
index 361f04d..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/TajoShellCommand.java
+++ /dev/null
@@ -1,129 +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.cli.tsql.commands;
-
-import org.apache.tajo.cli.tsql.TajoCli;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.conf.TajoConf;
-
-public abstract class TajoShellCommand {
-  public abstract String getCommand();
-  public String [] getAliases() {
-    return new String[] {};
-  }
-  public abstract void invoke(String [] command) throws Exception;
-  public abstract String getUsage();
-  public abstract String getDescription();
-  public void printHelp() {
-    context.getOutput().print(getCommand());
-    context.getOutput().print(" - ");
-    context.getOutput().println(getDescription());
-  }
-
-  protected TajoCli.TajoCliContext context;
-  protected TajoClient client;
-  protected int maxColumn;
-
-  public TajoShellCommand(TajoCli.TajoCliContext context) {
-    maxColumn = context.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java
deleted file mode 100644
index b540ca1..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/UnsetCommand.java
+++ /dev/null
@@ -1,53 +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.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-client/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java
deleted file mode 100644
index 9c4aa25..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/commands/VersionCommand.java
+++ /dev/null
@@ -1,49 +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.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";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/client/TajoHAClientUtil.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoHAClientUtil.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoHAClientUtil.java
deleted file mode 100644
index 7267b10..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoHAClientUtil.java
+++ /dev/null
@@ -1,87 +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.
- */
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.client;
-
-import com.google.protobuf.ServiceException;
-import org.apache.tajo.cli.tsql.TajoCli.TajoCliContext;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ha.HAServiceUtil;
-
-import java.io.IOException;
-
-public class TajoHAClientUtil {
-  /**
-   * In TajoMaster HA mode, if TajoCli can't connect existing active master,
-   * this should try to connect new active master.
-   *
-   * @param conf
-   * @param client
-   * @return
-   * @throws IOException
-   * @throws ServiceException
-   */
-  public static TajoClient getTajoClient(TajoConf conf, TajoClient client)
-      throws IOException, ServiceException {
-    return getTajoClient(conf, client, null);
-  }
-
-  public static TajoClient getTajoClient(TajoConf conf, TajoClient client,
-      TajoCliContext context) throws IOException, ServiceException {
-
-    if (conf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
-
-      if (!HAServiceUtil.isMasterAlive(conf.getVar(
-        TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS), conf)) {
-        TajoClient tajoClient = null;
-        String baseDatabase = client.getBaseDatabase();
-        conf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS,
-          HAServiceUtil.getMasterClientName(conf));
-        client.close();
-        tajoClient = new TajoClientImpl(conf, baseDatabase);
-
-        if (context != null && context.getCurrentDatabase() != null) {
-          tajoClient.selectDatabase(context.getCurrentDatabase());
-        }
-        return tajoClient;
-      } else {
-        return client;
-      }
-    } else {
-      return client;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-core/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml
index c713e09..ce9db73 100644
--- a/tajo-core/pom.xml
+++ b/tajo-core/pom.xml
@@ -269,6 +269,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-cli</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
       <artifactId>tajo-jdbc</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
index da25fe6..3603b79 100644
--- a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
+++ b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
@@ -293,8 +293,6 @@ public class QueryExecutorServlet extends HttpServlet {
     public void run() {
       startTime = System.currentTimeMillis();
       try {
-        tajoClient = TajoHAClientUtil.getTajoClient(tajoConf, tajoClient);
-
         if (!tajoClient.getCurrentDatabase().equals(database))
           tajoClient.selectDatabase(database);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index 3df2681..2ddc517 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -133,6 +133,7 @@
                       run cp -r $ROOT/tajo-algebra/target/tajo-algebra-${project.version}/* .
                       run cp -r $ROOT/tajo-plan/target/tajo-plan-${project.version}/* .
                       run cp -r $ROOT/tajo-client/target/tajo-client-${project.version}/* .
+                      run cp -r $ROOT/tajo-cli/target/tajo-cli-${project.version}/* .
                       run cp -r $ROOT/tajo-catalog/target/tajo-catalog-${project.version}/* .
                       run cp -r $ROOT/tajo-storage/target/tajo-storage-${project.version}/* .
                       run cp -r $ROOT/tajo-pullserver/target/tajo-pullserver-${project.version}.jar .

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index ad62bba..30f864c 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -773,6 +773,11 @@
       </dependency>
       <dependency>
         <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-cli</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
         <artifactId>tajo-jdbc</artifactId>
         <version>${tajo.version}</version>
       </dependency>


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

Posted by hy...@apache.org.
TAJO-1269: Separate cli from tajo-client.

Closes #374


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

Branch: refs/heads/master
Commit: 42d79cf5b5dae99cd8af3b1244db1f709d4268e6
Parents: 3f336f5
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Feb 5 00:58:43 2015 -0800
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Feb 5 00:58:43 2015 -0800

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 pom.xml                                         |   1 +
 tajo-cli/pom.xml                                | 337 +++++++++
 .../org/apache/tajo/cli/tools/TajoAdmin.java    | 459 ++++++++++++
 .../org/apache/tajo/cli/tools/TajoDump.java     | 202 ++++++
 .../org/apache/tajo/cli/tools/TajoGetConf.java  | 161 +++++
 .../org/apache/tajo/cli/tools/TajoHAAdmin.java  | 210 ++++++
 .../cli/tsql/DefaultTajoCliOutputFormatter.java | 211 ++++++
 .../cli/tsql/InvalidStatementException.java     |  25 +
 .../org/apache/tajo/cli/tsql/ParsedResult.java  |  53 ++
 .../org/apache/tajo/cli/tsql/SimpleParser.java  | 387 ++++++++++
 .../java/org/apache/tajo/cli/tsql/TajoCli.java  | 686 ++++++++++++++++++
 .../tajo/cli/tsql/TajoCliOutputFormatter.java   |  97 +++
 .../apache/tajo/cli/tsql/TajoFileHistory.java   |  41 ++
 .../tsql/commands/ConnectDatabaseCommand.java   |  72 ++
 .../cli/tsql/commands/CopyrightCommand.java     |  65 ++
 .../cli/tsql/commands/DescFunctionCommand.java  | 136 ++++
 .../cli/tsql/commands/DescTableCommand.java     | 137 ++++
 .../tsql/commands/ExecExternalShellCommand.java | 124 ++++
 .../tajo/cli/tsql/commands/ExitCommand.java     |  52 ++
 .../tajo/cli/tsql/commands/HdfsCommand.java     |  58 ++
 .../tajo/cli/tsql/commands/HelpCommand.java     | 133 ++++
 .../cli/tsql/commands/ListDatabaseCommand.java  |  50 ++
 .../tajo/cli/tsql/commands/SetCommand.java      | 127 ++++
 .../cli/tsql/commands/TajoAdminCommand.java     |  58 ++
 .../cli/tsql/commands/TajoGetConfCommand.java   |  58 ++
 .../cli/tsql/commands/TajoHAAdminCommand.java   |  58 ++
 .../cli/tsql/commands/TajoShellCommand.java     | 129 ++++
 .../tajo/cli/tsql/commands/UnsetCommand.java    |  53 ++
 .../tajo/cli/tsql/commands/VersionCommand.java  |  49 ++
 .../org/apache/tajo/cli/tools/TajoAdmin.java    | 457 ------------
 .../org/apache/tajo/cli/tools/TajoDump.java     | 202 ------
 .../org/apache/tajo/cli/tools/TajoGetConf.java  | 161 -----
 .../org/apache/tajo/cli/tools/TajoHAAdmin.java  | 211 ------
 .../cli/tsql/DefaultTajoCliOutputFormatter.java | 211 ------
 .../cli/tsql/InvalidStatementException.java     |  25 -
 .../org/apache/tajo/cli/tsql/ParsedResult.java  |  53 --
 .../org/apache/tajo/cli/tsql/SimpleParser.java  | 388 ----------
 .../java/org/apache/tajo/cli/tsql/TajoCli.java  | 701 -------------------
 .../tajo/cli/tsql/TajoCliOutputFormatter.java   |  98 ---
 .../apache/tajo/cli/tsql/TajoFileHistory.java   |  41 --
 .../tsql/commands/ConnectDatabaseCommand.java   |  72 --
 .../cli/tsql/commands/CopyrightCommand.java     |  65 --
 .../cli/tsql/commands/DescFunctionCommand.java  | 136 ----
 .../cli/tsql/commands/DescTableCommand.java     | 137 ----
 .../tsql/commands/ExecExternalShellCommand.java | 124 ----
 .../tajo/cli/tsql/commands/ExitCommand.java     |  52 --
 .../tajo/cli/tsql/commands/HdfsCommand.java     |  58 --
 .../tajo/cli/tsql/commands/HelpCommand.java     | 133 ----
 .../cli/tsql/commands/ListDatabaseCommand.java  |  50 --
 .../tajo/cli/tsql/commands/SetCommand.java      | 127 ----
 .../cli/tsql/commands/TajoAdminCommand.java     |  58 --
 .../cli/tsql/commands/TajoGetConfCommand.java   |  58 --
 .../cli/tsql/commands/TajoHAAdminCommand.java   |  58 --
 .../cli/tsql/commands/TajoShellCommand.java     | 129 ----
 .../tajo/cli/tsql/commands/UnsetCommand.java    |  53 --
 .../tajo/cli/tsql/commands/VersionCommand.java  |  49 --
 .../apache/tajo/client/TajoHAClientUtil.java    |  87 ---
 tajo-core/pom.xml                               |   4 +
 .../tajo/webapp/QueryExecutorServlet.java       |   2 -
 tajo-dist/pom.xml                               |   1 +
 tajo-project/pom.xml                            |   5 +
 62 files changed, 4241 insertions(+), 3996 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 48f436f..06ced2d 100644
--- a/CHANGES
+++ b/CHANGES
@@ -29,6 +29,8 @@ Release 0.10.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1269: Separate cli from tajo-client. (hyunsik)
+
     TAJO-1328: Fix deprecated property names in the catalog configuration
     document. (jihun)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8e5dd5e..7793550 100644
--- a/pom.xml
+++ b/pom.xml
@@ -92,6 +92,7 @@
     <module>tajo-pullserver</module>
     <module>tajo-dist</module>
     <module>tajo-thirdparty/asm</module>
+    <module>tajo-cli</module>
   </modules>
 
   <build>

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-cli/pom.xml b/tajo-cli/pom.xml
new file mode 100644
index 0000000..36e6118
--- /dev/null
+++ b/tajo-cli/pom.xml
@@ -0,0 +1,337 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.10.0-SNAPSHOT</version>
+    <relativePath>../tajo-project</relativePath>
+  </parent>
+  <artifactId>tajo-cli</artifactId>
+  <packaging>jar</packaging>
+  <name>Tajo CLI tools</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    <metrics.version>3.0.1</metrics.version>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.2</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-dependencies</id>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeScope>runtime</includeScope>
+              <outputDirectory>${project.build.directory}/lib</outputDirectory>
+              <overWriteReleases>false</overWriteReleases>
+              <overWriteSnapshots>false</overWriteSnapshots>
+              <overWriteIfNewer>true</overWriteIfNewer>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-pmd-plugin</artifactId>
+        <version>2.7.1</version>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-rpc</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+      <version>1.2</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>protobuf-java</artifactId>
+          <groupId>com.google.protobuf</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <version>2.4</version>
+    </dependency>
+    <dependency>
+      <groupId>jline</groupId>
+      <artifactId>jline</artifactId>
+    </dependency>
+  </dependencies>
+
+
+  <profiles>
+    <profile>
+      <id>all-dependencies</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <exclusions>
+            <exclusion>
+              <artifactId>protobuf-java</artifactId>
+              <groupId>com.google.protobuf</groupId>
+            </exclusion>
+            <exclusion>
+              <artifactId>hadoop-mapreduce-client-app</artifactId>
+              <groupId>org.apache.hadoop</groupId>
+            </exclusion>
+            <exclusion>
+              <artifactId>hadoop-yarn-api</artifactId>
+              <groupId>org.apache.hadoop</groupId>
+            </exclusion>
+            <exclusion>
+              <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+              <groupId>org.apache.hadoop</groupId>
+            </exclusion>
+            <exclusion>
+              <artifactId>hadoop-mapreduce-client-core</artifactId>
+              <groupId>org.apache.hadoop</groupId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
+
+
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+        <property>
+          <name>tar|rpm|deb</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                    <echo file="${project.build.directory}/dist-layout-stitching.sh">
+                      run() {
+                      echo "\$ ${@}"
+                      "${@}"
+                      res=$?
+                      if [ $res != 0 ]; then
+                      echo
+                      echo "Failed!"
+                      echo
+                      exit $res
+                      fi
+                      }
+
+                      ROOT=`cd ${basedir}/..;pwd`
+                      echo
+                      echo "Current directory `pwd`"
+                      echo
+                      run rm -rf ${project.artifactId}-${project.version}
+                      run mkdir ${project.artifactId}-${project.version}
+                      run cd ${project.artifactId}-${project.version}
+                      run cp -r ${basedir}/target/${project.artifactId}-${project.version}*.jar .
+                      echo
+                      echo "Tajo Client dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo
+                    </echo>
+                    <exec executable="sh" dir="${project.build.directory}" failonerror="true">
+                      <arg line="./dist-layout-stitching.sh" />
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+        <version>2.4</version>
+        <configuration>
+          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java
new file mode 100644
index 0000000..18b2d99
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java
@@ -0,0 +1,459 @@
+/**
+ * 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.tools;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.cli.*;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.client.*;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ClientProtos.BriefQueryInfo;
+import org.apache.tajo.ipc.ClientProtos.WorkerResourceInfo;
+import org.apache.tajo.service.ServiceTracker;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.ha.HAServiceUtil;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.net.InetSocketAddress;
+import java.sql.SQLException;
+import java.text.DecimalFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TajoAdmin {
+  private static final org.apache.commons.cli.Options options;
+  private static DecimalFormat decimalF = new DecimalFormat("###.0");
+  private enum WorkerStatus {
+    RUNNING,
+    LOST,
+    DECOMMISSIONED
+  }
+
+  final static String DASHLINE_LEN5 = "-----";
+  final static String DASHLINE_LEN10 = "----------";
+  final static String DASHLINE_LEN12 = "------------";
+  final static String DASHLINE_LEN25 = "-------------------------";
+  final static String DATE_FORMAT  = "yyyy-MM-dd HH:mm:ss";
+
+  static {
+    options = new Options();
+    options.addOption("h", "host", true, "Tajo server host");
+    options.addOption("p", "port", true, "Tajo server port");
+    options.addOption("list", null, false, "Show Tajo query list");
+    options.addOption("cluster", null, false, "Show Cluster Info");
+    options.addOption("showmasters", null, false, "gets list of tajomasters in the cluster");
+    options.addOption("desc", null, false, "Show Query Description");
+    options.addOption("kill", null, true, "Kill a running query");
+  }
+
+  private TajoConf tajoConf;
+  private TajoClient tajoClient;
+  private ServiceTracker serviceTracker;
+  private Writer writer;
+
+  public TajoAdmin(TajoConf tajoConf, Writer writer) {
+    this(tajoConf, writer, null);
+  }
+
+  public TajoAdmin(TajoConf tajoConf, Writer writer, TajoClient tajoClient) {
+    this.tajoConf = tajoConf;
+    this.writer = writer;
+    this.tajoClient = tajoClient;
+  }
+
+  private void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp( "admin [options]", options );
+  }
+
+  public void runCommand(String[] args) throws Exception {
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd = parser.parse(options, args);
+
+    String param = "";
+    int cmdType = 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 queryId = null;
+
+    if (cmd.hasOption("list")) {
+      cmdType = 1;
+    } else if (cmd.hasOption("desc")) {
+      cmdType = 2;
+    } else if (cmd.hasOption("cluster")) {
+      cmdType = 3;
+    } else if (cmd.hasOption("kill")) {
+      cmdType = 4;
+      queryId = cmd.getOptionValue("kill");
+    } else if (cmd.hasOption("showmasters")) {
+      cmdType = 5;
+    }
+
+    // if there is no "-h" option,
+    if(hostName == null) {
+      if (tajoConf.getVar(TajoConf.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 = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
+      }
+    }
+    if (port == null) {
+      if (tajoConf.getVar(TajoConf.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(tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
+      }
+    }
+
+    if (cmdType == 0) {
+      printUsage();
+      return;
+    }
+
+
+    if ((hostName == null) ^ (port == null)) {
+      System.err.println("ERROR: cannot find valid Tajo server address");
+      return;
+    } else if (hostName != null && port != null) {
+      tajoConf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName + ":" + port);
+      tajoClient = new TajoClientImpl(tajoConf);
+    } else if (hostName == null && port == null) {
+      tajoClient = new TajoClientImpl(tajoConf);
+    }
+
+    switch (cmdType) {
+      case 1:
+        processList(writer);
+        break;
+      case 2:
+        processDesc(writer);
+        break;
+      case 3:
+        processCluster(writer);
+        break;
+      case 4:
+        processKill(writer, queryId);
+        break;
+      case 5:
+        processMasters(writer);
+        break;
+      default:
+        printUsage();
+        break;
+    }
+
+    writer.flush();
+  }
+
+  private void processDesc(Writer writer) throws ParseException, IOException,
+      ServiceException, SQLException {
+
+    List<BriefQueryInfo> queryList = tajoClient.getRunningQueryList();
+    SimpleDateFormat df = new SimpleDateFormat(DATE_FORMAT);
+    int id = 1;
+    for (BriefQueryInfo queryInfo : queryList) {
+        String queryId = String.format("q_%s_%04d",
+                                       queryInfo.getQueryId().getId(),
+                                       queryInfo.getQueryId().getSeq());
+
+        writer.write("Id: " + id);
+        writer.write("\n");
+        id++;
+        writer.write("Query Id: " + queryId);
+        writer.write("\n");
+        writer.write("Started Time: " + df.format(queryInfo.getStartTime()));
+        writer.write("\n");
+
+        writer.write("Query State: " + queryInfo.getState().name());
+        writer.write("\n");
+        long end = queryInfo.getFinishTime();
+        long start = queryInfo.getStartTime();
+        String executionTime = decimalF.format((end-start) / 1000) + " sec";
+        if (TajoClientUtil.isQueryComplete(queryInfo.getState())) {
+          writer.write("Finished Time: " + df.format(queryInfo.getFinishTime()));
+          writer.write("\n");
+        }
+        writer.write("Execution Time: " + executionTime);
+        writer.write("\n");
+        writer.write("Query Progress: " + queryInfo.getProgress());
+        writer.write("\n");
+        writer.write("Query Statement:");
+        writer.write("\n");
+        writer.write(queryInfo.getQuery());
+        writer.write("\n");
+        writer.write("\n");
+    }
+  }
+
+  private void processCluster(Writer writer) throws ParseException, IOException,
+      ServiceException, SQLException {
+
+    List<WorkerResourceInfo> workerList = tajoClient.getClusterInfo();
+
+    int runningQueryMasterTasks = 0;
+
+    List<WorkerResourceInfo> liveWorkers = new ArrayList<WorkerResourceInfo>();
+    List<WorkerResourceInfo> deadWorkers = new ArrayList<WorkerResourceInfo>();
+    List<WorkerResourceInfo> decommissionWorkers = new ArrayList<WorkerResourceInfo>();
+
+    List<WorkerResourceInfo> liveQueryMasters = new ArrayList<WorkerResourceInfo>();
+    List<WorkerResourceInfo> deadQueryMasters = new ArrayList<WorkerResourceInfo>();
+
+    for (WorkerResourceInfo eachWorker : workerList) {
+      if(eachWorker.getQueryMasterMode() == true) {
+        if(eachWorker.getWorkerStatus().equals(WorkerStatus.RUNNING.toString())) {
+          liveQueryMasters.add(eachWorker);
+          runningQueryMasterTasks += eachWorker.getNumQueryMasterTasks();
+        }
+        if(eachWorker.getWorkerStatus().equals(WorkerStatus.LOST.toString())) {
+          deadQueryMasters.add(eachWorker);
+        }
+      }
+
+      if(eachWorker.getTaskRunnerMode() == true) {
+        if(eachWorker.getWorkerStatus().equals(WorkerStatus.RUNNING.toString())) {
+          liveWorkers.add(eachWorker);
+        } else if(eachWorker.getWorkerStatus().equals(WorkerStatus.LOST.toString())) {
+          deadWorkers.add(eachWorker);
+        } else if(eachWorker.getWorkerStatus().equals(WorkerStatus.DECOMMISSIONED.toString())) {
+          decommissionWorkers.add(eachWorker);
+        }
+      }
+    }
+
+    String fmtInfo = "%1$-5s %2$-5s %3$-5s%n";
+    String infoLine = String.format(fmtInfo, "Live", "Dead", "Tasks");
+
+    writer.write("Query Master\n");
+    writer.write("============\n\n");
+    writer.write(infoLine);
+    String line = String.format(fmtInfo, DASHLINE_LEN5, DASHLINE_LEN5, DASHLINE_LEN5);
+    writer.write(line);
+
+    line = String.format(fmtInfo, liveQueryMasters.size(),
+                         deadQueryMasters.size(), runningQueryMasterTasks);
+    writer.write(line);
+    writer.write("\n");
+
+    writer.write("Live QueryMasters\n");
+    writer.write("=================\n\n");
+
+    if (liveQueryMasters.isEmpty()) {
+      writer.write("No Live QueryMasters\n");
+    } else {
+      String fmtQueryMasterLine = "%1$-25s %2$-5s %3$-5s %4$-10s %5$-10s%n";
+      line = String.format(fmtQueryMasterLine, "QueryMaster", "Port", "Query",
+                           "Heap", "Status");
+      writer.write(line);
+      line = String.format(fmtQueryMasterLine, DASHLINE_LEN25, DASHLINE_LEN5,
+              DASHLINE_LEN5, DASHLINE_LEN10, DASHLINE_LEN10);
+      writer.write(line);
+      for (WorkerResourceInfo queryMaster : liveQueryMasters) {
+        TajoProtos.WorkerConnectionInfoProto connInfo = queryMaster.getConnectionInfo();
+        String queryMasterHost = String.format("%s:%d", connInfo.getHost(), connInfo.getQueryMasterPort());
+        String heap = String.format("%d MB", queryMaster.getMaxHeap() / 1024 / 1024);
+        line = String.format(fmtQueryMasterLine,
+            queryMasterHost,
+            connInfo.getClientPort(),
+            queryMaster.getNumQueryMasterTasks(),
+            heap,
+            queryMaster.getWorkerStatus());
+        writer.write(line);
+      }
+
+      writer.write("\n\n");
+    }
+
+    if (!deadQueryMasters.isEmpty()) {
+      writer.write("Dead QueryMasters\n");
+      writer.write("=================\n\n");
+
+      String fmtQueryMasterLine = "%1$-25s %2$-5s %3$-10s%n";
+      line = String.format(fmtQueryMasterLine, "QueryMaster", "Port", "Status");
+      writer.write(line);
+      line = String.format(fmtQueryMasterLine, DASHLINE_LEN25, DASHLINE_LEN5, DASHLINE_LEN10);
+      writer.write(line);
+
+      for (WorkerResourceInfo queryMaster : deadQueryMasters) {
+        TajoProtos.WorkerConnectionInfoProto connInfo = queryMaster.getConnectionInfo();
+        String queryMasterHost = String.format("%s:%d", connInfo.getHost(), connInfo.getQueryMasterPort());
+        line = String.format(fmtQueryMasterLine,
+            queryMasterHost,
+            connInfo.getClientPort(),
+            queryMaster.getWorkerStatus());
+        writer.write(line);
+      }
+
+      writer.write("\n\n");
+    }
+
+    writer.write("Worker\n");
+    writer.write("======\n\n");
+
+    String fmtWorkerInfo = "%1$-5s %2$-5s%n";
+    String workerInfoLine = String.format(fmtWorkerInfo, "Live", "Dead");
+    writer.write(workerInfoLine);
+    line = String.format(fmtWorkerInfo, DASHLINE_LEN5, DASHLINE_LEN5);
+    writer.write(line);
+
+    line = String.format(fmtWorkerInfo, liveWorkers.size(), deadWorkers.size());
+    writer.write(line);
+    writer.write("\n");
+
+    writer.write("Live Workers\n");
+    writer.write("============\n\n");
+    if(liveWorkers.isEmpty()) {
+      writer.write("No Live Workers\n\n");
+    } else {
+      writeWorkerInfo(writer, liveWorkers);
+    }
+
+    writer.write("Dead Workers\n");
+    writer.write("============\n\n");
+    if(deadWorkers.isEmpty()) {
+      writer.write("No Dead Workers\n\n");
+    } else {
+      writeWorkerInfo(writer, deadWorkers);
+    }
+  }
+
+  private void writeWorkerInfo(Writer writer, List<WorkerResourceInfo> workers) throws ParseException,
+      IOException, ServiceException, SQLException {
+    String fmtWorkerLine = "%1$-25s %2$-5s %3$-5s %4$-10s %5$-10s %6$-12s %7$-10s%n";
+    String line = String.format(fmtWorkerLine,
+        "Worker", "Port", "Tasks",
+        "Mem", "Disk",
+        "Heap", "Status");
+    writer.write(line);
+    line = String.format(fmtWorkerLine,
+        DASHLINE_LEN25, DASHLINE_LEN5, DASHLINE_LEN5,
+        DASHLINE_LEN10, DASHLINE_LEN10,
+        DASHLINE_LEN12, DASHLINE_LEN10);
+    writer.write(line);
+
+    for (WorkerResourceInfo worker : workers) {
+      TajoProtos.WorkerConnectionInfoProto connInfo = worker.getConnectionInfo();
+      String workerHost = String.format("%s:%d", connInfo.getHost(), connInfo.getPeerRpcPort());
+      String mem = String.format("%d/%d", worker.getUsedMemoryMB(),
+          worker.getMemoryMB());
+      String disk = String.format("%.2f/%.2f", worker.getUsedDiskSlots(),
+          worker.getDiskSlots());
+      String heap = String.format("%d/%d MB", worker.getFreeHeap()/1024/1024,
+          worker.getMaxHeap()/1024/1024);
+
+      line = String.format(fmtWorkerLine, workerHost,
+          connInfo.getPullServerPort(),
+          worker.getNumRunningTasks(),
+          mem, disk, heap, worker.getWorkerStatus());
+      writer.write(line);
+    }
+    writer.write("\n\n");
+  }
+
+  private void processList(Writer writer) throws ParseException, IOException,
+      ServiceException, SQLException {
+
+    List<BriefQueryInfo> queryList = tajoClient.getRunningQueryList();
+    SimpleDateFormat df = new SimpleDateFormat(DATE_FORMAT);
+    StringBuilder builder = new StringBuilder();
+
+    /* print title */
+    builder.append(StringUtils.rightPad("QueryId", 21));
+    builder.append(StringUtils.rightPad("State", 20));
+    builder.append(StringUtils.rightPad("StartTime", 20));
+    builder.append(StringUtils.rightPad("Query", 30)).append("\n");
+
+    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 20), 21));
+    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 19), 20));
+    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 19), 20));
+    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 29), 30)).append("\n");
+    writer.write(builder.toString());
+
+    builder = new StringBuilder();
+    for (BriefQueryInfo queryInfo : queryList) {
+      builder.append(StringUtils.rightPad(new QueryId(queryInfo.getQueryId()).toString(), 21));
+      builder.append(StringUtils.rightPad(queryInfo.getState().name(), 20));
+      builder.append(StringUtils.rightPad(df.format(queryInfo.getStartTime()), 20));
+      builder.append(StringUtils.abbreviate(queryInfo.getQuery(), 30)).append("\n");
+    }
+    writer.write(builder.toString());
+  }
+
+  public void processKill(Writer writer, String queryIdStr)
+      throws IOException, ServiceException {
+    QueryStatus status = tajoClient.killQuery(TajoIdUtils.parseQueryId(queryIdStr));
+    if (status.getState() == TajoProtos.QueryState.QUERY_KILLED) {
+      writer.write(queryIdStr + " is killed successfully.\n");
+    } else if (status.getState() == TajoProtos.QueryState.QUERY_KILL_WAIT) {
+      writer.write(queryIdStr + " will be finished after a while.\n");
+    } else {
+      writer.write("ERROR:" + status.getErrorMessage());
+    }
+  }
+
+  private void processMasters(Writer writer) throws ParseException, IOException,
+      ServiceException, SQLException {
+
+    if (tajoConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+
+      List<String> list = HAServiceUtil.getMasters(tajoConf);
+      int i = 0;
+      for (String master : list) {
+        if (i > 0) {
+          writer.write(" ");
+        }
+        writer.write(master);
+        i++;
+      }
+      writer.write("\n");
+    } else {
+      String confMasterServiceAddr = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS);
+      InetSocketAddress masterAddress = NetUtils.createSocketAddr(confMasterServiceAddr);
+      writer.write(masterAddress.getHostName());
+      writer.write("\n");
+    }
+  }
+
+  public static void main(String [] args) throws Exception {
+    TajoConf conf = new TajoConf();
+
+    Writer writer = new PrintWriter(System.out);
+    try {
+      TajoAdmin admin = new TajoAdmin(conf, writer);
+      admin.runCommand(args);
+    } finally {
+      writer.close();
+      System.exit(0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
new file mode 100644
index 0000000..7f38a5d
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
@@ -0,0 +1,202 @@
+/**
+ * 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.tools;
+
+import com.google.protobuf.ServiceException;
+
+import org.apache.commons.cli.*;
+import org.apache.tajo.auth.UserRoleInfo;
+import org.apache.tajo.catalog.CatalogConstants;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.DDLBuilder;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.util.Pair;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.sql.SQLException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.List;
+
+public class TajoDump {
+  private static final org.apache.commons.cli.Options options;
+
+  static {
+    options = new Options();
+    options.addOption("h", "host", true, "Tajo server host");
+    options.addOption("p", "port", true, "Tajo server port");
+    options.addOption("a", "all", false, "dump all table DDLs");
+  }
+
+  private static void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp( "tajo-dump [options] [database name]", options);
+  }
+
+  private static Pair<String, Integer> getConnectionAddr(TajoConf conf, CommandLine cmd) {
+    String hostName = null;
+    Integer port = null;
+    if (cmd.hasOption("h")) {
+      hostName = cmd.getOptionValue("h");
+    }
+    if (cmd.hasOption("p")) {
+      port = Integer.parseInt(cmd.getOptionValue("p"));
+    }
+
+    if(hostName == null) {
+      if (conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != null) {
+        hostName = conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
+      }
+    }
+    if (port == null) {
+      if (conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != null) {
+        port = Integer.parseInt(conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
+      }
+    }
+    return new Pair<String, Integer>(hostName, port);
+  }
+
+  public static void main(String [] args) throws ParseException, IOException, ServiceException, SQLException {
+    final TajoConf conf = new TajoConf();
+    final CommandLineParser parser = new PosixParser();
+    final CommandLine cmd = parser.parse(options, args);
+    final Pair<String, Integer> hostAndPort = getConnectionAddr(conf, cmd);
+    final String hostName = hostAndPort.getFirst();
+    final Integer port = hostAndPort.getSecond();
+    final UserRoleInfo userInfo = UserRoleInfo.getCurrentUser();
+
+    String baseDatabaseName = null;
+    if (cmd.getArgList().size() > 0) {
+      baseDatabaseName = (String) cmd.getArgList().get(0);
+    }
+
+    boolean isDumpingAllDatabases = cmd.hasOption('a');
+
+    // Neither two choices
+    if (!isDumpingAllDatabases && baseDatabaseName == null) {
+      printUsage();
+      System.exit(-1);
+    }
+
+    TajoClient client = null;
+    if ((hostName == null) ^ (port == null)) {
+      System.err.println("ERROR: cannot find any TajoMaster rpc address in arguments and tajo-site.xml.");
+      System.exit(-1);
+    } else if (hostName != null && port != null) {
+      conf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName+":"+port);
+      client = new TajoClientImpl(conf);
+    } else {
+      client = new TajoClientImpl(conf);
+    }
+
+    PrintWriter writer = new PrintWriter(System.out);
+    dump(client, userInfo, baseDatabaseName, isDumpingAllDatabases, true, true, writer);
+
+    System.exit(0);
+  }
+  
+  private static boolean isAcceptableDumpingDatabase(String databaseName) {
+    return (databaseName == null || !databaseName.equalsIgnoreCase(CatalogConstants.INFORMATION_SCHEMA_DB_NAME));
+  }
+
+  public static void dump(TajoClient client, UserRoleInfo userInfo, String baseDatabaseName,
+                   boolean isDumpingAllDatabases, boolean includeUserName, boolean includeDate, PrintWriter out)
+      throws SQLException, ServiceException {
+    printHeader(out, userInfo, includeUserName, includeDate);
+
+    if (isDumpingAllDatabases) {
+      // sort database names in an ascending lexicographic order of the names.
+      List<String> sorted = new ArrayList<String>(client.getAllDatabaseNames());
+      Collections.sort(sorted);
+
+      for (String databaseName : sorted) {
+        if (isAcceptableDumpingDatabase(databaseName)) {
+          dumpDatabase(client, databaseName, out);
+        }
+      }
+    } else {
+      dumpDatabase(client, baseDatabaseName, out);
+    }
+    out.flush();
+  }
+
+  private static void printHeader(PrintWriter writer, UserRoleInfo userInfo, boolean includeUSerName,
+                                  boolean includeDate) {
+    writer.write("--\n");
+    writer.write("-- Tajo database dump\n");
+    if (includeUSerName) {
+      writer.write("--\n-- Dump user: " + userInfo.getUserName() + "\n");
+    }
+    if (includeDate) {
+      writer.write("--\n-- Dump date: " + toDateString() + "\n");
+    }
+    writer.write("--\n");
+    writer.write("\n");
+  }
+
+  private static void dumpDatabase(TajoClient client, String databaseName, PrintWriter writer)
+      throws SQLException, ServiceException {
+    writer.write("\n");
+    writer.write("--\n");
+    writer.write(String.format("-- Database name: %s%n", CatalogUtil.denormalizeIdentifier(databaseName)));
+    writer.write("--\n");
+    writer.write("\n");
+    writer.write(String.format("CREATE DATABASE IF NOT EXISTS %s;", CatalogUtil.denormalizeIdentifier(databaseName)));
+    writer.write("\n\n");
+
+    // returned list is immutable.
+    List<String> tableNames = TUtil.newList(client.getTableList(databaseName));
+    Collections.sort(tableNames);
+    for (String tableName : tableNames) {
+      try {
+        TableDesc table = client.getTableDesc(CatalogUtil.buildFQName(databaseName, tableName));
+        
+        if (table.getMeta().getStoreType() == StoreType.SYSTEM) {
+          continue;
+        }
+        
+        if (table.isExternal()) {
+          writer.write(DDLBuilder.buildDDLForExternalTable(table));
+        } else {
+          writer.write(DDLBuilder.buildDDLForBaseTable(table));
+        }
+        writer.write("\n\n");
+      } catch (Exception e) {
+        // dump for each table can throw any exception. We need to skip the exception case.
+        // here, the error message prints out via stderr.
+        System.err.println("ERROR:" + tableName + "," + e.getMessage());
+      }
+    }
+  }
+
+  private static String toDateString() {
+    DateFormat df = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss");
+    java.util.Date today = Calendar.getInstance().getTime();
+    return df.format(today);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java
new file mode 100644
index 0000000..aa7620b
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java
@@ -0,0 +1,161 @@
+/**
+ * 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.tools;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.cli.*;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.conf.TajoConf;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.sql.SQLException;
+
+public class TajoGetConf {
+  private static final org.apache.commons.cli.Options options;
+
+  static {
+    options = new Options();
+    options.addOption("h", "host", true, "Tajo server host");
+    options.addOption("p", "port", true, "Tajo server port");
+  }
+
+  private TajoConf tajoConf;
+  private TajoClient tajoClient;
+  private Writer writer;
+
+  public final static String defaultLeftPad = " ";
+  public final static String defaultDescPad = "   ";
+
+  public TajoGetConf(TajoConf tajoConf, Writer writer) {
+    this(tajoConf, writer, null);
+  }
+
+  public TajoGetConf(TajoConf tajoConf, Writer writer, TajoClient tajoClient) {
+    this.tajoConf = tajoConf;
+    this.writer = writer;
+    this.tajoClient = tajoClient;
+  }
+
+  private void printUsage(boolean tsqlMode) {
+    if (!tsqlMode) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp( "getconf <key> [options]", options );
+    }
+    System.out.println(defaultLeftPad + "key" + defaultDescPad + "gets a specific key from the configuration");
+  }
+
+  public void runCommand(String[] args) throws Exception {
+    runCommand(args, true);
+  }
+
+  public void runCommand(String[] args, boolean tsqlMode) throws Exception {
+    CommandLineParser parser = new PosixParser();
+
+    if (args.length == 0) {
+      printUsage(tsqlMode);
+      return;
+    }
+
+    CommandLine cmd = parser.parse(options, args);
+
+    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 param;
+    if (cmd.getArgs().length > 1) {
+      printUsage(tsqlMode);
+      return;
+    } else {
+      param = cmd.getArgs()[0];
+    }
+
+    // if there is no "-h" option,
+    if(hostName == null) {
+      if (tajoConf.getVar(TajoConf.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 = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
+      }
+    }
+    if (port == null) {
+      if (tajoConf.getVar(TajoConf.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(tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
+      }
+    }
+
+    if ((hostName == null) ^ (port == null)) {
+      return;
+    } else if (hostName != null && port != null) {
+      tajoConf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName + ":" + port);
+      tajoClient = new TajoClientImpl(tajoConf);
+    } else if (hostName == null && port == null) {
+      tajoClient = new TajoClientImpl(tajoConf);
+    }
+
+    processConfKey(writer, param);
+    writer.flush();
+  }
+
+  private void processConfKey(Writer writer, String param) throws ParseException, IOException,
+      ServiceException, SQLException {
+    String value = tajoConf.getTrimmed(param);
+
+    // If there is no value in the configuration file, we need to find all ConfVars.
+    if (value == null) {
+      for(TajoConf.ConfVars vars : TajoConf.ConfVars.values()) {
+        if (vars.varname.equalsIgnoreCase(param)) {
+          value = tajoConf.getVar(vars);
+          break;
+        }
+      }
+    }
+
+    if (value != null) {
+      writer.write(value);
+    } else {
+      writer.write("Configuration " + param + " is missing.");
+    }
+
+    writer.write("\n");
+  }
+
+  public static void main(String [] args) throws Exception {
+    TajoConf conf = new TajoConf();
+
+    Writer writer = new PrintWriter(System.out);
+    try {
+      TajoGetConf admin = new TajoGetConf(conf, writer);
+      admin.runCommand(args, false);
+    } finally {
+      writer.close();
+      System.exit(0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java
new file mode 100644
index 0000000..4f2d024
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java
@@ -0,0 +1,210 @@
+/**
+ * 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.tools;
+
+import com.google.protobuf.ServiceException;
+import org.apache.commons.cli.*;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ha.HAServiceUtil;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Writer;
+
+public class TajoHAAdmin {
+  private static final Options options;
+
+  static {
+    options = new Options();
+    options.addOption("h", "host", true, "Tajo server host");
+    options.addOption("p", "port", true, "Tajo server port");
+    options.addOption("transitionToActive", null, true, "Transitions the master into Active state");
+    options.addOption("transitionToBackup", null, true, "Transitions the master into Backup state");
+    options.addOption("getState", null, true, "Returns the state of the master");
+    options.addOption("formatHA", null, false, "Format HA status on share storage");
+  }
+
+  private TajoConf tajoConf;
+  private TajoClient tajoClient;
+  private Writer writer;
+
+  public TajoHAAdmin(TajoConf tajoConf, Writer writer) {
+    this(tajoConf, writer, null);
+  }
+
+  public TajoHAAdmin(TajoConf tajoConf, Writer writer, TajoClient tajoClient) {
+    this.tajoConf = tajoConf;
+    this.writer = writer;
+    this.tajoClient = tajoClient;
+  }
+
+  private void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp( "haadmin [options]", options );
+  }
+
+  public void runCommand(String[] args) throws Exception {
+    if(args.length == 1 &&
+        (args[0].equalsIgnoreCase("-transitionToActive")
+            || args[0].equalsIgnoreCase("-transitionToBackup")
+            || args[0].equalsIgnoreCase("-getState"))) {
+      writer.write("Not enough arguments: expected 1 but got 0\n");
+      writer.flush();
+      return;
+    }
+
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd = parser.parse(options, args);
+
+    String param = "";
+    int cmdType = 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"));
+    }
+
+    if (cmd.hasOption("transitionToActive")) {
+      cmdType = 1;
+      param = cmd.getOptionValue("transitionToActive");
+    } else if (cmd.hasOption("transitionToBackup")) {
+      cmdType = 2;
+      param = cmd.getOptionValue("transitionToBackup");
+    } else if (cmd.hasOption("getState")) {
+      cmdType = 3;
+      param = cmd.getOptionValue("getState");
+    } else if (cmd.hasOption("formatHA")) {
+      cmdType = 4;
+    }
+
+    // if there is no "-h" option,
+    if(hostName == null) {
+      if (tajoConf.getVar(TajoConf.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 = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
+      }
+    }
+    if (port == null) {
+      if (tajoConf.getVar(TajoConf.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(tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
+      }
+    }
+
+    if (cmdType == 0) {
+      printUsage();
+      return;
+    }
+
+
+    if ((hostName == null) ^ (port == null)) {
+      System.err.println("ERROR: cannot find valid Tajo server address");
+      return;
+    } else if (hostName != null && port != null) {
+      tajoConf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName + ":" + port);
+      tajoClient = new TajoClientImpl(tajoConf);
+    } else if (hostName == null && port == null) {
+      tajoClient = new TajoClientImpl(tajoConf);
+    }
+
+    if (!tajoConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+      writer.write("HA is not enabled for this tajo cluster.");
+    } else {
+      switch (cmdType) {
+        case 1:
+          writer.write("Not Yet Implemented\n");
+          break;
+        case 2:
+          writer.write("Not Yet Implemented\n");
+          break;
+        case 3:
+          getState(writer, param);
+          break;
+        case 4:
+          formatHA(writer);
+          break;
+        default:
+          printUsage();
+          break;
+      }
+    }
+
+    writer.flush();
+  }
+
+  private void getState(Writer writer, String param) throws ParseException, IOException,
+      ServiceException {
+
+    int retValue = HAServiceUtil.getState(param, tajoConf);
+
+    switch (retValue) {
+      case 1:
+        writer.write("The master is active.\n");
+        break;
+      case 0:
+        writer.write("The master is backup.\n");
+        break;
+      case -1:
+        writer.write("Finding failed. - master:" + param + "\n");
+        break;
+      default:
+        writer.write("Cannot find the master. - master:" + param + "\n");
+        break;
+    }
+  }
+
+  private void formatHA(Writer writer) throws ParseException, IOException,
+      ServiceException {
+    int retValue = HAServiceUtil.formatHA(tajoConf);
+
+    switch (retValue) {
+      case 1:
+        writer.write("Formatting finished successfully.\n");
+        break;
+      case 0:
+        writer.write("If you want to format the ha information, you must shutdown tajo masters "
+            + " before formatting.\n");
+        break;
+      default:
+        writer.write("Cannot format ha information.\n");
+        break;
+    }
+  }
+
+  public static void main(String [] args) throws Exception {
+    TajoConf conf = new TajoConf();
+
+    Writer writer = new PrintWriter(System.out);
+    try {
+      TajoHAAdmin admin = new TajoHAAdmin(conf, writer);
+      admin.runCommand(args);
+    } finally {
+      writer.close();
+      System.exit(0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java
new file mode 100644
index 0000000..5cbe77b
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli.tsql;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.client.QueryStatus;
+import org.apache.tajo.util.FileUtil;
+
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+
+public class DefaultTajoCliOutputFormatter implements TajoCliOutputFormatter {
+  private int printPauseRecords;
+  private boolean printPause;
+  private boolean printErrorTrace;
+  private String nullChar;
+  public static char QUIT_COMMAND = 'q';
+
+  @Override
+  public void init(TajoCli.TajoCliContext context) {
+    this.printPause = context.getBool(SessionVars.CLI_PAGING_ENABLED);
+    this.printPauseRecords = context.getInt(SessionVars.CLI_PAGE_ROWS);
+    this.printErrorTrace = context.getBool(SessionVars.CLI_DISPLAY_ERROR_TRACE);
+    this.nullChar = context.get(SessionVars.CLI_NULL_CHAR);
+  }
+
+  @Override
+  public void setScriptMode() {
+    this.printPause = false;
+  }
+
+  private String getQuerySuccessMessage(TableDesc tableDesc, float responseTime, int totalPrintedRows, String postfix,
+                                        boolean endOfTuple) {
+    TableStats stat = tableDesc.getStats();
+    String volume = stat == null ? (endOfTuple ? "0 B" : "unknown bytes") :
+        FileUtil.humanReadableByteCount(stat.getNumBytes(), false);
+    long resultRows = stat == null ? TajoConstants.UNKNOWN_ROW_NUMBER : stat.getNumRows();
+
+    String displayRowNum;
+    if (resultRows == TajoConstants.UNKNOWN_ROW_NUMBER) {
+
+      if (endOfTuple) {
+        displayRowNum = totalPrintedRows + " rows";
+      } else {
+        displayRowNum = "unknown row number";
+      }
+
+    } else {
+      displayRowNum = resultRows + " rows";
+    }
+    return "(" + displayRowNum + ", " + getResponseTimeReadable(responseTime) + ", " + volume + " " + postfix + ")";
+  }
+
+  protected String getResponseTimeReadable(float responseTime) {
+    return responseTime + " sec";
+  }
+
+  @Override
+  public void printResult(PrintWriter sout, InputStream sin, TableDesc tableDesc,
+                          float responseTime, ResultSet res) throws Exception {
+    long resultRows = tableDesc.getStats() == null ? -1 : tableDesc.getStats().getNumRows();
+    if (resultRows == -1) {
+      resultRows = Integer.MAX_VALUE;
+    }
+
+    if (res == null) {
+      sout.println(getQuerySuccessMessage(tableDesc, responseTime, 0, "inserted", true));
+      return;
+    }
+    ResultSetMetaData rsmd = res.getMetaData();
+    int numOfColumns = rsmd.getColumnCount();
+    for (int i = 1; i <= numOfColumns; i++) {
+      if (i > 1) sout.print(",  ");
+      String columnName = rsmd.getColumnName(i);
+      sout.print(columnName);
+    }
+    sout.println("\n-------------------------------");
+
+    int numOfPrintedRows = 0;
+    int totalPrintedRows = 0;
+    boolean endOfTuple = true;
+    while (res.next()) {
+      for (int i = 1; i <= numOfColumns; i++) {
+        if (i > 1) sout.print(",  ");
+        String columnValue = res.getString(i);
+        if(res.wasNull()){
+          sout.print(nullChar);
+        } else {
+          sout.print(columnValue);
+        }
+      }
+      sout.println();
+      sout.flush();
+      numOfPrintedRows++;
+      totalPrintedRows++;
+      if (printPause && printPauseRecords > 0 && totalPrintedRows < resultRows && numOfPrintedRows >= printPauseRecords) {
+        if (resultRows < Integer.MAX_VALUE) {
+          sout.print("(" + totalPrintedRows + "/" + resultRows + " rows, continue... 'q' is quit)");
+        } else {
+          sout.print("(" + totalPrintedRows + " rows, continue... 'q' is quit)");
+        }
+        sout.flush();
+        if (sin != null) {
+          if (sin.read() == QUIT_COMMAND) {
+            endOfTuple = false;
+            sout.println();
+            break;
+          }
+        }
+        numOfPrintedRows = 0;
+        sout.println();
+      }
+    }
+    sout.println(getQuerySuccessMessage(tableDesc, responseTime, totalPrintedRows, "selected", endOfTuple));
+    sout.flush();
+  }
+
+  @Override
+  public void printNoResult(PrintWriter sout) {
+    sout.println("(0 rows)");
+    sout.flush();
+  }
+
+  @Override
+  public void printProgress(PrintWriter sout, QueryStatus status) {
+    sout.println("Progress: " + (int)(status.getProgress() * 100.0f)
+        + "%, response time: "
+        + getResponseTimeReadable((float)((status.getFinishTime() - status.getSubmitTime()) / 1000.0)));
+    sout.flush();
+  }
+
+  @Override
+  public void printMessage(PrintWriter sout, String message) {
+    sout.println(message);
+    sout.flush();
+  }
+
+  @Override
+  public void printErrorMessage(PrintWriter sout, Throwable t) {
+    sout.println(parseErrorMessage(t.getMessage()));
+    if (printErrorTrace) {
+      sout.println(ExceptionUtils.getStackTrace(t));
+    }
+    sout.flush();
+  }
+
+  @Override
+  public void printErrorMessage(PrintWriter sout, String message) {
+    sout.println(parseErrorMessage(message));
+    sout.flush();
+  }
+
+  @Override
+  public void printKilledMessage(PrintWriter sout, QueryId queryId) {
+    sout.println(TajoCli.KILL_PREFIX + queryId);
+    sout.flush();
+  }
+
+  @Override
+  public void printErrorMessage(PrintWriter sout, QueryStatus status) {
+    if (status.getErrorMessage() != null && !status.getErrorMessage().isEmpty()) {
+      printErrorMessage(sout, parseErrorMessage(status.getErrorMessage()));
+    } else {
+      printErrorMessage(sout, "No error message");
+    }
+    if (printErrorTrace && status.getErrorTrace() != null && !status.getErrorTrace().isEmpty()) {
+      sout.println(status.getErrorTrace());
+    }
+    sout.flush();
+  }
+
+  public static String parseErrorMessage(String message) {
+    if (message == null) {
+      return TajoCli.ERROR_PREFIX + "No error message";
+    }
+    String[] lines = message.split("\n");
+    message = lines[0];
+
+    int index = message.lastIndexOf(TajoCli.ERROR_PREFIX);
+    if (index < 0) {
+      message = TajoCli.ERROR_PREFIX + message;
+    } else {
+      message = message.substring(index);
+    }
+
+    return message;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/InvalidStatementException.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/InvalidStatementException.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/InvalidStatementException.java
new file mode 100644
index 0000000..2fed9fe
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/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.tsql;
+
+public class InvalidStatementException extends Exception {
+  public InvalidStatementException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.java
new file mode 100644
index 0000000..7894731
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.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;
+
+
+public class ParsedResult {
+  public static enum StatementType {
+    META,
+    STATEMENT
+  }
+
+  private final StatementType type;
+  private final String historyStatement;
+  private final String statement;
+
+  public ParsedResult(StatementType type, String statement, String historyStatement) {
+    this.type = type;
+    this.statement = statement;
+    this.historyStatement = historyStatement;
+  }
+
+  public StatementType getType() {
+    return type;
+  }
+
+  public String getHistoryStatement() {
+    return historyStatement.trim();
+  }
+
+  public String getStatement() {
+    return statement.trim();
+  }
+
+  public String toString() {
+    return "(" + type.name() + ") " + historyStatement;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java
new file mode 100644
index 0000000..cc772a3
--- /dev/null
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java
@@ -0,0 +1,387 @@
+/**
+ * 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.cli.tsql.ParsedResult.StatementType;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * 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
+    INVALID,       // Invalid Statement
+    STATEMENT_EOS, // End State (End of Statement)
+    META_EOS       // End State (End of Statement)
+  }
+
+  ParsingState state = START_STATE;
+  int lineNum;
+
+  /**
+   * It will be used to store a query statement into Jline history.
+   * the query statement for history does not include unnecessary white spaces and new line.
+   */
+  private StringBuilder historyAppender = new StringBuilder();
+  /**
+   * It will be used to submit a query statement to the TajoMaster. It just contains a raw query statement string.
+   */
+  private StringBuilder rawAppender = 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();
+
+    // if parsing continues, it means that the previous line is broken by '\n'.
+    // So, we should add new line to rawAppender.
+    if (isStatementContinue()) {
+      rawAppender.append("\n");
+    }
+
+    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 (isEndOfMeta(character)) {
+            state = ParsingState.META_EOS;
+          } else if (Character.isWhitespace(character)) {
+            // skip
+          }
+        }
+
+        if (state == ParsingState.META_EOS) {
+          historyAppender.append(str.subSequence(lineStartIdx, idx - 1).toString());
+          appendToRawStatement(str.subSequence(lineStartIdx, idx - 1).toString(), true);
+        } else {
+          historyAppender.append(str.subSequence(lineStartIdx, idx).toString());
+          appendToRawStatement(str.subSequence(lineStartIdx, idx).toString(), true);
+        }
+
+      } else if (isInlineCommentStart(chars, idx)) {
+        idx = consumeInlineComment(chars, idx);
+        appendToRawStatement(str.subSequence(lineStartIdx, idx).toString(), true);
+
+      /////////////////////////////////
+      //    TOK_START     -> STATEMENT
+      // or TOK_STATEMENT -> STATEMENT
+      ////////////////////////////////
+      } else if (isStatementContinue() || isStatementStart(chars[idx])) {
+        if (!isStatementContinue()) { // TOK_START -> STATEMENT
+          state = ParsingState.STATEMENT;
+          rawAppender.append("\n");
+        }
+
+        while (!isTerminateState(state) && idx < chars.length) {
+          char character = chars[idx++];
+
+          ///////////////////////////////////////////////////////
+          // in-statement loop BEGIN
+          ///////////////////////////////////////////////////////
+          if (isEndOfStatement(character)) {
+            state = ParsingState.STATEMENT_EOS;
+
+          } else if (state == ParsingState.STATEMENT && character == '\n') {
+            appendToBothStatements(chars, lineStartIdx, idx, 1); // omit new line chacter '\n' from history statement
+            lineStartIdx = idx;
+
+          } else if (state == ParsingState.STATEMENT && character == '\'') { // TOK_STATEMENT -> WITHIN_QUOTE
+            state = ParsingState.WITHIN_QUOTE;
+
+            if (idx < chars.length) {
+              character = chars[idx++];
+            } else {
+              continue;
+            }
+
+
+            // idx points the characters followed by the current character. So, we should use 'idx - 1'
+            // in order to point the current character.
+          } else if (state == ParsingState.STATEMENT && idx < chars.length && isInlineCommentStart(chars, idx - 1)) {
+            idx++;
+            appendToBothStatements(chars, lineStartIdx, idx, 2); // omit two dash characters '--' from history statement
+            int commentStartIdx = idx;
+            idx = consumeInlineComment(chars, idx);
+            appendToRawStatement(str.subSequence(commentStartIdx, idx).toString(), true);
+            lineStartIdx = idx;
+          }
+          ///////////////////////////////////////////////////////
+          // in-statement loop END
+          ///////////////////////////////////////////////////////
+
+          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;
+            }
+          }
+        }
+
+        // After all characters are consumed
+
+        if (state == ParsingState.STATEMENT_EOS) { // If one query statement is terminated
+          appendToBothStatements(chars, lineStartIdx, idx - 1); // skip semicolon (;)
+        } else {
+          appendToBothStatements(chars, lineStartIdx, idx);
+
+          // if it is not within quote and there is no space between lines, adds a space.
+          if (state == ParsingState.STATEMENT && (historyAppender.charAt(historyAppender.length() - 1) != ' ')) {
+            historyAppender.append(" ");
+            rawAppender.append("\n");
+          }
+        }
+      } else { // skip unknown character
+        idx++;
+      }
+
+      lineNum++;
+      statements.addAll(doProcessEndOfStatement(state == ParsingState.META));
+    }
+
+    return statements;
+  }
+
+  /**
+   * Append the range of characters into a given StringBuilder instance.
+   *
+   * @param chars Characters
+   * @param fromIdx start character index
+   * @param toIdx end character index
+   */
+  private void appendToStatement(StringBuilder builder, char[] chars, int fromIdx, int toIdx) {
+    builder.append(chars, fromIdx, toIdx - fromIdx);
+  }
+
+  /**
+   * Append the range of characters into both history and raw appenders. It omits the number of characters specified by
+   * <code>omitCharNums</code>.
+   *
+   *
+   * @param chars Characters
+   * @param fromIdx start character index
+   * @param toIdx end character index
+   * @param omitCharNums how many characters will be omitted from history statement
+   */
+  private void appendToBothStatements(char[] chars, int fromIdx, int toIdx, int omitCharNums) {
+    appendToStatement(historyAppender, chars, fromIdx, toIdx - omitCharNums);
+    if (historyAppender.charAt(historyAppender.length() - 1) != ' ') {
+      historyAppender.append(" ");
+    }
+    appendToStatement(rawAppender, chars, fromIdx, toIdx);
+  }
+
+  /**
+   * Append the range of characters into both history and raw appenders.
+   *
+   *
+   * @param chars Characters
+   * @param fromIdx start character index
+   * @param toIdx end character index
+   */
+  private void appendToBothStatements(char[] chars, int fromIdx, int toIdx) {
+    historyAppender.append(chars, fromIdx, toIdx - fromIdx);
+    rawAppender.append(chars, fromIdx, toIdx - fromIdx);
+  }
+
+  private int consumeInlineComment(char [] chars, int currentIdx) {
+    currentIdx++;
+    while (currentIdx < chars.length && !isNewLine(chars[currentIdx])) {
+      currentIdx++;
+    }
+    return currentIdx;
+  }
+
+  private void appendToRawStatement(String str, boolean addLF) {
+    if (!str.isEmpty() && !"\n".equals(str) &&
+        rawAppender.length() > 0 && addLF && rawAppender.charAt(rawAppender.length() - 1) != '\n') {
+      rawAppender.append(str);
+    } else {
+      rawAppender.append(str);
+    }
+  }
+
+  private static boolean isEndOfMeta(char character) {
+    return character == ';' || character == '\n';
+  }
+
+  private static boolean isEndOfStatement(char character) {
+    return character == ';';
+  }
+
+  /**
+   * It checks if inline comment '--' begins.
+   * @param chars
+   * @param idx
+   * @return
+   */
+  private boolean isInlineCommentStart(char[] chars, int idx) {
+    if (idx >= chars.length - 1) {
+      return false;
+    }
+    return (state == ParsingState.STATEMENT || state == ParsingState.TOK_START) &&
+        (chars[idx] == '-' && chars[idx + 1] == '-');
+  }
+
+  private boolean isNewLine(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;
+  }
+
+  /**
+   * process all parsed statements so far and return a list of parsed results.
+   *
+   * @param endOfFile TRUE if the end of file.
+   * @return the list of parsed results, each of result contains one query statement or meta command.
+   * @throws InvalidStatementException
+   */
+  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 historyStatement = historyAppender.toString();
+      String rawStatement = rawAppender.toString();
+      if (state == ParsingState.META_EOS) {
+        parsedResults.add(new ParsedResult(StatementType.META, rawStatement, historyStatement));
+        state = ParsingState.TOK_START;
+      } else if (state == ParsingState.STATEMENT_EOS) {
+        parsedResults.add(new ParsedResult(StatementType.STATEMENT, rawStatement, historyStatement));
+      } else {
+        throw new InvalidStatementException("ERROR: " + errorMessage);
+      }
+
+      // reset all states
+      historyAppender.delete(0, historyAppender.length());
+      rawAppender.delete(0, rawAppender.length());
+      state = START_STATE;
+    }
+
+    return parsedResults;
+  }
+
+  /**
+   * It manually triggers the end of file.
+   *
+   * @return the list of parsed results, each of result contains one query statement or meta command.
+   * @throws InvalidStatementException
+   */
+  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() + "]: " + historyAppender.toString();
+  }
+}


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

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java b/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java
deleted file mode 100644
index 88b8e0f..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoAdmin.java
+++ /dev/null
@@ -1,457 +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.cli.tools;
-
-import com.google.protobuf.ServiceException;
-import org.apache.commons.cli.*;
-import org.apache.commons.lang.StringUtils;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.client.*;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ipc.ClientProtos.BriefQueryInfo;
-import org.apache.tajo.ipc.ClientProtos.WorkerResourceInfo;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.ha.HAServiceUtil;
-import org.apache.tajo.util.TajoIdUtils;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.Writer;
-import java.net.InetSocketAddress;
-import java.sql.SQLException;
-import java.text.DecimalFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.List;
-
-public class TajoAdmin {
-  private static final org.apache.commons.cli.Options options;
-  private static DecimalFormat decimalF = new DecimalFormat("###.0");
-  private enum WorkerStatus {
-    RUNNING,
-    LOST,
-    DECOMMISSIONED
-  }
-
-  final static String DASHLINE_LEN5 = "-----";
-  final static String DASHLINE_LEN10 = "----------";
-  final static String DASHLINE_LEN12 = "------------";
-  final static String DASHLINE_LEN25 = "-------------------------";
-  final static String DATE_FORMAT  = "yyyy-MM-dd HH:mm:ss";
-
-  static {
-    options = new Options();
-    options.addOption("h", "host", true, "Tajo server host");
-    options.addOption("p", "port", true, "Tajo server port");
-    options.addOption("list", null, false, "Show Tajo query list");
-    options.addOption("cluster", null, false, "Show Cluster Info");
-    options.addOption("showmasters", null, false, "gets list of tajomasters in the cluster");
-    options.addOption("desc", null, false, "Show Query Description");
-    options.addOption("kill", null, true, "Kill a running query");
-  }
-
-  private TajoConf tajoConf;
-  private TajoClient tajoClient;
-  private Writer writer;
-
-  public TajoAdmin(TajoConf tajoConf, Writer writer) {
-    this(tajoConf, writer, null);
-  }
-
-  public TajoAdmin(TajoConf tajoConf, Writer writer, TajoClient tajoClient) {
-    this.tajoConf = tajoConf;
-    this.writer = writer;
-    this.tajoClient = tajoClient;
-  }
-
-  private void printUsage() {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp( "admin [options]", options );
-  }
-
-  public void runCommand(String[] args) throws Exception {
-    CommandLineParser parser = new PosixParser();
-    CommandLine cmd = parser.parse(options, args);
-
-    String param = "";
-    int cmdType = 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 queryId = null;
-
-    if (cmd.hasOption("list")) {
-      cmdType = 1;
-    } else if (cmd.hasOption("desc")) {
-      cmdType = 2;
-    } else if (cmd.hasOption("cluster")) {
-      cmdType = 3;
-    } else if (cmd.hasOption("kill")) {
-      cmdType = 4;
-      queryId = cmd.getOptionValue("kill");
-    } else if (cmd.hasOption("showmasters")) {
-      cmdType = 5;
-    }
-
-    // if there is no "-h" option,
-    if(hostName == null) {
-      if (tajoConf.getVar(TajoConf.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 = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
-      }
-    }
-    if (port == null) {
-      if (tajoConf.getVar(TajoConf.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(tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
-      }
-    }
-
-    if (cmdType == 0) {
-      printUsage();
-      return;
-    }
-
-
-    if ((hostName == null) ^ (port == null)) {
-      System.err.println("ERROR: cannot find valid Tajo server address");
-      return;
-    } else if (hostName != null && port != null) {
-      tajoConf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName + ":" + port);
-      tajoClient = new TajoClientImpl(tajoConf);
-    } else if (hostName == null && port == null) {
-      tajoClient = new TajoClientImpl(tajoConf);
-    }
-
-    switch (cmdType) {
-      case 1:
-        processList(writer);
-        break;
-      case 2:
-        processDesc(writer);
-        break;
-      case 3:
-        processCluster(writer);
-        break;
-      case 4:
-        processKill(writer, queryId);
-        break;
-      case 5:
-        processMasters(writer);
-        break;
-      default:
-        printUsage();
-        break;
-    }
-
-    writer.flush();
-  }
-
-  private void processDesc(Writer writer) throws ParseException, IOException,
-      ServiceException, SQLException {
-    tajoClient = TajoHAClientUtil.getTajoClient(tajoConf, tajoClient);
-    List<BriefQueryInfo> queryList = tajoClient.getRunningQueryList();
-    SimpleDateFormat df = new SimpleDateFormat(DATE_FORMAT);
-    int id = 1;
-    for (BriefQueryInfo queryInfo : queryList) {
-        String queryId = String.format("q_%s_%04d",
-                                       queryInfo.getQueryId().getId(),
-                                       queryInfo.getQueryId().getSeq());
-
-        writer.write("Id: " + id);
-        writer.write("\n");
-        id++;
-        writer.write("Query Id: " + queryId);
-        writer.write("\n");
-        writer.write("Started Time: " + df.format(queryInfo.getStartTime()));
-        writer.write("\n");
-
-        writer.write("Query State: " + queryInfo.getState().name());
-        writer.write("\n");
-        long end = queryInfo.getFinishTime();
-        long start = queryInfo.getStartTime();
-        String executionTime = decimalF.format((end-start) / 1000) + " sec";
-        if (TajoClientUtil.isQueryComplete(queryInfo.getState())) {
-          writer.write("Finished Time: " + df.format(queryInfo.getFinishTime()));
-          writer.write("\n");
-        }
-        writer.write("Execution Time: " + executionTime);
-        writer.write("\n");
-        writer.write("Query Progress: " + queryInfo.getProgress());
-        writer.write("\n");
-        writer.write("Query Statement:");
-        writer.write("\n");
-        writer.write(queryInfo.getQuery());
-        writer.write("\n");
-        writer.write("\n");
-    }
-  }
-
-  private void processCluster(Writer writer) throws ParseException, IOException,
-      ServiceException, SQLException {
-    tajoClient = TajoHAClientUtil.getTajoClient(tajoConf, tajoClient);
-    List<WorkerResourceInfo> workerList = tajoClient.getClusterInfo();
-
-    int runningQueryMasterTasks = 0;
-
-    List<WorkerResourceInfo> liveWorkers = new ArrayList<WorkerResourceInfo>();
-    List<WorkerResourceInfo> deadWorkers = new ArrayList<WorkerResourceInfo>();
-    List<WorkerResourceInfo> decommissionWorkers = new ArrayList<WorkerResourceInfo>();
-
-    List<WorkerResourceInfo> liveQueryMasters = new ArrayList<WorkerResourceInfo>();
-    List<WorkerResourceInfo> deadQueryMasters = new ArrayList<WorkerResourceInfo>();
-
-    for (WorkerResourceInfo eachWorker : workerList) {
-      if(eachWorker.getQueryMasterMode() == true) {
-        if(eachWorker.getWorkerStatus().equals(WorkerStatus.RUNNING.toString())) {
-          liveQueryMasters.add(eachWorker);
-          runningQueryMasterTasks += eachWorker.getNumQueryMasterTasks();
-        }
-        if(eachWorker.getWorkerStatus().equals(WorkerStatus.LOST.toString())) {
-          deadQueryMasters.add(eachWorker);
-        }
-      }
-
-      if(eachWorker.getTaskRunnerMode() == true) {
-        if(eachWorker.getWorkerStatus().equals(WorkerStatus.RUNNING.toString())) {
-          liveWorkers.add(eachWorker);
-        } else if(eachWorker.getWorkerStatus().equals(WorkerStatus.LOST.toString())) {
-          deadWorkers.add(eachWorker);
-        } else if(eachWorker.getWorkerStatus().equals(WorkerStatus.DECOMMISSIONED.toString())) {
-          decommissionWorkers.add(eachWorker);
-        }
-      }
-    }
-
-    String fmtInfo = "%1$-5s %2$-5s %3$-5s%n";
-    String infoLine = String.format(fmtInfo, "Live", "Dead", "Tasks");
-
-    writer.write("Query Master\n");
-    writer.write("============\n\n");
-    writer.write(infoLine);
-    String line = String.format(fmtInfo, DASHLINE_LEN5, DASHLINE_LEN5, DASHLINE_LEN5);
-    writer.write(line);
-
-    line = String.format(fmtInfo, liveQueryMasters.size(),
-                         deadQueryMasters.size(), runningQueryMasterTasks);
-    writer.write(line);
-    writer.write("\n");
-
-    writer.write("Live QueryMasters\n");
-    writer.write("=================\n\n");
-
-    if (liveQueryMasters.isEmpty()) {
-      writer.write("No Live QueryMasters\n");
-    } else {
-      String fmtQueryMasterLine = "%1$-25s %2$-5s %3$-5s %4$-10s %5$-10s%n";
-      line = String.format(fmtQueryMasterLine, "QueryMaster", "Port", "Query",
-                           "Heap", "Status");
-      writer.write(line);
-      line = String.format(fmtQueryMasterLine, DASHLINE_LEN25, DASHLINE_LEN5,
-              DASHLINE_LEN5, DASHLINE_LEN10, DASHLINE_LEN10);
-      writer.write(line);
-      for (WorkerResourceInfo queryMaster : liveQueryMasters) {
-        TajoProtos.WorkerConnectionInfoProto connInfo = queryMaster.getConnectionInfo();
-        String queryMasterHost = String.format("%s:%d", connInfo.getHost(), connInfo.getQueryMasterPort());
-        String heap = String.format("%d MB", queryMaster.getMaxHeap() / 1024 / 1024);
-        line = String.format(fmtQueryMasterLine,
-            queryMasterHost,
-            connInfo.getClientPort(),
-            queryMaster.getNumQueryMasterTasks(),
-            heap,
-            queryMaster.getWorkerStatus());
-        writer.write(line);
-      }
-
-      writer.write("\n\n");
-    }
-
-    if (!deadQueryMasters.isEmpty()) {
-      writer.write("Dead QueryMasters\n");
-      writer.write("=================\n\n");
-
-      String fmtQueryMasterLine = "%1$-25s %2$-5s %3$-10s%n";
-      line = String.format(fmtQueryMasterLine, "QueryMaster", "Port", "Status");
-      writer.write(line);
-      line = String.format(fmtQueryMasterLine, DASHLINE_LEN25, DASHLINE_LEN5, DASHLINE_LEN10);
-      writer.write(line);
-
-      for (WorkerResourceInfo queryMaster : deadQueryMasters) {
-        TajoProtos.WorkerConnectionInfoProto connInfo = queryMaster.getConnectionInfo();
-        String queryMasterHost = String.format("%s:%d", connInfo.getHost(), connInfo.getQueryMasterPort());
-        line = String.format(fmtQueryMasterLine,
-            queryMasterHost,
-            connInfo.getClientPort(),
-            queryMaster.getWorkerStatus());
-        writer.write(line);
-      }
-
-      writer.write("\n\n");
-    }
-
-    writer.write("Worker\n");
-    writer.write("======\n\n");
-
-    String fmtWorkerInfo = "%1$-5s %2$-5s%n";
-    String workerInfoLine = String.format(fmtWorkerInfo, "Live", "Dead");
-    writer.write(workerInfoLine);
-    line = String.format(fmtWorkerInfo, DASHLINE_LEN5, DASHLINE_LEN5);
-    writer.write(line);
-
-    line = String.format(fmtWorkerInfo, liveWorkers.size(), deadWorkers.size());
-    writer.write(line);
-    writer.write("\n");
-
-    writer.write("Live Workers\n");
-    writer.write("============\n\n");
-    if(liveWorkers.isEmpty()) {
-      writer.write("No Live Workers\n\n");
-    } else {
-      writeWorkerInfo(writer, liveWorkers);
-    }
-
-    writer.write("Dead Workers\n");
-    writer.write("============\n\n");
-    if(deadWorkers.isEmpty()) {
-      writer.write("No Dead Workers\n\n");
-    } else {
-      writeWorkerInfo(writer, deadWorkers);
-    }
-  }
-
-  private void writeWorkerInfo(Writer writer, List<WorkerResourceInfo> workers) throws ParseException,
-      IOException, ServiceException, SQLException {
-    String fmtWorkerLine = "%1$-25s %2$-5s %3$-5s %4$-10s %5$-10s %6$-12s %7$-10s%n";
-    String line = String.format(fmtWorkerLine,
-        "Worker", "Port", "Tasks",
-        "Mem", "Disk",
-        "Heap", "Status");
-    writer.write(line);
-    line = String.format(fmtWorkerLine,
-        DASHLINE_LEN25, DASHLINE_LEN5, DASHLINE_LEN5,
-        DASHLINE_LEN10, DASHLINE_LEN10,
-        DASHLINE_LEN12, DASHLINE_LEN10);
-    writer.write(line);
-
-    for (WorkerResourceInfo worker : workers) {
-      TajoProtos.WorkerConnectionInfoProto connInfo = worker.getConnectionInfo();
-      String workerHost = String.format("%s:%d", connInfo.getHost(), connInfo.getPeerRpcPort());
-      String mem = String.format("%d/%d", worker.getUsedMemoryMB(),
-          worker.getMemoryMB());
-      String disk = String.format("%.2f/%.2f", worker.getUsedDiskSlots(),
-          worker.getDiskSlots());
-      String heap = String.format("%d/%d MB", worker.getFreeHeap()/1024/1024,
-          worker.getMaxHeap()/1024/1024);
-
-      line = String.format(fmtWorkerLine, workerHost,
-          connInfo.getPullServerPort(),
-          worker.getNumRunningTasks(),
-          mem, disk, heap, worker.getWorkerStatus());
-      writer.write(line);
-    }
-    writer.write("\n\n");
-  }
-
-  private void processList(Writer writer) throws ParseException, IOException,
-      ServiceException, SQLException {
-    tajoClient = TajoHAClientUtil.getTajoClient(tajoConf, tajoClient);
-    List<BriefQueryInfo> queryList = tajoClient.getRunningQueryList();
-    SimpleDateFormat df = new SimpleDateFormat(DATE_FORMAT);
-    StringBuilder builder = new StringBuilder();
-
-    /* print title */
-    builder.append(StringUtils.rightPad("QueryId", 21));
-    builder.append(StringUtils.rightPad("State", 20));
-    builder.append(StringUtils.rightPad("StartTime", 20));
-    builder.append(StringUtils.rightPad("Query", 30)).append("\n");
-
-    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 20), 21));
-    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 19), 20));
-    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 19), 20));
-    builder.append(StringUtils.rightPad(StringUtils.repeat("-", 29), 30)).append("\n");
-    writer.write(builder.toString());
-
-    builder = new StringBuilder();
-    for (BriefQueryInfo queryInfo : queryList) {
-      builder.append(StringUtils.rightPad(new QueryId(queryInfo.getQueryId()).toString(), 21));
-      builder.append(StringUtils.rightPad(queryInfo.getState().name(), 20));
-      builder.append(StringUtils.rightPad(df.format(queryInfo.getStartTime()), 20));
-      builder.append(StringUtils.abbreviate(queryInfo.getQuery(), 30)).append("\n");
-    }
-    writer.write(builder.toString());
-  }
-
-  public void processKill(Writer writer, String queryIdStr)
-      throws IOException, ServiceException {
-    QueryStatus status = tajoClient.killQuery(TajoIdUtils.parseQueryId(queryIdStr));
-    if (status.getState() == TajoProtos.QueryState.QUERY_KILLED) {
-      writer.write(queryIdStr + " is killed successfully.\n");
-    } else if (status.getState() == TajoProtos.QueryState.QUERY_KILL_WAIT) {
-      writer.write(queryIdStr + " will be finished after a while.\n");
-    } else {
-      writer.write("ERROR:" + status.getErrorMessage());
-    }
-  }
-
-  private void processMasters(Writer writer) throws ParseException, IOException,
-      ServiceException, SQLException {
-    tajoClient = TajoHAClientUtil.getTajoClient(tajoConf, tajoClient);
-    if (tajoConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
-
-      List<String> list = HAServiceUtil.getMasters(tajoConf);
-      int i = 0;
-      for (String master : list) {
-        if (i > 0) {
-          writer.write(" ");
-        }
-        writer.write(master);
-        i++;
-      }
-      writer.write("\n");
-    } else {
-      String confMasterServiceAddr = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS);
-      InetSocketAddress masterAddress = NetUtils.createSocketAddr(confMasterServiceAddr);
-      writer.write(masterAddress.getHostName());
-      writer.write("\n");
-    }
-  }
-
-  public static void main(String [] args) throws Exception {
-    TajoConf conf = new TajoConf();
-
-    Writer writer = new PrintWriter(System.out);
-    try {
-      TajoAdmin admin = new TajoAdmin(conf, writer);
-      admin.runCommand(args);
-    } finally {
-      writer.close();
-      System.exit(0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoDump.java b/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
deleted file mode 100644
index 7f38a5d..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
+++ /dev/null
@@ -1,202 +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.cli.tools;
-
-import com.google.protobuf.ServiceException;
-
-import org.apache.commons.cli.*;
-import org.apache.tajo.auth.UserRoleInfo;
-import org.apache.tajo.catalog.CatalogConstants;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.DDLBuilder;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.client.TajoClientImpl;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.util.Pair;
-import org.apache.tajo.util.TUtil;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.sql.SQLException;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Collections;
-import java.util.List;
-
-public class TajoDump {
-  private static final org.apache.commons.cli.Options options;
-
-  static {
-    options = new Options();
-    options.addOption("h", "host", true, "Tajo server host");
-    options.addOption("p", "port", true, "Tajo server port");
-    options.addOption("a", "all", false, "dump all table DDLs");
-  }
-
-  private static void printUsage() {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp( "tajo-dump [options] [database name]", options);
-  }
-
-  private static Pair<String, Integer> getConnectionAddr(TajoConf conf, CommandLine cmd) {
-    String hostName = null;
-    Integer port = null;
-    if (cmd.hasOption("h")) {
-      hostName = cmd.getOptionValue("h");
-    }
-    if (cmd.hasOption("p")) {
-      port = Integer.parseInt(cmd.getOptionValue("p"));
-    }
-
-    if(hostName == null) {
-      if (conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != null) {
-        hostName = conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
-      }
-    }
-    if (port == null) {
-      if (conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != null) {
-        port = Integer.parseInt(conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
-      }
-    }
-    return new Pair<String, Integer>(hostName, port);
-  }
-
-  public static void main(String [] args) throws ParseException, IOException, ServiceException, SQLException {
-    final TajoConf conf = new TajoConf();
-    final CommandLineParser parser = new PosixParser();
-    final CommandLine cmd = parser.parse(options, args);
-    final Pair<String, Integer> hostAndPort = getConnectionAddr(conf, cmd);
-    final String hostName = hostAndPort.getFirst();
-    final Integer port = hostAndPort.getSecond();
-    final UserRoleInfo userInfo = UserRoleInfo.getCurrentUser();
-
-    String baseDatabaseName = null;
-    if (cmd.getArgList().size() > 0) {
-      baseDatabaseName = (String) cmd.getArgList().get(0);
-    }
-
-    boolean isDumpingAllDatabases = cmd.hasOption('a');
-
-    // Neither two choices
-    if (!isDumpingAllDatabases && baseDatabaseName == null) {
-      printUsage();
-      System.exit(-1);
-    }
-
-    TajoClient client = null;
-    if ((hostName == null) ^ (port == null)) {
-      System.err.println("ERROR: cannot find any TajoMaster rpc address in arguments and tajo-site.xml.");
-      System.exit(-1);
-    } else if (hostName != null && port != null) {
-      conf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName+":"+port);
-      client = new TajoClientImpl(conf);
-    } else {
-      client = new TajoClientImpl(conf);
-    }
-
-    PrintWriter writer = new PrintWriter(System.out);
-    dump(client, userInfo, baseDatabaseName, isDumpingAllDatabases, true, true, writer);
-
-    System.exit(0);
-  }
-  
-  private static boolean isAcceptableDumpingDatabase(String databaseName) {
-    return (databaseName == null || !databaseName.equalsIgnoreCase(CatalogConstants.INFORMATION_SCHEMA_DB_NAME));
-  }
-
-  public static void dump(TajoClient client, UserRoleInfo userInfo, String baseDatabaseName,
-                   boolean isDumpingAllDatabases, boolean includeUserName, boolean includeDate, PrintWriter out)
-      throws SQLException, ServiceException {
-    printHeader(out, userInfo, includeUserName, includeDate);
-
-    if (isDumpingAllDatabases) {
-      // sort database names in an ascending lexicographic order of the names.
-      List<String> sorted = new ArrayList<String>(client.getAllDatabaseNames());
-      Collections.sort(sorted);
-
-      for (String databaseName : sorted) {
-        if (isAcceptableDumpingDatabase(databaseName)) {
-          dumpDatabase(client, databaseName, out);
-        }
-      }
-    } else {
-      dumpDatabase(client, baseDatabaseName, out);
-    }
-    out.flush();
-  }
-
-  private static void printHeader(PrintWriter writer, UserRoleInfo userInfo, boolean includeUSerName,
-                                  boolean includeDate) {
-    writer.write("--\n");
-    writer.write("-- Tajo database dump\n");
-    if (includeUSerName) {
-      writer.write("--\n-- Dump user: " + userInfo.getUserName() + "\n");
-    }
-    if (includeDate) {
-      writer.write("--\n-- Dump date: " + toDateString() + "\n");
-    }
-    writer.write("--\n");
-    writer.write("\n");
-  }
-
-  private static void dumpDatabase(TajoClient client, String databaseName, PrintWriter writer)
-      throws SQLException, ServiceException {
-    writer.write("\n");
-    writer.write("--\n");
-    writer.write(String.format("-- Database name: %s%n", CatalogUtil.denormalizeIdentifier(databaseName)));
-    writer.write("--\n");
-    writer.write("\n");
-    writer.write(String.format("CREATE DATABASE IF NOT EXISTS %s;", CatalogUtil.denormalizeIdentifier(databaseName)));
-    writer.write("\n\n");
-
-    // returned list is immutable.
-    List<String> tableNames = TUtil.newList(client.getTableList(databaseName));
-    Collections.sort(tableNames);
-    for (String tableName : tableNames) {
-      try {
-        TableDesc table = client.getTableDesc(CatalogUtil.buildFQName(databaseName, tableName));
-        
-        if (table.getMeta().getStoreType() == StoreType.SYSTEM) {
-          continue;
-        }
-        
-        if (table.isExternal()) {
-          writer.write(DDLBuilder.buildDDLForExternalTable(table));
-        } else {
-          writer.write(DDLBuilder.buildDDLForBaseTable(table));
-        }
-        writer.write("\n\n");
-      } catch (Exception e) {
-        // dump for each table can throw any exception. We need to skip the exception case.
-        // here, the error message prints out via stderr.
-        System.err.println("ERROR:" + tableName + "," + e.getMessage());
-      }
-    }
-  }
-
-  private static String toDateString() {
-    DateFormat df = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss");
-    java.util.Date today = Calendar.getInstance().getTime();
-    return df.format(today);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java b/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java
deleted file mode 100644
index aa7620b..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoGetConf.java
+++ /dev/null
@@ -1,161 +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.cli.tools;
-
-import com.google.protobuf.ServiceException;
-import org.apache.commons.cli.*;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.client.TajoClientImpl;
-import org.apache.tajo.conf.TajoConf;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.Writer;
-import java.sql.SQLException;
-
-public class TajoGetConf {
-  private static final org.apache.commons.cli.Options options;
-
-  static {
-    options = new Options();
-    options.addOption("h", "host", true, "Tajo server host");
-    options.addOption("p", "port", true, "Tajo server port");
-  }
-
-  private TajoConf tajoConf;
-  private TajoClient tajoClient;
-  private Writer writer;
-
-  public final static String defaultLeftPad = " ";
-  public final static String defaultDescPad = "   ";
-
-  public TajoGetConf(TajoConf tajoConf, Writer writer) {
-    this(tajoConf, writer, null);
-  }
-
-  public TajoGetConf(TajoConf tajoConf, Writer writer, TajoClient tajoClient) {
-    this.tajoConf = tajoConf;
-    this.writer = writer;
-    this.tajoClient = tajoClient;
-  }
-
-  private void printUsage(boolean tsqlMode) {
-    if (!tsqlMode) {
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp( "getconf <key> [options]", options );
-    }
-    System.out.println(defaultLeftPad + "key" + defaultDescPad + "gets a specific key from the configuration");
-  }
-
-  public void runCommand(String[] args) throws Exception {
-    runCommand(args, true);
-  }
-
-  public void runCommand(String[] args, boolean tsqlMode) throws Exception {
-    CommandLineParser parser = new PosixParser();
-
-    if (args.length == 0) {
-      printUsage(tsqlMode);
-      return;
-    }
-
-    CommandLine cmd = parser.parse(options, args);
-
-    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 param;
-    if (cmd.getArgs().length > 1) {
-      printUsage(tsqlMode);
-      return;
-    } else {
-      param = cmd.getArgs()[0];
-    }
-
-    // if there is no "-h" option,
-    if(hostName == null) {
-      if (tajoConf.getVar(TajoConf.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 = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
-      }
-    }
-    if (port == null) {
-      if (tajoConf.getVar(TajoConf.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(tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
-      }
-    }
-
-    if ((hostName == null) ^ (port == null)) {
-      return;
-    } else if (hostName != null && port != null) {
-      tajoConf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName + ":" + port);
-      tajoClient = new TajoClientImpl(tajoConf);
-    } else if (hostName == null && port == null) {
-      tajoClient = new TajoClientImpl(tajoConf);
-    }
-
-    processConfKey(writer, param);
-    writer.flush();
-  }
-
-  private void processConfKey(Writer writer, String param) throws ParseException, IOException,
-      ServiceException, SQLException {
-    String value = tajoConf.getTrimmed(param);
-
-    // If there is no value in the configuration file, we need to find all ConfVars.
-    if (value == null) {
-      for(TajoConf.ConfVars vars : TajoConf.ConfVars.values()) {
-        if (vars.varname.equalsIgnoreCase(param)) {
-          value = tajoConf.getVar(vars);
-          break;
-        }
-      }
-    }
-
-    if (value != null) {
-      writer.write(value);
-    } else {
-      writer.write("Configuration " + param + " is missing.");
-    }
-
-    writer.write("\n");
-  }
-
-  public static void main(String [] args) throws Exception {
-    TajoConf conf = new TajoConf();
-
-    Writer writer = new PrintWriter(System.out);
-    try {
-      TajoGetConf admin = new TajoGetConf(conf, writer);
-      admin.runCommand(args, false);
-    } finally {
-      writer.close();
-      System.exit(0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java b/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java
deleted file mode 100644
index ef8fee9..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tools/TajoHAAdmin.java
+++ /dev/null
@@ -1,211 +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.cli.tools;
-
-import com.google.protobuf.ServiceException;
-import org.apache.commons.cli.*;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.client.TajoClientImpl;
-import org.apache.tajo.client.TajoHAClientUtil;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ha.HAServiceUtil;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.Writer;
-
-public class TajoHAAdmin {
-  private static final Options options;
-
-  static {
-    options = new Options();
-    options.addOption("h", "host", true, "Tajo server host");
-    options.addOption("p", "port", true, "Tajo server port");
-    options.addOption("transitionToActive", null, true, "Transitions the master into Active state");
-    options.addOption("transitionToBackup", null, true, "Transitions the master into Backup state");
-    options.addOption("getState", null, true, "Returns the state of the master");
-    options.addOption("formatHA", null, false, "Format HA status on share storage");
-  }
-
-  private TajoConf tajoConf;
-  private TajoClient tajoClient;
-  private Writer writer;
-
-  public TajoHAAdmin(TajoConf tajoConf, Writer writer) {
-    this(tajoConf, writer, null);
-  }
-
-  public TajoHAAdmin(TajoConf tajoConf, Writer writer, TajoClient tajoClient) {
-    this.tajoConf = tajoConf;
-    this.writer = writer;
-    this.tajoClient = tajoClient;
-  }
-
-  private void printUsage() {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp( "haadmin [options]", options );
-  }
-
-  public void runCommand(String[] args) throws Exception {
-    if(args.length == 1 &&
-        (args[0].equalsIgnoreCase("-transitionToActive")
-            || args[0].equalsIgnoreCase("-transitionToBackup")
-            || args[0].equalsIgnoreCase("-getState"))) {
-      writer.write("Not enough arguments: expected 1 but got 0\n");
-      writer.flush();
-      return;
-    }
-
-    CommandLineParser parser = new PosixParser();
-    CommandLine cmd = parser.parse(options, args);
-
-    String param = "";
-    int cmdType = 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"));
-    }
-
-    if (cmd.hasOption("transitionToActive")) {
-      cmdType = 1;
-      param = cmd.getOptionValue("transitionToActive");
-    } else if (cmd.hasOption("transitionToBackup")) {
-      cmdType = 2;
-      param = cmd.getOptionValue("transitionToBackup");
-    } else if (cmd.hasOption("getState")) {
-      cmdType = 3;
-      param = cmd.getOptionValue("getState");
-    } else if (cmd.hasOption("formatHA")) {
-      cmdType = 4;
-    }
-
-    // if there is no "-h" option,
-    if(hostName == null) {
-      if (tajoConf.getVar(TajoConf.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 = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
-      }
-    }
-    if (port == null) {
-      if (tajoConf.getVar(TajoConf.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(tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
-      }
-    }
-
-    if (cmdType == 0) {
-      printUsage();
-      return;
-    }
-
-
-    if ((hostName == null) ^ (port == null)) {
-      System.err.println("ERROR: cannot find valid Tajo server address");
-      return;
-    } else if (hostName != null && port != null) {
-      tajoConf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName + ":" + port);
-      tajoClient = new TajoClientImpl(tajoConf);
-    } else if (hostName == null && port == null) {
-      tajoClient = new TajoClientImpl(tajoConf);
-    }
-
-    if (!tajoConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
-      writer.write("HA is not enabled for this tajo cluster.");
-    } else {
-      switch (cmdType) {
-        case 1:
-          writer.write("Not Yet Implemented\n");
-          break;
-        case 2:
-          writer.write("Not Yet Implemented\n");
-          break;
-        case 3:
-          getState(writer, param);
-          break;
-        case 4:
-          formatHA(writer);
-          break;
-        default:
-          printUsage();
-          break;
-      }
-    }
-
-    writer.flush();
-  }
-
-  private void getState(Writer writer, String param) throws ParseException, IOException,
-      ServiceException {
-    tajoClient = TajoHAClientUtil.getTajoClient(tajoConf, tajoClient);
-    int retValue = HAServiceUtil.getState(param, tajoConf);
-
-    switch (retValue) {
-      case 1:
-        writer.write("The master is active.\n");
-        break;
-      case 0:
-        writer.write("The master is backup.\n");
-        break;
-      case -1:
-        writer.write("Finding failed. - master:" + param + "\n");
-        break;
-      default:
-        writer.write("Cannot find the master. - master:" + param + "\n");
-        break;
-    }
-  }
-
-  private void formatHA(Writer writer) throws ParseException, IOException,
-      ServiceException {
-    int retValue = HAServiceUtil.formatHA(tajoConf);
-
-    switch (retValue) {
-      case 1:
-        writer.write("Formatting finished successfully.\n");
-        break;
-      case 0:
-        writer.write("If you want to format the ha information, you must shutdown tajo masters "
-            + " before formatting.\n");
-        break;
-      default:
-        writer.write("Cannot format ha information.\n");
-        break;
-    }
-  }
-
-  public static void main(String [] args) throws Exception {
-    TajoConf conf = new TajoConf();
-
-    Writer writer = new PrintWriter(System.out);
-    try {
-      TajoHAAdmin admin = new TajoHAAdmin(conf, writer);
-      admin.runCommand(args);
-    } finally {
-      writer.close();
-      System.exit(0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java
deleted file mode 100644
index 5cbe77b..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/DefaultTajoCliOutputFormatter.java
+++ /dev/null
@@ -1,211 +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.cli.tsql;
-
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.SessionVars;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.client.QueryStatus;
-import org.apache.tajo.util.FileUtil;
-
-import java.io.InputStream;
-import java.io.PrintWriter;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-
-public class DefaultTajoCliOutputFormatter implements TajoCliOutputFormatter {
-  private int printPauseRecords;
-  private boolean printPause;
-  private boolean printErrorTrace;
-  private String nullChar;
-  public static char QUIT_COMMAND = 'q';
-
-  @Override
-  public void init(TajoCli.TajoCliContext context) {
-    this.printPause = context.getBool(SessionVars.CLI_PAGING_ENABLED);
-    this.printPauseRecords = context.getInt(SessionVars.CLI_PAGE_ROWS);
-    this.printErrorTrace = context.getBool(SessionVars.CLI_DISPLAY_ERROR_TRACE);
-    this.nullChar = context.get(SessionVars.CLI_NULL_CHAR);
-  }
-
-  @Override
-  public void setScriptMode() {
-    this.printPause = false;
-  }
-
-  private String getQuerySuccessMessage(TableDesc tableDesc, float responseTime, int totalPrintedRows, String postfix,
-                                        boolean endOfTuple) {
-    TableStats stat = tableDesc.getStats();
-    String volume = stat == null ? (endOfTuple ? "0 B" : "unknown bytes") :
-        FileUtil.humanReadableByteCount(stat.getNumBytes(), false);
-    long resultRows = stat == null ? TajoConstants.UNKNOWN_ROW_NUMBER : stat.getNumRows();
-
-    String displayRowNum;
-    if (resultRows == TajoConstants.UNKNOWN_ROW_NUMBER) {
-
-      if (endOfTuple) {
-        displayRowNum = totalPrintedRows + " rows";
-      } else {
-        displayRowNum = "unknown row number";
-      }
-
-    } else {
-      displayRowNum = resultRows + " rows";
-    }
-    return "(" + displayRowNum + ", " + getResponseTimeReadable(responseTime) + ", " + volume + " " + postfix + ")";
-  }
-
-  protected String getResponseTimeReadable(float responseTime) {
-    return responseTime + " sec";
-  }
-
-  @Override
-  public void printResult(PrintWriter sout, InputStream sin, TableDesc tableDesc,
-                          float responseTime, ResultSet res) throws Exception {
-    long resultRows = tableDesc.getStats() == null ? -1 : tableDesc.getStats().getNumRows();
-    if (resultRows == -1) {
-      resultRows = Integer.MAX_VALUE;
-    }
-
-    if (res == null) {
-      sout.println(getQuerySuccessMessage(tableDesc, responseTime, 0, "inserted", true));
-      return;
-    }
-    ResultSetMetaData rsmd = res.getMetaData();
-    int numOfColumns = rsmd.getColumnCount();
-    for (int i = 1; i <= numOfColumns; i++) {
-      if (i > 1) sout.print(",  ");
-      String columnName = rsmd.getColumnName(i);
-      sout.print(columnName);
-    }
-    sout.println("\n-------------------------------");
-
-    int numOfPrintedRows = 0;
-    int totalPrintedRows = 0;
-    boolean endOfTuple = true;
-    while (res.next()) {
-      for (int i = 1; i <= numOfColumns; i++) {
-        if (i > 1) sout.print(",  ");
-        String columnValue = res.getString(i);
-        if(res.wasNull()){
-          sout.print(nullChar);
-        } else {
-          sout.print(columnValue);
-        }
-      }
-      sout.println();
-      sout.flush();
-      numOfPrintedRows++;
-      totalPrintedRows++;
-      if (printPause && printPauseRecords > 0 && totalPrintedRows < resultRows && numOfPrintedRows >= printPauseRecords) {
-        if (resultRows < Integer.MAX_VALUE) {
-          sout.print("(" + totalPrintedRows + "/" + resultRows + " rows, continue... 'q' is quit)");
-        } else {
-          sout.print("(" + totalPrintedRows + " rows, continue... 'q' is quit)");
-        }
-        sout.flush();
-        if (sin != null) {
-          if (sin.read() == QUIT_COMMAND) {
-            endOfTuple = false;
-            sout.println();
-            break;
-          }
-        }
-        numOfPrintedRows = 0;
-        sout.println();
-      }
-    }
-    sout.println(getQuerySuccessMessage(tableDesc, responseTime, totalPrintedRows, "selected", endOfTuple));
-    sout.flush();
-  }
-
-  @Override
-  public void printNoResult(PrintWriter sout) {
-    sout.println("(0 rows)");
-    sout.flush();
-  }
-
-  @Override
-  public void printProgress(PrintWriter sout, QueryStatus status) {
-    sout.println("Progress: " + (int)(status.getProgress() * 100.0f)
-        + "%, response time: "
-        + getResponseTimeReadable((float)((status.getFinishTime() - status.getSubmitTime()) / 1000.0)));
-    sout.flush();
-  }
-
-  @Override
-  public void printMessage(PrintWriter sout, String message) {
-    sout.println(message);
-    sout.flush();
-  }
-
-  @Override
-  public void printErrorMessage(PrintWriter sout, Throwable t) {
-    sout.println(parseErrorMessage(t.getMessage()));
-    if (printErrorTrace) {
-      sout.println(ExceptionUtils.getStackTrace(t));
-    }
-    sout.flush();
-  }
-
-  @Override
-  public void printErrorMessage(PrintWriter sout, String message) {
-    sout.println(parseErrorMessage(message));
-    sout.flush();
-  }
-
-  @Override
-  public void printKilledMessage(PrintWriter sout, QueryId queryId) {
-    sout.println(TajoCli.KILL_PREFIX + queryId);
-    sout.flush();
-  }
-
-  @Override
-  public void printErrorMessage(PrintWriter sout, QueryStatus status) {
-    if (status.getErrorMessage() != null && !status.getErrorMessage().isEmpty()) {
-      printErrorMessage(sout, parseErrorMessage(status.getErrorMessage()));
-    } else {
-      printErrorMessage(sout, "No error message");
-    }
-    if (printErrorTrace && status.getErrorTrace() != null && !status.getErrorTrace().isEmpty()) {
-      sout.println(status.getErrorTrace());
-    }
-    sout.flush();
-  }
-
-  public static String parseErrorMessage(String message) {
-    if (message == null) {
-      return TajoCli.ERROR_PREFIX + "No error message";
-    }
-    String[] lines = message.split("\n");
-    message = lines[0];
-
-    int index = message.lastIndexOf(TajoCli.ERROR_PREFIX);
-    if (index < 0) {
-      message = TajoCli.ERROR_PREFIX + message;
-    } else {
-      message = message.substring(index);
-    }
-
-    return message;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tsql/InvalidStatementException.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/InvalidStatementException.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/InvalidStatementException.java
deleted file mode 100644
index 2fed9fe..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/InvalidStatementException.java
+++ /dev/null
@@ -1,25 +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.cli.tsql;
-
-public class InvalidStatementException extends Exception {
-  public InvalidStatementException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.java
deleted file mode 100644
index 7894731..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/ParsedResult.java
+++ /dev/null
@@ -1,53 +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.cli.tsql;
-
-
-public class ParsedResult {
-  public static enum StatementType {
-    META,
-    STATEMENT
-  }
-
-  private final StatementType type;
-  private final String historyStatement;
-  private final String statement;
-
-  public ParsedResult(StatementType type, String statement, String historyStatement) {
-    this.type = type;
-    this.statement = statement;
-    this.historyStatement = historyStatement;
-  }
-
-  public StatementType getType() {
-    return type;
-  }
-
-  public String getHistoryStatement() {
-    return historyStatement.trim();
-  }
-
-  public String getStatement() {
-    return statement.trim();
-  }
-
-  public String toString() {
-    return "(" + type.name() + ") " + historyStatement;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java
deleted file mode 100644
index 56315a8..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/SimpleParser.java
+++ /dev/null
@@ -1,388 +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.cli.tsql;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.tajo.cli.tsql.ParsedResult.StatementType.META;
-import static org.apache.tajo.cli.tsql.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
-    INVALID,       // Invalid Statement
-    STATEMENT_EOS, // End State (End of Statement)
-    META_EOS       // End State (End of Statement)
-  }
-
-  ParsingState state = START_STATE;
-  int lineNum;
-
-  /**
-   * It will be used to store a query statement into Jline history.
-   * the query statement for history does not include unnecessary white spaces and new line.
-   */
-  private StringBuilder historyAppender = new StringBuilder();
-  /**
-   * It will be used to submit a query statement to the TajoMaster. It just contains a raw query statement string.
-   */
-  private StringBuilder rawAppender = 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();
-
-    // if parsing continues, it means that the previous line is broken by '\n'.
-    // So, we should add new line to rawAppender.
-    if (isStatementContinue()) {
-      rawAppender.append("\n");
-    }
-
-    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 (isEndOfMeta(character)) {
-            state = ParsingState.META_EOS;
-          } else if (Character.isWhitespace(character)) {
-            // skip
-          }
-        }
-
-        if (state == ParsingState.META_EOS) {
-          historyAppender.append(str.subSequence(lineStartIdx, idx - 1).toString());
-          appendToRawStatement(str.subSequence(lineStartIdx, idx - 1).toString(), true);
-        } else {
-          historyAppender.append(str.subSequence(lineStartIdx, idx).toString());
-          appendToRawStatement(str.subSequence(lineStartIdx, idx).toString(), true);
-        }
-
-      } else if (isInlineCommentStart(chars, idx)) {
-        idx = consumeInlineComment(chars, idx);
-        appendToRawStatement(str.subSequence(lineStartIdx, idx).toString(), true);
-
-      /////////////////////////////////
-      //    TOK_START     -> STATEMENT
-      // or TOK_STATEMENT -> STATEMENT
-      ////////////////////////////////
-      } else if (isStatementContinue() || isStatementStart(chars[idx])) {
-        if (!isStatementContinue()) { // TOK_START -> STATEMENT
-          state = ParsingState.STATEMENT;
-          rawAppender.append("\n");
-        }
-
-        while (!isTerminateState(state) && idx < chars.length) {
-          char character = chars[idx++];
-
-          ///////////////////////////////////////////////////////
-          // in-statement loop BEGIN
-          ///////////////////////////////////////////////////////
-          if (isEndOfStatement(character)) {
-            state = ParsingState.STATEMENT_EOS;
-
-          } else if (state == ParsingState.STATEMENT && character == '\n') {
-            appendToBothStatements(chars, lineStartIdx, idx, 1); // omit new line chacter '\n' from history statement
-            lineStartIdx = idx;
-
-          } else if (state == ParsingState.STATEMENT && character == '\'') { // TOK_STATEMENT -> WITHIN_QUOTE
-            state = ParsingState.WITHIN_QUOTE;
-
-            if (idx < chars.length) {
-              character = chars[idx++];
-            } else {
-              continue;
-            }
-
-
-            // idx points the characters followed by the current character. So, we should use 'idx - 1'
-            // in order to point the current character.
-          } else if (state == ParsingState.STATEMENT && idx < chars.length && isInlineCommentStart(chars, idx - 1)) {
-            idx++;
-            appendToBothStatements(chars, lineStartIdx, idx, 2); // omit two dash characters '--' from history statement
-            int commentStartIdx = idx;
-            idx = consumeInlineComment(chars, idx);
-            appendToRawStatement(str.subSequence(commentStartIdx, idx).toString(), true);
-            lineStartIdx = idx;
-          }
-          ///////////////////////////////////////////////////////
-          // in-statement loop END
-          ///////////////////////////////////////////////////////
-
-          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;
-            }
-          }
-        }
-
-        // After all characters are consumed
-
-        if (state == ParsingState.STATEMENT_EOS) { // If one query statement is terminated
-          appendToBothStatements(chars, lineStartIdx, idx - 1); // skip semicolon (;)
-        } else {
-          appendToBothStatements(chars, lineStartIdx, idx);
-
-          // if it is not within quote and there is no space between lines, adds a space.
-          if (state == ParsingState.STATEMENT && (historyAppender.charAt(historyAppender.length() - 1) != ' ')) {
-            historyAppender.append(" ");
-            rawAppender.append("\n");
-          }
-        }
-      } else { // skip unknown character
-        idx++;
-      }
-
-      lineNum++;
-      statements.addAll(doProcessEndOfStatement(state == ParsingState.META));
-    }
-
-    return statements;
-  }
-
-  /**
-   * Append the range of characters into a given StringBuilder instance.
-   *
-   * @param chars Characters
-   * @param fromIdx start character index
-   * @param toIdx end character index
-   */
-  private void appendToStatement(StringBuilder builder, char[] chars, int fromIdx, int toIdx) {
-    builder.append(chars, fromIdx, toIdx - fromIdx);
-  }
-
-  /**
-   * Append the range of characters into both history and raw appenders. It omits the number of characters specified by
-   * <code>omitCharNums</code>.
-   *
-   *
-   * @param chars Characters
-   * @param fromIdx start character index
-   * @param toIdx end character index
-   * @param omitCharNums how many characters will be omitted from history statement
-   */
-  private void appendToBothStatements(char[] chars, int fromIdx, int toIdx, int omitCharNums) {
-    appendToStatement(historyAppender, chars, fromIdx, toIdx - omitCharNums);
-    if (historyAppender.charAt(historyAppender.length() - 1) != ' ') {
-      historyAppender.append(" ");
-    }
-    appendToStatement(rawAppender, chars, fromIdx, toIdx);
-  }
-
-  /**
-   * Append the range of characters into both history and raw appenders.
-   *
-   *
-   * @param chars Characters
-   * @param fromIdx start character index
-   * @param toIdx end character index
-   */
-  private void appendToBothStatements(char[] chars, int fromIdx, int toIdx) {
-    historyAppender.append(chars, fromIdx, toIdx - fromIdx);
-    rawAppender.append(chars, fromIdx, toIdx - fromIdx);
-  }
-
-  private int consumeInlineComment(char [] chars, int currentIdx) {
-    currentIdx++;
-    while (currentIdx < chars.length && !isNewLine(chars[currentIdx])) {
-      currentIdx++;
-    }
-    return currentIdx;
-  }
-
-  private void appendToRawStatement(String str, boolean addLF) {
-    if (!str.isEmpty() && !"\n".equals(str) &&
-        rawAppender.length() > 0 && addLF && rawAppender.charAt(rawAppender.length() - 1) != '\n') {
-      rawAppender.append(str);
-    } else {
-      rawAppender.append(str);
-    }
-  }
-
-  private static boolean isEndOfMeta(char character) {
-    return character == ';' || character == '\n';
-  }
-
-  private static boolean isEndOfStatement(char character) {
-    return character == ';';
-  }
-
-  /**
-   * It checks if inline comment '--' begins.
-   * @param chars
-   * @param idx
-   * @return
-   */
-  private boolean isInlineCommentStart(char[] chars, int idx) {
-    if (idx >= chars.length - 1) {
-      return false;
-    }
-    return (state == ParsingState.STATEMENT || state == ParsingState.TOK_START) &&
-        (chars[idx] == '-' && chars[idx + 1] == '-');
-  }
-
-  private boolean isNewLine(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;
-  }
-
-  /**
-   * process all parsed statements so far and return a list of parsed results.
-   *
-   * @param endOfFile TRUE if the end of file.
-   * @return the list of parsed results, each of result contains one query statement or meta command.
-   * @throws InvalidStatementException
-   */
-  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 historyStatement = historyAppender.toString();
-      String rawStatement = rawAppender.toString();
-      if (state == ParsingState.META_EOS) {
-        parsedResults.add(new ParsedResult(META, rawStatement, historyStatement));
-        state = ParsingState.TOK_START;
-      } else if (state == ParsingState.STATEMENT_EOS) {
-        parsedResults.add(new ParsedResult(STATEMENT, rawStatement, historyStatement));
-      } else {
-        throw new InvalidStatementException("ERROR: " + errorMessage);
-      }
-
-      // reset all states
-      historyAppender.delete(0, historyAppender.length());
-      rawAppender.delete(0, rawAppender.length());
-      state = START_STATE;
-    }
-
-    return parsedResults;
-  }
-
-  /**
-   * It manually triggers the end of file.
-   *
-   * @return the list of parsed results, each of result contains one query statement or meta command.
-   * @throws InvalidStatementException
-   */
-  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() + "]: " + historyAppender.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/42d79cf5/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
deleted file mode 100644
index db7f981..0000000
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
+++ /dev/null
@@ -1,701 +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.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.ipc.*;
-import org.apache.tajo.TajoProtos.QueryState;
-import org.apache.tajo.catalog.TableDesc;
-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.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;
-
-import static org.apache.tajo.cli.tsql.ParsedResult.StatementType.META;
-import static org.apache.tajo.cli.tsql.ParsedResult.StatementType.STATEMENT;
-import static org.apache.tajo.cli.tsql.SimpleParser.ParsingState;
-
-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 {
-      checkMasterStatus();
-      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() == 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() == 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 {
-    checkMasterStatus();
-    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 {
-    checkMasterStatus();
-    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 {
-    checkMasterStatus();
-    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();
-    }
-  }
-
-  private void checkMasterStatus() throws IOException, ServiceException {
-    String sessionId = client.getSessionId() != null ? client.getSessionId().getId() : null;
-    client = TajoHAClientUtil.getTajoClient(conf, client, context);
-    if(sessionId != null && (client.getSessionId() == null ||
-        !sessionId.equals(client.getSessionId().getId()))) {
-      commands.clear();
-      initHistory();
-      initCommands();
-    }
-  }
-
-  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());
-  }
-}


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

Posted by hy...@apache.org.
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";
+  }
+}