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/09/02 13:06:57 UTC

[34/39] tajo git commit: Basic pushdown to PostgreSQL has been completed.

Basic pushdown to PostgreSQL has been completed.


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

Branch: refs/heads/TAJO-1730
Commit: fa819881573f89fadd415ce5bd9b1a0280be3702
Parents: d3a016a
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Aug 31 17:11:44 2015 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Aug 31 17:11:44 2015 +0900

----------------------------------------------------------------------
 .../java/org/apache/tajo/catalog/TableUtil.java |  51 ---
 .../apache/tajo/catalog/TestFunctionDesc.java   |  19 --
 .../tajo/catalog/store/AbstractDBStore.java     |   4 +-
 .../tajo/client/CatalogAdminClientImpl.java     |   8 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |  13 +-
 .../test/java/org/apache/tajo/TpchTestBase.java |   4 +-
 .../src/test/resources/tpch/customer.tbl        |  10 +-
 .../src/test/resources/tpch/lineitem.tbl        |  10 +-
 .../src/test/resources/tpch/nation.tbl          |  50 +--
 .../src/test/resources/tpch/orders.tbl          |   6 +-
 .../src/test/resources/tpch/partsupp.tbl        |   6 +-
 .../src/test/resources/tpch/region.tbl          |  10 +-
 .../src/test/resources/tpch/supplier.tbl        |   6 +-
 .../org/apache/tajo/storage/StorageService.java |   5 +
 .../java/org/apache/tajo/util/FileUtil.java     | 117 ++-----
 .../org/apache/tajo/util/JavaResourceUtil.java  |  46 +++
 .../java/org/apache/tajo/util/StringUtils.java  |  37 ++-
 .../org/apache/tajo/util/TestFileUtils.java     |  65 +---
 .../apache/tajo/cli/tools/TestDDLBuilder.java   |  10 +-
 .../tajo/engine/parser/TestSQLAnalyzer.java     | 139 +++++----
 .../apache/tajo/engine/query/TestJoinQuery.java |   4 +-
 .../org/apache/tajo/storage/TestRowFile.java    |   8 +-
 .../tajo/engine/codegen/EvalCodeGenerator.java  |   4 +-
 .../engine/planner/PhysicalPlannerImpl.java     |   2 +
 .../physical/OperatorPushableScanExec.java      |  56 ++++
 .../engine/planner/physical/SeqScanExec.java    |  12 +-
 .../apache/tajo/master/exec/QueryExecutor.java  |   1 +
 .../org/apache/tajo/plan/LogicalPlanner.java    |  34 +-
 .../apache/tajo/plan/expr/AlgebraicUtil.java    |   2 +-
 .../org/apache/tajo/plan/expr/EvalType.java     |  14 +-
 .../tajo/plan/expr/SimpleEvalNodeVisitor.java   |   9 +-
 .../plan/exprrewrite/rules/ConstantFolding.java |   2 +-
 .../exprrewrite/rules/ConstantPropagation.java  |   2 +-
 .../org/apache/tajo/plan/logical/ScanNode.java  |  30 +-
 .../rules/CommonConditionReduceRule.java        |   2 +-
 .../tajo/plan/serder/EvalNodeSerializer.java    |   6 +-
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  35 +--
 tajo-plan/src/main/proto/Plan.proto             |  13 +-
 .../apache/tajo/storage/AbstractScanner.java    |  10 +
 .../org/apache/tajo/storage/MergeScanner.java   |  11 +
 .../org/apache/tajo/storage/NullScanner.java    |  11 +
 .../org/apache/tajo/storage/PlanPushable.java   |  36 +++
 .../java/org/apache/tajo/storage/Scanner.java   |  15 +
 .../org/apache/tajo/storage/Tablespace.java     |  47 +--
 .../apache/tajo/storage/TablespaceManager.java  |  23 +-
 .../storage/fragment/FragmentConvertor.java     |   5 +-
 .../src/main/resources/storage-default.xml      |   6 +-
 .../src/test/resources/storage-default.xml      |   4 +
 .../apache/tajo/storage/hbase/HBaseScanner.java |  10 +
 .../tajo/storage/hbase/HBaseTablespace.java     |  11 +-
 .../org/apache/tajo/storage/FileScanner.java    |  12 +
 .../org/apache/tajo/storage/FileTablespace.java |  13 +-
 .../tajo/storage/text/CSVLineDeserializer.java  |   8 +-
 .../tajo/storage/TestDelimitedTextFile.java     |  12 +-
 .../org/apache/tajo/storage/TestLineReader.java |   3 +-
 .../org/apache/tajo/storage/TestStorages.java   |  22 +-
 .../apache/tajo/storage/avro/TestAvroUtil.java  |   3 +-
 .../apache/tajo/storage/json/TestJsonSerDe.java |   4 +-
 .../tajo/storage/jdbc/ConnectionInfo.java       |   2 +-
 .../apache/tajo/storage/jdbc/JdbcFragment.java  |  32 +-
 .../storage/jdbc/JdbcMetadataProviderBase.java  |  15 +-
 .../apache/tajo/storage/jdbc/JdbcScanner.java   | 310 +++++++++++++++++++
 .../tajo/storage/jdbc/JdbcTablespace.java       |  34 +-
 .../apache/tajo/storage/jdbc/SQLBuilder.java    | 172 ++++++++++
 .../storage/jdbc/SQLExpressionGenerator.java    | 216 +++++++++++++
 .../storage/jdbc/JdbcTablespaceTestBase.java    |  23 --
 .../tajo/storage/mysql/MySQLTablespace.java     |  29 ++
 .../tajo/storage/mysql/MysqlJdbcScanner.java    |  40 +++
 .../tajo/storage/mysql/EmbedMySQLServer.java    |   4 +-
 .../mysql/TestMySQLMetadataProvider.java        |   4 +-
 .../src/test/resources/tpch/customer.sql        |  10 -
 .../src/test/resources/tpch/lineitem.sql        |  18 --
 .../src/test/resources/tpch/mysql/customer.sql  |  10 +
 .../src/test/resources/tpch/mysql/lineitem.sql  |  18 ++
 .../src/test/resources/tpch/mysql/nation.sql    |   6 +
 .../src/test/resources/tpch/mysql/orders.sql    |  11 +
 .../src/test/resources/tpch/mysql/part.sql      |  11 +
 .../src/test/resources/tpch/mysql/partsupp.sql  |   7 +
 .../src/test/resources/tpch/mysql/region.sql    |   5 +
 .../src/test/resources/tpch/mysql/supplier.sql  |   9 +
 .../src/test/resources/tpch/nation.sql          |   6 -
 .../src/test/resources/tpch/orders.sql          |  11 -
 .../src/test/resources/tpch/part.sql            |  11 -
 .../src/test/resources/tpch/partsupp.sql        |   7 -
 .../src/test/resources/tpch/region.sql          |   5 -
 .../src/test/resources/tpch/supplier.sql        |   9 -
 .../tajo/storage/pgsql/PgSQLJdbcScanner.java    |  36 +++
 .../tajo/storage/pgsql/PgSQLTablespace.java     |  32 ++
 .../tajo/storage/pgsql/EmbedPgSQLServer.java    |  38 ++-
 .../storage/pgsql/TestPgSQLEndPointTests.java   |  26 +-
 .../tajo/storage/pgsql/TestPgSQLQueryTests.java |  31 +-
 .../pgsql/TestPgSQLSimpleQueryTests.java        |  47 +++
 .../queries/TestPgSQLQueryTests/select_all.sql  |   0
 .../TestPgSQLQueryTests/testSimpleFilter.sql    |   1 +
 .../TestPgSQLSimpleQueryTests/testSelectAll.sql |   1 +
 .../testSelectLimit.sql                         |   1 +
 .../TestPgSQLQueryTests/select_all.result       |   0
 .../TestPgSQLQueryTests/testSimpleFilter.result |   5 +
 .../testSelectAll.result                        |   7 +
 .../testSelectLimit.result                      |   5 +
 .../src/test/resources/tpch/customer.sql        |  10 -
 .../src/test/resources/tpch/lineitem.sql        |  18 --
 .../src/test/resources/tpch/nation.sql          |   6 -
 .../src/test/resources/tpch/orders.sql          |  11 -
 .../src/test/resources/tpch/part.sql            |  11 -
 .../src/test/resources/tpch/partsupp.sql        |   7 -
 .../src/test/resources/tpch/pgsql/customer.sql  |  10 +
 .../src/test/resources/tpch/pgsql/lineitem.sql  |  18 ++
 .../src/test/resources/tpch/pgsql/nation.sql    |   6 +
 .../src/test/resources/tpch/pgsql/orders.sql    |  11 +
 .../src/test/resources/tpch/pgsql/part.sql      |  11 +
 .../src/test/resources/tpch/pgsql/partsupp.sql  |   7 +
 .../src/test/resources/tpch/pgsql/region.sql    |   5 +
 .../src/test/resources/tpch/pgsql/supplier.sql  |   9 +
 .../src/test/resources/tpch/region.sql          |   5 -
 .../src/test/resources/tpch/supplier.sql        |   9 -
 116 files changed, 1794 insertions(+), 743 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableUtil.java
