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 2013/10/04 13:29:30 UTC

git commit: TAJO-220: Implement catalog dump feature. (hyunsik)

Updated Branches:
  refs/heads/master 4449d9c48 -> b9baf526f


TAJO-220: Implement catalog dump feature. (hyunsik)


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

Branch: refs/heads/master
Commit: b9baf526f59bf668ddd9a80e53afd8360a10a6e8
Parents: 4449d9c
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Oct 4 20:28:07 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Oct 4 20:28:07 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 pom.xml                                         |   1 +
 .../org/apache/tajo/catalog/DDLBuilder.java     |  90 +++++++++++++
 .../java/org/apache/tajo/catalog/Options.java   |   4 +
 .../org/apache/tajo/catalog/TestDDLBuilder.java |  48 +++++++
 .../org/apache/tajo/engine/parser/SQLLexer.g4   |   2 +-
 .../main/java/org/apache/tajo/cli/TajoCli.java  |  18 +--
 .../java/org/apache/tajo/client/TajoClient.java |  12 +-
 .../java/org/apache/tajo/client/TajoDump.java   | 131 +++++++++++++++++++
 .../src/test/results/testBuildDDL.result        |   5 +
 tajo-dist/src/main/bin/tajo                     |  14 +-
 tajo-dist/src/main/bin/tajo_dump                |  22 ++++
 12 files changed, 318 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f57ac21..3632870 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@ Release 0.2.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-220: Implement catalog dump feature. (hyunsik)
