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/08/20 15:36:39 UTC

[3/3] tajo git commit: TAJO-1792: tajo-cluster-tests is not available when it is used as an external maven module.

TAJO-1792: tajo-cluster-tests is not available when it is used as an external maven module.

Closes #704


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

Branch: refs/heads/master
Commit: e296a0d1f0696a52e95d8b34e016d7a272c9a1a6
Parents: 2d2f192
Author: Hyunsik Choi <hy...@apache.org>
Authored: Thu Aug 20 22:34:35 2015 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Thu Aug 20 22:34:35 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../org/apache/tajo/jdbc/MetaDataTuple.java     | 200 ++++++
 .../apache/tajo/jdbc/TajoMetaDataResultSet.java |  74 +++
 tajo-cluster-tests/pom.xml                      |   7 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |  21 +-
 .../test/java/org/apache/tajo/TpchTestBase.java |  23 +-
 .../src/test/resources/tpch/customer.tbl        |   5 +
 .../src/test/resources/tpch/empty_orders.tbl    |   0
 .../src/test/resources/tpch/lineitem.tbl        |   5 +
 .../src/test/resources/tpch/nation.tbl          |  25 +
 .../src/test/resources/tpch/orders.tbl          |   3 +
 .../src/test/resources/tpch/part.tbl            |   4 +
 .../src/test/resources/tpch/partsupp.tbl        |   3 +
 .../src/test/resources/tpch/region.tbl          |   5 +
 .../src/test/resources/tpch/supplier.tbl        |   3 +
 .../java/org/apache/tajo/util/FileUtil.java     |  18 +-
 tajo-core-tests/pom.xml                         |   4 +
 .../apache/tajo/engine/query/TestJoinQuery.java |   1 -
 .../org/apache/tajo/jdbc/TestResultSet.java     | 224 -------
 .../java/org/apache/tajo/jdbc/TestSQLState.java |  91 ---
 .../tajo/jdbc/TestTajoDatabaseMetaData.java     | 504 ---------------
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java | 607 -------------------
 .../apache/tajo/querymaster/TestKillQuery.java  |   6 +-
 .../dataset/TestTajoJdbc/table1/table1.tbl      |   5 -
 .../TestTajoJdbc/create_table_with_date_ddl.sql |  10 -
 .../TestTajoDatabaseMetaData/getSchemas1.result |   3 -
 .../TestTajoDatabaseMetaData/getSchemas2.result |   3 -
 .../getTableTypes.result                        |   3 -
 .../TestTajoDatabaseMetaData/getTables1.result  |   4 -
 .../TestTajoDatabaseMetaData/getTables2.result  |   4 -
 .../TestTajoJdbc/testSortWithDateTime.result    |   7 -
 .../src/test/resources/tpch/customer.tbl        |   5 -
 .../src/test/resources/tpch/empty_orders.tbl    |   0
 .../src/test/resources/tpch/lineitem.tbl        |   5 -
 .../src/test/resources/tpch/nation.tbl          |  25 -
 .../src/test/resources/tpch/orders.tbl          |   3 -
 .../src/test/resources/tpch/part.tbl            |   4 -
 .../src/test/resources/tpch/partsupp.tbl        |   3 -
 .../src/test/resources/tpch/region.tbl          |   5 -
 .../src/test/resources/tpch/supplier.tbl        |   3 -
 tajo-core/pom.xml                               |  12 +-
 .../java/org/apache/tajo/benchmark/TPCH.java    |   8 -
 .../tajo/engine/function/FunctionLoader.java    |  11 +-
 tajo-jdbc/pom.xml                               |  78 +++
 .../org/apache/tajo/jdbc/MetaDataTuple.java     | 200 ------
 .../apache/tajo/jdbc/TajoDatabaseMetaData.java  |   1 -
 .../apache/tajo/jdbc/TajoMetaDataResultSet.java |  74 ---
 .../org/apache/tajo/jdbc/TestResultSet.java     | 224 +++++++
 .../java/org/apache/tajo/jdbc/TestSQLState.java |  91 +++
 .../tajo/jdbc/TestTajoDatabaseMetaData.java     | 504 +++++++++++++++
 .../java/org/apache/tajo/jdbc/TestTajoJdbc.java | 607 +++++++++++++++++++
 .../dataset/TestTajoJdbc/table1/table1.tbl      |   5 +
 .../TestTajoJdbc/create_table_with_date_ddl.sql |  10 +
 .../TestTajoDatabaseMetaData/getSchemas1.result |   3 +
 .../TestTajoDatabaseMetaData/getSchemas2.result |   3 +
 .../getTableTypes.result                        |   3 +
 .../TestTajoDatabaseMetaData/getTables1.result  |   4 +
 .../TestTajoDatabaseMetaData/getTables2.result  |   4 +
 .../TestTajoJdbc/testSortWithDateTime.result    |   7 +
 .../function/python/PythonScriptEngine.java     | 147 +++--
 60 files changed, 2012 insertions(+), 1912 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 1c2a9e7..d7baf63 100644
--- a/CHANGES
+++ b/CHANGES
@@ -34,6 +34,9 @@ Release 0.11.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1792: tajo-cluster-tests is not available when it is used as an 
+    external maven module. (hyunsik)
+
     TAJO-1775: TAJO-1775: HCatalogStore need to be deprecated. (jaehwa)
 
     TAJO-1745: Add positive and negative test methods. (hyunsik)

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-client/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
new file mode 100644
index 0000000..bd078f6
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/MetaDataTuple.java
@@ -0,0 +1,200 @@
+/**
+ * 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.jdbc;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.IntervalDatum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.exception.TajoRuntimeException;
+import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.datetime.TimeMeta;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class MetaDataTuple implements Tuple {
+  List<Datum> values = new ArrayList<Datum>();
+
+  public MetaDataTuple(int size) {
+    values = new ArrayList<Datum>(size);
+    for(int i = 0; i < size; i++) {
+      values.add(NullDatum.get());
+    }
+  }
+
+  @Override
+  public int size() {
+    return values.size();
+  }
+
+  @Override
+  public boolean contains(int fieldid) {
+    return false;
+  }
+
+  @Override
+  public boolean isBlank(int fieldid) {
+    return values.get(fieldid) == null;
+  }
+
+  @Override
+  public boolean isBlankOrNull(int fieldid) {
+    return values.get(fieldid) == null || values.get(fieldid).isNull();
+  }
+
+  @Override
+  public void put(int fieldId, Tuple tuple) {
+    this.put(fieldId, tuple.asDatum(fieldId));
+  }
+
+  @Override
+  public void clear() {
+    values.clear();
+  }
+
+  @Override
+  public void put(int fieldId, Datum value) {
+    values.set(fieldId, value);
+  }
+
+  @Override
+  public void put(Datum[] values) {
+    for (int i = 0; i < values.length; i++) {
+      this.values.set(i, values[i]);
+    }
+  }
+
+  @Override
+  public TajoDataTypes.Type type(int fieldId) {
+    return values.get(fieldId).type();
+  }
+
+  @Override
+  public int size(int fieldId) {
+    return values.get(fieldId).size();
+  }
+
+  @Override
+  public void clearOffset() {
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+
+  @Override
+  public Datum asDatum(int fieldId) {
+    return values.get(fieldId);
+  }
+
+  @Override
+  public void setOffset(long offset) {
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+
+  @Override
+  public long getOffset() {
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+
+  @Override
+  public boolean getBool(int fieldId) {
+    return values.get(fieldId).asBool();
+  }
+
+  @Override
+  public byte getByte(int fieldId) {
+    return values.get(fieldId).asByte();
+  }
+
+  @Override
+  public char getChar(int fieldId) {
+    return values.get(fieldId).asChar();
+  }
+
+  @Override
+  public byte [] getBytes(int fieldId) {
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+
+  @Override
+  public byte[] getTextBytes(int fieldId) {
+    return values.get(fieldId).asTextBytes();
+  }
+
+  @Override
+  public short getInt2(int fieldId) {
+    return values.get(fieldId).asInt2();
+  }
+
+  @Override
+  public int getInt4(int fieldId) {
+    return values.get(fieldId).asInt4();
+  }
+
+  @Override
+  public long getInt8(int fieldId) {
+    return values.get(fieldId).asInt8();
+  }
+
+  @Override
+  public float getFloat4(int fieldId) {
+    return values.get(fieldId).asFloat4();
+  }
+
+  @Override
+  public double getFloat8(int fieldId) {
+    return values.get(fieldId).asFloat8();
+  }
+
+  @Override
+  public String getText(int fieldId) {
+    return values.get(fieldId).asChars();
+  }
+
+  @Override
+  public TimeMeta getTimeDate(int fieldId) {
+    return values.get(fieldId).asTimeMeta();
+  }
+
+  @Override
+  public ProtobufDatum getProtobufDatum(int fieldId) {
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+
+  @Override
+  public IntervalDatum getInterval(int fieldId) {
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+
+  @Override
+  public char[] getUnicodeChars(int fieldId) {
+    return values.get(fieldId).asUnicodeChars();
+  }
+
+  @Override
+  public Tuple clone() throws CloneNotSupportedException {
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+
+  @Override
+  public Datum[] getValues(){
+    throw new TajoRuntimeException(new UnsupportedException());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java
new file mode 100644
index 0000000..9fba40a
--- /dev/null
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoMetaDataResultSet.java
@@ -0,0 +1,74 @@
+/**
+ * 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.jdbc;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+public class TajoMetaDataResultSet extends TajoResultSetBase {
+  private List<MetaDataTuple> values;
+
+  public TajoMetaDataResultSet(Schema schema, List<MetaDataTuple> values) {
+    super(null, schema, null);
+    setDataTuples(values);
+  }
+
+  public TajoMetaDataResultSet(List<String> columns, List<Type> types, List<MetaDataTuple> values) {
+    super(null, new Schema(), null);
+    int index = 0;
+    if(columns != null) {
+      for(String columnName: columns) {
+        schema.addColumn(columnName, types.get(index++));
+      }
+    }
+    setDataTuples(values);
+  }
+
+  protected void setDataTuples(List<MetaDataTuple> values) {
+    this.values = values;
+    this.totalRow = values == null ? 0 : values.size();
+  }
+
+  @Override
+  protected Tuple nextTuple() throws IOException {
+    if(curRow >= totalRow) {
+      return null;
+    }
+    return values.get(curRow);
+  }
+
+  @Override
+  public void close() throws SQLException {
+  }
+
+  @Override
+  public String getString(int fieldId) throws SQLException {
+    return cur.getText(fieldId - 1);
+  }
+
+  @Override
+  public String getString(String name) throws SQLException {
+    return cur.getText(findColumn(name));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/pom.xml b/tajo-cluster-tests/pom.xml
index 6fbd5d8..2816048 100644
--- a/tajo-cluster-tests/pom.xml
+++ b/tajo-cluster-tests/pom.xml
@@ -57,12 +57,7 @@
           <excludes>
             <exclude>derby.log</exclude>
             <exclude>benchmark/**</exclude>
-            <exclude>src/test/tpch/**</exclude>
-            <exclude>src/test/resources/dataset/**</exclude>
-            <exclude>src/test/resources/queries/**</exclude>
-            <exclude>src/test/resources/results/**</exclude>
-            <exclude>src/main/resources/META-INF/services/*</exclude>
-            <exclude>src/main/resources/webapps/static/js/*</exclude>
+            <exclude>src/test/resources/tpch/**</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
index bfc7d7b..85b72f2 100644
--- a/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-cluster-tests/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -161,23 +161,18 @@ public class QueryTestCaseBase {
   protected static LogicalPlanVerifier postVerifier;
 
   /** the base path of dataset directories */
