You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2014/04/11 16:16:50 UTC

git commit: TAJO-732: Support executing LINUX shell command and HDFS command. (hyoungjun kim via hyunsik)

Repository: tajo
Updated Branches:
  refs/heads/master 38f5316af -> 5ae608b12


TAJO-732: Support executing LINUX shell command and HDFS command. (hyoungjun kim via hyunsik)


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

Branch: refs/heads/master
Commit: 5ae608b1261f1b754f2bb58de793c539bd98360f
Parents: 38f5316
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Apr 11 23:16:30 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Apr 11 23:16:30 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../tajo/cli/ExecExternalShellCommand.java      | 116 +++++++++++++++++++
 .../java/org/apache/tajo/cli/HdfsCommand.java   |  57 +++++++++
 .../java/org/apache/tajo/cli/HelpCommand.java   |   7 ++
 .../org/apache/tajo/cli/TajoAdminCommand.java   |  57 +++++++++
 .../main/java/org/apache/tajo/cli/TajoCli.java  |  18 ++-
 .../java/org/apache/tajo/client/TajoAdmin.java  |  87 ++++++++------
 .../master/querymaster/QueryMasterTask.java     |   4 +-
 .../tajo/cli/TestExecExternalShellCommand.java  |  46 ++++++++
 .../org/apache/tajo/cli/TestHdfsCommand.java    |  46 ++++++++
 10 files changed, 401 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/5ae608b1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9cdcb5e..a207f04 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -143,6 +143,9 @@ Release 0.8.0 - unreleased
 
   IMPROVEMENTS
 
+    TAJO-732: Support executing LINUX shell command and HDFS command.
+    (hyoungjun kim via hyunsik)
+
     TAJO-745: APIs in TajoClient and JDBC should be case sensitive.
 
     TAJO-743: Change the default resource allocation policy of leaf tasks. (jinho)