+
     TAJO-216: Improve FilterPushDownRule and Implement physical operators 
     for outer join. (camelia_c via hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 136a5f1..1288547 100644
--- a/pom.xml
+++ b/pom.xml
@@ -313,6 +313,7 @@
             <exclude>**/*.tbl</exclude>
             <exclude>**/*.jsp</exclude>
             <exclude>**/web.xml</exclude>
+            <exclude>**/*.result</exclude>
             <!-- generated content -->
             <exclude>**/target/**</exclude>
             <exclude>**/*.log</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
new file mode 100644
index 0000000..87d32ca
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.catalog;
+
+import org.apache.tajo.common.TajoDataTypes;
+
+import java.util.Map;
+
+public class DDLBuilder {
+
+  public static String buildDDL(TableDesc desc) {
+    StringBuilder sb = new StringBuilder();
+
+    sb.append("--\n")
+      .append("-- Name: ").append(desc.getName()).append("; Type: TABLE;")
+      .append(" Storage: ").append(desc.getMeta().getStoreType().name());
+    sb.append("\n-- Path: ").append(desc.getPath());
+    sb.append("\n--\n");
+    sb.append("CREATE EXTERNAL TABLE ").append(desc.getName());
+    buildSchema(sb, desc.getMeta().getSchema());
+    buildUsingClause(sb, desc.getMeta());
+    buildWithClause(sb, desc.getMeta());
+    buildLocationClause(sb, desc);
+
+    sb.append(";");
+    return sb.toString();
+  }
+
+  private static void buildSchema(StringBuilder sb, Schema schema) {
+    boolean first = true;
+
+    sb.append(" (");
+    for (Column column : schema.toArray()) {
+      if (first) {
+        first = false;
+      } else {
+        sb.append(", ");
+      }
+
+      sb.append(column.getColumnName()).append(" ");
+      TajoDataTypes.DataType dataType = column.getDataType();
+      sb.append(dataType.getType().name());
+      if (column.getDataType().hasLength() && column.getDataType().getLength() > 0) {
+        sb.append(" (").append(column.getDataType().getLength()).append(")");
+      }
+    }
+    sb.append(")");
+  }
+
+  private static void buildUsingClause(StringBuilder sb, TableMeta meta) {
+    sb.append(" USING " + meta.getStoreType().name());
+  }
+
+  private static void buildWithClause(StringBuilder sb, TableMeta meta) {
+    Options options = meta.getOptions();
+    if (options != null && options.size() > 0) {
+      boolean first = true;
+      sb.append(" WITH (");
+      for (Map.Entry<String, String> entry : meta.getOptions().getAllKeyValus().entrySet()) {
+        if (first) {
+          first = false;
+        } else {
+          sb.append(", ");
+        }
+        sb.append("'").append(entry.getKey()).append("'='").append(entry.getValue()).append("'");
+      }
+      sb.append(")");
+    }
+  }
+
+  private static void buildLocationClause(StringBuilder sb, TableDesc desc) {
+    sb.append(" LOCATION '").append(desc.getPath()).append("'");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Options.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Options.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Options.java
index 230496c..e770b11 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Options.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Options.java
@@ -58,6 +58,10 @@ public class Options implements ProtoObject<KeyValueSetProto>, Cloneable, GsonOb
 	public static Options create(Options options) {
     return new Options(options);
   }
+
+  public int size() {
+    return keyVals.size();
+  }
 	
 	public void put(String key, String val) {
 		this.keyVals.put(key, val);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestDDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestDDLBuilder.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestDDLBuilder.java
new file mode 100644
index 0000000..876abb4
--- /dev/null
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestDDLBuilder.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.catalog;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.util.FileUtil;
+import org.junit.Test;
+
+import java.io.File;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestDDLBuilder {
+  @Test
+  public void testBuildDDL() throws Exception {
+    Schema schema = new Schema();
+    schema.addColumn("name", TajoDataTypes.Type.BLOB);
+    schema.addColumn("addr", TajoDataTypes.Type.TEXT);
+    TableMeta meta = CatalogUtil.newTableMeta(schema, CatalogProtos.StoreType.CSV);
+    meta.putOption("csv.delimiter", "|");
+    meta.putOption(TableMeta.COMPRESSION_CODEC, GzipCodec.class.getName());
+
+
+    TableDesc desc = new TableDescImpl("table1", meta, new Path("/table1"));
+
+    assertEquals(FileUtil.readTextFile(new File("src/test/results/testBuildDDL.result")), DDLBuilder.buildDDL(desc));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
index 26bcb67..03583cc 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4
@@ -349,7 +349,7 @@ BlockComment
     ;
 
 LineComment
-    :   '//' ~[\r\n]* -> skip
+    :   '--' ~[\r\n]* -> skip
     ;
 
 /*

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
index f2e9999..8980eb7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
@@ -267,8 +267,12 @@ public class TajoCli {
 
   public int executeStatements(String line) throws Exception {
 
+    // TODO - comment handling and multi line queries should be improved
+    // remove comments
+    String filtered = line.replaceAll("--[^\\r\\n]*", "").trim();
+
     String stripped;
-    for (String statement : line.split(";")) {
+    for (String statement : filtered.split(";")) {
       stripped = StringUtils.chomp(statement);
       if (StringUtils.isBlank(stripped)) {
         continue;
@@ -430,16 +434,6 @@ public class TajoCli {
     public abstract String getDescription();
   }
 
-  private void showTables() throws ServiceException {
-    List<String> tableList = client.getTableList();
-    if (tableList.size() == 0) {
-      sout.println("No Relation Found");
-    }
-    for (String table : tableList) {
-      sout.println(table);
-    }
-  }
-
   private String toFormattedString(TableDesc desc) {
     StringBuilder sb = new StringBuilder();
     sb.append("\ntable name: ").append(desc.getName()).append("\n");
@@ -507,7 +501,7 @@ public class TajoCli {
 
     @Override
     public String getDescription() {
-      return "list CLI commands";
+      return "show table description";
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
index e030b86..5e75403 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoProtos.QueryState;
+import org.apache.tajo.annotation.ThreadSafe;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
@@ -47,9 +48,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-/**
- * TajoClient is ThreadSafe
- */
+@ThreadSafe
 public class TajoClient {
   private final Log LOG = LogFactory.getLog(TajoClient.class);
 
@@ -88,9 +87,6 @@ public class TajoClient {
     } catch (Exception e) {
       throw new IOException(e);
     }
-
-    LOG.info("connected to tajo cluster (" +
-        org.apache.tajo.util.NetUtils.normalizeInetSocketAddress(addr) + ")");
   }
 
   public void close() {
@@ -188,8 +184,8 @@ public class TajoClient {
       queryMasterConnectionMap.put(queryId, service);
       queryMasterClientMap.put(queryId, client);
 
-      LOG.info("Connected to Query Master (qid=" + queryId + ", addr=" +
-          NetUtils.normalizeInetSocketAddress(addr) + ")");
+      LOG.info("Connected to Query Master (qid=" + queryId + ", addr=" + NetUtils.normalizeInetSocketAddress(addr)
+          + ")");
     } catch (Exception e) {
       LOG.error(e.getMessage());
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoDump.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoDump.java
new file mode 100644
index 0000000..84d5646
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoDump.java
@@ -0,0 +1,131 @@
+/**
+ * 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.common.collect.Lists;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.cli.*;
+import org.apache.commons.cli.Options;
+import org.apache.tajo.catalog.DDLBuilder;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.function.builtin.Date;
+
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+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] [table_name]", options );
+  }
+
+  public static void main(String [] args) throws ParseException, IOException, ServiceException {
+    TajoConf conf = new TajoConf();
+
+    CommandLineParser parser = new PosixParser();
+    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"));
+    }
+
+    // if there is no "-h" option,
+    if(hostName == null) {
+      if (conf.getVar(TajoConf.ConfVars.CLIENT_SERVICE_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.CLIENT_SERVICE_ADDRESS).split(":")[0];
+      }
+    }
+    if (port == null) {
+      if (conf.getVar(TajoConf.ConfVars.CLIENT_SERVICE_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.CLIENT_SERVICE_ADDRESS).split(":")[1]);
+      }
+    }
+
+    TajoClient client = null;
+    if ((hostName == null) ^ (port == null)) {
+      System.err.println("ERROR: cannot find valid Tajo server address");
+      System.exit(-1);
+    } else if (hostName != null && port != null) {
+      conf.setVar(TajoConf.ConfVars.CLIENT_SERVICE_ADDRESS, hostName+":"+port);
+      client = new TajoClient(conf);
+    } else if (hostName == null && port == null) {
+      client = new TajoClient(conf);
+    }
+
+    List<TableDesc> tableDescList = Lists.newArrayList();
+
+    if (cmd.hasOption("a")) {
+      for (String tableName : client.getTableList()) {
+        tableDescList.add(client.getTableDesc(tableName));
+      }
+    } else if (cmd.getArgs().length > 0) {
+      for (String tableName : cmd.getArgs()) {
+        tableDescList.add(client.getTableDesc(tableName));
+      }
+    } else {
+      printUsage();
+    }
+
+
+    Writer writer = new PrintWriter(System.out);
+    writer.write("--\n");
+    writer.write("-- Tajo database dump\n");
+    writer.write("-- Dump date: " + toDateString() + "\n");
+    writer.write("--\n");
+    writer.write("\n");
+    for (TableDesc tableDesc : tableDescList) {
+      writer.write(DDLBuilder.buildDDL(tableDesc));
+      writer.write("\n\n");
+    }
+    writer.flush();
+    writer.close();
+    System.exit(0);
+  }
+
+  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/incubator-tajo/blob/b9baf526/tajo-core/tajo-core-backend/src/test/results/testBuildDDL.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/results/testBuildDDL.result b/tajo-core/tajo-core-backend/src/test/results/testBuildDDL.result
new file mode 100644
index 0000000..4fd87c2
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/results/testBuildDDL.result
@@ -0,0 +1,5 @@
+--
+-- Name: table1; Type: TABLE; Storage: CSV
+-- Path: /table1
+--
+CREATE EXTERNAL TABLE table1 (name BLOB, addr TEXT) USING CSV WITH ('csv.delimiter'='|', 'compression.codec'='org.apache.hadoop.io.compress.GzipCodec') LOCATION '/table1';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-dist/src/main/bin/tajo
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo
index baf7c7b..fb62acf 100755
--- a/tajo-dist/src/main/bin/tajo
+++ b/tajo-dist/src/main/bin/tajo
@@ -280,18 +280,12 @@ elif [ "$COMMAND" = "worker" ] ; then
 elif [ "$COMMAND" = "catalog" ] ; then
   CLASS='org.apache.tajo.catalog.CatalogServer'
   TAJO_OPTS="$TAJO_OPTS $TAJO_CATALOG_OPTS"
-elif [ "$COMMAND" = "catutil" ] ; then
-  CLASS='org.apache.tajo.catalog.CatalogUtil'
-  TAJO_OPTS="$TAJO_OPTS $TAJO_CATALOG_OPTS"
-elif [ "$COMMAND" = "cluster" ] ; then
-  CLASS='org.apache.tajo.engine.cluster.ClusterUtil'
-  TAJO_OPTS="$TAJO_OPTS $TAJO_ZKSERVER_OPTS"
 elif [ "$COMMAND" = "cli" ] ; then
   CLASS='org.apache.tajo.cli.TajoCli'
-  TAJO_OPTS="$TAJO_OPTS $TAJO_ZKSERVER_OPTS"
-elif [ "$COMMAND" = "benchmark" ] ; then
-  CLASS='org.apache.tajo.benchmark.Driver'
-  TAJO_OPTS="$TAJO_OPTS $TAJO_ZKSERVER_OPTS"
+  TAJO_OPTS="$TAJO_OPTS $TAJO_CLI_OPTS"
+elif [ "$COMMAND" = "dump" ] ; then
+  CLASS='org.apache.tajo.client.TajoDump'
+  TAJO_OPTS="$TAJO_OPTS $TAJO_DUMP_OPTS"
 else
   CLASS=$COMMAND
 fi

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/b9baf526/tajo-dist/src/main/bin/tajo_dump
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo_dump b/tajo-dist/src/main/bin/tajo_dump
new file mode 100755
index 0000000..fe0f50f
--- /dev/null
+++ b/tajo-dist/src/main/bin/tajo_dump
@@ -0,0 +1,22 @@
+#!/usr/bin/env bash
+
+# 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.
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/tajo dump "$@"