-  protected static final Path datasetBasePath;
+  protected static Path datasetBasePath;
   /** the base path of query directories */
-  protected static final Path queryBasePath;
+  protected static Path queryBasePath;
   /** the base path of result directories */
-  protected static final Path resultBasePath;
+  protected static Path resultBasePath;
 
   static {
     testBase = TpchTestBase.getInstance();
     testingCluster = testBase.getTestingCluster();
     conf = testBase.getTestingCluster().getConfiguration();
     catalog = testBase.getTestingCluster().getMaster().getCatalog();
-    URL datasetBaseURL = ClassLoader.getSystemResource("dataset");
-    datasetBasePath = new Path(datasetBaseURL.toString());
-    URL queryBaseURL = ClassLoader.getSystemResource("queries");
-    queryBasePath = new Path(queryBaseURL.toString());
-    URL resultBaseURL = ClassLoader.getSystemResource("results");
-    resultBasePath = new Path(resultBaseURL.toString());
+
 
     GlobalEngine engine = testingCluster.getMaster().getContext().getGlobalEngine();
     sqlParser = engine.getAnalyzer();
@@ -206,8 +201,14 @@ public class QueryTestCaseBase {
 
   @BeforeClass
   public static void setUpClass() throws Exception {
-    conf = testBase.getTestingCluster().getConfiguration();
     client = testBase.getTestingCluster().newTajoClient();
+
+    URL datasetBaseURL = ClassLoader.getSystemResource("dataset");
+    datasetBasePath = new Path(datasetBaseURL.toString());
+    URL queryBaseURL = ClassLoader.getSystemResource("queries");
+    queryBasePath = new Path(queryBaseURL.toString());
+    URL resultBaseURL = ClassLoader.getSystemResource("results");
+    resultBasePath = new Path(resultBaseURL.toString());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/java/org/apache/tajo/TpchTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/java/org/apache/tajo/TpchTestBase.java b/tajo-cluster-tests/src/test/java/org/apache/tajo/TpchTestBase.java
index 2824f9a..027e735 100644
--- a/tajo-cluster-tests/src/test/java/org/apache/tajo/TpchTestBase.java
+++ b/tajo-cluster-tests/src/test/java/org/apache/tajo/TpchTestBase.java
@@ -21,12 +21,16 @@ package org.apache.tajo;
 import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
 import org.apache.tajo.benchmark.TPCH;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.KeyValueSet;
 
 import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.sql.ResultSet;
 import java.util.Map;
@@ -68,10 +72,14 @@ public class TpchTestBase {
       schemas[i] = tpch.getSchema(names[i]);
     }
 
-    File file;
+    // create a temporal table
+    File tpchTablesDir = new File(new File(CommonTestingUtil.getTestDir().toUri()), "tpch");
+
     for (int i = 0; i < names.length; i++) {
-      file = TPCH.getDataFile(names[i]);
-      paths[i] = file.getAbsolutePath();
+      String str = FileUtil.readTextFileFromResource("tpch/" + names[i] + ".tbl");
+      Path tablePath = new Path(new Path(tpchTablesDir.toURI()), names[i] + ".tbl");
+      FileUtil.writeTextToFile(str, tablePath);
+      paths[i] = tablePath.toString();
     }
     try {
       Thread.sleep(1000);
@@ -99,11 +107,10 @@ public class TpchTestBase {
     return util.getTestingCluster();
   }
 
-  public void tearDown() throws IOException {
-    try {
-      Thread.sleep(2000);
-    } catch (InterruptedException e) {
+  public String getPath(String tableName) {
+    if (!nameMap.containsKey(tableName)) {
+      throw new RuntimeException("No such a table name '" + tableName + "'");
     }
-    util.shutdown();
+    return paths[nameMap.get(tableName)];
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/customer.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/customer.tbl b/tajo-cluster-tests/src/test/resources/tpch/customer.tbl
new file mode 100644
index 0000000..4f684c6
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/customer.tbl
@@ -0,0 +1,5 @@
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/empty_orders.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/empty_orders.tbl b/tajo-cluster-tests/src/test/resources/tpch/empty_orders.tbl
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/lineitem.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/lineitem.tbl b/tajo-cluster-tests/src/test/resources/tpch/lineitem.tbl
new file mode 100644
index 0000000..e3beac9
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/lineitem.tbl
@@ -0,0 +1,5 @@
+1|1|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the|
+1|1|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |
+2|2|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|
+3|2|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|
+3|3|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/nation.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/nation.tbl b/tajo-cluster-tests/src/test/resources/tpch/nation.tbl
new file mode 100644
index 0000000..ed3fd5b
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/nation.tbl
@@ -0,0 +1,25 @@
+0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|
+1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|
+2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |
+3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|
+4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|
+5|ETHIOPIA|0|ven packages wake quickly. regu|
+6|FRANCE|3|refully final requests. regular, ironi|
+7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|
+8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|
+9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|
+10|IRAN|4|efully alongside of the slyly final dependencies. |
+11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|
+12|JAPAN|2|ously. final, express gifts cajole a|
+13|JORDAN|4|ic deposits are blithely about the carefully regular pa|
+14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|
+15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|
+16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|
+17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|
+18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|
+19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|
+20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|
+21|VIETNAM|2|hely enticingly express accounts. even, final |
+22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|
+23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|
+24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/orders.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/orders.tbl b/tajo-cluster-tests/src/test/resources/tpch/orders.tbl
new file mode 100644
index 0000000..15a1b6f
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/orders.tbl
@@ -0,0 +1,3 @@
+1|3|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |
+2|4|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|
+3|2|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/part.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/part.tbl b/tajo-cluster-tests/src/test/resources/tpch/part.tbl
new file mode 100644
index 0000000..6e6fa72
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/part.tbl
@@ -0,0 +1,4 @@
+1|goldenrod lavender spring chocolate lace|Manufacturer#1|Brand#13|PROMO BURNISHED COPPER|7|JUMBO PKG|901.00|ly. slyly ironi
+2|blush thistle blue yellow saddle|Manufacturer#1|Brand#13|LARGE BRUSHED BRASS|15|LG CASE|902.00|lar accounts amo
+3|spring green yellow purple cornsilk|Manufacturer#4|Brand#42|STANDARD POLISHED BRASS|21|WRAP CASE|903.00|egular deposits hag
+4|cornflower chocolate smoke green pink|Manufacturer#3|Brand#34|SMALL PLATED BRASS|14|MED DRUM|904.00|p furiously r
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/partsupp.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/partsupp.tbl b/tajo-cluster-tests/src/test/resources/tpch/partsupp.tbl
new file mode 100644
index 0000000..a6211e6
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/partsupp.tbl
@@ -0,0 +1,3 @@
+1|2|3325|771.64|, even theodolites. regular, final theodolites eat after the carefully pending foxes. furiously regular deposits sleep slyly. carefully bold realms above the ironic dependencies haggle careful|
+2|3|8895|1.01|nic accounts. final accounts sleep furiously about the ironic, bold packages. regular, regular accounts|
+3|4|4651|920.92|ilent foxes affix furiously quickly unusual requests. even packages across the carefully even theodolites nag above the sp|

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/region.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/region.tbl b/tajo-cluster-tests/src/test/resources/tpch/region.tbl
new file mode 100644
index 0000000..c5ebb63
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/region.tbl
@@ -0,0 +1,5 @@
+0|AFRICA|lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to |
+1|AMERICA|hs use ironic, even requests. s|
+2|ASIA|ges. thinly even pinto beans ca|
+3|EUROPE|ly final courts cajole furiously final excuse|
+4|MIDDLE EAST|uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl|

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-cluster-tests/src/test/resources/tpch/supplier.tbl
----------------------------------------------------------------------
diff --git a/tajo-cluster-tests/src/test/resources/tpch/supplier.tbl b/tajo-cluster-tests/src/test/resources/tpch/supplier.tbl
new file mode 100644
index 0000000..a6fafb3
--- /dev/null
+++ b/tajo-cluster-tests/src/test/resources/tpch/supplier.tbl
@@ -0,0 +1,3 @@
+2|Supplier#000000002|89eJ5ksX3ImxJQBvxObC,|5|15-679-861-2259|4032.68| slyly bold instructions. idle dependen|
+3|Supplier#000000003|q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3|1|11-383-516-1199|4192.40|blithely silent requests after the express dependencies are sl|
+4|Supplier#000000004|Bk7ah4CK8SYQTepEmvMkkgMwg|15|25-843-787-7479|4641.08|riously even requests above the exp|

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
index a7f8691..5216eb8 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
@@ -22,6 +22,7 @@ import com.google.protobuf.Message;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.conf.TajoConf;
 
 import java.io.*;
 import java.net.URL;
@@ -132,8 +133,17 @@ public class FileUtil {
     return fileData.toString();
   }
 
-  public static void writeTextToStream(String text, OutputStream outputStream)
-      throws IOException {
+  public static void writeTextToFile(String text, Path path) throws IOException {
+    FileSystem fs = path.getFileSystem(new TajoConf());
+    if (!fs.exists(path.getParent())) {
+      fs.mkdirs(path.getParent());
+    }
+    FSDataOutputStream out = fs.create(path);
+    out.write(text.getBytes());
+    out.close();
+  }
+
+  public static void writeTextToStream(String text, OutputStream outputStream) throws IOException {
     try {
       outputStream.write(text.getBytes());
     } finally {
@@ -149,10 +159,6 @@ public class FileUtil {
     return String.format("%.1f %sB", bytes / Math.pow(unit, exp), pre);
   }
 
-  public static boolean isLocalPath(Path path) {
-    return path.toUri().getScheme().equals("file");
-  }
-
 
   /**
    * Close the Closeable objects and <b>ignore</b> any {@link IOException} or

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core-tests/pom.xml b/tajo-core-tests/pom.xml
index 38ce8be..a4c0587 100644
--- a/tajo-core-tests/pom.xml
+++ b/tajo-core-tests/pom.xml
@@ -253,6 +253,10 @@
           <groupId>com.sun.jersey.jersey-test-framework</groupId>
           <artifactId>jersey-test-framework-grizzly2</artifactId>
         </exclusion>
+        <exclusion>
+          <artifactId>netty-all</artifactId>
+          <groupId>io.netty</groupId>
+        </exclusion>
       </exclusions>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
index 177d1cb..e5b9b98 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -41,7 +41,6 @@ import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.junit.runners.Parameterized.Parameters;
 
-import java.io.File;
 import java.io.OutputStream;
 import java.sql.SQLException;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestResultSet.java b/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
deleted file mode 100644
index 0c83fd0..0000000
--- a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
+++ /dev/null
@@ -1,224 +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.jdbc;
-
-import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.IntegrationTest;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.TpchTestBase;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.util.KeyValueSet;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.IOException;
-import java.sql.*;
-import java.util.Calendar;
-import java.util.List;
-import java.util.TimeZone;
-
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class TestResultSet {
-  private static TajoTestingCluster util;
-  private static TajoConf conf;
-  private static TableDesc desc;
-  private static FileTablespace sm;
-  private static TableMeta scoreMeta;
-  private static Schema scoreSchema;
-  private static List<ByteString> serializedData;
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    util = TpchTestBase.getInstance().getTestingCluster();
-    conf = util.getConfiguration();
-    sm = TablespaceManager.getDefault();
-
-    scoreSchema = new Schema();
-    scoreSchema.addColumn("deptname", Type.TEXT);
-    scoreSchema.addColumn("score", Type.INT4);
-    scoreMeta = CatalogUtil.newTableMeta("TEXT");
-    TableStats stats = new TableStats();
-
-    Path p = new Path(sm.getTableUri("default", "score"));
-    sm.getFileSystem().mkdirs(p);
-    Appender appender = sm.getAppender(scoreMeta, scoreSchema, new Path(p, "score"));
-    RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(scoreSchema);
-    serializedData = Lists.newArrayList();
-    appender.init();
-
-    int deptSize = 100;
-    int tupleNum = 10000;
-    Tuple tuple;
-    long written = 0;
-    for (int i = 0; i < tupleNum; i++) {
-      tuple = new VTuple(2);
-      String key = "test" + (i % deptSize);
-      tuple.put(0, DatumFactory.createText(key));
-      tuple.put(1, DatumFactory.createInt4(i + 1));
-      written += key.length() + Integer.SIZE;
-      appender.addTuple(tuple);
-      serializedData.add(ByteString.copyFrom(encoder.toBytes(tuple)));
-    }
-    appender.close();
-    stats.setNumRows(tupleNum);
-    stats.setNumBytes(written);
-    stats.setAvgRows(tupleNum);
-    stats.setNumBlocks(1000);
-    stats.setNumShuffleOutputs(100);
-    desc = new TableDesc(CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "score"),
-        scoreSchema, scoreMeta, p.toUri());
-    desc.setStats(stats);
-  }
-
-  @AfterClass
-  public static void terminate() throws IOException {
-
-  }
-
-  @Test
-  public void testMemoryResultSet() throws Exception {
-    TajoMemoryResultSet rs = new TajoMemoryResultSet(null, desc.getSchema(),
-        serializedData, desc.getStats().getNumRows().intValue(), null);
-
-    ResultSetMetaData meta = rs.getMetaData();
-    assertNotNull(meta);
-    Schema schema = scoreSchema;
-    assertEquals(schema.size(), meta.getColumnCount());
-    for (int i = 0; i < meta.getColumnCount(); i++) {
-      assertEquals(schema.getColumn(i).getSimpleName(), meta.getColumnName(i + 1));
-      assertEquals(schema.getColumn(i).getQualifier(), meta.getTableName(i + 1));
-    }
-
-    int i = 0;
-    assertTrue(rs.isBeforeFirst());
-    for (; rs.next(); i++) {
-      assertEquals("test"+i%100, rs.getString(1));
-      assertEquals("test"+i%100, rs.getString("deptname"));
-      assertEquals(i+1, rs.getInt(2));
-      assertEquals(i+1, rs.getInt("score"));
-    }
-    assertEquals(10000, i);
-    assertTrue(rs.isAfterLast());
-  }
-
-  @Test
-  public void testDateTimeType() throws Exception {
-    // HiveCatalog does not support date type, time type in hive-0.12.0
-    if(util.isHiveCatalogStoreRunning()) return;
-
-    ResultSet res = null;
-    TajoClient client = util.newTajoClient();
-    try {
-      String tableName = "datetimetable";
-      String query = "select col1, col2, col3 from " + tableName;
-
-      String [] table = new String[] {tableName};
-      Schema schema = new Schema();
-      schema.addColumn("col1", Type.DATE);
-      schema.addColumn("col2", Type.TIME);
-      schema.addColumn("col3", Type.TIMESTAMP);
-      Schema [] schemas = new Schema[] {schema};
-      String [] data = {
-          "2014-01-01|01:00:00|2014-01-01 01:00:00"
-      };
-      KeyValueSet tableOptions = new KeyValueSet();
-      tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-
-      res = TajoTestingCluster
-          .run(table, schemas, tableOptions, new String[][]{data}, query, client);
-
-      assertTrue(res.next());
-
-      Date date = res.getDate(1);
-      assertNotNull(date);
-      assertEquals(Date.valueOf("2014-01-01"), date);
-
-      date = res.getDate("col1");
-      assertNotNull(date);
-      assertEquals(Date.valueOf("2014-01-01"), date);
-
-      Time time = res.getTime(2);
-      assertNotNull(time);
-      assertEquals(Time.valueOf("01:00:00"), time);
-
-      time = res.getTime("col2");
-      assertNotNull(time);
-      assertEquals(Time.valueOf("01:00:00"), time);
-
-      Timestamp timestamp = res.getTimestamp(3);
-      assertNotNull(timestamp);
-      assertEquals(Timestamp.valueOf("2014-01-01 01:00:00"), timestamp);
-
-      timestamp = res.getTimestamp("col3");
-      assertNotNull(timestamp);
-      assertEquals(Timestamp.valueOf("2014-01-01 01:00:00"), timestamp);
-
-      // assert with timezone
-      Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("GMT+9"));
-      date = res.getDate(1, cal);
-      assertNotNull(date);
-      assertEquals("2014-01-01", date.toString());
-
-      date = res.getDate("col1", cal);
-      assertNotNull(date);
-      assertEquals("2014-01-01", date.toString());
-
-      time = res.getTime(2, cal);
-      assertNotNull(time);
-      assertEquals("10:00:00", time.toString());
-
-      time = res.getTime("col2", cal);
-      assertNotNull(time);
-      assertEquals("10:00:00", time.toString());
-
-      timestamp = res.getTimestamp(3, cal);
-      assertNotNull(timestamp);
-      assertEquals("2014-01-01 10:00:00.0", timestamp.toString());
-
-      timestamp = res.getTimestamp("col3", cal);
-      assertNotNull(timestamp);
-      assertEquals("2014-01-01 10:00:00.0", timestamp.toString());
-    } finally {
-      if (res != null) {
-        res.close();
-      }
-
-      client.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestSQLState.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestSQLState.java b/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestSQLState.java
deleted file mode 100644
index e711524..0000000
--- a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestSQLState.java
+++ /dev/null
@@ -1,91 +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.jdbc;
-
-import com.google.common.collect.Maps;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.client.QueryClient;
-import org.apache.tajo.client.QueryStatus;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.net.InetSocketAddress;
-import java.sql.*;
-import java.util.*;
-
-import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class TestSQLState extends QueryTestCaseBase {
-  private static InetSocketAddress tajoMasterAddress;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    tajoMasterAddress = testingCluster.getMaster().getTajoMasterClientService().getBindAddress();
-    Class.forName("org.apache.tajo.jdbc.TajoDriver").newInstance();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-  }
-
-  static String buildConnectionUri(String hostName, int port, String databaseName) {
-    return "jdbc:tajo://" + hostName + ":" + port + "/" + databaseName;
-  }
-
-  private Connection makeConnection() throws SQLException {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-    assertTrue(conn.isValid(100));
-
-    return conn;
-  }
-
-  public void assertSQLState(String sql, String sqlState) throws SQLException {
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet res = null;
-
-    try {
-      conn = makeConnection();
-      stmt = conn.createStatement();
-      res = stmt.executeQuery(sql);
-    } catch (SQLException se) {
-      assertEquals(sqlState, se.getSQLState());
-    } catch (Throwable t) {
-      fail(t.getMessage());
-    } finally {
-      CatalogUtil.closeQuietly(stmt, res);
-      CatalogUtil.closeQuietly(conn);
-    }
-  }
-
-  @Test
-  public void testSyntaxError() throws Exception {
-    assertSQLState("selec x,y,x from lineitem", "42601");
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java b/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java
deleted file mode 100644
index 8ee6755..0000000
--- a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoDatabaseMetaData.java
+++ /dev/null
@@ -1,504 +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.jdbc;
-
-import com.google.common.collect.Sets;
-import org.apache.tajo.QueryTestCaseBase;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.common.type.TajoTypeUtil;
-import org.apache.tajo.util.TUtil;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.net.InetSocketAddress;
-import java.sql.*;
-import java.util.*;
-
-import static org.junit.Assert.*;
-
-public class TestTajoDatabaseMetaData extends QueryTestCaseBase {
-  private static InetSocketAddress tajoMasterAddress;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    tajoMasterAddress = testingCluster.getMaster().getTajoMasterClientService().getBindAddress();
-    Class.forName("org.apache.tajo.jdbc.TajoDriver").newInstance();
-  }
-
-  public static List<String> getListFromResultSet(ResultSet resultSet, String columnName) throws SQLException {
-    List<String> list = new ArrayList<String>();
-    while(resultSet.next()) {
-      list.add(resultSet.getString(columnName));
-    }
-    return list;
-  }
-
-  @Test
-  public void testSetAndGetCatalogAndSchema() throws Exception {
-    String connUri = TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-
-    assertDatabaseNotExists("jdbc_test1");
-    PreparedStatement pstmt = conn.prepareStatement("CREATE DATABASE jdbc_test1");
-    pstmt.executeUpdate();
-    assertDatabaseExists("jdbc_test1");
-    pstmt.close();
-
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      assertDatabaseNotExists("Jdbc_Test2");
-      pstmt = conn.prepareStatement("CREATE DATABASE \"Jdbc_Test2\"");
-      pstmt.executeUpdate();
-      assertDatabaseExists("Jdbc_Test2");
-      pstmt.close();
-    }
-
-    conn.setCatalog("jdbc_test1");
-    assertEquals("jdbc_test1", conn.getCatalog());
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      conn.setCatalog("Jdbc_Test2");
-      assertEquals("Jdbc_Test2", conn.getCatalog());
-    }
-    conn.setCatalog("jdbc_test1");
-    assertEquals("jdbc_test1", conn.getCatalog());
-
-    ResultSet resultSet = conn.getMetaData().getSchemas();
-    assertResultSet(resultSet, "getSchemas1.result");
-    resultSet.close();
-
-    resultSet = conn.getMetaData().getSchemas("jdbc_test1", "%");
-    assertResultSet(resultSet, "getSchemas2.result");
-    resultSet.close();
-
-    resultSet = conn.getMetaData().getTableTypes();
-    assertResultSet(resultSet, "getTableTypes.result");
-    resultSet.close();
-
-    conn.setCatalog(TajoConstants.DEFAULT_DATABASE_NAME);
-    pstmt = conn.prepareStatement("DROP DATABASE jdbc_test1");
-    pstmt.executeUpdate();
-    pstmt.close();
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      pstmt = conn.prepareStatement("DROP DATABASE \"Jdbc_Test2\"");
-      pstmt.executeUpdate();
-      pstmt.close();
-    }
-
-    conn.close();
-  }
-
-  @Test
-  public void testGetCatalogsAndTables() throws Exception {
-    String connUri = TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
-    Connection defaultConnect = DriverManager.getConnection(connUri);
-
-    DatabaseMetaData dbmd = defaultConnect.getMetaData();
-    List<String> existingDatabases = getListFromResultSet(dbmd.getCatalogs(), "TABLE_CAT");
-
-    // create database "jdbc_test1" and its tables
-    assertDatabaseNotExists("jdbc_test3");
-    PreparedStatement pstmt = defaultConnect.prepareStatement("CREATE DATABASE jdbc_test3");
-    pstmt.executeUpdate();
-    assertDatabaseExists("jdbc_test3");
-    pstmt.close();
-    pstmt = defaultConnect.prepareStatement("CREATE TABLE jdbc_test3.table1 (age int)");
-    pstmt.executeUpdate();
-    pstmt.close();
-    pstmt = defaultConnect.prepareStatement("CREATE TABLE jdbc_test3.table2 (age int)");
-    pstmt.executeUpdate();
-    pstmt.close();
-
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      // create database "jdbc_test2" and its tables
-      assertDatabaseNotExists("Jdbc_Test4");
-      pstmt = defaultConnect.prepareStatement("CREATE DATABASE \"Jdbc_Test4\"");
-      pstmt.executeUpdate();
-      assertDatabaseExists("Jdbc_Test4");
-      pstmt.close();
-
-      pstmt = defaultConnect.prepareStatement("CREATE TABLE \"Jdbc_Test4\".table3 (age int)");
-      pstmt.executeUpdate();
-      pstmt.close();
-      pstmt = defaultConnect.prepareStatement("CREATE TABLE \"Jdbc_Test4\".table4 (age int)");
-      pstmt.executeUpdate();
-      pstmt.close();
-    }
-
-    // verify getCatalogs()
-    dbmd = defaultConnect.getMetaData();
-    List<String> newDatabases = getListFromResultSet(dbmd.getCatalogs(), "TABLE_CAT");
-
-    newDatabases.removeAll(existingDatabases);
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      assertEquals(2, newDatabases.size());
-    } else {
-      assertEquals(1, newDatabases.size());
-    }
-    assertTrue(newDatabases.contains("jdbc_test3"));
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      assertTrue(newDatabases.contains("Jdbc_Test4"));
-    }
-
-    // verify getTables()
-    ResultSet res = defaultConnect.getMetaData().getTables("jdbc_test3", null, null, null);
-    assertResultSet(res, "getTables1.result");
-    res.close();
-
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      res = defaultConnect.getMetaData().getTables("Jdbc_Test4", null, null, null);
-      assertResultSet(res, "getTables2.result");
-      res.close();
-    }
-
-    defaultConnect.close();
-
-    // jdbc1_test database connection test
-    String jdbcTest1ConnUri =
-        TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(), "jdbc_test3");
-    Connection jdbcTest1Conn = DriverManager.getConnection(jdbcTest1ConnUri);
-    assertEquals("jdbc_test3", jdbcTest1Conn.getCatalog());
-    jdbcTest1Conn.close();
-
-    client.selectDatabase("default");
-    executeString("DROP TABLE jdbc_test3.table1");
-    executeString("DROP TABLE jdbc_test3.table2");
-    executeString("DROP DATABASE jdbc_test3");
-
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      String jdbcTest2ConnUri =
-          TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(), "Jdbc_Test4");
-      Connection jdbcTest2Conn = DriverManager.getConnection(jdbcTest2ConnUri);
-      assertEquals("Jdbc_Test4", jdbcTest2Conn.getCatalog());
-      jdbcTest2Conn.close();
-
-      client.selectDatabase("default");
-      executeString("DROP TABLE \"Jdbc_Test4\".table3");
-      executeString("DROP TABLE \"Jdbc_Test4\".table4");
-      executeString("DROP DATABASE \"Jdbc_Test4\"");
-    }
-  }
-
-  @Test
-  public void testGetTablesWithPattern() throws Exception {
-    String connUri = TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        TajoConstants.DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-
-    Map<String,List<String>> tables = new HashMap<String,List<String>>();
-    assertDatabaseNotExists("db_1");
-    executeString("CREATE DATABASE db_1");
-    assertDatabaseExists("db_1");
-    for (int i = 0; i < 3; i++) {
-      String tableName = "tb_" + i;
-      TUtil.putToNestedList(tables, "db_1", tableName);
-      executeString("CREATE TABLE db_1." + tableName + " (age int)");
-    }
-    for (int i = 0; i < 3; i++) {
-      String tableName = "table_" + i + "_ptn";
-      TUtil.putToNestedList(tables, "db_1", tableName);
-      executeString("CREATE TABLE db_1." + tableName + " (age int)");
-    }
-
-    assertDatabaseNotExists("db_2");
-    executeString("CREATE DATABASE db_2");
-    assertDatabaseExists("db_2");
-    for (int i = 0; i < 3; i++) {
-      String tableName = "tb_" + i;
-      TUtil.putToNestedList(tables, "db_2", tableName);
-      executeString("CREATE TABLE db_2." + tableName + " (age int)");
-    }
-    for (int i = 0; i < 3; i++) {
-      String tableName = "table_" + i + "_ptn";
-      TUtil.putToNestedList(tables, "db_2", tableName);
-      executeString("CREATE TABLE db_2." + tableName + " (age int)");
-    }
-
-    // all wildcard test
-    Set<String> tableList =
-        Sets.newHashSet(getListFromResultSet(conn.getMetaData().getTables("db_2", null, "%", null), "TABLE_NAME"));
-    assertEquals(Sets.newHashSet(tables.get("db_2")), tableList);
-
-    // leading wildcard test
-    tableList =
-        Sets.newHashSet(getListFromResultSet(conn.getMetaData().getTables("db_2", null, "%_ptn", null), "TABLE_NAME"));
-    assertEquals(Sets.newHashSet("table_0_ptn", "table_1_ptn", "table_2_ptn"), tableList);
-
-    // tailing wildcard test
-    tableList =
-        Sets.newHashSet(getListFromResultSet(conn.getMetaData().getTables("db_2", null, "tb_%", null), "TABLE_NAME"));
-    assertEquals(Sets.newHashSet("tb_0", "tb_1", "tb_2"), tableList);
-
-    ResultSet resultSet = conn.getMetaData().getTables(null, null, "tb\\_%", null);
-    int i = 0;
-    while(resultSet.next()) {
-      tables.get(resultSet.getString("TABLE_CAT")).contains(resultSet.getString("TABLE_NAME"));
-      i++;
-    }
-    assertEquals(6, i);
-
-    executeString("DROP DATABASE db_1");
-    executeString("DROP DATABASE db_2");
-  }
-
-  private static String getTestColName(String dbName, String tableName, int i) {
-    if (i % 2 == 1) {
-      return CatalogUtil.denormalizeIdentifier(dbName + "_" + tableName + "_col") + " int";
-    } else {
-      return CatalogUtil.denormalizeIdentifier(dbName + "_" + tableName + "_COL") + " int";
-    }
-  }
-
-  @Test
-  public void testGetColumnsWithPattern() throws Exception {
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      String connUri = TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-          TajoConstants.DEFAULT_DATABASE_NAME);
-      Connection conn = DriverManager.getConnection(connUri);
-
-      // Below creates the following 12 tables
-      // db<i>.tb<j>, i = {1,2}, 0 <= j < 2
-      // db<i>.table_<j>, i = {1,2}, 0 <= j < 2
-
-      Map<String,List<String>> tables = new HashMap<String,List<String>>();
-      for (int j = 1; j <= 2; j++) {
-        String dbName = "db" + j;
-        assertDatabaseNotExists(dbName);
-        executeString("CREATE DATABASE " + dbName).close();
-        assertDatabaseExists(dbName);
-        for (int i = 3; i < 6; i++) {
-          String tableName = "tb" + i;
-
-
-          if (i % 2 == 0) {
-            tableName = tableName.toUpperCase();
-          }
-
-          TUtil.putToNestedList(tables, dbName, tableName);
-
-          executeString("CREATE TABLE " + dbName + "." + CatalogUtil.denormalizeIdentifier(tableName) +
-              " (" + getTestColName(dbName, tableName, 1) +
-              ") PARTITION BY COLUMN (" + getTestColName(dbName, tableName, 2) + ")").close();
-          assertTableExists(dbName + "." + tableName);
-        }
-        for (int i = 3; i < 6; i++) {
-          String tableName = "table" + i;
-
-
-          if (i % 2 == 0) {
-            tableName = tableName.toUpperCase();
-          }
-
-          TUtil.putToNestedList(tables, dbName, tableName);
-
-          executeString("CREATE TABLE " + dbName + "." + CatalogUtil.denormalizeIdentifier(tableName) +
-              " (" + getTestColName(dbName, tableName, 1) +
-              ") PARTITION BY COLUMN (" + getTestColName(dbName, tableName, 2) + ")").close();
-          assertTableExists(dbName + "." + tableName);
-        }
-      }
-
-      // all wildcard test on columns
-      Set<String> columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "tb3", "%"),
-              "COLUMN_NAME"));
-      assertEquals(Sets.newHashSet("db2_tb3_col", "db2_tb3_COL"), columnList);
-
-      // tailing wildcard + case sensitive test on columns
-      columnList = Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "tb3", "%col"),
-          "COLUMN_NAME"));
-      assertEquals(Sets.newHashSet("db2_tb3_col"), columnList);
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "tb3", "%COL"),
-              "COLUMN_NAME"));
-      assertEquals(Sets.newHashSet("db2_tb3_COL"), columnList);
-
-      // tailing wildcard test on columns
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "tb3", "db2\\_tb3\\_%"),
-              "COLUMN_NAME"));
-      assertEquals(Sets.newHashSet("db2_tb3_col", "db2_tb3_COL"), columnList);
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "%", "db2\\_tb3\\_%"),
-              "COLUMN_NAME"));
-      assertEquals(Sets.newHashSet("db2_tb3_col", "db2_tb3_COL"), columnList);
-
-      // leading wildcard test on tables
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db1", null, "%3", "%"),
-              "COLUMN_NAME"));
-      assertEquals(
-          Sets.newHashSet(
-              "db1_tb3_col", "db1_tb3_COL",
-              "db1_table3_col", "db1_table3_COL"),
-          columnList);
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "%3", "%"),
-              "COLUMN_NAME"));
-      assertEquals(
-          Sets.newHashSet(
-              "db2_tb3_col", "db2_tb3_COL",
-              "db2_table3_col", "db2_table3_COL"),
-          columnList);
-
-      // tailing wildcard + case sensitive test on tables
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "TABLE%", "%"),
-              "COLUMN_NAME"));
-      assertEquals(
-          Sets.newHashSet(
-              "db2_TABLE4_col", "db2_TABLE4_COL"), columnList);
-
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "TABLE4", "%"),
-              "COLUMN_NAME"));
-      assertEquals(
-          Sets.newHashSet(
-              "db2_TABLE4_col", "db2_TABLE4_COL"),
-          columnList);
-
-      // tailing wildcard test on tables
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns("db2", null, "table%", "%"),
-              "COLUMN_NAME"));
-      assertEquals(
-          Sets.newHashSet(
-              "db2_table3_col", "db2_table3_COL",
-              "db2_table5_col", "db2_table5_COL"),
-          columnList);
-
-      // wildcard test on database
-      columnList =
-          Sets.newHashSet(getListFromResultSet(conn.getMetaData().getColumns(null, null, "%3", "db1_tb3%"),
-              "COLUMN_NAME"));
-      assertEquals(Sets.newHashSet("db1_tb3_col", "db1_tb3_COL"), columnList);
-
-      executeString("DROP DATABASE db1");
-      executeString("DROP DATABASE db2");
-    }
-  }
-
-  @Test
-  public void testEmptyMetaInfo() throws Exception {
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      String connUri = TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-          TajoConstants.DEFAULT_DATABASE_NAME);
-      Connection conn = DriverManager.getConnection(connUri);
-
-      try {
-        DatabaseMetaData meta = conn.getMetaData();
-
-        ResultSet res = meta.getProcedures(null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getProcedureColumns(null, null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getUDTs(null, null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getColumnPrivileges(null, null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getTablePrivileges(null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getBestRowIdentifier(null, null, null, 0, false);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getVersionColumns(null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getPrimaryKeys(null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getImportedKeys(null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getExportedKeys(null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getCrossReference(null, null, null, null, null, null);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getIndexInfo(null, null, null, false, false);
-        assertNotNull(res);
-        assertFalse(res.next());
-
-        res = meta.getClientInfoProperties();
-        assertNotNull(res);
-        assertFalse(res.next());
-      } finally {
-        conn.close();
-      }
-    }
-  }
-
-  @Test
-  public void testGetTypeInfo() throws Exception {
-    if (!testingCluster.isHiveCatalogStoreRunning()) {
-      String connUri = TestTajoJdbc.buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-          TajoConstants.DEFAULT_DATABASE_NAME);
-      Connection conn = DriverManager.getConnection(connUri);
-
-      try {
-        DatabaseMetaData meta = conn.getMetaData();
-
-        ResultSet res = meta.getTypeInfo();
-
-        assertNotNull(res);
-
-        int numTypes = 0;
-
-        String[] columnNames = {"TYPE_NAME", "DATA_TYPE", "PRECISION", "LITERAL_PREFIX", "LITERAL_SUFFIX",
-            "CREATE_PARAMS", "NULLABLE", "CASE_SENSITIVE", "SEARCHABLE", "UNSIGNED_ATTRIBUTE",
-            "FIXED_PREC_SCALE", "AUTO_INCREMENT", "LOCAL_TYPE_NAME", "MINIMUM_SCALE", "MAXIMUM_SCALE",
-            "SQL_DATA_TYPE", "SQL_DATETIME_SUB", "NUM_PREC_RADIX"};
-
-        while (res.next()) {
-          for (int i = 0; i < columnNames.length; i++) {
-            Object value = res.getObject(columnNames[i]);
-            if (i == 15 || i == 16) {
-              assertNull(value);
-            } else {
-              assertNotNull(value);
-            }
-          }
-          numTypes++;
-        }
-
-        assertEquals(numTypes, TajoTypeUtil.getTypeInfos().size());
-      } finally {
-        conn.close();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java b/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
deleted file mode 100644
index e6d01fe..0000000
--- a/tajo-core-tests/src/test/java/org/apache/tajo/jdbc/TestTajoJdbc.java
+++ /dev/null
@@ -1,607 +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.jdbc;
-
-import com.google.common.collect.Maps;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.client.QueryStatus;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.net.InetSocketAddress;
-import java.sql.*;
-import java.util.*;
-
-import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class TestTajoJdbc extends QueryTestCaseBase {
-  private static InetSocketAddress tajoMasterAddress;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    tajoMasterAddress = testingCluster.getMaster().getTajoMasterClientService().getBindAddress();
-    Class.forName("org.apache.tajo.jdbc.TajoDriver").newInstance();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-  }
-
-  public static String buildConnectionUri(String hostName, int port, String databaseName) {
-    return "jdbc:tajo://" + hostName + ":" + port + "/" + databaseName;
-  }
-
-  @Test
-  public void testAcceptURL() throws SQLException {
-    TajoDriver driver = new TajoDriver();
-    assertTrue(driver.acceptsURL("jdbc:tajo:"));
-    assertFalse(driver.acceptsURL("jdbc:taju:"));
-  }
-
-  @Test(expected = SQLException.class)
-  public void testGetConnection() throws SQLException {
-    DriverManager.getConnection("jdbc:taju://" + tajoMasterAddress.getHostName() + ":" + tajoMasterAddress.getPort()
-      + "/default");
-  }
-
-  @Test
-  public void testStatement() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-      DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-    assertTrue(conn.isValid(100));
-
-    Statement stmt = null;
-    ResultSet res = null;
-    try {
-      stmt = conn.createStatement();
-
-      res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-        "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
-
-      try {
-        Map<String, Integer> result = Maps.newHashMap();
-        result.put("NO", 3);
-        result.put("RF", 2);
-
-        assertNotNull(res);
-        assertTrue(res.next());
-        assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-        assertTrue(res.next());
-        assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-        assertFalse(res.next());
-
-        ResultSetMetaData rsmd = res.getMetaData();
-        assertEquals(3, rsmd.getColumnCount());
-        assertEquals("l_returnflag", rsmd.getColumnName(1));
-        assertEquals("l_linestatus", rsmd.getColumnName(2));
-        assertEquals("count_order", rsmd.getColumnName(3));
-      } finally {
-        res.close();
-      }
-    } finally {
-      if (res != null) {
-        res.close();
-      }
-      if (stmt != null) {
-        stmt.close();
-      }
-      if (conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-  @Test
-  public void testPreparedStatement() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-      TajoConstants.DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-    assertTrue(conn.isValid(100));
-
-    PreparedStatement stmt = null;
-    ResultSet res = null;
-    try {
-      /*
-      test data set
-      1,17.0,N
-      1,36.0,N
-      2,38.0,N
-      3,45.0,R
-      3,49.0,R
-      */
-
-      String sql =
-        "select l_orderkey, l_quantity, l_returnflag from lineitem where l_quantity > ? and l_returnflag = ?";
-
-      stmt = conn.prepareStatement(sql);
-
-      stmt.setInt(1, 20);
-      stmt.setString(2, "N");
-
-      res = stmt.executeQuery();
-
-      ResultSetMetaData rsmd = res.getMetaData();
-      assertEquals(3, rsmd.getColumnCount());
-      assertEquals("l_orderkey", rsmd.getColumnName(1));
-      assertEquals("l_quantity", rsmd.getColumnName(2));
-      assertEquals("l_returnflag", rsmd.getColumnName(3));
-
-      try {
-        int numRows = 0;
-        String[] resultData = {"136.0N", "238.0N"};
-        while (res.next()) {
-          assertEquals(resultData[numRows],
-            ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
-          numRows++;
-        }
-        assertEquals(2, numRows);
-      } finally {
-        res.close();
-      }
-
-      stmt.setInt(1, 20);
-      stmt.setString(2, "R");
-
-      res = stmt.executeQuery();
-
-      rsmd = res.getMetaData();
-      assertEquals(3, rsmd.getColumnCount());
-      assertEquals("l_orderkey", rsmd.getColumnName(1));
-      assertEquals("l_quantity", rsmd.getColumnName(2));
-      assertEquals("l_returnflag", rsmd.getColumnName(3));
-
-      try {
-        int numRows = 0;
-        String[] resultData = {"345.0R", "349.0R"};
-        while (res.next()) {
-          assertEquals(resultData[numRows],
-            ("" + res.getObject(1).toString() + res.getObject(2).toString() + res.getObject(3).toString()));
-          numRows++;
-        }
-        assertEquals(2, numRows);
-      } finally {
-        res.close();
-      }
-    } finally {
-      if (res != null) {
-        res.close();
-      }
-      if (stmt != null) {
-        stmt.close();
-      }
-      if (conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-  @Test
-  public void testDatabaseMetaDataGetTable() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-      TajoConstants.DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-    assertTrue(conn.isValid(100));
-
-    DatabaseMetaData dbmd = conn.getMetaData();
-
-    ResultSet rs = null;
-
-    try {
-      rs = dbmd.getTables("default", null, null, null);
-
-      ResultSetMetaData rsmd = rs.getMetaData();
-      int numCols = rsmd.getColumnCount();
-      assertEquals(5, numCols);
-
-      Set<String> retrivedViaJavaAPI = new HashSet<String>(client.getTableList("default"));
-
-      Set<String> retrievedViaJDBC = new HashSet<String>();
-      while (rs.next()) {
-        retrievedViaJDBC.add(rs.getString("TABLE_NAME"));
-      }
-      assertEquals(retrievedViaJDBC, retrivedViaJavaAPI);
-    } finally {
-      if (rs != null) {
-        rs.close();
-      }
-    }
-
-    assertTrue(conn.isValid(100));
-    conn.close();
-  }
-
-  @Test
-  public void testDatabaseMetaDataGetColumns() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-      TajoConstants.DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-    assertTrue(conn.isValid(100));
-
-    DatabaseMetaData dbmd = conn.getMetaData();
-    ResultSet rs = null;
-
-    try {
-      String tableName = "lineitem";
-      rs = dbmd.getColumns(null, null, tableName, null);
-
-      ResultSetMetaData rsmd = rs.getMetaData();
-      int numCols = rsmd.getColumnCount();
-
-      assertEquals(22, numCols);
-      int numColumns = 0;
-
-      TableDesc tableDesc = client.getTableDesc(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, tableName));
-      assertNotNull(tableDesc);
-
-      List<Column> columns = tableDesc.getSchema().getRootColumns();
-
-      while (rs.next()) {
-        assertEquals(tableName, rs.getString("TABLE_NAME"));
-        assertEquals(columns.get(numColumns).getSimpleName(), rs.getString("COLUMN_NAME"));
-        // TODO assert type
-        numColumns++;
-      }
-
-      assertEquals(16, numColumns);
-    } finally {
-      if (rs != null) {
-        rs.close();
-      }
-    }
-
-    assertTrue(conn.isValid(100));
-    conn.close();
-    assertFalse(conn.isValid(100));
-  }
-
-  @Test
-  public void testMultipleConnections() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-      TajoConstants.DEFAULT_DATABASE_NAME);
-
-    Connection[] conns = new Connection[2];
-    conns[0] = DriverManager.getConnection(connUri);
-    conns[1] = DriverManager.getConnection(connUri);
-
-    try {
-      for (int i = 0; i < conns.length; i++) {
-        Statement stmt = null;
-        ResultSet res = null;
-        try {
-          stmt = conns[i].createStatement();
-
-          res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-            "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
-
-          try {
-            Map<String, Integer> result = Maps.newHashMap();
-            result.put("NO", 3);
-            result.put("RF", 2);
-
-            assertNotNull(res);
-            assertTrue(res.next());
-            assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-            assertTrue(res.next());
-            assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-            assertFalse(res.next());
-
-            ResultSetMetaData rsmd = res.getMetaData();
-            assertEquals(3, rsmd.getColumnCount());
-            assertEquals("l_returnflag", rsmd.getColumnName(1));
-            assertEquals("l_linestatus", rsmd.getColumnName(2));
-            assertEquals("count_order", rsmd.getColumnName(3));
-          } finally {
-            res.close();
-          }
-        } finally {
-          if (res != null) {
-            res.close();
-          }
-          if (stmt != null) {
-            stmt.close();
-          }
-        }
-      }
-    } finally {
-      assertTrue(conns[0].isValid(100));
-      conns[0].close();
-      assertFalse(conns[0].isValid(100));
-      assertTrue(conns[1].isValid(100));
-      conns[1].close();
-      assertFalse(conns[1].isValid(100));
-    }
-  }
-
-  @Test
-  public void testMultipleConnectionsSequentialClose() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-      DEFAULT_DATABASE_NAME);
-
-    Connection[] conns = new Connection[2];
-    conns[0] = DriverManager.getConnection(connUri);
-    conns[1] = DriverManager.getConnection(connUri);
-
-    try {
-      for (int i = 0; i < conns.length; i++) {
-        Statement stmt = null;
-        ResultSet res = null;
-        try {
-          stmt = conns[i].createStatement();
-
-          res = stmt.executeQuery("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
-            "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
-
-          try {
-            Map<String, Integer> result = Maps.newHashMap();
-            result.put("NO", 3);
-            result.put("RF", 2);
-
-            assertNotNull(res);
-            assertTrue(res.next());
-            assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-            assertTrue(res.next());
-            assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-            assertFalse(res.next());
-
-            ResultSetMetaData rsmd = res.getMetaData();
-            assertEquals(3, rsmd.getColumnCount());
-            assertEquals("l_returnflag", rsmd.getColumnName(1));
-            assertEquals("l_linestatus", rsmd.getColumnName(2));
-            assertEquals("count_order", rsmd.getColumnName(3));
-          } finally {
-            res.close();
-          }
-        } finally {
-          if (res != null) {
-            res.close();
-          }
-          if (stmt != null) {
-            stmt.close();
-          }
-          conns[i].close();
-        }
-      }
-    } finally {
-      if (!conns[0].isClosed()) {
-        assertTrue(conns[0].isValid(100));
-        conns[0].close();
-        assertFalse(conns[0].isValid(100));
-      }
-      if (!conns[1].isClosed()) {
-        assertTrue(conns[1].isValid(100));
-        conns[1].close();
-        assertFalse(conns[1].isValid(100));
-      }
-    }
-  }
-
-  @Test
-  public void testCreateTableWithDateAndTimestamp() throws Exception {
-    String tableName = CatalogUtil.normalizeIdentifier("testCreateTableWithDateAndTimestamp");
-
-    int result;
-    Statement stmt = null;
-    ResultSet res = null;
-    Connection conn = null;
-    try {
-      String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
-      conn = DriverManager.getConnection(connUri);
-      assertTrue(conn.isValid(100));
-
-      stmt = conn.createStatement();
-      result = stmt.executeUpdate("create table " + tableName + " (id int, name text, score double"
-        + ", register_date timestamp, update_date date, send_date time)");
-      assertEquals(result, 1);
-
-      res = stmt.executeQuery("select * from " + tableName);
-      assertFalse(res.next());
-
-      ResultSetMetaData rsmd = res.getMetaData();
-      assertNotNull(rsmd);
-      assertEquals(6, rsmd.getColumnCount());
-
-      assertEquals("id", rsmd.getColumnName(1));
-      assertEquals("name", rsmd.getColumnName(2));
-      assertEquals("score", rsmd.getColumnName(3));
-      assertEquals("register_date", rsmd.getColumnName(4));
-      assertEquals("update_date", rsmd.getColumnName(5));
-      assertEquals("send_date", rsmd.getColumnName(6));
-
-      assertEquals("integer", rsmd.getColumnTypeName(1));
-      assertEquals("varchar", rsmd.getColumnTypeName(2));
-      assertEquals("double", rsmd.getColumnTypeName(3));
-      assertEquals("timestamp", rsmd.getColumnTypeName(4));
-      assertEquals("date", rsmd.getColumnTypeName(5));
-      assertEquals("time", rsmd.getColumnTypeName(6));
-
-    } finally {
-      cleanupQuery(res);
-      if (stmt != null) {
-        stmt.close();
-      }
-
-      if(conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-  @Test
-  public void testSortWithDateTime() throws Exception {
-    Statement stmt = null;
-    ResultSet res = null;
-    Connection conn = null;
-    int result;
-
-    // skip this test if catalog uses HiveCatalogStore.
-    // It is because HiveCatalogStore does not support Time data type.
-    try {
-      if (!testingCluster.isHiveCatalogStoreRunning()) {
-        executeDDL("create_table_with_date_ddl.sql", "table1");
-
-        String connUri = buildConnectionUri(tajoMasterAddress.getHostName(),
-          tajoMasterAddress.getPort(), "TestTajoJdbc");
-
-        conn = DriverManager.getConnection(connUri);
-        assertTrue(conn.isValid(100));
-
-        stmt = conn.createStatement();
-        res = stmt.executeQuery("select col1, col2, col3 from table1 order by col1, col2, col3");
-
-        ResultSetMetaData rsmd = res.getMetaData();
-        assertNotNull(rsmd);
-        assertEquals(3, rsmd.getColumnCount());
-
-        assertEquals("timestamp", rsmd.getColumnTypeName(1));
-        assertEquals("date", rsmd.getColumnTypeName(2));
-        assertEquals("time", rsmd.getColumnTypeName(3));
-
-        assertResultSet(res);
-
-        result = stmt.executeUpdate("drop table table1");
-        assertEquals(result, 1);
-
-      }
-    } finally {
-      cleanupQuery(res);
-      if (stmt != null) {
-        stmt.close();
-      }
-
-      if(conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-
-  @Test
-  public void testAlterTableAddPartition() throws Exception {
-    Statement stmt = null;
-    ResultSet resultSet = null;
-    int retCode = 0;
-    Connection conn = null;
-    int result;
-    String errorMessage = null;
-
-    // skip this test if catalog uses HiveCatalogStore.
-    // It is because HiveCatalogStore does not support Time data type.
-    try {
-      if (!testingCluster.isHiveCatalogStoreRunning()) {
-        String connUri = buildConnectionUri(tajoMasterAddress.getHostName(),
-          tajoMasterAddress.getPort(), "TestTajoJdbc");
-
-        conn = DriverManager.getConnection(connUri);
-        assertTrue(conn.isValid(100));
-
-        String tableName = CatalogUtil.normalizeIdentifier("testAlterTablePartition");
-        resultSet = executeString(
-          "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
-        resultSet.close();
-
-        stmt = conn.createStatement();
-        result  = stmt.executeUpdate("alter table " + tableName + " add partition (key = 0.1)");
-        assertEquals(result, 1);
-     }
-    } finally {      
-      cleanupQuery(resultSet);
-      if (stmt != null) {
-        stmt.close();
-      }
-
-      if(conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-  @Test
-  public void testMaxRows() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-      DEFAULT_DATABASE_NAME);
-    Connection conn = DriverManager.getConnection(connUri);
-    assertTrue(conn.isValid(100));
-    Statement stmt = null;
-    ResultSet res = null;
-    //Parameter value setting for test.
-    final int maxRowsNum = 3;
-    int resultRowsNum = 0, returnMaxRows = 0;
-    try {
-      stmt = conn.createStatement();
-      //set maxRows(3)
-      stmt.setMaxRows(maxRowsNum);
-      //get MaxRows
-      returnMaxRows = stmt.getMaxRows();
-      res = stmt.executeQuery("select * from lineitem");
-      assertNotNull(res);
-      while (res.next()) {
-        //Actuality result Rows.
-        resultRowsNum++;	
-      }
-      //The test success, if maxRowsNum and resultRowsNum and returnMaxRows is same.
-      assertTrue(maxRowsNum == resultRowsNum && maxRowsNum == returnMaxRows);
-    } finally {
-      if (res != null) {
-        cleanupQuery(res);
-      }
-      if (stmt != null) {
-        stmt.close();
-      }
-      if (conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-  @Test
-  public final void testCancel() throws Exception {
-    String connUri = buildConnectionUri(tajoMasterAddress.getHostName(), tajoMasterAddress.getPort(),
-        DEFAULT_DATABASE_NAME);
-    Properties props = new Properties();
-    props.setProperty(SessionVars.BLOCK_ON_RESULT.keyname(), "false");
-
-    Connection conn = new JdbcConnection(connUri, props);
-    PreparedStatement statement = conn.prepareStatement("select sleep(1) from lineitem");
-    try {
-      assertTrue("should have result set", statement.execute());
-      TajoResultSetBase result = (TajoResultSetBase) statement.getResultSet();
-      Thread.sleep(1000);   // todo query master is not killed properly if it's compiling the query (use 100, if you want see)
-      statement.cancel();
-
-      QueryStatus status = client.getQueryStatus(result.getQueryId());
-      assertEquals(TajoProtos.QueryState.QUERY_KILLED, status.getState());
-    } finally {
-      if (statement != null) {
-        statement.close();
-      }
-      if (conn != null) {
-        conn.close();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
index 0e3e63e..2aecad3 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
@@ -75,14 +75,12 @@ public class TestKillQuery {
     cluster.startMiniClusterInLocal(1);
     conf = cluster.getConfiguration();
     client = cluster.newTajoClient();
-    File file = TPCH.getDataFile("lineitem");
     client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) "
-        + "using text location 'file://" + file.getAbsolutePath() + "'");
+        + "using text location '" + TpchTestBase.getInstance().getPath("lineitem") + "'");
     assertTrue(client.existTable("default.lineitem"));
 
-    file = TPCH.getDataFile("customer");
     client.executeQueryAndGetResult("create external table default.customer (c_custkey int, c_name text) "
-        + "using text location 'file://" + file.getAbsolutePath() + "'");
+        + "using text location '" + TpchTestBase.getInstance().getPath("customer") + "'");
     assertTrue(client.existTable("default.customer"));
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl b/tajo-core-tests/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
deleted file mode 100644
index 52fa2fe..0000000
--- a/tajo-core-tests/src/test/resources/dataset/TestTajoJdbc/table1/table1.tbl
+++ /dev/null
@@ -1,5 +0,0 @@
-1997-11-09 20:34:56|1996-04-12|15:34:56
-1997-11-09 20:34:56|1996-03-13|19:34:56
-1993-11-09 20:34:56|1997-01-28|08:34:56
-1995-11-09 20:34:56|1994-02-02|17:34:56
-1995-11-09 20:34:56|1993-11-09|20:34:56

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql b/tajo-core-tests/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
deleted file mode 100644
index caeeaf9..0000000
--- a/tajo-core-tests/src/test/resources/queries/TestTajoJdbc/create_table_with_date_ddl.sql
+++ /dev/null
@@ -1,10 +0,0 @@
--- Sort Table
--- It is used in TestSortQuery::testSortWithDate
-
-create external table table1 (
-  col1 timestamp,
-	col2 date,
-	col3 time
-) using text
-with ('text.delimiter'='|', 'text.null'='NULL')
-location ${table.path};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas1.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas1.result b/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas1.result
deleted file mode 100644
index 0a013d9..0000000
--- a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas1.result
+++ /dev/null
@@ -1,3 +0,0 @@
-TABLE_SCHEM,TABLE_CATALOG
--------------------------------
-,jdbc_test1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas2.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas2.result b/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas2.result
deleted file mode 100644
index 0a013d9..0000000
--- a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getSchemas2.result
+++ /dev/null
@@ -1,3 +0,0 @@
-TABLE_SCHEM,TABLE_CATALOG
--------------------------------
-,jdbc_test1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTableTypes.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTableTypes.result b/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTableTypes.result
deleted file mode 100644
index 29b90ab..0000000
--- a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTableTypes.result
+++ /dev/null
@@ -1,3 +0,0 @@
-TABLE_TYPE
--------------------------------
-TABLE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/e296a0d1/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTables1.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTables1.result b/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTables1.result
deleted file mode 100644
index 201d325..0000000
--- a/tajo-core-tests/src/test/resources/results/TestTajoDatabaseMetaData/getTables1.result
+++ /dev/null
@@ -1,4 +0,0 @@
-TABLE_CAT,TABLE_SCHEM,TABLE_NAME,TABLE_TYPE,REMARKS
--------------------------------
-jdbc_test3,,table1,TABLE,
-jdbc_test3,,table2,TABLE,
\ No newline at end of file