http://git-wip-us.apache.org/repos/asf/tajo/blob/5ae608b1/tajo-client/src/main/java/org/apache/tajo/cli/ExecExternalShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/ExecExternalShellCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/ExecExternalShellCommand.java
new file mode 100644
index 0000000..ea2db28
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/ExecExternalShellCommand.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import java.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 {
+    if (command.length < 2) {
+      context.getOutput().println("ERROR: No shell command.");
+      return;
+    }
+
+    StringBuilder shellCommand = new StringBuilder();
+    String prefix = "";
+    for(int i = 1; i < command.length; i++) {
+      shellCommand.append(prefix).append(command[i]);
+      prefix = " ";
+    }
+
+    String[] execCommand = new String[3];
+    execCommand[0] = "/bin/bash";
+    execCommand[1] = "-c";
+    execCommand[2] = shellCommand.toString();
+
+    PrintWriter sout = context.getOutput();
+
+    CountDownLatch latch = new CountDownLatch(2);
+    Process process = Runtime.getRuntime().exec(execCommand);
+    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);
+    }
+  }
+
+  @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/5ae608b1/tajo-client/src/main/java/org/apache/tajo/cli/HdfsCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/HdfsCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/HdfsCommand.java
new file mode 100644
index 0000000..5b4c12e
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/HdfsCommand.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import org.apache.hadoop.fs.FsShell;
+
+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/5ae608b1/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
index 87d9900..53bc404 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/HelpCommand.java
@@ -57,6 +57,13 @@ public class HelpCommand extends TajoShellCommand {
     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");

http://git-wip-us.apache.org/repos/asf/tajo/blob/5ae608b1/tajo-client/src/main/java/org/apache/tajo/cli/TajoAdminCommand.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/TajoAdminCommand.java b/tajo-client/src/main/java/org/apache/tajo/cli/TajoAdminCommand.java
new file mode 100644
index 0000000..267c7a4
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/TajoAdminCommand.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import org.apache.tajo.client.TajoAdmin;
+
+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/5ae608b1/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java b/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
index c277c4c..508b8bb 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/TajoCli.java
@@ -71,7 +71,10 @@ public class TajoCli {
       ConnectDatabaseCommand.class,
       ListDatabaseCommand.class,
       SetCommand.class,
-      UnsetCommand.class
+      UnsetCommand.class,
+      ExecExternalShellCommand.class,
+      HdfsCommand.class,
+      TajoAdminCommand.class
   };
   private final Map<String, TajoShellCommand> commands = new TreeMap<String, TajoShellCommand>();
 
@@ -106,12 +109,17 @@ public class TajoCli {
     public PrintWriter getOutput() {
       return sout;
     }
+
+    public TajoConf getConf() {
+      return conf;
+    }
   }
 
   public TajoCli(TajoConf c, String [] args, InputStream in, OutputStream out) throws Exception {
     this.conf = new TajoConf(c);
     this.sin = in;
     this.reader = new ConsoleReader(sin, out);
+    this.reader.setExpandEvents(false);
     this.sout = new PrintWriter(reader.getOutput());
 
     CommandLineParser parser = new PosixParser();
@@ -153,7 +161,7 @@ public class TajoCli {
 
     if ((hostName == null) ^ (port == null)) {
       System.err.println("ERROR: cannot find valid Tajo server address");
-      System.exit(-1);
+      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 TajoClient(conf, baseDatabase);
@@ -209,7 +217,7 @@ public class TajoCli {
          cmd = (TajoShellCommand) cons.newInstance(context);
       } catch (Exception e) {
         System.err.println(e.getMessage());
-        System.exit(-1);
+        throw new RuntimeException(e.getMessage());
       }
       commands.put(cmd.getCommand(), cmd);
     }
@@ -247,7 +255,6 @@ public class TajoCli {
 
     SimpleParser parser = new SimpleParser();
     while((line = reader.readLine(currentPrompt + "> ")) != null) {
-
       if (line.equals("")) {
         continue;
       }
@@ -278,8 +285,7 @@ public class TajoCli {
   public int executeMetaCommand(String line) throws Exception {
     String [] metaCommands = line.split(";");
     for (String metaCommand : metaCommands) {
-      String arguments [];
-      arguments = metaCommand.split(" ");
+      String arguments [] = metaCommand.split(" ");
 
       TajoShellCommand invoked = commands.get(arguments[0]);
       if (invoked == null) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/5ae608b1/tajo-client/src/main/java/org/apache/tajo/client/TajoAdmin.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoAdmin.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoAdmin.java
index 0cc8110..9a0478c 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoAdmin.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoAdmin.java
@@ -65,12 +65,26 @@ public class TajoAdmin {
     options.addOption("kill", null, true, "Kill a running query");
   }
 
-  private static void printUsage() {
+  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 );
   }
 
-  private static String getQueryState(QueryState state) {
+  private String getQueryState(QueryState state) {
     String stateStr = "FAILED";
 
     if (TajoClient.isQueryRunnning(state)) {
@@ -82,9 +96,7 @@ public class TajoAdmin {
     return stateStr;
   }
 
-  public static void main(String [] args) throws ParseException, IOException, ServiceException, SQLException {
-    TajoConf conf = new TajoConf();
-
+  public void runCommand(String[] args) throws Exception {
     CommandLineParser parser = new PosixParser();
     CommandLine cmd = parser.parse(options, args);
 
@@ -115,49 +127,48 @@ public class TajoAdmin {
 
     // if there is no "-h" option,
     if(hostName == null) {
-      if (conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != 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 = conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
+        hostName = tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[0];
       }
     }
     if (port == null) {
-      if (conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS) != 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(conf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
+        port = Integer.parseInt(tajoConf.getVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS).split(":")[1]);
       }
     }
 
     if (cmdType == 0) {
-        printUsage();
-        System.exit(0);
+      printUsage();
+      return;
     }
 
-    TajoClient client = null;
+
     if ((hostName == null) ^ (port == null)) {
       System.err.println("ERROR: cannot find valid Tajo server address");
-      System.exit(-1);
+      return;
     } else if (hostName != null && port != null) {
-      conf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName+":"+port);
-      client = new TajoClient(conf);
+      tajoConf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS, hostName + ":" + port);
+      tajoClient = new TajoClient(tajoConf);
     } else if (hostName == null && port == null) {
-      client = new TajoClient(conf);
+      tajoClient = new TajoClient(tajoConf);
     }
 
-    Writer writer = new PrintWriter(System.out);
     switch (cmdType) {
       case 1:
-        processList(writer, client);
+        processList(writer);
         break;
       case 2:
-        processDesc(writer, client);
+        processDesc(writer);
         break;
       case 3:
-        processCluster(writer, client);
+        processCluster(writer);
         break;
       case 4:
-        processKill(writer, client, queryId);
+        processKill(writer, queryId);
         break;
       default:
         printUsage();
@@ -165,14 +176,11 @@ public class TajoAdmin {
     }
 
     writer.flush();
-    writer.close();
-
-    System.exit(0);
   }
 
-  public static void processDesc(Writer writer, TajoClient client) throws ParseException, IOException,
+  private void processDesc(Writer writer) throws ParseException, IOException,
       ServiceException, SQLException {
-    List<BriefQueryInfo> queryList = client.getRunningQueryList();
+    List<BriefQueryInfo> queryList = tajoClient.getRunningQueryList();
     SimpleDateFormat df = new SimpleDateFormat(DATE_FORMAT);
     int id = 1;
     for (BriefQueryInfo queryInfo : queryList) {
@@ -209,9 +217,9 @@ public class TajoAdmin {
     }
   }
 
-  public static void processCluster(Writer writer, TajoClient client) throws ParseException, IOException,
+  private void processCluster(Writer writer) throws ParseException, IOException,
       ServiceException, SQLException {
-    List<WorkerResourceInfo> workerList = client.getClusterInfo();
+    List<WorkerResourceInfo> workerList = tajoClient.getClusterInfo();
 
     int runningQueryMasterTasks = 0;
 
@@ -339,7 +347,7 @@ public class TajoAdmin {
     }
   }
 
-  private static void writeWorkerInfo(Writer writer, List<WorkerResourceInfo> workers) throws ParseException,
+  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,
@@ -373,9 +381,9 @@ public class TajoAdmin {
     writer.write("\n\n");
   }
 
-  public static void processList(Writer writer, TajoClient client) throws ParseException, IOException,
+  private void processList(Writer writer) throws ParseException, IOException,
       ServiceException, SQLException {
-    List<BriefQueryInfo> queryList = client.getRunningQueryList();
+    List<BriefQueryInfo> queryList = tajoClient.getRunningQueryList();
     SimpleDateFormat df = new SimpleDateFormat(DATE_FORMAT);
     String fmt = "%1$-20s %2$-7s %3$-20s %4$-30s%n";
     String line = String.format(fmt, "QueryId", "State", 
@@ -397,13 +405,26 @@ public class TajoAdmin {
     }
   }
 
-  public static void processKill(Writer writer, TajoClient client, String queryIdStr)
+  public void processKill(Writer writer, String queryIdStr)
       throws IOException, ServiceException {
-    boolean killedSuccessfully = client.killQuery(TajoIdUtils.parseQueryId(queryIdStr));
+    boolean killedSuccessfully = tajoClient.killQuery(TajoIdUtils.parseQueryId(queryIdStr));
     if (killedSuccessfully) {
       writer.write(queryIdStr + " is killed successfully.\n");
     } else {
       writer.write("killing query is failed.");
     }
   }
+
+  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/5ae608b1/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index 1402c9f..23b0def 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -265,7 +265,9 @@ public class QueryMasterTask extends CompositeService {
     @Override
     public void handle(LocalTaskEvent event) {
       TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId());
-      proxy.killTaskAttempt(event.getTaskAttemptId());
+      if (proxy != null) {
+        proxy.killTaskAttempt(event.getTaskAttemptId());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/5ae608b1/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java
new file mode 100644
index 0000000..9c6e760
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestExecExternalShellCommand.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.conf.TajoConf;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestExecExternalShellCommand {
+  @Test
+  public void testCommand() throws Exception {
+    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    TajoCli cli = new TajoCli(tajoConf, new String[]{}, null, out);
+
+    cli.executeMetaCommand("\\! echo \"this is test\"");
+    String consoleResult = new String(out.toByteArray());
+    assertEquals("this is test\n", consoleResult);
+
+    cli.executeMetaCommand("\\! error_command");
+    consoleResult = new String(out.toByteArray());
+    assertEquals("this is test\nERROR: /bin/bash: error_command: command not found\n", consoleResult);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/5ae608b1/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java
new file mode 100644
index 0000000..b51835f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/cli/TestHdfsCommand.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.cli;
+
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.conf.TajoConf;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHdfsCommand {
+  @Test
+  public void testHdfCommand() throws Exception {
+    TajoConf tajoConf = TpchTestBase.getInstance().getTestingCluster().getConfiguration();
+
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+    System.setOut(new PrintStream(out));
+    System.setErr(new PrintStream(out));
+    TajoCli cli = new TajoCli(tajoConf, new String[]{}, null, out);
+
+    cli.executeMetaCommand("\\dfs -test");
+    String consoleResult = new String(out.toByteArray());
+    assertEquals("-test: Not enough arguments: expected 1 but got 0\n" +
+        "Usage: hadoop fs [generic options] -test -[defsz] <path>\n", consoleResult);
+  }
+}