deleted file mode 100644
index 76aec33..0000000
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableUtil.java
+++ /dev/null
@@ -1,51 +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.catalog;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableProto;
-import org.apache.tajo.util.FileUtil;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-public class TableUtil {
-  public static TableMeta getTableMeta(Configuration conf, Path tablePath) 
-      throws IOException {
-    TableMeta meta = null;
-    
-    FileSystem fs = tablePath.getFileSystem(conf);
-    
-    Path tableMetaPath = new Path(tablePath, ".meta");
-    if(!fs.exists(tableMetaPath)) {
-      throw new FileNotFoundException(".meta file not found in "+tablePath.toString());
-    }
-    FSDataInputStream tableMetaIn = 
-      fs.open(tableMetaPath);
-
-    TableProto tableProto = (TableProto) FileUtil.loadProto(tableMetaIn, 
-      TableProto.getDefaultInstance());
-    meta = new TableMeta(tableProto);
-
-    return meta;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
index 16e85f0..e8170ee 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
@@ -71,25 +71,6 @@ public class TestFunctionDesc {
     assertEquals(Type.INT4, desc.getReturnType().getType());
     assertArrayEquals(CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT8),
         desc.getParamTypes());
-
-    CommonTestingUtil.getTestDir(TEST_PATH);
-    File save = new File(TEST_PATH + "/save.dat");
-    FileUtil.writeProto(save, desc.getProto());
-
-    FunctionDescProto proto = FunctionDescProto.getDefaultInstance();
-    proto = (FunctionDescProto) FileUtil.loadProto(save, proto);
-
-    FunctionDesc newDesc = new FunctionDesc(proto);
-
-    assertEquals("sum", newDesc.getFunctionName());
-    assertEquals(TestSum.class, newDesc.getLegacyFuncClass());
-    assertEquals(FunctionType.GENERAL, newDesc.getFuncType());
-    assertEquals(Type.INT4, newDesc.getReturnType().getType());
-
-    assertArrayEquals(CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT8),
-        newDesc.getParamTypes());
-
-    assertEquals(desc.getProto(), newDesc.getProto());
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
index cd8dcb6..7eeaabd 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
@@ -36,7 +36,7 @@ import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.exception.*;
-import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.JavaResourceUtil;
 import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TUtil;
 
@@ -187,7 +187,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
   public String readSchemaFile(String path) {
     try {
-      return FileUtil.readTextFileFromResource("schemas/" + path);
+      return JavaResourceUtil.readTextFromResource("schemas/" + path);
     } catch (IOException e) {
       throw new TajoInternalError(e);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
index af1278d..d5ade14 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/CatalogAdminClientImpl.java
@@ -87,15 +87,15 @@ public class CatalogAdminClientImpl implements CatalogAdminClient {
   }
 
   @Override
-  public void dropDatabase(final String databaseName) throws UndefinedDatabaseException {
+  public void dropDatabase(final String databaseName)
+      throws UndefinedDatabaseException, InsufficientPrivilegeException {
 
     try {
       final BlockingInterface stub = conn.getTMStub();
       final ReturnState state = stub.dropDatabase(null, conn.getSessionedString(databaseName));
 
-      if (isThisError(state, ResultCode.UNDEFINED_DATABASE)) {
-        throw new UndefinedDatabaseException(state);
-      }
+      throwsIfThisError(state, UndefinedDatabaseException.class);
+      throwsIfThisError(state, InsufficientPrivilegeException.class);
       ensureOk(state);
 
     } catch (ServiceException e) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/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 6796921..6638958 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
@@ -40,6 +40,7 @@ import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.exception.InsufficientPrivilegeException;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.master.GlobalEngine;
@@ -225,11 +226,19 @@ public class QueryTestCaseBase {
     // if the current database is "default", shouldn't drop it.
     if (!currentDatabase.equals(TajoConstants.DEFAULT_DATABASE_NAME)) {
       for (String tableName : catalog.getAllTableNames(currentDatabase)) {
-        client.updateQuery("DROP TABLE IF EXISTS " + tableName);
+        try {
+          client.updateQuery("DROP TABLE IF EXISTS " + tableName);
+        } catch (InsufficientPrivilegeException i) {
+          LOG.warn("relation '" + tableName + "' is read only.");
+        }
       }
 
       client.selectDatabase(TajoConstants.DEFAULT_DATABASE_NAME);
-      client.dropDatabase(currentDatabase);
+      try {
+        client.dropDatabase(currentDatabase);
+      } catch (InsufficientPrivilegeException e) {
+        LOG.warn("database '" + currentDatabase + "' is read only.");
+      }
     }
     client.close();
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/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 027e735..62f7477 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
@@ -27,10 +27,10 @@ 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.JavaResourceUtil;
 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;
@@ -76,7 +76,7 @@ public class TpchTestBase {
     File tpchTablesDir = new File(new File(CommonTestingUtil.getTestDir().toUri()), "tpch");
 
     for (int i = 0; i < names.length; i++) {
-      String str = FileUtil.readTextFileFromResource("tpch/" + names[i] + ".tbl");
+      String str = JavaResourceUtil.readTextFromResource("tpch/" + names[i] + ".tbl");
       Path tablePath = new Path(new Path(tpchTablesDir.toURI()), names[i] + ".tbl");
       FileUtil.writeTextToFile(str, tablePath);
       paths[i] = tablePath.toString();

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/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
index 4f684c6..d1114fa 100644
--- a/tajo-cluster-tests/src/test/resources/tpch/customer.tbl
+++ b/tajo-cluster-tests/src/test/resources/tpch/customer.tbl
@@ -1,5 +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|
+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/fa819881/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
index e3beac9..47a0ea2 100644
--- a/tajo-cluster-tests/src/test/resources/tpch/lineitem.tbl
+++ b/tajo-cluster-tests/src/test/resources/tpch/lineitem.tbl
@@ -1,5 +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|
+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/fa819881/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
index ed3fd5b..bd42b90 100644
--- a/tajo-cluster-tests/src/test/resources/tpch/nation.tbl
+++ b/tajo-cluster-tests/src/test/resources/tpch/nation.tbl
@@ -1,25 +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|
+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/fa819881/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
index 15a1b6f..09fcbf1 100644
--- a/tajo-cluster-tests/src/test/resources/tpch/orders.tbl
+++ b/tajo-cluster-tests/src/test/resources/tpch/orders.tbl
@@ -1,3 +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|
+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/fa819881/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
index a6211e6..3912eb7 100644
--- a/tajo-cluster-tests/src/test/resources/tpch/partsupp.tbl
+++ b/tajo-cluster-tests/src/test/resources/tpch/partsupp.tbl
@@ -1,3 +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|
+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/fa819881/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
index c5ebb63..ae117c2 100644
--- a/tajo-cluster-tests/src/test/resources/tpch/region.tbl
+++ b/tajo-cluster-tests/src/test/resources/tpch/region.tbl
@@ -1,5 +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|
+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/fa819881/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
index a6fafb3..253301c 100644
--- a/tajo-cluster-tests/src/test/resources/tpch/supplier.tbl
+++ b/tajo-cluster-tests/src/test/resources/tpch/supplier.tbl
@@ -1,3 +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|
+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/fa819881/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java
index 4e5741b..44349ba 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java
@@ -18,7 +18,10 @@
 
 package org.apache.tajo.storage;
 
+import org.apache.tajo.exception.UnsupportedException;
+
 import javax.annotation.Nullable;
+import java.io.IOException;
 import java.net.URI;
 
 /**
@@ -35,4 +38,6 @@ public interface StorageService {
    * @return Table URI
    */
   URI getTableURI(@Nullable String spaceName, String databaseName, String tableName);
+
+  long getTableVolumn(URI uri) throws UnsupportedException;
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/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 5216eb8..430d49d 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
@@ -18,104 +18,28 @@
 
 package org.apache.tajo.util;
 
-import com.google.protobuf.Message;
 import org.apache.commons.logging.Log;
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.tajo.conf.TajoConf;
 
 import java.io.*;
-import java.net.URL;
 import java.nio.charset.Charset;
 
+/**
+ * Simple File Utilities
+ */
 public class FileUtil {
-  public static void writeProto(File file, Message proto) throws IOException {
-    FileOutputStream stream = null;
-    try {
-      stream = new FileOutputStream(file);
-      stream.write(proto.toByteArray());
-    } finally {
-      IOUtils.closeStream(stream);
-    }
-  }
-
-  public static void writeProto(OutputStream out, Message proto) throws IOException {
-    out.write(proto.toByteArray());
-  }
-
-  public static void writeProto(FileSystem fs, Path path, Message proto) throws IOException {
-    FSDataOutputStream stream = fs.create(path);
-    try {
-      stream.write(proto.toByteArray());
-    } finally {
-      IOUtils.closeStream(stream);
-    }
-  }
-
-  public static Message loadProto(File file, Message proto) throws IOException {
-    FileInputStream in = null;
-    try {
-      in = new FileInputStream(file);
-      Message.Builder builder = proto.newBuilderForType().mergeFrom(in);
-      return builder.build();
-    } finally {
-      IOUtils.closeStream(in);
-    }
-  }
-
-  public static Message loadProto(InputStream in, Message proto) throws IOException {
-    Message.Builder builder = proto.newBuilderForType().mergeFrom(in);
-    return builder.build();
-  }
-
-  public static Message loadProto(FileSystem fs,
-                                  Path path, Message proto) throws IOException {
-    FSDataInputStream in = null;
-    try {
-      in = new FSDataInputStream(fs.open(path));
-      Message.Builder builder = proto.newBuilderForType().mergeFrom(in);
-      return builder.build();
-    } finally {
-      IOUtils.closeStream(in);
-    }
-  }
-
-  public static URL getResourcePath(String resource) throws IOException {
-    return ClassLoader.getSystemResource(resource);
-  }
 
   /**
-   * It returns a string from a text file found in classpath.
+   * Read a file stored in a local file system and return the string contents.
    *
-   * @param resource Resource file name
-   * @return String contents if exists. Otherwise, it will return null.
+   * @param file File
+   * @return String contents
    * @throws IOException
    */
-  public static String readTextFileFromResource(String resource) throws IOException {
-    InputStream stream = ClassLoader.getSystemResourceAsStream(resource);
-    if (stream != null) {
-      return readTextFromStream(stream);
-    } else {
-      return null;
-    }
-  }
-
-  public static String readTextFromStream(InputStream inputStream)
-      throws IOException {
-    try {
-      StringBuilder fileData = new StringBuilder(1000);
-      byte[] buf = new byte[1024];
-      int numRead;
-      while ((numRead = inputStream.read(buf)) != -1) {
-        String readData = new String(buf, 0, numRead, Charset.defaultCharset());
-        fileData.append(readData);
-      }
-      return fileData.toString();
-    } finally {
-      IOUtils.closeStream(inputStream);
-    }
-  }
-
   public static String readTextFile(File file) throws IOException {
     StringBuilder fileData = new StringBuilder(1000);
     BufferedReader reader = new BufferedReader(new FileReader(file));
@@ -133,6 +57,13 @@ public class FileUtil {
     return fileData.toString();
   }
 
+  /**
+   * Write a string into a file
+   *
+   * @param text
+   * @param path File path
+   * @throws IOException
+   */
   public static void writeTextToFile(String text, Path path) throws IOException {
     FileSystem fs = path.getFileSystem(new TajoConf());
     if (!fs.exists(path.getParent())) {
@@ -143,6 +74,22 @@ public class FileUtil {
     out.close();
   }
 
+  public static String readTextFromStream(InputStream inputStream)
+      throws IOException {
+    try {
+      StringBuilder fileData = new StringBuilder(1000);
+      byte[] buf = new byte[1024];
+      int numRead;
+      while ((numRead = inputStream.read(buf)) != -1) {
+        String readData = new String(buf, 0, numRead, Charset.defaultCharset());
+        fileData.append(readData);
+      }
+      return fileData.toString();
+    } finally {
+      IOUtils.closeStream(inputStream);
+    }
+  }
+
   public static void writeTextToStream(String text, OutputStream outputStream) throws IOException {
     try {
       outputStream.write(text.getBytes());

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-common/src/main/java/org/apache/tajo/util/JavaResourceUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/JavaResourceUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/JavaResourceUtil.java
new file mode 100644
index 0000000..cd19b5e
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/util/JavaResourceUtil.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+
+public class JavaResourceUtil {
+
+  public static URL getResourceURL(String resource) throws IOException {
+    return ClassLoader.getSystemResource(resource);
+  }
+
+  /**
+   * It returns a string from a text file found in classpath.
+   *
+   * @param resource Resource file name
+   * @return String contents if exists. Otherwise, it will return null.
+   * @throws IOException
+   */
+  public static String readTextFromResource(String resource) throws IOException {
+    InputStream stream = ClassLoader.getSystemResourceAsStream(resource);
+    if (stream != null) {
+      return FileUtil.readTextFromStream(stream);
+    } else {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
index 7a9f0c3..aa426c8 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.util;
 
+import com.google.common.base.Function;
 import io.netty.util.CharsetUtil;
 import org.apache.commons.lang.CharUtils;
 import org.apache.commons.lang.StringEscapeUtils;
@@ -405,7 +406,17 @@ public class StringUtils {
    * @return A joined string
    */
   public static String join(Object[] objects) {
-    return join(objects, ", ", 0, objects.length);
+    return join(objects, ",");
+  }
+
+  /**
+   * Concatenate all objects' string with the delimiter ", "
+   *
+   * @param objects Iterable objects
+   * @return A joined string
+   */
+  public static String join(Object[] objects, String delimiter) {
+    return join(objects, delimiter, 0, objects.length);
   }
 
   /**
@@ -430,6 +441,28 @@ public class StringUtils {
    * @return A joined string
    */
   public static String join(Object[] objects, String delimiter, int startIndex, int length) {
+    return join(objects, delimiter, startIndex, length, new Function<Object, String>() {
+      @Override
+      public String apply(Object input) {
+        return input.toString();
+      }
+    });
+  }
+
+
+  /**
+   * Concatenate all objects' string with a delimiter string
+   *
+   * @param objects object array
+   * @param delimiter Delimiter string
+   * @param f convert from a type to string
+   * @return A joined string
+   */
+  public static <T> String join(T [] objects, String delimiter, Function<T, String> f) {
+    return join(objects, delimiter, 0, objects.length, f);
+  }
+
+  public static <T> String join(T [] objects, String delimiter, int startIndex, int length, Function<T, String> f) {
     boolean first = true;
     StringBuilder sb = new StringBuilder();
     int endIndex = startIndex + length;
@@ -440,7 +473,7 @@ public class StringUtils {
         sb.append(delimiter);
       }
 
-      sb.append(objects[i].toString());
+      sb.append(f.apply(objects[i]));
     }
 
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-common/src/test/java/org/apache/tajo/util/TestFileUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/util/TestFileUtils.java b/tajo-common/src/test/java/org/apache/tajo/util/TestFileUtils.java
index 1e7d3c3..90b1a39 100644
--- a/tajo-common/src/test/java/org/apache/tajo/util/TestFileUtils.java
+++ b/tajo-common/src/test/java/org/apache/tajo/util/TestFileUtils.java
@@ -18,21 +18,11 @@
 
 package org.apache.tajo.util;
 
-import com.google.protobuf.Message;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.tajo.util.TestProtos.TestMessageProto;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Test;
-import org.apache.tajo.util.TestProtos.TestMessageProto;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
 
 public class TestFileUtils {
 	private static final String TEST_PATH = "target/test-data/TestFileUTils";
@@ -56,52 +46,9 @@ public class TestFileUtils {
 	
 	@After
 	public void tearDown() throws Exception {
-		File testDir = new File(TEST_PATH);
-		if(testDir.exists()) {
-			testDir.delete();
-		}
-	}
-
-	@Test
-	public final void testWriteLoadProtoFromFile() throws IOException {		
-		File file = new File(TEST_PATH+"/file.bin");
-		file.createNewFile();
-		FileUtil.writeProto(file, proto);
-		
-		Message defaultInstance = TestMessageProto.getDefaultInstance();
-		TestMessageProto message = (TestMessageProto) 
-			FileUtil.loadProto(new File(TEST_PATH+"/file.bin"), defaultInstance);
-		
-		assertEquals(proto, message);
-	}
-
-	@Test
-	public final void testWriteLoadProtoFromStream() throws IOException {
-		FileOutputStream out = new FileOutputStream(new File(TEST_PATH+"/file.bin"));		
-		FileUtil.writeProto(out, proto);
-		
-		
-		FileInputStream in = new FileInputStream(new File(TEST_PATH+"/file.bin"));
-		Message defaultInstance = TestMessageProto.getDefaultInstance();
-		TestMessageProto message = (TestMessageProto) 
-			FileUtil.loadProto(in, defaultInstance);
-		
-		assertEquals(proto, message);
-	}
-
-	@Test
-	public final void testWriteLoadProtoFromPath() throws IOException {	
-		Path path = new Path(TEST_PATH+"/file.bin");
-    Configuration conf = new Configuration();
-    FileSystem localFS = FileSystem.getLocal(conf);
-		FileUtil.writeProto(localFS, path, proto);
-		
-		Message defaultInstance = TestMessageProto.getDefaultInstance();
-		TestMessageProto message = (TestMessageProto) 
-			FileUtil.loadProto(localFS, new Path(TEST_PATH+"/file.bin"),
-          defaultInstance);
-		
-		assertEquals(proto, message);
-	}
-	
+    File testDir = new File(TEST_PATH);
+    if (testDir.exists()) {
+      testDir.delete();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core-tests/src/test/java/org/apache/tajo/cli/tools/TestDDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/cli/tools/TestDDLBuilder.java b/tajo-core-tests/src/test/java/org/apache/tajo/cli/tools/TestDDLBuilder.java
index 06a54c4..bb8cf1b 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/cli/tools/TestDDLBuilder.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/cli/tools/TestDDLBuilder.java
@@ -25,7 +25,7 @@ import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.JavaResourceUtil;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -61,7 +61,7 @@ public class TestDDLBuilder {
     TableDesc desc = new TableDesc("db1.table1", schema1, meta1, new Path("/table1").toUri());
     desc.setPartitionMethod(partitionMethod1);
     desc.setExternal(true);
-    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLForExternalTable.result"),
+    assertEquals(JavaResourceUtil.readTextFromResource("results/testDDLBuilder/testBuildDDLForExternalTable.result"),
         DDLBuilder.buildDDLForExternalTable(desc));
   }
 
@@ -87,20 +87,20 @@ public class TestDDLBuilder {
     TableDesc desc = new TableDesc("db1.TABLE2", schema2, meta1, new Path("/table1").toUri());
     desc.setPartitionMethod(partitionMethod2);
     desc.setExternal(true);
-    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName1.result"),
+    assertEquals(JavaResourceUtil.readTextFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName1.result"),
         DDLBuilder.buildDDLForExternalTable(desc));
 
     desc = new TableDesc("db1.TABLE1", schema2, meta1, new Path("/table1").toUri());
     desc.setPartitionMethod(partitionMethod2);
     desc.setExternal(false);
-    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName2.result"),
+    assertEquals(JavaResourceUtil.readTextFromResource("results/testDDLBuilder/testBuildDDLQuotedTableName2.result"),
         DDLBuilder.buildDDLForBaseTable(desc));
   }
 
   @Test
   public void testBuildDDLForBaseTable() throws Exception {
     TableDesc desc = new TableDesc("db1.table2", schema1, meta1, new Path("/table1").toUri());
-    assertEquals(FileUtil.readTextFileFromResource("results/testDDLBuilder/testBuildDDLForBaseTable.result"),
+    assertEquals(JavaResourceUtil.readTextFromResource("results/testDDLBuilder/testBuildDDLForBaseTable.result"),
         DDLBuilder.buildDDLForBaseTable(desc));
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core-tests/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
index e9a9305..f53fbfd 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
@@ -22,10 +22,9 @@ import org.antlr.v4.runtime.ANTLRInputStream;
 import org.antlr.v4.runtime.CommonTokenStream;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.engine.parser.SQLParser.SqlContext;
-import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.JavaResourceUtil;
 import org.junit.Test;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
 
@@ -51,8 +50,8 @@ public class TestSQLAnalyzer {
   }
 
   public void assertParseResult(String sqlFileName, String resultFileName) throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/TestSQLAnalyzer/" + sqlFileName);
-    String result = FileUtil.readTextFileFromResource("results/TestSQLAnalyzer/" + resultFileName);
+    String sql = JavaResourceUtil.readTextFromResource("queries/TestSQLAnalyzer/" + sqlFileName);
+    String result = JavaResourceUtil.readTextFromResource("results/TestSQLAnalyzer/" + resultFileName);
 
     Expr expr = parseQuery(sql);
     assertEquals(result.trim(), expr.toJson().trim());
@@ -61,223 +60,223 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testSelect1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/select_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/select_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSelect2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/select_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/select_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSelect3() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/select_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/select_3.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSelect4() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/select_4.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/select_4.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSelect5() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/select_5.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/select_5.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testAsterisk1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/asterisk_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/asterisk_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testAsterisk2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/asterisk_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/asterisk_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testAsterisk3() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/asterisk_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/asterisk_3.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testAsterisk4() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/asterisk_4.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/asterisk_4.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testGroupby1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/groupby_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/groupby_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin3() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_3.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin4() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_4.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_4.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin5() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_5.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_5.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin6() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_6.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_6.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin7() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_7.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_7.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin8() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_8.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_8.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin9() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_9.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_9.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin10() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_10.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_10.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testJoin11() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/join_11.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/join_11.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSet1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/set_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/set_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSet2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/set_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/set_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSet3() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/set_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/set_3.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testSet4() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/set_4.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/set_4.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testDropTable() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/drop_table.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/drop_table.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable3() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_3.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable4() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_4.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_4.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable5() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_5.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_5.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable6() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_6.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_6.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable7() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_7.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_7.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable8() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_8.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_8.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable9() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_9.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_9.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTable10() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_10.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_10.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testCreateTableLike1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_like_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_like_1.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.CreateTable, expr.getType());
     CreateTable createTable = (CreateTable) expr;
@@ -286,7 +285,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testCreateTablePartitionByHash1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_partition_by_hash_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_partition_by_hash_1.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.CreateTable, expr.getType());
     CreateTable createTable = (CreateTable) expr;
@@ -299,7 +298,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testCreateTablePartitionByHash2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_partition_by_hash_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_partition_by_hash_2.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.CreateTable, expr.getType());
     CreateTable createTable = (CreateTable) expr;
@@ -313,7 +312,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testCreateTablePartitionByRange() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_partition_by_range.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_partition_by_range.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.CreateTable, expr.getType());
     CreateTable createTable = (CreateTable) expr;
@@ -326,7 +325,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testCreateTablePartitionByList() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_partition_by_list.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_partition_by_list.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.CreateTable, expr.getType());
     CreateTable createTable = (CreateTable) expr;
@@ -351,7 +350,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testCreateTablePartitionByColumn() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/create_table_partition_by_column.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/create_table_partition_by_column.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.CreateTable, expr.getType());
     CreateTable createTable = (CreateTable) expr;
@@ -366,7 +365,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableAddPartition1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_add_partition_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_add_partition_1.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -384,7 +383,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableAddPartition2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_add_partition_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_add_partition_2.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -403,7 +402,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableAddPartition3() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_add_partition_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_add_partition_3.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -425,7 +424,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableAddPartition4() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_add_partition_4.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_add_partition_4.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -440,7 +439,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableAddPartition5() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_add_partition_5.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_add_partition_5.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -455,7 +454,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableDropPartition1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_drop_partition_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_drop_partition_1.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -474,7 +473,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableDropPartition2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_drop_partition_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_drop_partition_2.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -496,7 +495,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableDropPartition3() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_drop_partition_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_drop_partition_3.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -512,7 +511,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableDropPartition4() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_drop_partition_4.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_drop_partition_4.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -528,7 +527,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableSetProperty1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_set_property_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_set_property_1.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -540,7 +539,7 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testAlterTableSetProperty2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_set_property_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_set_property_2.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -553,7 +552,7 @@ public class TestSQLAnalyzer {
   @Test
   public void testAlterTableSetProperty3() throws IOException {
     // update multiple table properties with a single 'SET PROPERTY' sql
-    String sql = FileUtil.readTextFileFromResource("queries/default/alter_table_set_property_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/alter_table_set_property_3.sql");
     Expr expr = parseQuery(sql);
     assertEquals(OpType.AlterTable, expr.getType());
     AlterTable alterTable = (AlterTable)expr;
@@ -567,73 +566,73 @@ public class TestSQLAnalyzer {
 
   @Test
   public void testTableSubQuery1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/table_subquery1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/table_subquery1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testTableSubQuery2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/table_subquery2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/table_subquery2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInSubquery1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/in_subquery_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/in_subquery_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInSubquery2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/in_subquery_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/in_subquery_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testExistsPredicate1() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/exists_predicate_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/exists_predicate_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testExistsPredicate2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/exists_predicate_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/exists_predicate_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInsertIntoTable() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/insert_into_select_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/insert_into_select_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInsertIntoLocation() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/insert_into_select_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/insert_into_select_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInsertIntoTable2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/insert_into_select_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/insert_into_select_3.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInsertOverwriteIntoTable() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/insert_overwrite_into_select_1.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/insert_overwrite_into_select_1.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInsertOverwriteIntoLocation() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/insert_overwrite_into_select_2.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/insert_overwrite_into_select_2.sql");
     parseQuery(sql);
   }
 
   @Test
   public void testInsertOverwriteIntoTable2() throws IOException {
-    String sql = FileUtil.readTextFileFromResource("queries/default/insert_overwrite_into_select_3.sql");
+    String sql = JavaResourceUtil.readTextFromResource("queries/default/insert_overwrite_into_select_3.sql");
     parseQuery(sql);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/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 e5b9b98..8652f52 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
@@ -37,7 +37,7 @@ import org.apache.tajo.datum.Int4Datum;
 import org.apache.tajo.datum.TextDatum;
 import org.apache.tajo.exception.TajoException;
 import org.apache.tajo.storage.*;
-import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.JavaResourceUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.junit.runners.Parameterized.Parameters;
 
@@ -244,7 +244,7 @@ public class TestJoinQuery extends QueryTestCaseBase {
     TableMeta tableMeta = table.getMeta();
     Schema schema = table.getLogicalSchema();
 
-    String[] rows = FileUtil.readTextFileFromResource("tpch/" + tableName + ".tbl").split("\n");
+    String[] rows = JavaResourceUtil.readTextFromResource("tpch/" + tableName + ".tbl").split("\n");
 
     assertTrue(rows.length > 0);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestRowFile.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestRowFile.java b/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestRowFile.java
index e45dd75..c1d5a97 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestRowFile.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestRowFile.java
@@ -74,13 +74,10 @@ public class TestRowFile {
     FileTablespace sm = (FileTablespace) TablespaceManager.get(cluster.getDefaultFileSystem().getUri()).get();
 
     Path tablePath = new Path("/test");
-    Path metaPath = new Path(tablePath, ".meta");
     Path dataPath = new Path(tablePath, "test.tbl");
     FileSystem fs = sm.getFileSystem();
     fs.mkdirs(tablePath);
 
-    FileUtil.writeProto(fs, metaPath, meta.getProto());
-
     Appender appender = sm.getAppender(meta, schema, dataPath);
     appender.enableStats();
     appender.init();
@@ -105,14 +102,11 @@ public class TestRowFile {
     assertEquals(tupleNum, stat.getNumRows().longValue());
 
     FileStatus file = fs.getFileStatus(dataPath);
-    TableProto proto = (TableProto) FileUtil.loadProto(
-        cluster.getDefaultFileSystem(), metaPath, TableProto.getDefaultInstance());
-    meta = new TableMeta(proto);
     FileFragment fragment = new FileFragment("test.tbl", dataPath, 0, file.getLen());
 
     int tupleCnt = 0;
     start = System.currentTimeMillis();
-    Scanner scanner = sm.getScanner(meta, schema, fragment);
+    Scanner scanner = sm.getScanner(meta, schema, fragment, null);
     scanner.init();
     while ((tuple=scanner.next()) != null) {
       tupleCnt++;

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java
index cc740e7..8b87ea9 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenerator.java
@@ -130,7 +130,7 @@ public class EvalCodeGenerator extends SimpleEvalNodeVisitor<EvalCodeGenContext>
     }
   }
 
-  public EvalNode visitUnaryEval(EvalCodeGenContext context, Stack<EvalNode> stack, UnaryEval unary) {
+  public EvalNode visitUnaryEval(EvalCodeGenContext context, UnaryEval unary, Stack<EvalNode> stack) {
     stack.push(unary);
     if (unary.getType() == EvalType.CAST) {
       visitCast(context, stack, (CastEval) unary);
@@ -349,7 +349,7 @@ public class EvalCodeGenerator extends SimpleEvalNodeVisitor<EvalCodeGenContext>
     return cast;
   }
 
-  public EvalNode visitField(EvalCodeGenContext context, Stack<EvalNode> stack, FieldEval field) {
+  public EvalNode visitField(EvalCodeGenContext context, FieldEval field, Stack<EvalNode> stack) {
 
     if (field.getValueType().getType() == TajoDataTypes.Type.NULL_TYPE) {
       context.pushNullOfThreeValuedLogic();

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
index 5bbf3a9..19a27af 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
@@ -120,6 +120,8 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
     PhysicalExec leftExec;
     PhysicalExec rightExec;
 
+    //ctx.getEnforcer().getEnforceProperties().get
+
     switch (logicalNode.getType()) {
 
       case ROOT:

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/OperatorPushableScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/OperatorPushableScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/OperatorPushableScanExec.java
new file mode 100644
index 0000000..66ae844
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/OperatorPushableScanExec.java
@@ -0,0 +1,56 @@
+/**
+ * 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.engine.planner.physical;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+public class OperatorPushableScanExec extends PhysicalExec {
+  private final LogicalNode plan;
+  
+
+  public OperatorPushableScanExec(TaskAttemptContext context, Schema inSchema, Schema outSchema, LogicalNode plan) {
+    super(context, inSchema, outSchema);
+    this.plan = plan;
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    return null;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+
+  @Override
+  public float getProgress() {
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
index b49fa40..66ad005 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
@@ -171,9 +171,6 @@ public class SeqScanExec extends ScanExec {
       scanIt = new FilterScanIterator(scanner, qual);
 
     } else {
-      if (scanner.isSelectable()) { // TODO - isSelectable should be moved to FormatProperty
-        scanner.setFilter(qual);
-      }
       scanIt = new FullScanIterator(scanner);
     }
   }
@@ -236,10 +233,17 @@ public class SeqScanExec extends ScanExec {
       this.scanner = tablespace.getScanner(
           meta,
           plan.getPhysicalSchema(),
-          fragments[0],
+          FragmentConvertor.convert(context.getConf(), fragments[0]),
           projected);
     }
 
+    if (scanner.isSelectable()) { // TODO - isSelectable should be moved to FormatProperty
+      scanner.setFilter(qual);
+    }
+
+    if (plan.hasLimit()) {
+      scanner.setLimit(plan.getLimit());
+    }
     scanner.init();
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
index 39013df..624cf4b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@ -257,6 +257,7 @@ public class QueryExecutor {
     if (plan.getRootBlock().hasNode(NodeType.LIMIT)) {
       LimitNode limitNode = plan.getRootBlock().getNode(NodeType.LIMIT);
       maxRow = (int) limitNode.getFetchFirstNum();
+      scanNode.setLimit(maxRow);
     }
     if (desc.getStats().getNumRows() == 0) {
       desc.getStats().setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
index 4b17b0e..457b2ab 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
@@ -24,9 +24,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.BuiltinStorages;
 import org.apache.tajo.OverridableConf;
@@ -35,18 +32,13 @@ import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.algebra.WindowSpec;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.exception.UndefinedColumnException;
-import org.apache.tajo.exception.UndefinedTableException;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.IndexMethod;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.exception.ExceptionUtil;
-import org.apache.tajo.exception.TajoException;
-import org.apache.tajo.exception.TajoInternalError;
-import org.apache.tajo.exception.NotImplementedException;
+import org.apache.tajo.exception.*;
 import org.apache.tajo.plan.LogicalPlan.QueryBlock;
 import org.apache.tajo.plan.algebra.BaseAlgebraVisitor;
 import org.apache.tajo.plan.expr.*;
@@ -1317,7 +1309,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     QueryBlock block = context.queryBlock;
 
     ScanNode scanNode = block.getNodeFromExpr(expr);
-    updatePhysicalInfo(context, scanNode.getTableDesc());
+    updatePhysicalInfo(scanNode.getTableDesc());
 
     // Find expression which can be evaluated at this relation node.
     // Except for column references, additional expressions used in select list, where clause, order-by clauses
@@ -1376,24 +1368,16 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     return targets;
   }
 
-  private void updatePhysicalInfo(PlanContext planContext, TableDesc desc) {
-    if (desc.getUri() != null &&
-        !desc.getMeta().getStoreType().equals("SYSTEM") &&
-        !desc.getMeta().getStoreType().equals("FAKEFILE") && // FAKEFILE is used for test
-        PlannerUtil.isFileStorageType(desc.getMeta().getStoreType())) {
+  private void updatePhysicalInfo(TableDesc desc) {
+
+    // FAKEFILE is used for test{
+    if (!desc.getMeta().getStoreType().equals("SYSTEM") && !desc.getMeta().getStoreType().equals("FAKEFILE")) {
       try {
-        Path path = new Path(desc.getUri());
-        FileSystem fs = path.getFileSystem(planContext.queryContext.getConf());
-        FileStatus status = fs.getFileStatus(path);
-        if (desc.getStats() != null && (status.isDirectory() || status.isFile())) {
-          ContentSummary summary = fs.getContentSummary(path);
-          if (summary != null) {
-            long volume = summary.getLength();
-            desc.getStats().setNumBytes(volume);
-          }
+        if (desc.getStats() != null) {
+          desc.getStats().setNumBytes(storage.getTableVolumn(desc.getUri()));
         }
       } catch (Throwable t) {
-        LOG.warn(t, t);
+        LOG.warn(desc.getName() + " does not support Tablespace::getTableVolume().");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java
index d06c1d3..19d5d16 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java
@@ -156,7 +156,7 @@ public class AlgebraicUtil {
     }
 
     @Override
-    public EvalNode visitUnaryEval(Object context, Stack<EvalNode> stack, UnaryEval unaryEval) {
+    public EvalNode visitUnaryEval(Object context, UnaryEval unaryEval, Stack<EvalNode> stack) {
       stack.push(unaryEval);
       EvalNode child = visit(context, unaryEval.getChild(), stack);
       stack.pop();

http://git-wip-us.apache.org/repos/asf/tajo/blob/fa819881/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java
index 2c2a52f..35390a5 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java
@@ -21,10 +21,11 @@ package org.apache.tajo.plan.expr;
 public enum EvalType {
   // Unary expression
   NOT(NotEval.class, "!"),
+  SIGNED(SignedEval.class),
 
   // Binary expression
-  AND(BinaryEval.class),
-  OR(BinaryEval.class),
+  AND(BinaryEval.class, "AND"),
+  OR(BinaryEval.class, "OR"),
   EQUAL(BinaryEval.class, "="),
   IS_NULL(IsNullEval.class),
   NOT_EQUAL(BinaryEval.class, "<>"),
@@ -49,19 +50,18 @@ public enum EvalType {
   FUNCTION(GeneralFunctionEval.class),
 
   // String operator or pattern matching predicates
-  LIKE(LikePredicateEval.class),
-  SIMILAR_TO(SimilarToPredicateEval.class),
-  REGEX(RegexPredicateEval.class),
+  LIKE(LikePredicateEval.class, "LIKE"),
+  SIMILAR_TO(SimilarToPredicateEval.class, "SIMILAR TO"),
+  REGEX(RegexPredicateEval.class, "REGEX"),
   CONCATENATE(BinaryEval.class, "||"),
 
   // Other predicates
   BETWEEN(BetweenPredicateEval.class),
   CASE(CaseWhenEval.class),
   IF_THEN(CaseWhenEval.IfThenEval.class),
-  IN(InEval.class),
+  IN(InEval.class, "IN"),
 
   // Value or Reference
-  SIGNED(SignedEval.class),
   CAST(CastEval.class),
   ROW_CONSTANT(RowConstantEval.class),
   FIELD(FieldEval.class),