Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into hbase_storage

Conflicts:
	tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
diff --git a/BUILDING b/BUILDING
index 974fac8..f318f88 100644
--- a/BUILDING
+++ b/BUILDING
@@ -44,6 +44,8 @@
   * Use -Dtar to create a TAR with the distribution (using -Pdist)
   * Use -Dhadoop.version to build with the specific hadoop version (-Dhadoop.version=2.5.1)
     * Currently, 2.3.0 or higher are supported.
+  * Use -Dhbase.version to build with the specific hbase version (-Dhbase.version=0.98.7-hadoop2)
+    * Currently, 0.98.x-hadoop2 or higher are tested.
 
  Tests options:
   * Use -DskipTests to skip tests when running the following Maven goals:
diff --git a/CHANGES b/CHANGES
index 8ecd795..f0efc1e 100644
--- a/CHANGES
+++ b/CHANGES
@@ -8,6 +8,10 @@
     TAJO-1222: DelimitedTextFile should be tolerant against parsing errors.
     (hyunsik)
 
+    TAJO-1131: Supports Inserting or Creating table into 
+    the HBase mapped table.(Hyoungjun Kim)
+
+
     TAJO-1026: Implement Query history persistency manager.(Hyoungjun Kim)
 
     TAJO-233: Support PostgreSQL CatalogStore. (Jihun Kang via hyunsik)
@@ -18,6 +22,9 @@
 
   IMPROVEMENT
 
+    TAJO-1122: Refactor the tajo-storage project structure.
+    (Hyoungjun Kim)
+    
     TAJO-1165: Needs to show error messages on query_executor.jsp. 
     (Jihun Kang via jaehwa)
 
@@ -52,6 +59,8 @@
     TAJO-1125: Separate logical plan and optimizer into a maven module.
     (hyunsik)
 
+    TAJO-1123: Use Fragment instead of FileFragment.(Hyoungjun Kim)
+
     TAJO-1092: Improve the function system to allow other function 
     implementation types. (hyunsik)
 
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
index 4056ed3..0216128 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
@@ -22,7 +22,7 @@
 import com.google.gson.*;
 import com.google.gson.annotations.Expose;
 import com.google.gson.annotations.SerializedName;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.apache.tajo.util.TUtil;
 
 import java.util.ArrayList;
diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java
index b53a6db..6a41caf 100644
--- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java
+++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/Expr.java
@@ -22,7 +22,7 @@
 import com.google.gson.annotations.Expose;
 import com.google.gson.annotations.SerializedName;
 import org.apache.tajo.algebra.LiteralValue.LiteralType;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 
 import java.lang.reflect.Type;
 
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java
index 69d5be4..3a3eb66 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java
@@ -23,7 +23,7 @@
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTableDescProto;
 
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
index f2d9b9c..aca2460 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@ -282,6 +282,8 @@
       return StoreType.TEXTFILE;
     } else if (typeStr.equalsIgnoreCase(StoreType.JSON.name())) {
       return StoreType.JSON;
+    } else if (typeStr.equalsIgnoreCase(StoreType.HBASE.name())) {
+      return StoreType.HBASE;
     } else {
       return null;
     }
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
index aceb6f1..f6acfae 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
@@ -25,7 +25,7 @@
 import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 
 /**
  * Describes a column. It is an immutable object.
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
index 23d39f2..5aea768 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
@@ -22,7 +22,7 @@
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.annotation.NotNull;
 import org.apache.tajo.function.*;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
index 296230b..eae52e5 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
@@ -25,13 +25,12 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.catalog.exception.AlreadyExistsFieldException;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
-import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
 import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.util.TUtil;
 
 import java.util.*;
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java
index 5b10dfd..3390146 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SortSpec.java
@@ -22,7 +22,7 @@
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.SortSpecProto;
 
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
index ce167e1..001b71a 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
@@ -30,7 +30,7 @@
 import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
 
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
index 2d95e6b..6378d42 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableMeta.java
@@ -28,7 +28,7 @@
 import org.apache.tajo.catalog.proto.CatalogProtos.TableProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableProtoOrBuilder;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.util.KeyValueSet;
 
 import java.util.Map;
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java
index 2ecbe98..5090a7d 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java
@@ -25,7 +25,7 @@
 import org.apache.tajo.function.Function;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.json.*;
+import org.apache.tajo.storage.json.*;
 import org.apache.tajo.util.TUtil;
 
 import java.lang.reflect.Type;
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java
index 14722c3..849d970 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/FunctionAdapter.java
@@ -20,8 +20,8 @@
 
 import com.google.gson.*;
 import org.apache.tajo.function.Function;
-import org.apache.tajo.json.CommonGsonHelper;
-import org.apache.tajo.json.GsonSerDerAdapter;
+import org.apache.tajo.storage.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.GsonSerDerAdapter;
 
 import java.lang.reflect.Type;
 
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java
index 3f4304a..33d504e 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/TableMetaAdapter.java
@@ -20,11 +20,11 @@
 
 import com.google.common.base.Preconditions;
 import com.google.gson.*;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.json.GsonSerDerAdapter;
+import org.apache.tajo.storage.json.GsonSerDerAdapter;
 
 import java.lang.reflect.Type;
 
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java
index d775ba8..9d520d4 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionDesc.java
@@ -23,7 +23,7 @@
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 
 /**
  * <code>PartitionDesc</code> presents a table partition.
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java
index 721a7a0..adc5620 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/partition/PartitionMethodDesc.java
@@ -26,7 +26,7 @@
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.util.TUtil;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.PartitionType;
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java
index 1bd0276..ef6e35a 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStats.java
@@ -30,7 +30,7 @@
 import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.util.TUtil;
 
 public class ColumnStats implements ProtoObject<CatalogProtos.ColumnStatsProto>, Cloneable, GsonObject {
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java
index dd358ae..9df01da 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java
@@ -26,7 +26,7 @@
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
 import org.apache.tajo.common.ProtoObject;
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java
index 6a538b8..8842746 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/Function.java
@@ -21,7 +21,7 @@
 import com.google.common.base.Objects;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.datum.Datum;
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index f29bc6c..26951dd 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -37,6 +37,7 @@
   AVRO = 9;
   TEXTFILE = 10;
   JSON = 11;
+  HBASE = 12;
 }
 
 enum OrderType {
@@ -69,16 +70,8 @@
 
 message FragmentProto {
   required string id = 1;
-  required bytes contents = 2;
-}
-
-message FileFragmentProto {
-  required string id = 1;
-  required string path = 2;
-  required int64 startOffset = 3;
-  required int64 length = 4;
-  repeated string hosts = 7;
-  repeated int32 diskIds = 8;
+  required string storeType = 2;
+  required bytes contents = 3;
 }
 
 message TableProto {
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
index 2950a96..4187150 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
@@ -113,7 +113,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
diff --git a/tajo-client/pom.xml b/tajo-client/pom.xml
index 7fc8c74..c023db8 100644
--- a/tajo-client/pom.xml
+++ b/tajo-client/pom.xml
@@ -195,7 +195,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
diff --git a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
index d4886cf..6c5006e 100644
--- a/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
+++ b/tajo-client/src/main/java/org/apache/tajo/cli/tsql/TajoCli.java
@@ -25,13 +25,13 @@
 import jline.console.ConsoleReader;
 import org.apache.commons.cli.*;
 import org.apache.tajo.*;
+import org.apache.tajo.ipc.*;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.cli.tsql.commands.*;
 import org.apache.tajo.client.*;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.util.FileUtil;
 
 import java.io.*;
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
index e99c0ff..b6e47c7 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
@@ -30,11 +30,9 @@
 import org.apache.tajo.client.QueryClient;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.MergeScanner;
-import org.apache.tajo.storage.Scanner;
-import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 import java.io.IOException;
 import java.sql.SQLException;
@@ -92,7 +90,7 @@
         totalRow = INFINITE_ROW_NUM;
       }
 
-      List<FileFragment> frags = getFragments(new Path(desc.getPath()));
+      List<Fragment> frags = getFragments(new Path(desc.getPath()));
       scanner = new MergeScanner(conf, desc.getSchema(), desc.getMeta(), frags);
     }
   }
@@ -111,9 +109,9 @@
     }
   }
 
-  private List<FileFragment> getFragments(Path tablePath)
+  private List<Fragment> getFragments(Path tablePath)
       throws IOException {
-    List<FileFragment> fragments = Lists.newArrayList();
+    List<Fragment> fragments = Lists.newArrayList();
     FileStatus[] files = fs.listStatus(tablePath, new PathFilter() {
       @Override
       public boolean accept(Path path) {
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java
new file mode 100644
index 0000000..ba76d63
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+import org.apache.tajo.validation.Validator;
+
+public enum QueryVars implements ConfigKey {
+  COMMAND_TYPE,
+  STAGING_DIR,
+  OUTPUT_TABLE_NAME,
+  OUTPUT_TABLE_PATH,
+  OUTPUT_PARTITIONS,
+  OUTPUT_OVERWRITE,
+  OUTPUT_AS_DIRECTORY,
+  OUTPUT_PER_FILE_SIZE,
+  ;
+
+  QueryVars() {
+  }
+
+  @Override
+  public String keyname() {
+    return name().toLowerCase();
+  }
+
+  @Override
+  public ConfigType type() {
+    return ConfigType.QUERY;
+  }
+
+  @Override
+  public Class<?> valueClass() {
+    return null;
+  }
+
+  @Override
+  public Validator validator() {
+    return null;
+  }
+}
+
+
diff --git a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
index 1cc28af..de09f09 100644
--- a/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/TajoConstants.java
@@ -37,8 +37,9 @@
   public static final String SYSTEM_HA_DIR_NAME = "ha";
   public static final String SYSTEM_HA_ACTIVE_DIR_NAME = "active";
   public static final String SYSTEM_HA_BACKUP_DIR_NAME = "backup";
-  public static final int UNKNOWN_ROW_NUMBER = -1;
 
+  public static final int UNKNOWN_ROW_NUMBER = -1;
+  public static final int UNKNOWN_LENGTH = -1;
 
   private TajoConstants() {}
 }
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index 312abfb..91f05b6 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -302,6 +302,11 @@
     HISTORY_EXPIRY_TIME_DAY("tajo.history.expiry-time-day", 7),
 
     // Misc -------------------------------------------------------------------
+    // Fragment
+    // When making physical plan, the length of fragment is used to determine the physical operation.
+    // Some storage does not know the size of the fragment.
+    // In this case PhysicalPlanner uses this value to determine.
+    FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH("tajo.fragment.alternative.unknown.length", (long)(512 * 1024 * 1024)),
 
     // Geo IP
     GEOIP_DATA("tajo.function.geoip-database-location", ""),
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
index 442db71..dcf966c 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
@@ -23,8 +23,8 @@
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.exception.InvalidCastException;
 import org.apache.tajo.exception.InvalidOperationException;
-import org.apache.tajo.json.CommonGsonHelper;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.GsonObject;
 
 import static org.apache.tajo.common.TajoDataTypes.Type;
 
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/ClassNameSerializer.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/ClassNameSerializer.java
similarity index 97%
rename from tajo-common/src/main/java/org/apache/tajo/json/ClassNameSerializer.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/ClassNameSerializer.java
index c7ec7ae..7069134 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/ClassNameSerializer.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/ClassNameSerializer.java
@@ -19,7 +19,7 @@
 /**
  * 
  */
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 import com.google.gson.*;
 import org.apache.tajo.util.ClassUtil;
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/CommonGsonHelper.java
similarity index 98%
rename from tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/CommonGsonHelper.java
index 8994b11..e2d20cf 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/CommonGsonHelper.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/CommonGsonHelper.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/DataTypeAdapter.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/DataTypeAdapter.java
similarity index 97%
rename from tajo-common/src/main/java/org/apache/tajo/json/DataTypeAdapter.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/DataTypeAdapter.java
index fc15aea..e8765c9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/DataTypeAdapter.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/DataTypeAdapter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 import com.google.gson.*;
 import org.apache.tajo.common.TajoDataTypes;
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/DatumAdapter.java
similarity index 98%
rename from tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/DatumAdapter.java
index d65559d..b176720 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/DatumAdapter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 import com.google.gson.*;
 import org.apache.tajo.common.TajoDataTypes;
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/GsonHelper.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonHelper.java
similarity index 97%
rename from tajo-common/src/main/java/org/apache/tajo/json/GsonHelper.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/GsonHelper.java
index 8c6d891..2e193ba 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/GsonHelper.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonHelper.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 import com.google.gson.*;
 
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonObject.java
similarity index 95%
rename from tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/GsonObject.java
index 3c49a21..aa4a82d 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/GsonObject.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonObject.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 public interface GsonObject {
   public String toJson();
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/GsonSerDerAdapter.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonSerDerAdapter.java
similarity index 96%
rename from tajo-common/src/main/java/org/apache/tajo/json/GsonSerDerAdapter.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/GsonSerDerAdapter.java
index 80a40f8..493a997 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/GsonSerDerAdapter.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/GsonSerDerAdapter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 import com.google.gson.JsonDeserializer;
 import com.google.gson.JsonSerializer;
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java b/tajo-common/src/main/java/org/apache/tajo/storage/json/PathSerializer.java
similarity index 96%
rename from tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
rename to tajo-common/src/main/java/org/apache/tajo/storage/json/PathSerializer.java
index c286ae3..6d9d32f 100644
--- a/tajo-common/src/main/java/org/apache/tajo/json/PathSerializer.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/json/PathSerializer.java
@@ -19,7 +19,7 @@
 /**
  * 
  */
-package org.apache.tajo.json;
+package org.apache.tajo.storage.json;
 
 import com.google.gson.*;
 import org.apache.hadoop.fs.Path;
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java b/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
index 8e3eb2a..a52651b 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
@@ -22,8 +22,8 @@
 import com.google.common.base.Preconditions;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.json.CommonGsonHelper;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.GsonObject;
 
 import java.util.HashMap;
 import java.util.Map;
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
index 0ceb2b2..dcf5c4e 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
@@ -119,6 +119,15 @@
     return new ArrayList<T>();
   }
 
+  public static List<Integer> newIntList(int...items) {
+    List<Integer> list = new ArrayList<Integer>();
+    for (int item: items) {
+      list.add(item);
+    }
+
+    return list;
+  }
+
   public static <T> List<T> newList(T...items) {
     List<T> list = new ArrayList<T>();
     for (T t : items) {
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java
index 4dcbbee..d42b496 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestBytesDatum.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.datum;
 
 import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.apache.tajo.util.Bytes;
 import org.junit.Test;
 
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java
index 41b4dca..56673f2 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestDateDatum.java
@@ -20,7 +20,7 @@
 
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.exception.InvalidCastException;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java
index 14ebc7c..0ef905d 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestInet4Datum.java
@@ -18,7 +18,7 @@
 
 package org.apache.tajo.datum;
 
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.junit.Before;
 import org.junit.Test;
 
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java
index ea641ec..a130c0b 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimeDatum.java
@@ -21,7 +21,7 @@
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.exception.InvalidCastException;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.junit.Test;
 
 import static org.junit.Assert.assertArrayEquals;
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
index 5886083..f4b5c97 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestTimestampDatum.java
@@ -21,7 +21,7 @@
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.exception.InvalidCastException;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml
index 060ac1b..e4baa09 100644
--- a/tajo-core/pom.xml
+++ b/tajo-core/pom.xml
@@ -246,7 +246,16 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hbase</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
@@ -397,6 +406,33 @@
       <artifactId>gmetric4j</artifactId>
       <version>1.0.3</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop-compat</artifactId>
+      <version>${hbase.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop2-compat</artifactId>
+      <version>${hbase.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>
@@ -456,8 +492,8 @@
         <dependency>
           <groupId>org.apache.tajo</groupId>
           <artifactId>tajo-hcatalog</artifactId>
-          <scope>test</scope>
           <version>${tajo.version}</version>
+          <scope>test</scope>
           <exclusions>
             <exclusion>
               <groupId>com.google.protobuf</groupId>
diff --git a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
index beba248..9b63a24 100644
--- a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
+++ b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
@@ -89,11 +89,11 @@
   ;
 
 create_table_statement
-  : CREATE EXTERNAL TABLE (if_not_exists)? table_name table_elements USING file_type=identifier
-    (param_clause)? (table_partitioning_clauses)? (LOCATION path=Character_String_Literal)
-  | CREATE TABLE (if_not_exists)? table_name table_elements (USING file_type=identifier)?
+  : CREATE EXTERNAL TABLE (if_not_exists)? table_name table_elements USING storage_type=identifier
+    (param_clause)? (table_partitioning_clauses)? (LOCATION path=Character_String_Literal)?
+  | CREATE TABLE (if_not_exists)? table_name table_elements (USING storage_type=identifier)?
     (param_clause)? (table_partitioning_clauses)? (AS query_expression)?
-  | CREATE TABLE (if_not_exists)? table_name (USING file_type=identifier)?
+  | CREATE TABLE (if_not_exists)? table_name (USING storage_type=identifier)?
     (param_clause)? (table_partitioning_clauses)? AS query_expression
   | CREATE TABLE (if_not_exists)? table_name LIKE like_table_name=table_name
   ;
@@ -1559,7 +1559,7 @@
 
 insert_statement
   : INSERT (OVERWRITE)? INTO table_name (LEFT_PAREN column_name_list RIGHT_PAREN)? query_expression
-  | INSERT (OVERWRITE)? INTO LOCATION path=Character_String_Literal (USING file_type=identifier (param_clause)?)? query_expression
+  | INSERT (OVERWRITE)? INTO LOCATION path=Character_String_Literal (USING storage_type=identifier (param_clause)?)? query_expression
   ;
 
 /*
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToCharLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToCharLong.java
new file mode 100644
index 0000000..5fed940
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/string/ToCharLong.java
@@ -0,0 +1,55 @@
+/**
+ * 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.function.string;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+import org.apache.tajo.plan.function.GeneralFunction;
+import org.apache.tajo.storage.Tuple;
+
+import java.text.DecimalFormat;
+
+@Description(
+    functionName = "to_char",
+    description = "convert integer to string.",
+    example = "> SELECT to_char(125, '00999');\n"
+        + "00125",
+    returnType = TajoDataTypes.Type.TEXT,
+    paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.INT8, TajoDataTypes.Type.TEXT})}
+)
+
+public class ToCharLong extends GeneralFunction {
+  DecimalFormat df = null;
+
+  public ToCharLong() {
+    super(new Column[]{new Column("val", TajoDataTypes.Type.INT8), new Column("format", TajoDataTypes.Type.TEXT)});
+  }
+
+  @Override
+  public Datum eval(Tuple params) {
+    if (df == null) {
+      df = new DecimalFormat(params.get(1).asChars());
+    }
+    return new TextDatum(df.format(params.get(0).asInt8()));
+  }
+}
\ No newline at end of file
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
index 57f2536..1d10e67 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/json/CoreGsonHelper.java
@@ -31,7 +31,7 @@
 import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.function.Function;
-import org.apache.tajo.json.*;
+import org.apache.tajo.storage.json.*;
 import org.apache.tajo.plan.serder.EvalNodeAdapter;
 import org.apache.tajo.plan.serder.LogicalNodeAdapter;
 import org.apache.tajo.util.TUtil;
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 03b10c9..40e5f8a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -29,6 +29,7 @@
 import org.apache.tajo.algebra.LiteralValue.LiteralType;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.engine.parser.SQLParser.*;
+import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.StringUtils;
 
@@ -62,6 +63,7 @@
     try {
       context = parser.sql();
     } catch (SQLParseError e) {
+      e.printStackTrace();
       throw new SQLSyntaxError(e);
     }
     return visitSql(context);
@@ -1162,12 +1164,14 @@
       createTable.setExternal();
 
       ColumnDefinition[] elements = getDefinitions(ctx.table_elements());
-      String fileType = ctx.file_type.getText();
-      String path = stripQuote(ctx.path.getText());
-
+      String storageType = ctx.storage_type.getText();
       createTable.setTableElements(elements);
-      createTable.setStorageType(fileType);
-      createTable.setLocation(path);
+      createTable.setStorageType(storageType);
+
+      if (PlannerUtil.isFileStorageType(storageType)) {
+        String path = stripQuote(ctx.path.getText());
+        createTable.setLocation(path);
+      }
     } else {
       if (checkIfExist(ctx.table_elements())) {
         ColumnDefinition[] elements = getDefinitions(ctx.table_elements());
@@ -1175,7 +1179,7 @@
       }
 
       if (checkIfExist(ctx.USING())) {
-        String fileType = ctx.file_type.getText();
+        String fileType = ctx.storage_type.getText();
         createTable.setStorageType(fileType);
       }
 
@@ -1449,7 +1453,7 @@
       insertExpr.setLocation(stripQuote(ctx.path.getText()));
 
       if (ctx.USING() != null) {
-        insertExpr.setStorageType(ctx.file_type.getText());
+        insertExpr.setStorageType(ctx.storage_type.getText());
 
         if (ctx.param_clause() != null) {
           insertExpr.setParams(escapeTableMeta(getParams(ctx.param_clause())));
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 6806609..3880bcb 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
@@ -29,11 +29,13 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.SortSpec;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.SortSpecProto;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.physical.*;
@@ -47,11 +49,9 @@
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.IndexUtil;
@@ -77,11 +77,9 @@
   private static final int UNGENERATED_PID = -1;
 
   protected final TajoConf conf;
-  protected final StorageManager sm;
 
-  public PhysicalPlannerImpl(final TajoConf conf, final StorageManager sm) {
+  public PhysicalPlannerImpl(final TajoConf conf) {
     this.conf = conf;
-    this.sm = sm;
   }
 
   public PhysicalExec createPlan(final TaskAttemptContext context, final LogicalNode logicalPlan)
@@ -250,11 +248,10 @@
   public long estimateSizeRecursive(TaskAttemptContext ctx, String [] tableIds) throws IOException {
     long size = 0;
     for (String tableId : tableIds) {
-      // TODO - CSV is a hack.
-      List<FileFragment> fragments = FragmentConvertor.convert(ctx.getConf(), CatalogProtos.StoreType.CSV,
-          ctx.getTables(tableId));
-      for (FileFragment frag : fragments) {
-        size += frag.getEndKey();
+      FragmentProto[] fragmentProtos = ctx.getTables(tableId);
+      List<Fragment> fragments = FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
+      for (Fragment frag : fragments) {
+        size += StorageManager.getFragmentLength(ctx.getConf(), frag);
       }
     }
     return size;
@@ -446,13 +443,13 @@
     leftSortNode.setSortSpecs(sortSpecs[0]);
     leftSortNode.setInSchema(leftExec.getSchema());
     leftSortNode.setOutSchema(leftExec.getSchema());
-    ExternalSortExec outerSort = new ExternalSortExec(context, sm, leftSortNode, leftExec);
+    ExternalSortExec outerSort = new ExternalSortExec(context, leftSortNode, leftExec);
 
     SortNode rightSortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
     rightSortNode.setSortSpecs(sortSpecs[1]);
     rightSortNode.setInSchema(rightExec.getSchema());
     rightSortNode.setOutSchema(rightExec.getSchema());
-    ExternalSortExec innerSort = new ExternalSortExec(context, sm, rightSortNode, rightExec);
+    ExternalSortExec innerSort = new ExternalSortExec(context, rightSortNode, rightExec);
 
     LOG.info("Join (" + plan.getPID() +") chooses [Merge Join]");
     return new MergeJoinExec(context, plan, outerSort, innerSort, sortSpecs[0], sortSpecs[1]);
@@ -543,13 +540,13 @@
     leftSortNode2.setSortSpecs(sortSpecs2[0]);
     leftSortNode2.setInSchema(leftExec.getSchema());
     leftSortNode2.setOutSchema(leftExec.getSchema());
-    ExternalSortExec outerSort2 = new ExternalSortExec(context, sm, leftSortNode2, leftExec);
+    ExternalSortExec outerSort2 = new ExternalSortExec(context, leftSortNode2, leftExec);
 
     SortNode rightSortNode2 = LogicalPlan.createNodeWithoutPID(SortNode.class);
     rightSortNode2.setSortSpecs(sortSpecs2[1]);
     rightSortNode2.setInSchema(rightExec.getSchema());
     rightSortNode2.setOutSchema(rightExec.getSchema());
-    ExternalSortExec innerSort2 = new ExternalSortExec(context, sm, rightSortNode2, rightExec);
+    ExternalSortExec innerSort2 = new ExternalSortExec(context, rightSortNode2, rightExec);
 
     return new RightOuterMergeJoinExec(context, plan, outerSort2, innerSort2, sortSpecs2[0], sortSpecs2[1]);
   }
@@ -634,13 +631,13 @@
     leftSortNode.setSortSpecs(sortSpecs3[0]);
     leftSortNode.setInSchema(leftExec.getSchema());
     leftSortNode.setOutSchema(leftExec.getSchema());
-    ExternalSortExec outerSort3 = new ExternalSortExec(context, sm, leftSortNode, leftExec);
+    ExternalSortExec outerSort3 = new ExternalSortExec(context, leftSortNode, leftExec);
 
     SortNode rightSortNode = LogicalPlan.createNodeWithoutPID(SortNode.class);
     rightSortNode.setSortSpecs(sortSpecs3[1]);
     rightSortNode.setInSchema(rightExec.getSchema());
     rightSortNode.setOutSchema(rightExec.getSchema());
-    ExternalSortExec innerSort3 = new ExternalSortExec(context, sm, rightSortNode, rightExec);
+    ExternalSortExec innerSort3 = new ExternalSortExec(context, rightSortNode, rightExec);
 
     return new MergeFullOuterJoinExec(context, plan, outerSort3, innerSort3, sortSpecs3[0], sortSpecs3[1]);
   }
@@ -768,7 +765,7 @@
     switch (plan.getShuffleType()) {
     case HASH_SHUFFLE:
     case SCATTERED_HASH_SHUFFLE:
-      return new HashShuffleFileWriteExec(ctx, sm, plan, subOp);
+      return new HashShuffleFileWriteExec(ctx, plan, subOp);
 
     case RANGE_SHUFFLE:
       SortExec sortExec = PhysicalPlanUtil.findExecutor(subOp, SortExec.class);
@@ -783,7 +780,7 @@
           specs[i] = new SortSpec(columns[i]);
         }
       }
-      return new RangeShuffleFileWriteExec(ctx, sm, subOp, plan.getInSchema(), plan.getInSchema(), sortSpecs);
+      return new RangeShuffleFileWriteExec(ctx, subOp, plan.getInSchema(), plan.getInSchema(), sortSpecs);
 
     case NONE_SHUFFLE:
       // if there is no given NULL CHAR property in the table property and the query is neither CTAS or INSERT,
@@ -869,7 +866,7 @@
     sortNode.setInSchema(child.getSchema());
     sortNode.setOutSchema(child.getSchema());
 
-    ExternalSortExec sortExec = new ExternalSortExec(context, sm, sortNode, child);
+    ExternalSortExec sortExec = new ExternalSortExec(context, sortNode, child);
     LOG.info("The planner chooses [Sort-based Column Partitioned Store] algorithm");
     return new SortBasedColPartitionStoreExec(context, storeTableNode, sortExec);
   }
@@ -896,10 +893,10 @@
     // Since the default intermediate file format is raw file, it is not problem right now.
     if (checkIfSortEquivalance(ctx, scanNode, node)) {
       if (ctx.getTable(scanNode.getCanonicalName()) == null) {
-        return new SeqScanExec(ctx, sm, scanNode, null);
+        return new SeqScanExec(ctx, scanNode, null);
       }
       FragmentProto [] fragments = ctx.getTables(scanNode.getCanonicalName());
-      return new ExternalSortExec(ctx, sm, (SortNode) node.peek(), fragments);
+      return new ExternalSortExec(ctx, (SortNode) node.peek(), fragments);
     } else {
       Enforcer enforcer = ctx.getEnforcer();
 
@@ -919,25 +916,26 @@
         if (scanNode instanceof PartitionedTableScanNode) {
           if (broadcastFlag) {
             PartitionedTableScanNode partitionedTableScanNode = (PartitionedTableScanNode) scanNode;
-            List<FileFragment> fileFragments = TUtil.newList();
+            List<Fragment> fileFragments = TUtil.newList();
+            FileStorageManager fileStorageManager = (FileStorageManager)StorageManager.getFileStorageManager(ctx.getConf());
             for (Path path : partitionedTableScanNode.getInputPaths()) {
-              fileFragments.addAll(TUtil.newList(sm.split(scanNode.getCanonicalName(), path)));
+              fileFragments.addAll(TUtil.newList(fileStorageManager.split(scanNode.getCanonicalName(), path)));
             }
 
             FragmentProto[] fragments =
                 FragmentConvertor.toFragmentProtoArray(fileFragments.toArray(new FileFragment[fileFragments.size()]));
 
             ctx.addFragments(scanNode.getCanonicalName(), fragments);
-            return new PartitionMergeScanExec(ctx, sm, scanNode, fragments);
+            return new PartitionMergeScanExec(ctx, scanNode, fragments);
           }
         }
       }
 
       if (ctx.getTable(scanNode.getCanonicalName()) == null) {
-        return new SeqScanExec(ctx, sm, scanNode, null);
+        return new SeqScanExec(ctx, scanNode, null);
       }
       FragmentProto [] fragments = ctx.getTables(scanNode.getCanonicalName());
-      return new SeqScanExec(ctx, sm, scanNode, fragments);
+      return new SeqScanExec(ctx, scanNode, fragments);
     }
   }
 
@@ -997,7 +995,7 @@
     sortNode.setSortSpecs(sortSpecs);
     sortNode.setInSchema(subOp.getSchema());
     sortNode.setOutSchema(subOp.getSchema());
-    ExternalSortExec sortExec = new ExternalSortExec(ctx, sm, sortNode, subOp);
+    ExternalSortExec sortExec = new ExternalSortExec(ctx, sortNode, subOp);
     LOG.info("The planner chooses [Sort Aggregation] in (" + TUtil.arrayToString(sortSpecs) + ")");
     return new SortAggregateExec(ctx, groupbyNode, sortExec);
   }
@@ -1038,7 +1036,7 @@
       sortNode.setSortSpecs(sortSpecs);
       sortNode.setInSchema(subOp.getSchema());
       sortNode.setOutSchema(subOp.getSchema());
-      child = new ExternalSortExec(context, sm, sortNode, subOp);
+      child = new ExternalSortExec(context, sortNode, subOp);
       LOG.info("The planner chooses [Sort Aggregation] in (" + TUtil.arrayToString(sortSpecs) + ")");
     }
 
@@ -1101,7 +1099,7 @@
     sortNode.setSortSpecs(sortSpecs.toArray(new SortSpec[]{}));
     sortNode.setInSchema(distinctNode.getInSchema());
     sortNode.setOutSchema(distinctNode.getInSchema());
-    ExternalSortExec sortExec = new ExternalSortExec(context, sm, sortNode, subOp);
+    ExternalSortExec sortExec = new ExternalSortExec(context, sortNode, subOp);
 
     return sortExec;
   }
@@ -1132,7 +1130,7 @@
       sortNode.setSortSpecs(sortSpecs);
       sortNode.setInSchema(subOp.getSchema());
       sortNode.setOutSchema(eachGroupbyNode.getInSchema());
-      ExternalSortExec sortExec = new ExternalSortExec(ctx, sm, sortNode, subOp);
+      ExternalSortExec sortExec = new ExternalSortExec(ctx, sortNode, subOp);
 
       sortAggregateExec[index++] = new SortAggregateExec(ctx, eachGroupbyNode, sortExec);
     }
@@ -1160,7 +1158,7 @@
       if (algorithm == SortEnforce.SortAlgorithm.IN_MEMORY_SORT) {
         return new MemSortExec(context, sortNode, child);
       } else {
-        return new ExternalSortExec(context, sm, sortNode, child);
+        return new ExternalSortExec(context, sortNode, child);
       }
     }
 
@@ -1169,7 +1167,7 @@
 
   public SortExec createBestSortPlan(TaskAttemptContext context, SortNode sortNode,
                                      PhysicalExec child) throws IOException {
-    return new ExternalSortExec(context, sm, sortNode, child);
+    return new ExternalSortExec(context, sortNode, child);
   }
 
   public PhysicalExec createIndexScanExec(TaskAttemptContext ctx,
@@ -1181,14 +1179,15 @@
 
     FragmentProto [] fragmentProtos = ctx.getTables(annotation.getTableName());
     List<FileFragment> fragments =
-        FragmentConvertor.convert(ctx.getConf(), ctx.getDataChannel().getStoreType(), fragmentProtos);
+        FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
 
     String indexName = IndexUtil.getIndexNameOfFrag(fragments.get(0), annotation.getSortKeys());
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(ctx.getConf());
     Path indexPath = new Path(sm.getTablePath(annotation.getTableName()), "index");
 
     TupleComparator comp = new BaseTupleComparator(annotation.getKeySchema(),
         annotation.getSortKeys());
-    return new BSTIndexScanExec(ctx, sm, annotation, fragments.get(0), new Path(indexPath, indexName),
+    return new BSTIndexScanExec(ctx, annotation, fragments.get(0), new Path(indexPath, indexName),
         annotation.getKeySchema(), comp, annotation.getDatum());
 
   }
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java
index 77eb32d..aecb364 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/ExecutionBlock.java
@@ -82,6 +82,8 @@
       } else if (node instanceof TableSubQueryNode) {
         TableSubQueryNode subQuery = (TableSubQueryNode) node;
         s.add(s.size(), subQuery.getSubQuery());
+      } else if (node instanceof StoreTableNode) {
+        store = (StoreTableNode)node;
       }
     }
   }
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
index f699607..6adc523 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.engine.planner.physical;
 
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.tajo.catalog.Schema;
@@ -47,8 +48,7 @@
 
   private float progress;
 
-  public BSTIndexScanExec(TaskAttemptContext context,
-                          StorageManager sm , ScanNode scanNode ,
+  public BSTIndexScanExec(TaskAttemptContext context, ScanNode scanNode ,
        FileFragment fragment, Path fileName , Schema keySchema,
        TupleComparator comparator , Datum[] datum) throws IOException {
     super(context, scanNode.getInSchema(), scanNode.getOutSchema());
@@ -61,7 +61,8 @@
     this.fileScanner.init();
     this.projector = new Projector(context, inSchema, outSchema, scanNode.getTargets());
 
-    this.reader = new BSTIndex(sm.getFileSystem().getConf()).
+    FileSystem fs = fileName.getFileSystem(context.getConf());
+    this.reader = new BSTIndex(fs.getConf()).
         getIndexReader(fileName, keySchema, comparator);
     this.reader.open();
   }
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
index ec239de..8ee4e2f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
@@ -35,6 +35,7 @@
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.StoreTableNode;
 import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.FileStorageManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.unit.StorageUnit;
@@ -120,6 +121,7 @@
     super.init();
 
     storeTablePath = context.getOutputPath();
+
     FileSystem fs = storeTablePath.getFileSystem(context.getConf());
     if (!fs.exists(storeTablePath.getParent())) {
       fs.mkdirs(storeTablePath.getParent());
@@ -160,7 +162,8 @@
       actualFilePath = new Path(lastFileName + "_" + suffixId);
     }
 
-    appender = StorageManager.getStorageManager(context.getConf()).getAppender(meta, outSchema, actualFilePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(context.getConf()))
+        .getAppender(meta, outSchema, actualFilePath);
 
     appender.enableStats();
     appender.init();
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
index 121e6bd..4e19114 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ExternalSortExec.java
@@ -106,7 +106,7 @@
   /** total bytes of input data */
   private long sortAndStoredBytes;
 
-  private ExternalSortExec(final TaskAttemptContext context, final StorageManager sm, final SortNode plan)
+  private ExternalSortExec(final TaskAttemptContext context, final SortNode plan)
       throws PhysicalPlanningException {
     super(context, plan.getInSchema(), plan.getOutSchema(), null, plan.getSortKeys());
 
@@ -128,10 +128,9 @@
     localFS = new RawLocalFileSystem();
   }
 
-  public ExternalSortExec(final TaskAttemptContext context,
-                          final StorageManager sm, final SortNode plan,
+  public ExternalSortExec(final TaskAttemptContext context,final SortNode plan,
                           final CatalogProtos.FragmentProto[] fragments) throws PhysicalPlanningException {
-    this(context, sm, plan);
+    this(context, plan);
 
     mergedInputFragments = TUtil.newList();
     for (CatalogProtos.FragmentProto proto : fragments) {
@@ -140,10 +139,9 @@
     }
   }
 
-  public ExternalSortExec(final TaskAttemptContext context,
-                          final StorageManager sm, final SortNode plan, final PhysicalExec child)
+  public ExternalSortExec(final TaskAttemptContext context, final SortNode plan, final PhysicalExec child)
       throws IOException {
-    this(context, sm, plan);
+    this(context, plan);
     setChild(child);
   }
 
@@ -175,7 +173,7 @@
 
     long chunkWriteStart = System.currentTimeMillis();
     Path outputPath = getChunkPathForWrite(0, chunkId);
-    final RawFileAppender appender = new RawFileAppender(context.getConf(), inSchema, meta, outputPath);
+    final RawFileAppender appender = new RawFileAppender(context.getConf(), null, inSchema, meta, outputPath);
     appender.init();
     for (Tuple t : tupleBlock) {
       appender.addTuple(t);
@@ -473,7 +471,7 @@
       final Path outputPath = getChunkPathForWrite(level + 1, nextRunId);
       info(LOG, mergeFanout + " files are being merged to an output file " + outputPath.getName());
       long mergeStartTime = System.currentTimeMillis();
-      final RawFileAppender output = new RawFileAppender(context.getConf(), inSchema, meta, outputPath);
+      final RawFileAppender output = new RawFileAppender(context.getConf(), null, inSchema, meta, outputPath);
       output.init();
       final Scanner merger = createKWayMerger(inputFiles, startIdx, mergeFanout);
       merger.init();
@@ -857,7 +855,7 @@
     if (finalOutputFiles != null) {
       for (FileFragment frag : finalOutputFiles) {
         File tmpFile = new File(localFS.makeQualified(frag.getPath()).toUri());
-        if (frag.getStartKey() == 0 && frag.getEndKey() == tmpFile.length()) {
+        if (frag.getStartKey() == 0 && frag.getLength() == tmpFile.length()) {
           localFS.delete(frag.getPath(), true);
           LOG.info("Delete file: " + frag);
         }
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java
index 5bf80fd..d051fb6 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java
@@ -55,7 +55,7 @@
   private HashShuffleAppenderManager hashShuffleAppenderManager;
   private int numHashShuffleBufferTuples;
 
-  public HashShuffleFileWriteExec(TaskAttemptContext context, final StorageManager sm,
+  public HashShuffleFileWriteExec(TaskAttemptContext context,
                                   final ShuffleFileWriteNode plan, final PhysicalExec child) throws IOException {
     super(context, plan.getInSchema(), plan.getOutSchema(), child);
     Preconditions.checkArgument(plan.hasShuffleKeys());
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
index 0569c1b..5297e2c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PartitionMergeScanExec.java
@@ -44,25 +44,22 @@
   private List<SeqScanExec> scanners = Lists.newArrayList();
   private Iterator<SeqScanExec> iterator;
 
-  private StorageManager sm;
-
   private float progress;
   protected TableStats inputStats;
 
-  public PartitionMergeScanExec(TaskAttemptContext context, StorageManager sm,
+  public PartitionMergeScanExec(TaskAttemptContext context,
                                 ScanNode plan, CatalogProtos.FragmentProto[] fragments) throws IOException {
     super(context, plan.getInSchema(), plan.getOutSchema());
 
     this.plan = plan;
     this.fragments = fragments;
-    this.sm = sm;
 
     inputStats = new TableStats();
   }
 
   public void init() throws IOException {
     for (CatalogProtos.FragmentProto fragment : fragments) {
-      SeqScanExec scanExec = new SeqScanExec(context, sm, (ScanNode) PlannerUtil.clone(null, plan),
+      SeqScanExec scanExec = new SeqScanExec(context, (ScanNode) PlannerUtil.clone(null, plan),
           new CatalogProtos.FragmentProto[] {fragment});
       scanners.add(scanExec);
     }
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
index a63b838..247b373 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
@@ -34,10 +34,7 @@
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.PersistentStoreNode;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 
@@ -141,9 +138,7 @@
                                          int currentDepth, int maxDepth) throws IOException {
     // Intermediate directory
     if (fs.isDirectory(path)) {
-
-      FileStatus[] files = fs.listStatus(path, StorageManager.hiddenFileFilter);
-
+      FileStatus[] files = fs.listStatus(path, FileStorageManager.hiddenFileFilter);
       if (files != null && files.length > 0) {
 
         for (FileStatus eachFile : files) {
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
index 79993da..585d73a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
@@ -27,6 +27,8 @@
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TextDatum;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.index.bst.BSTIndex;
@@ -51,7 +53,7 @@
   private FileAppender appender;
   private TableMeta meta;
 
-  public RangeShuffleFileWriteExec(final TaskAttemptContext context, final StorageManager sm,
+  public RangeShuffleFileWriteExec(final TaskAttemptContext context,
                                    final PhysicalExec child, final Schema inSchema, final Schema outSchema,
                                    final SortSpec[] sortSpecs) throws IOException {
     super(context, inSchema, outSchema, child);
@@ -78,8 +80,8 @@
         context.getDataChannel().getStoreType() : CatalogProtos.StoreType.RAW);
     FileSystem fs = new RawLocalFileSystem();
     fs.mkdirs(storeTablePath);
-    this.appender = (FileAppender) StorageManager.getStorageManager(context.getConf()).getAppender(meta,
-        outSchema, new Path(storeTablePath, "output"));
+    this.appender = (FileAppender) ((FileStorageManager)StorageManager.getFileStorageManager(context.getConf()))
+        .getAppender(meta, outSchema, new Path(storeTablePath, "output"));
     this.appender.enableStats();
     this.appender.init();
     this.indexWriter = bst.getIndexWriter(new Path(storeTablePath, "index"),
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 759b19c..3aed7ba 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
@@ -40,6 +40,7 @@
 import org.apache.tajo.plan.rewrite.rules.PartitionedTableRewriter;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.worker.TaskAttemptContext;
 
@@ -67,7 +68,7 @@
 
   private boolean cacheRead = false;
 
-  public SeqScanExec(TaskAttemptContext context, StorageManager sm, ScanNode plan,
+  public SeqScanExec(TaskAttemptContext context, ScanNode plan,
                      CatalogProtos.FragmentProto [] fragments) throws IOException {
     super(context, plan.getInSchema(), plan.getOutSchema());
 
@@ -79,9 +80,8 @@
       String pathNameKey = "";
       if (fragments != null) {
         for (FragmentProto f : fragments) {
-          FileFragment fileFragement = FragmentConvertor.convert(
-              context.getConf(), plan.getTableDesc().getMeta().getStoreType(), f);
-          pathNameKey += fileFragement.getPath();
+          Fragment fragement = FragmentConvertor.convert(context.getConf(), f);
+          pathNameKey += fragement.getKey();
         }
       }
 
@@ -215,13 +215,13 @@
     if (fragments != null) {
       if (fragments.length > 1) {
         this.scanner = new MergeScanner(context.getConf(), plan.getPhysicalSchema(), plan.getTableDesc().getMeta(),
-            FragmentConvertor.<FileFragment>convert(context.getConf(), plan.getTableDesc().getMeta().getStoreType(),
-                fragments), projected
+            FragmentConvertor.convert(context.getConf(), fragments), projected
         );
       } else {
-        this.scanner = StorageManager.getStorageManager(
-            context.getConf()).getScanner(plan.getTableDesc().getMeta(), plan.getPhysicalSchema(), fragments[0],
-            projected);
+        StorageManager storageManager = StorageManager.getStorageManager(
+            context.getConf(), plan.getTableDesc().getMeta().getStoreType());
+        this.scanner = storageManager.getScanner(plan.getTableDesc().getMeta(),
+            plan.getPhysicalSchema(), fragments[0], projected);
       }
       scanner.init();
     }
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
index 0dc172c..3d3da5c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
@@ -23,12 +23,15 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.statistics.StatisticsUtil;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.plan.logical.InsertNode;
 import org.apache.tajo.plan.logical.PersistentStoreNode;
+import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.FileStorageManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.unit.StorageUnit;
@@ -78,30 +81,33 @@
   }
 
   public void openNewFile(int suffixId) throws IOException {
-    String prevFile = null;
+    Schema appenderSchema = (plan instanceof InsertNode) ? ((InsertNode) plan).getTableSchema() : outSchema;
 
-    lastFileName = context.getOutputPath();
-    if (suffixId > 0) {
-      prevFile = lastFileName.toString();
+    if (PlannerUtil.isFileStorageType(meta.getStoreType())) {
+      String prevFile = null;
 
-      lastFileName = new Path(lastFileName + "_" + suffixId);
-    }
+      lastFileName = context.getOutputPath();
 
-    if (plan instanceof InsertNode) {
-      InsertNode createTableNode = (InsertNode) plan;
-      appender = StorageManager.getStorageManager(context.getConf()).getAppender(meta,
-          createTableNode.getTableSchema(), context.getOutputPath());
+      if (suffixId > 0) {
+        prevFile = lastFileName.toString();
+        lastFileName = new Path(lastFileName + "_" + suffixId);
+      }
+
+      appender = ((FileStorageManager)StorageManager.getFileStorageManager(context.getConf()))
+          .getAppender(meta, appenderSchema, lastFileName);
+
+      if (suffixId > 0) {
+        LOG.info(prevFile + " exceeds " + SessionVars.MAX_OUTPUT_FILE_SIZE.keyname() + " (" + maxPerFileSize + " MB), " +
+            "The remain output will be written into " + lastFileName.toString());
+      }
     } else {
-      appender = StorageManager.getStorageManager(context.getConf()).getAppender(meta, outSchema, lastFileName);
+      appender = StorageManager.getStorageManager(context.getConf(), meta.getStoreType()).getAppender(
+          context.getQueryContext(),
+          context.getTaskId(), meta, appenderSchema, context.getQueryContext().getStagingDir());
     }
 
     appender.enableStats();
     appender.init();
-
-    if (suffixId > 0) {
-      LOG.info(prevFile + " exceeds " + SessionVars.MAX_OUTPUT_FILE_SIZE.keyname() + " (" + maxPerFileSize + " MB), " +
-          "The remain output will be written into " + lastFileName.toString());
-    }
   }
 
   /* (non-Javadoc)
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
index d8f7f08..488cae5 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
@@ -21,11 +21,11 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.ConfigKey;
 import org.apache.tajo.OverridableConf;
+import org.apache.tajo.QueryVars;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.master.session.Session;
-import org.apache.tajo.validation.Validator;
 import org.apache.tajo.plan.logical.NodeType;
 
 import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
@@ -34,41 +34,6 @@
  * QueryContent is a overridable config, and it provides a set of various configs for a query instance.
  */
 public class QueryContext extends OverridableConf {
-  public static enum QueryVars implements ConfigKey {
-    COMMAND_TYPE,
-    STAGING_DIR,
-    OUTPUT_TABLE_NAME,
-    OUTPUT_TABLE_PATH,
-    OUTPUT_PARTITIONS,
-    OUTPUT_OVERWRITE,
-    OUTPUT_AS_DIRECTORY,
-    OUTPUT_PER_FILE_SIZE,
-    ;
-
-    QueryVars() {
-    }
-
-    @Override
-    public String keyname() {
-      return name().toLowerCase();
-    }
-
-    @Override
-    public ConfigType type() {
-      return ConfigType.QUERY;
-    }
-
-    @Override
-    public Class<?> valueClass() {
-      return null;
-    }
-
-    @Override
-    public Validator validator() {
-      return null;
-    }
-  }
-
   public QueryContext(TajoConf conf) {
     super(conf, ConfigKey.ConfigType.QUERY);
   }
@@ -103,8 +68,8 @@
   }
 
   public Path getStagingDir() {
-    String strVal = get(QueryVars.STAGING_DIR);
-    return strVal != null ? new Path(strVal) : null;
+    String strVal = get(QueryVars.STAGING_DIR, "");
+    return strVal != null && !strVal.isEmpty() ? new Path(strVal) : null;
   }
 
   /**
@@ -127,7 +92,9 @@
   }
 
   public void setOutputPath(Path path) {
-    put(QueryVars.OUTPUT_TABLE_PATH, path.toUri().toString());
+    if (path != null) {
+      put(QueryVars.OUTPUT_TABLE_PATH, path.toUri().toString());
+    }
   }
 
   public Path getOutputPath() {
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
index aeb4e05..3bb1b5b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
@@ -125,12 +125,13 @@
     Tuple startTuple = new VTuple(target.size());
     Tuple endTuple = new VTuple(target.size());
     int i = 0;
+    int sortSpecIndex = 0;
 
     // In outer join, empty table could be searched.
     // As a result, min value and max value would be null.
     // So, we should put NullDatum for this case.
     for (Column col : target.getColumns()) {
-      if (sortSpecs[i].isAscending()) {
+      if (sortSpecs[sortSpecIndex].isAscending()) {
         if (statSet.get(col).getMinValue() != null)
           startTuple.put(i, statSet.get(col).getMinValue());
         else
@@ -164,6 +165,10 @@
         else
           endTuple.put(i, DatumFactory.createNullDatum());
       }
+      if (target.getColumns().size() == sortSpecs.length) {
+        // Not composite column sort
+        sortSpecIndex++;
+      }
       i++;
     }
     return new TupleRange(sortSpecs, startTuple, endTuple);
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java
index e4b98d4..406550d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/DefaultFragmentScheduleAlgorithm.java
@@ -21,6 +21,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.NetUtils;
 
 import java.util.*;
@@ -81,9 +82,12 @@
   @Override
   public void addFragment(FragmentPair fragmentPair) {
     String[] hosts = fragmentPair.getLeftFragment().getHosts();
-    int[] diskIds = fragmentPair.getLeftFragment().getDiskIds();
+    int[] diskIds = null;
+    if (fragmentPair.getLeftFragment() instanceof FileFragment) {
+      diskIds = ((FileFragment)fragmentPair.getLeftFragment()).getDiskIds();
+    }
     for (int i = 0; i < hosts.length; i++) {
-      addFragment(hosts[i], diskIds[i], fragmentPair);
+      addFragment(hosts[i], diskIds != null ? diskIds[i] : -1, fragmentPair);
     }
     fragmentNum++;
   }
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
index 77e3257..01137aa 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
@@ -44,6 +44,7 @@
 import org.apache.tajo.master.container.TajoContainerId;
 import org.apache.tajo.storage.DataLocation;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.worker.FetchImpl;
 
@@ -150,8 +151,8 @@
     super.stop();
   }
 
-  private FileFragment[] fragmentsForNonLeafTask;
-  private FileFragment[] broadcastFragmentsForNonLeafTask;
+  private Fragment[] fragmentsForNonLeafTask;
+  private Fragment[] broadcastFragmentsForNonLeafTask;
 
   LinkedList<TaskRequestEvent> taskRequestEvents = new LinkedList<TaskRequestEvent>();
   public void schedule() {
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java b/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
index 598b1c5..827386b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.master;
 
 import com.google.common.base.Objects;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 /**
  * FragmentPair consists of two fragments, a left fragment and a right fragment.
@@ -29,23 +29,23 @@
  * For other queries, it is assumed to have only a left fragment.
  */
 public class FragmentPair {
-  private FileFragment leftFragment;
-  private FileFragment rightFragment;
+  private Fragment leftFragment;
+  private Fragment rightFragment;
 
-  public FragmentPair(FileFragment left) {
+  public FragmentPair(Fragment left) {
     this.leftFragment = left;
   }
 
-  public FragmentPair(FileFragment left, FileFragment right) {
+  public FragmentPair(Fragment left, Fragment right) {
     this.leftFragment = left;
     this.rightFragment = right;
   }
 
-  public FileFragment getLeftFragment() {
+  public Fragment getLeftFragment() {
     return leftFragment;
   }
 
-  public FileFragment getRightFragment() {
+  public Fragment getRightFragment() {
     return rightFragment;
   }
 
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 9bf9a75..be7d54f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -40,6 +40,7 @@
 import org.apache.tajo.catalog.exception.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.conf.TajoConf;
@@ -302,6 +303,15 @@
         responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
       }
     } else { // it requires distributed execution. So, the query is forwarded to a query master.
+      StoreType storeType = PlannerUtil.getStoreType(plan);
+      if (storeType != null) {
+        StorageManager sm = StorageManager.getStorageManager(context.getConf(), storeType);
+        StorageProperty storageProperty = sm.getStorageProperty();
+        if (!storageProperty.isSupportsInsertInto()) {
+          throw new VerifyException("Inserting into non-file storage is not supported.");
+        }
+        sm.beforeInsertOrCATS(rootNode.getChild());
+      }
       context.getSystemMetrics().counter("Query", "numDMLQuery").inc();
       hookManager.doHooks(queryContext, plan);
 
@@ -314,6 +324,7 @@
         responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
         responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
         responseBuilder.setErrorMessage("Fail starting QueryMaster.");
+        LOG.error("Fail starting QueryMaster: " + sql);
       } else {
         responseBuilder.setIsForwarded(true);
         responseBuilder.setQueryId(queryInfo.getQueryId().getProto());
@@ -322,7 +333,8 @@
           responseBuilder.setQueryMasterHost(queryInfo.getQueryMasterHost());
         }
         responseBuilder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
-        LOG.info("Query is forwarded to " + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
+        LOG.info("Query " + queryInfo.getQueryId().toString() + "," + queryInfo.getSql() + "," +
+            " is forwarded to " + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
       }
     }
     SubmitQueryResponse response = responseBuilder.build();
@@ -515,6 +527,7 @@
     LOG.info("=============================================");
 
     annotatedPlanVerifier.verify(queryContext, state, plan);
+    verifyInsertTableSchema(queryContext, state, plan);
 
     if (!state.verified()) {
       StringBuilder sb = new StringBuilder();
@@ -527,6 +540,25 @@
     return plan;
   }
 
+  private void verifyInsertTableSchema(QueryContext queryContext, VerificationState state, LogicalPlan plan) {
+    StoreType storeType = PlannerUtil.getStoreType(plan);
+    if (storeType != null) {
+      LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+      if (rootNode.getChild().getType() == NodeType.INSERT) {
+        try {
+          TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+          InsertNode iNode = rootNode.getChild();
+          Schema outSchema = iNode.getChild().getOutSchema();
+
+          StorageManager.getStorageManager(queryContext.getConf(), storeType)
+              .verifyInsertTableSchema(tableDesc, outSchema);
+        } catch (Throwable t) {
+          state.addVerification(t.getMessage());
+        }
+      }
+    }
+  }
+
   /**
    * Alter a given table
    */
@@ -689,32 +721,18 @@
       meta = CatalogUtil.newTableMeta(createTable.getStorageType());
     }
 
-    if(createTable.isExternal()){
+    if(PlannerUtil.isFileStorageType(createTable.getStorageType()) && createTable.isExternal()){
       Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
-    } else {
-      String databaseName;
-      String tableName;
-      if (CatalogUtil.isFQTableName(createTable.getTableName())) {
-        databaseName = CatalogUtil.extractQualifier(createTable.getTableName());
-        tableName = CatalogUtil.extractSimpleName(createTable.getTableName());
-      } else {
-        databaseName = queryContext.getCurrentDatabase();
-        tableName = createTable.getTableName();
-      }
-
-      // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} )
-      Path tablePath = StorageUtil.concatPath(sm.getWarehouseDir(), databaseName, tableName);
-      createTable.setPath(tablePath);
     }
 
-    return createTableOnPath(queryContext, createTable.getTableName(), createTable.getTableSchema(),
-        meta, createTable.getPath(), createTable.isExternal(), createTable.getPartitionMethod(), ifNotExists);
+    return createTable(queryContext, createTable.getTableName(), createTable.getStorageType(),
+        createTable.getTableSchema(), meta, createTable.getPath(), createTable.isExternal(),
+        createTable.getPartitionMethod(), ifNotExists);
   }
 
-  public TableDesc createTableOnPath(QueryContext queryContext, String tableName, Schema schema, TableMeta meta,
-                                     Path path, boolean isExternal, PartitionMethodDesc partitionDesc,
-                                     boolean ifNotExists)
-      throws IOException {
+  public TableDesc createTable(QueryContext queryContext, String tableName, StoreType storeType,
+                               Schema schema, TableMeta meta, Path path, boolean isExternal,
+                               PartitionMethodDesc partitionDesc, boolean ifNotExists) throws IOException {
     String databaseName;
     String simpleTableName;
     if (CatalogUtil.isFQTableName(tableName)) {
@@ -738,39 +756,15 @@
       }
     }
 
-    FileSystem fs = path.getFileSystem(context.getConf());
-
-    if (isExternal) {
-      if(!fs.exists(path)) {
-        LOG.error("ERROR: " + path.toUri() + " does not exist");
-        throw new IOException("ERROR: " + path.toUri() + " does not exist");
-      }
-    } else {
-      fs.mkdirs(path);
-    }
-
-    long totalSize = 0;
-
-    try {
-      totalSize = sm.calculateSize(path);
-    } catch (IOException e) {
-      LOG.warn("Cannot calculate the size of the relation", e);
-    }
-
-    TableStats stats = new TableStats();
-    stats.setNumBytes(totalSize);
-
-    if (isExternal) { // if it is an external table, there is no way to know the exact row number without processing.
-      stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
-    }
-
     TableDesc desc = new TableDesc(CatalogUtil.buildFQName(databaseName, simpleTableName),
-        schema, meta, path.toUri(), isExternal);
-    desc.setStats(stats);
+        schema, meta, (path != null ? path.toUri(): null), isExternal);
+    
     if (partitionDesc != null) {
       desc.setPartitionMethod(partitionDesc);
     }
 
+    StorageManager.getStorageManager(queryContext.getConf(), storeType).createTable(desc, ifNotExists);
+
     if (catalog.createTable(desc)) {
       LOG.info("Table " + desc.getName() + " is created (" + desc.getStats().getNumBytes() + ")");
       return desc;
@@ -864,13 +858,13 @@
       }
     }
 
-    Path path = new Path(catalog.getTableDesc(qualifiedName).getPath());
+    TableDesc tableDesc = catalog.getTableDesc(qualifiedName);
     catalog.dropTable(qualifiedName);
 
     if (purge) {
       try {
-        FileSystem fs = path.getFileSystem(context.getConf());
-        fs.delete(path, true);
+        StorageManager.getStorageManager(queryContext.getConf(),
+            tableDesc.getMeta().getStoreType()).purgeTable(tableDesc);
       } catch (IOException e) {
         throw new InternalError(e.getMessage());
       }
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java b/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java
index 82fd6fc..56cf8e5 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GreedyFragmentScheduleAlgorithm.java
@@ -24,6 +24,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.tajo.master.DefaultFragmentScheduleAlgorithm.FragmentsPerDisk;
+import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.util.TUtil;
 
@@ -101,9 +102,12 @@
   @Override
   public void addFragment(FragmentPair fragmentPair) {
     String[] hosts = fragmentPair.getLeftFragment().getHosts();
-    int[] diskIds = fragmentPair.getLeftFragment().getDiskIds();
+    int[] diskIds = null;
+    if (fragmentPair.getLeftFragment() instanceof FileFragment) {
+      diskIds = ((FileFragment)fragmentPair.getLeftFragment()).getDiskIds();
+    }
     for (int i = 0; i < hosts.length; i++) {
-      addFragment(hosts[i], diskIds[i], fragmentPair);
+      addFragment(hosts[i], diskIds != null ? diskIds[i] : -1, fragmentPair);
     }
     totalFragmentNum++;
   }
@@ -276,23 +280,27 @@
 
   public void removeFragment(FragmentPair fragmentPair) {
     String [] hosts = fragmentPair.getLeftFragment().getHosts();
-    int[] diskIds = fragmentPair.getLeftFragment().getDiskIds();
+    int[] diskIds = null;
+    if (fragmentPair.getLeftFragment() instanceof FileFragment) {
+      diskIds = ((FileFragment)fragmentPair.getLeftFragment()).getDiskIds();
+    }
     for (int i = 0; i < hosts.length; i++) {
+      int diskId = diskIds == null ? -1 : diskIds[i];
       String normalizedHost = NetUtils.normalizeHost(hosts[i]);
       Map<Integer, FragmentsPerDisk> diskFragmentMap = fragmentHostMapping.get(normalizedHost);
 
       if (diskFragmentMap != null) {
-        FragmentsPerDisk fragmentsPerDisk = diskFragmentMap.get(diskIds[i]);
+        FragmentsPerDisk fragmentsPerDisk = diskFragmentMap.get(diskId);
         if (fragmentsPerDisk != null) {
           boolean isRemoved = fragmentsPerDisk.removeFragmentPair(fragmentPair);
           if (isRemoved) {
             if (fragmentsPerDisk.size() == 0) {
-              diskFragmentMap.remove(diskIds[i]);
+              diskFragmentMap.remove(diskId);
               if (diskFragmentMap.size() == 0) {
                 fragmentHostMapping.remove(normalizedHost);
               }
             }
-            HostAndDisk hostAndDisk = new HostAndDisk(normalizedHost, diskIds[i]);
+            HostAndDisk hostAndDisk = new HostAndDisk(normalizedHost, diskId);
             if (totalHostPriority.containsKey(hostAndDisk)) {
               PrioritizedHost prioritizedHost = totalHostPriority.get(hostAndDisk);
               updateHostPriority(prioritizedHost.hostAndDisk, prioritizedHost.priority-1);
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
index b2883cc..c691a98 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
@@ -25,7 +25,9 @@
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
 import org.apache.tajo.engine.planner.global.ExecutionBlock;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.query.QueryUnitRequest;
@@ -38,7 +40,9 @@
 import org.apache.tajo.master.querymaster.QueryUnitAttempt;
 import org.apache.tajo.master.querymaster.SubQuery;
 import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.worker.FetchImpl;
 
@@ -197,15 +201,17 @@
     if (event.getType() == EventType.T_SCHEDULE) {
       if (event instanceof FragmentScheduleEvent) {
         FragmentScheduleEvent castEvent = (FragmentScheduleEvent) event;
-        Collection<FileFragment> rightFragments = castEvent.getRightFragments();
+        Collection<Fragment> rightFragments = castEvent.getRightFragments();
         if (rightFragments == null || rightFragments.isEmpty()) {
           scheduledFragments.addFragment(new FragmentPair(castEvent.getLeftFragment(), null));
         } else {
-          for (FileFragment eachFragment: rightFragments) {
+          for (Fragment eachFragment: rightFragments) {
             scheduledFragments.addFragment(new FragmentPair(castEvent.getLeftFragment(), eachFragment));
           }
         }
-        initDiskBalancer(castEvent.getLeftFragment().getHosts(), castEvent.getLeftFragment().getDiskIds());
+        if (castEvent.getLeftFragment() instanceof FileFragment) {
+          initDiskBalancer(castEvent.getLeftFragment().getHosts(), ((FileFragment)castEvent.getLeftFragment()).getDiskIds());
+        }
       } else if (event instanceof FetchScheduleEvent) {
         FetchScheduleEvent castEvent = (FetchScheduleEvent) event;
         scheduledFetches.addFetch(castEvent.getFetches());
@@ -366,6 +372,7 @@
       long taskSize = adjustTaskSize();
       LOG.info("Adjusted task size: " + taskSize);
 
+      TajoConf conf = subQuery.getContext().getConf();
       // host local, disk local
       String normalized = NetUtils.normalizeHost(host);
       Integer diskId = hostDiskBalancerMap.get(normalized).getDiskId(container.containerID);
@@ -376,13 +383,14 @@
             break;
           }
 
-          if (assignedFragmentSize + fragmentPair.getLeftFragment().getEndKey() > taskSize) {
+          if (assignedFragmentSize +
+              StorageManager.getFragmentLength(conf, fragmentPair.getLeftFragment()) > taskSize) {
             break;
           } else {
             fragmentPairs.add(fragmentPair);
-            assignedFragmentSize += fragmentPair.getLeftFragment().getEndKey();
+            assignedFragmentSize += StorageManager.getFragmentLength(conf, fragmentPair.getLeftFragment());
             if (fragmentPair.getRightFragment() != null) {
-              assignedFragmentSize += fragmentPair.getRightFragment().getEndKey();
+              assignedFragmentSize += StorageManager.getFragmentLength(conf, fragmentPair.getRightFragment());
             }
           }
           scheduledFragments.removeFragment(fragmentPair);
@@ -398,13 +406,14 @@
             break;
           }
 
-          if (assignedFragmentSize + fragmentPair.getLeftFragment().getEndKey() > taskSize) {
+          if (assignedFragmentSize +
+              StorageManager.getFragmentLength(conf, fragmentPair.getLeftFragment()) > taskSize) {
             break;
           } else {
             fragmentPairs.add(fragmentPair);
-            assignedFragmentSize += fragmentPair.getLeftFragment().getEndKey();
+            assignedFragmentSize += StorageManager.getFragmentLength(conf, fragmentPair.getLeftFragment());
             if (fragmentPair.getRightFragment() != null) {
-              assignedFragmentSize += fragmentPair.getRightFragment().getEndKey();
+              assignedFragmentSize += StorageManager.getFragmentLength(conf, fragmentPair.getRightFragment());
             }
           }
           scheduledFragments.removeFragment(fragmentPair);
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java
index 768528d..37fa4fe 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java
@@ -33,7 +33,11 @@
 import org.apache.tajo.storage.RowStoreUtil;
 import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
 import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
@@ -41,7 +45,7 @@
 import java.util.List;
 
 public class NonForwardQueryResultScanner {
-  private static final int MAX_FILE_NUM_PER_SCAN = 100;
+  private static final int MAX_FRAGMENT_NUM_PER_SCAN = 100;
 
   private QueryId queryId;
   private String sessionId;
@@ -54,7 +58,7 @@
   private TajoConf tajoConf;
   private ScanNode scanNode;
 
-  private int currentFileIndex = 0;
+  private int currentFragmentIndex = 0;
 
   public NonForwardQueryResultScanner(TajoConf tajoConf, String sessionId,
                                       QueryId queryId,
@@ -76,23 +80,24 @@
   }
 
   private void initSeqScanExec() throws IOException {
-    FragmentProto[] fragments = PhysicalPlanUtil.getNonZeroLengthDataFiles(tajoConf, tableDesc,
-        currentFileIndex, MAX_FILE_NUM_PER_SCAN);
-    if (fragments != null && fragments.length > 0) {
+    List<Fragment> fragments = StorageManager.getStorageManager(tajoConf, tableDesc.getMeta().getStoreType())
+        .getNonForwardSplit(tableDesc, currentFragmentIndex, MAX_FRAGMENT_NUM_PER_SCAN);
+
+    if (fragments != null && !fragments.isEmpty()) {
+      FragmentProto[] fragmentProtos = FragmentConvertor.toFragmentProtoArray(fragments.toArray(new Fragment[]{}));
       this.taskContext = new TaskAttemptContext(
           new QueryContext(tajoConf), null,
           new QueryUnitAttemptId(new QueryUnitId(new ExecutionBlockId(queryId, 1), 0), 0),
-          fragments, null);
+          fragmentProtos, null);
 
       try {
         // scanNode must be clone cause SeqScanExec change target in the case of a partitioned table.
-        scanExec = new SeqScanExec(taskContext,
-            StorageManager.getStorageManager(tajoConf), (ScanNode)scanNode.clone(), fragments);
+        scanExec = new SeqScanExec(taskContext, (ScanNode)scanNode.clone(), fragmentProtos);
       } catch (CloneNotSupportedException e) {
         throw new IOException(e.getMessage(), e);
       }
       scanExec.init();
-      currentFileIndex += fragments.length;
+      currentFragmentIndex += fragments.size();
     }
   }
 
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
index 795983d..56fbb22 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -49,6 +49,7 @@
 import org.apache.tajo.master.rm.WorkerResourceManager;
 import org.apache.tajo.master.session.SessionManager;
 import org.apache.tajo.rpc.RpcChannelFactory;
+import org.apache.tajo.storage.FileStorageManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.util.*;
 import org.apache.tajo.util.history.HistoryReader;
@@ -106,7 +107,7 @@
 
   private CatalogServer catalogServer;
   private CatalogService catalog;
-  private StorageManager storeManager;
+  private FileStorageManager storeManager;
   private GlobalEngine globalEngine;
   private AsyncDispatcher dispatcher;
   private TajoMasterClientService tajoMasterClientService;
@@ -166,7 +167,7 @@
 
       // check the system directory and create if they are not created.
       checkAndInitializeSystemDirectories();
-      this.storeManager = StorageManager.getStorageManager(systemConf);
+      this.storeManager = (FileStorageManager)StorageManager.getFileStorageManager(systemConf, null);
 
       catalogServer = new CatalogServer(FunctionLoader.load());
       addIfService(catalogServer);
@@ -407,7 +408,7 @@
     return this.catalogServer;
   }
 
-  public StorageManager getStorageManager() {
+  public FileStorageManager getStorageManager() {
     return this.storeManager;
   }
 
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index 540bd71..77075c4 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -797,7 +797,8 @@
 
         TableDesc desc;
         try {
-          desc = context.getGlobalEngine().createTableOnPath(queryContext, request.getName(), schema,
+          desc = context.getGlobalEngine().createTable(queryContext, request.getName(),
+              meta.getStoreType(), schema,
               meta, path, true, partitionDesc, false);
         } catch (Exception e) {
           return TableResponse.newBuilder()
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java
index 8cc17cb..c38f99f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/FragmentScheduleEvent.java
@@ -20,22 +20,23 @@
 
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 import java.util.Collection;
 
 public class FragmentScheduleEvent extends TaskSchedulerEvent {
-  private final FileFragment leftFragment;
-  private final Collection<FileFragment> rightFragments;
+  private final Fragment leftFragment;
+  private final Collection<Fragment> rightFragments;
 
   public FragmentScheduleEvent(final EventType eventType, final ExecutionBlockId blockId,
-                               final FileFragment fragment) {
+                               final Fragment fragment) {
     this(eventType, blockId, fragment, null);
   }
 
   public FragmentScheduleEvent(final EventType eventType,
                                final ExecutionBlockId blockId,
-                               final FileFragment leftFragment,
-                               final Collection<FileFragment> rightFragments) {
+                               final Fragment leftFragment,
+                               final Collection<Fragment> rightFragments) {
     super(eventType, blockId);
     this.leftFragment = leftFragment;
     this.rightFragments = rightFragments;
@@ -45,11 +46,11 @@
     return this.rightFragments != null && !this.rightFragments.isEmpty();
   }
 
-  public FileFragment getLeftFragment() {
+  public Fragment getLeftFragment() {
     return leftFragment;
   }
 
-  public Collection<FileFragment> getRightFragments() { return rightFragments; }
+  public Collection<Fragment> getRightFragments() { return rightFragments; }
 
   @Override
   public String toString() {
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 6f80171..978480e 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -23,7 +23,6 @@
 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.hadoop.yarn.event.EventHandler;
@@ -32,31 +31,27 @@
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.SessionVars;
-import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.global.ExecutionBlock;
 import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
 import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.plan.logical.CreateTableNode;
-import org.apache.tajo.plan.logical.InsertNode;
-import org.apache.tajo.plan.logical.NodeType;
+import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.master.event.*;
+import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.util.history.QueryHistory;
 import org.apache.tajo.util.history.SubQueryHistory;
 
 import java.io.IOException;
-import java.text.NumberFormat;
 import java.util.*;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -72,7 +67,6 @@
   private Map<ExecutionBlockId, SubQuery> subqueries;
   private final EventHandler eventHandler;
   private final MasterPlan plan;
-  private final StorageManager sm;
   QueryMasterTask.QueryMasterTaskContext context;
   private ExecutionBlockCursor cursor;
 
@@ -214,7 +208,6 @@
     subqueries = Maps.newHashMap();
     this.eventHandler = eventHandler;
     this.plan = plan;
-    this.sm = context.getStorageManager();
     this.cursor = new ExecutionBlockCursor(plan, true);
 
     StringBuilder sb = new StringBuilder("\n=======================================================");
@@ -396,7 +389,7 @@
 
       query.setStartTime();
       SubQuery subQuery = new SubQuery(query.context, query.getPlan(),
-          query.getExecutionBlockCursor().nextBlock(), query.sm);
+          query.getExecutionBlockCursor().nextBlock());
       subQuery.setPriority(query.priority--);
       query.addSubQuery(subQuery);
 
@@ -421,6 +414,20 @@
       } else {
         finalState = QueryState.QUERY_ERROR;
       }
+      if (finalState != QueryState.QUERY_SUCCEEDED) {
+        SubQuery lastStage = query.getSubQuery(subQueryEvent.getExecutionBlockId());
+        if (lastStage != null && lastStage.getTableMeta() != null) {
+          StoreType storeType = lastStage.getTableMeta().getStoreType();
+          if (storeType != null) {
+            LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
+            try {
+              StorageManager.getStorageManager(query.systemConf, storeType).rollbackOutputCommit(rootNode.getChild());
+            } catch (IOException e) {
+              LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
+            }
+          }
+        }
+      }
       query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
       query.setFinishTime();
 
@@ -428,356 +435,27 @@
     }
 
     private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
-      MasterPlan masterPlan = query.getPlan();
-
-      ExecutionBlock terminal = query.getPlan().getTerminalBlock();
-      DataChannel finalChannel = masterPlan.getChannel(event.getExecutionBlockId(), terminal.getId());
-
-      QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
+      SubQuery lastStage = query.getSubQuery(event.getExecutionBlockId());
+      StoreType storeType = lastStage.getTableMeta().getStoreType();
       try {
-        Path finalOutputDir = commitOutputData(query);
+        LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
+        CatalogService catalog = lastStage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
+        TableDesc tableDesc =  PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+
+        Path finalOutputDir = StorageManager.getStorageManager(query.systemConf, storeType)
+            .commitOutputData(query.context.getQueryContext(),
+                lastStage.getId(), lastStage.getMasterPlan().getLogicalPlan(), lastStage.getSchema(), tableDesc);
+
+        QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
         hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
-      } catch (Throwable t) {
-        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(t)));
+      } catch (Exception e) {
+        query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
         return QueryState.QUERY_ERROR;
       }
 
       return QueryState.QUERY_SUCCEEDED;
     }
 
-    /**
-     * It moves a result data stored in a staging output dir into a final output dir.
-     */
-    public Path commitOutputData(Query query) throws IOException {
-      QueryContext queryContext = query.context.getQueryContext();
-      Path stagingResultDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);
-      Path finalOutputDir;
-      if (queryContext.hasOutputPath()) {
-        finalOutputDir = queryContext.getOutputPath();
-        try {
-          FileSystem fs = stagingResultDir.getFileSystem(query.systemConf);
-
-          if (queryContext.isOutputOverwrite()) { // INSERT OVERWRITE INTO
-
-            // It moves the original table into the temporary location.
-            // Then it moves the new result table into the original table location.
-            // Upon failed, it recovers the original table if possible.
-            boolean movedToOldTable = false;
-            boolean committed = false;
-            Path oldTableDir = new Path(queryContext.getStagingDir(), TajoConstants.INSERT_OVERWIRTE_OLD_TABLE_NAME);
-            ContentSummary summary = fs.getContentSummary(stagingResultDir);
-
-            if (queryContext.hasPartition() && summary.getFileCount() > 0L) {
-              // This is a map for existing non-leaf directory to rename. A key is current directory and a value is
-              // renaming directory.
-              Map<Path, Path> renameDirs = TUtil.newHashMap();
-              // This is a map for recovering existing partition directory. A key is current directory and a value is
-              // temporary directory to back up.
-              Map<Path, Path> recoveryDirs = TUtil.newHashMap();
-
-              try {
-                if (!fs.exists(finalOutputDir)) {
-                  fs.mkdirs(finalOutputDir);
-                }
-
-                visitPartitionedDirectory(fs, stagingResultDir, finalOutputDir, stagingResultDir.toString(),
-                    renameDirs, oldTableDir);
-
-                // Rename target partition directories
-                for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
-                  // Backup existing data files for recovering
-                  if (fs.exists(entry.getValue())) {
-                    String recoveryPathString = entry.getValue().toString().replaceAll(finalOutputDir.toString(),
-                        oldTableDir.toString());
-                    Path recoveryPath = new Path(recoveryPathString);
-                    fs.rename(entry.getValue(), recoveryPath);
-                    fs.exists(recoveryPath);
-                    recoveryDirs.put(entry.getValue(), recoveryPath);
-                  }
-                  // Delete existing directory
-                  fs.delete(entry.getValue(), true);
-                  // Rename staging directory to final output directory
-                  fs.rename(entry.getKey(), entry.getValue());
-                }
-
-              } catch (IOException ioe) {
-                // Remove created dirs
-                for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
-                  fs.delete(entry.getValue(), true);
-                }
-
-                // Recovery renamed dirs
-                for(Map.Entry<Path, Path> entry : recoveryDirs.entrySet()) {
-                  fs.delete(entry.getValue(), true);
-                  fs.rename(entry.getValue(), entry.getKey());
-                }
-
-                throw new IOException(ioe.getMessage());
-              }
-            } else { // no partition
-              try {
-
-                // if the final output dir exists, move all contents to the temporary table dir.
-                // Otherwise, just make the final output dir. As a result, the final output dir will be empty.
-                if (fs.exists(finalOutputDir)) {
-                  fs.mkdirs(oldTableDir);
-
-                  for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
-                    fs.rename(status.getPath(), oldTableDir);
-                  }
-
-                  movedToOldTable = fs.exists(oldTableDir);
-                } else { // if the parent does not exist, make its parent directory.
-                  fs.mkdirs(finalOutputDir);
-                }
-
-                // Move the results to the final output dir.
-                for (FileStatus status : fs.listStatus(stagingResultDir)) {
-                  fs.rename(status.getPath(), finalOutputDir);
-                }
-
-                // Check the final output dir
-                committed = fs.exists(finalOutputDir);
-
-              } catch (IOException ioe) {
-                // recover the old table
-                if (movedToOldTable && !committed) {
-
-                  // if commit is failed, recover the old data
-                  for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
-                    fs.delete(status.getPath(), true);
-                  }
-
-                  for (FileStatus status : fs.listStatus(oldTableDir)) {
-                    fs.rename(status.getPath(), finalOutputDir);
-                  }
-                }
-
-                throw new IOException(ioe.getMessage());
-              }
-            }
-          } else {
-            NodeType queryType = queryContext.getCommandType();
-
-            if (queryType == NodeType.INSERT) { // INSERT INTO an existing table
-
-              NumberFormat fmt = NumberFormat.getInstance();
-              fmt.setGroupingUsed(false);
-              fmt.setMinimumIntegerDigits(3);
-
-              if (queryContext.hasPartition()) {
-                for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
-                  if (eachFile.isFile()) {
-                    LOG.warn("Partition table can't have file in a staging dir: " + eachFile.getPath());
-                    continue;
-                  }
-                  moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, -1);
-                }
-              } else {
-                int maxSeq = StorageUtil.getMaxFileSequence(fs, finalOutputDir, false) + 1;
-                for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
-                  moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, maxSeq++);
-                }
-              }
-              // checking all file moved and remove empty dir
-              verifyAllFileMoved(fs, stagingResultDir);
-              FileStatus[] files = fs.listStatus(stagingResultDir);
-              if (files != null && files.length != 0) {
-                for (FileStatus eachFile: files) {
-                  LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
-                }
-              }
-            } else { // CREATE TABLE AS SELECT (CTAS)
-              if (fs.exists(finalOutputDir)) {
-                for (FileStatus status : fs.listStatus(stagingResultDir)) {
-                  fs.rename(status.getPath(), finalOutputDir);
-                }
-              } else {
-                fs.rename(stagingResultDir, finalOutputDir);
-              }
-              LOG.info("Moved from the staging dir to the output directory '" + finalOutputDir);
-            }
-          }
-
-          // remove the staging directory if the final output dir is given.
-          Path stagingDirRoot = queryContext.getStagingDir().getParent();
-          fs.delete(stagingDirRoot, true);
-
-        } catch (Throwable t) {
-          LOG.error(t);
-          throw new IOException(t);
-        }
-      } else {
-        finalOutputDir = new Path(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME);
-      }
-
-      return finalOutputDir;
-    }
-
-    /**
-     * This method sets a rename map which includes renamed staging directory to final output directory recursively.
-     * If there exists some data files, this delete it for duplicate data.
-     *
-     *
-     * @param fs
-     * @param stagingPath
-     * @param outputPath
-     * @param stagingParentPathString
-     * @throws IOException
-     */
-    private void visitPartitionedDirectory(FileSystem fs, Path stagingPath, Path outputPath,
-                                        String stagingParentPathString,
-                                        Map<Path, Path> renameDirs, Path oldTableDir) throws IOException {
-      FileStatus[] files = fs.listStatus(stagingPath);
-
-      for(FileStatus eachFile : files) {
-        if (eachFile.isDirectory()) {
-          Path oldPath = eachFile.getPath();
-
-          // Make recover directory.
-          String recoverPathString = oldPath.toString().replaceAll(stagingParentPathString,
-          oldTableDir.toString());
-          Path recoveryPath = new Path(recoverPathString);
-          if (!fs.exists(recoveryPath)) {
-            fs.mkdirs(recoveryPath);
-          }
-
-          visitPartitionedDirectory(fs, eachFile.getPath(), outputPath, stagingParentPathString,
-          renameDirs, oldTableDir);
-          // Find last order partition for renaming
-          String newPathString = oldPath.toString().replaceAll(stagingParentPathString,
-          outputPath.toString());
-          Path newPath = new Path(newPathString);
-          if (!isLeafDirectory(fs, eachFile.getPath())) {
-           renameDirs.put(eachFile.getPath(), newPath);
-          } else {
-            if (!fs.exists(newPath)) {
-             fs.mkdirs(newPath);
-            }
-          }
-        }
-      }
-    }
-
-    private boolean isLeafDirectory(FileSystem fs, Path path) throws IOException {
-      boolean retValue = false;
-
-      FileStatus[] files = fs.listStatus(path);
-      for (FileStatus file : files) {
-        if (fs.isDirectory(file.getPath())) {
-          retValue = true;
-          break;
-        }
-      }
-
-      return retValue;
-    }
-
-    private boolean verifyAllFileMoved(FileSystem fs, Path stagingPath) throws IOException {
-      FileStatus[] files = fs.listStatus(stagingPath);
-      if (files != null && files.length != 0) {
-        for (FileStatus eachFile: files) {
-          if (eachFile.isFile()) {
-            LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
-            return false;
-          } else {
-            if (verifyAllFileMoved(fs, eachFile.getPath())) {
-              fs.delete(eachFile.getPath(), false);
-            } else {
-              return false;
-            }
-          }
-        }
-      }
-
-      return true;
-    }
-
-    /**
-     * Attach the sequence number to a path.
-     *
-     * @param path Path
-     * @param seq sequence number
-     * @param nf Number format
-     * @return New path attached with sequence number
-     * @throws IOException
-     */
-    private String replaceFileNameSeq(Path path, int seq, NumberFormat nf) throws IOException {
-      String[] tokens = path.getName().split("-");
-      if (tokens.length != 4) {
-        throw new IOException("Wrong result file name:" + path);
-      }
-      return tokens[0] + "-" + tokens[1] + "-" + tokens[2] + "-" + nf.format(seq);
-    }
-
-    /**
-     * Attach the sequence number to the output file name and than move the file into the final result path.
-     *
-     * @param fs FileSystem
-     * @param stagingResultDir The staging result dir
-     * @param fileStatus The file status
-     * @param finalOutputPath Final output path
-     * @param nf Number format
-     * @param fileSeq The sequence number
-     * @throws IOException
-     */
-    private void moveResultFromStageToFinal(FileSystem fs, Path stagingResultDir,
-                                            FileStatus fileStatus, Path finalOutputPath,
-                                            NumberFormat nf,
-                                            int fileSeq) throws IOException {
-      if (fileStatus.isDirectory()) {
-        String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
-        if (subPath != null) {
-          Path finalSubPath = new Path(finalOutputPath, subPath);
-          if (!fs.exists(finalSubPath)) {
-            fs.mkdirs(finalSubPath);
-          }
-          int maxSeq = StorageUtil.getMaxFileSequence(fs, finalSubPath, false);
-          for (FileStatus eachFile : fs.listStatus(fileStatus.getPath())) {
-            moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputPath, nf, ++maxSeq);
-          }
-        } else {
-          throw new IOException("Wrong staging dir:" + stagingResultDir + "," + fileStatus.getPath());
-        }
-      } else {
-        String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
-        if (subPath != null) {
-          Path finalSubPath = new Path(finalOutputPath, subPath);
-          finalSubPath = new Path(finalSubPath.getParent(), replaceFileNameSeq(finalSubPath, fileSeq, nf));
-          if (!fs.exists(finalSubPath.getParent())) {
-            fs.mkdirs(finalSubPath.getParent());
-          }
-          if (fs.exists(finalSubPath)) {
-            throw new IOException("Already exists data file:" + finalSubPath);
-          }
-          boolean success = fs.rename(fileStatus.getPath(), finalSubPath);
-          if (success) {
-            LOG.info("Moving staging file[" + fileStatus.getPath() + "] + " +
-                "to final output[" + finalSubPath + "]");
-          } else {
-            LOG.error("Can't move staging file[" + fileStatus.getPath() + "] + " +
-                "to final output[" + finalSubPath + "]");
-          }
-        }
-      }
-    }
-
-    private String extractSubPath(Path parentPath, Path childPath) {
-      String parentPathStr = parentPath.toUri().getPath();
-      String childPathStr = childPath.toUri().getPath();
-
-      if (parentPathStr.length() > childPathStr.length()) {
-        return null;
-      }
-
-      int index = childPathStr.indexOf(parentPathStr);
-      if (index != 0) {
-        return null;
-      }
-
-      return childPathStr.substring(parentPathStr.length() + 1);
-    }
-
     private static interface QueryHook {
       boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir);
       void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext, Query query,
@@ -942,7 +620,7 @@
     private void executeNextBlock(Query query) {
       ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
       ExecutionBlock nextBlock = cursor.nextBlock();
-      SubQuery nextSubQuery = new SubQuery(query.context, query.getPlan(), nextBlock, query.sm);
+      SubQuery nextSubQuery = new SubQuery(query.context, query.getPlan(), nextBlock);
       nextSubQuery.setPriority(query.priority--);
       query.addSubQuery(nextSubQuery);
       nextSubQuery.handle(new SubQueryEvent(nextSubQuery.getId(), SubQueryEventType.SQ_INIT));
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
index 00b95ac..d4e0752 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
@@ -24,7 +24,7 @@
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.ipc.ClientProtos.QueryInfoProto;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.util.TajoIdUtils;
 import org.apache.tajo.util.history.History;
 
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
index b3b4dbb..e30b4f9 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -41,7 +41,6 @@
 import org.apache.tajo.rpc.NullCallback;
 import org.apache.tajo.rpc.RpcConnectionPool;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.util.HAServiceUtil;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.util.history.QueryHistory;
@@ -71,8 +70,6 @@
 
   private GlobalPlanner globalPlanner;
 
-  private StorageManager storageManager;
-
   private TajoConf systemConf;
 
   private Map<QueryId, QueryMasterTask> queryMasterTasks = Maps.newConcurrentMap();
@@ -116,8 +113,6 @@
       this.dispatcher = new TajoAsyncDispatcher("querymaster_" + System.currentTimeMillis());
       addIfService(dispatcher);
 
-      this.storageManager = StorageManager.getStorageManager(systemConf);
-
       globalPlanner = new GlobalPlanner(systemConf, workerContext);
 
       dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler());
@@ -373,10 +368,6 @@
       return clock;
     }
 
-    public StorageManager getStorageManager() {
-      return storageManager;
-    }
-
     public QueryMaster getQueryMaster() {
       return QueryMaster.this;
     }
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index 75d8ab6..1eaef0f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -35,10 +35,13 @@
 import org.apache.tajo.algebra.JsonHelper;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.plan.LogicalOptimizer;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.plan.rewrite.RewriteRule;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.plan.logical.LogicalNode;
@@ -54,10 +57,12 @@
 import org.apache.tajo.master.event.*;
 import org.apache.tajo.master.rm.TajoWorkerResourceManager;
 import org.apache.tajo.master.session.Session;
+import org.apache.tajo.plan.verifier.VerifyException;
 import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.rpc.NettyClientBase;
 import org.apache.tajo.rpc.RpcConnectionPool;
 import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageProperty;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.HAServiceUtil;
 import org.apache.tajo.util.metrics.TajoMetrics;
@@ -348,6 +353,8 @@
   }
 
   public synchronized void startQuery() {
+    StorageManager sm = null;
+    LogicalPlan plan = null;
     try {
       if (query != null) {
         LOG.warn("Query already started");
@@ -358,7 +365,29 @@
       LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
       Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
       jsonExpr = null; // remove the possible OOM
-      LogicalPlan plan = planner.createPlan(queryContext, expr);
+      plan = planner.createPlan(queryContext, expr);
+
+      StoreType storeType = PlannerUtil.getStoreType(plan);
+      if (storeType != null) {
+        sm = StorageManager.getStorageManager(systemConf, storeType);
+        StorageProperty storageProperty = sm.getStorageProperty();
+        if (storageProperty.isSortedInsert()) {
+          String tableName = PlannerUtil.getStoreTableName(plan);
+          LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+          TableDesc tableDesc =  PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+          if (tableDesc == null) {
+            throw new VerifyException("Can't get table meta data from catalog: " + tableName);
+          }
+          List<RewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
+              getQueryTaskContext().getQueryContext(), tableDesc);
+          if (storageSpecifiedRewriteRules != null) {
+            for (RewriteRule eachRule: storageSpecifiedRewriteRules) {
+              optimizer.addRuleAfterToJoinOpt(eachRule);
+            }
+          }
+        }
+      }
+
       optimizer.optimize(queryContext, plan);
 
       GlobalEngine.DistributedQueryHookManager hookManager = new GlobalEngine.DistributedQueryHookManager();
@@ -393,6 +422,15 @@
     } catch (Throwable t) {
       LOG.error(t.getMessage(), t);
       initError = t;
+
+      if (plan != null && sm != null) {
+        LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+        try {
+          sm.rollbackOutputCommit(rootNode.getChild());
+        } catch (IOException e) {
+          LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
+        }
+      }
     }
   }
 
@@ -441,8 +479,14 @@
     // Create Output Directory
     ////////////////////////////////////////////
 
+    String outputPath = context.get(QueryVars.OUTPUT_TABLE_PATH, "");
     if (context.isCreateTable() || context.isInsert()) {
-      stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
+      if (outputPath == null || outputPath.isEmpty()) {
+        // hbase
+        stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+      } else {
+        stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
+      }
     } else {
       stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
     }
@@ -570,10 +614,6 @@
       return queryId;
     }
 
-    public StorageManager getStorageManager() {
-      return queryMasterContext.getStorageManager();
-    }
-
     public Path getStagingDir() {
       return queryContext.getStagingDir();
     }
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
index 0f275e9..75402c2 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
@@ -43,6 +43,7 @@
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.storage.DataLocation;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TajoIdUtils;
@@ -265,8 +266,13 @@
 
     List<String> fragmentList = new ArrayList<String>();
     for (FragmentProto eachFragment : getAllFragments()) {
-      FileFragment fileFragment = FragmentConvertor.convert(FileFragment.class, eachFragment);
-      fragmentList.add(fileFragment.toString());
+      try {
+        Fragment fragment = FragmentConvertor.convert(systemConf, eachFragment);
+        fragmentList.add(fragment.toString());
+      } catch (Exception e) {
+        LOG.error(e.getMessage());
+        fragmentList.add("ERROR: " + eachFragment.getStoreType() + "," + eachFragment.getId() + ": " + e.getMessage());
+      }
     }
     queryUnitHistory.setFragments(fragmentList.toArray(new String[]{}));
 
@@ -313,15 +319,18 @@
 	  }
 	}
 
-  private void addDataLocation(FileFragment fragment) {
+  private void addDataLocation(Fragment fragment) {
     String[] hosts = fragment.getHosts();
-    int[] diskIds = fragment.getDiskIds();
+    int[] diskIds = null;
+    if (fragment instanceof FileFragment) {
+      diskIds = ((FileFragment)fragment).getDiskIds();
+    }
     for (int i = 0; i < hosts.length; i++) {
-      dataLocations.add(new DataLocation(hosts[i], diskIds[i]));
+      dataLocations.add(new DataLocation(hosts[i], diskIds == null ? -1 : diskIds[i]));
     }
   }
 
-  public void addFragment(FileFragment fragment, boolean useDataLocation) {
+  public void addFragment(Fragment fragment, boolean useDataLocation) {
     Set<FragmentProto> fragmentProtos;
     if (fragMap.containsKey(fragment.getTableName())) {
       fragmentProtos = fragMap.get(fragment.getTableName());
@@ -336,8 +345,8 @@
     totalFragmentNum++;
   }
 
-  public void addFragments(Collection<FileFragment> fragments) {
-    for (FileFragment eachFragment: fragments) {
+  public void addFragments(Collection<Fragment> fragments) {
+    for (Fragment eachFragment: fragments) {
       addFragment(eachFragment, false);
     }
   }
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
index 55b1895..a240ace 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -45,15 +45,18 @@
 import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
 import org.apache.tajo.master.TaskSchedulerContext;
 import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.plan.logical.SortNode.SortPurpose;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.FileStorageManager;
 import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.RowStoreUtil;
 import org.apache.tajo.storage.TupleRange;
 import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.unit.StorageUnit;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.Pair;
+import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.util.TajoIdUtils;
 import org.apache.tajo.worker.FetchImpl;
@@ -83,18 +86,20 @@
     MasterPlan masterPlan = subQuery.getMasterPlan();
     ExecutionBlock execBlock = subQuery.getBlock();
     QueryMasterTask.QueryMasterTaskContext masterContext = subQuery.getContext();
-    StorageManager storageManager = subQuery.getStorageManager();
 
     ScanNode[] scans = execBlock.getScanNodes();
 
     Path tablePath;
-    FileFragment[] fragments = new FileFragment[scans.length];
+    Fragment[] fragments = new Fragment[scans.length];
     long[] stats = new long[scans.length];
 
     // initialize variables from the child operators
     for (int i = 0; i < scans.length; i++) {
       TableDesc tableDesc = masterContext.getTableDescMap().get(scans[i].getCanonicalName());
       if (tableDesc == null) { // if it is a real table stored on storage
+        FileStorageManager storageManager =
+            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+
         tablePath = storageManager.getTablePath(scans[i].getTableName());
         if (execBlock.getUnionScanMap() != null && !execBlock.getUnionScanMap().isEmpty()) {
           for (Map.Entry<ExecutionBlockId, ExecutionBlockId> unionScanEntry: execBlock.getUnionScanMap().entrySet()) {
@@ -107,21 +112,23 @@
         }
         fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
       } else {
-        tablePath = new Path(tableDesc.getPath());
         try {
           stats[i] = GlobalPlanner.computeDescendentVolume(scans[i]);
         } catch (PlanningException e) {
           throw new IOException(e);
         }
 
+        StorageManager storageManager =
+            StorageManager.getStorageManager(subQuery.getContext().getConf(), tableDesc.getMeta().getStoreType());
+
         // if table has no data, storageManager will return empty FileFragment.
         // So, we need to handle FileFragment by its size.
         // If we don't check its size, it can cause IndexOutOfBoundsException.
-        List<FileFragment> fileFragments = storageManager.getSplits(scans[i].getCanonicalName(), tableDesc.getMeta(), tableDesc.getSchema(), tablePath);
+        List<Fragment> fileFragments = storageManager.getSplits(scans[i].getCanonicalName(), tableDesc);
         if (fileFragments.size() > 0) {
           fragments[i] = fileFragments.get(0);
         } else {
-          fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+          fragments[i] = new FileFragment(scans[i].getCanonicalName(), new Path(tableDesc.getPath()), 0, 0, new String[]{UNKNOWN_HOST});
         }
       }
     }
@@ -268,14 +275,14 @@
         //select intermediate scan and stats
         ScanNode[] intermediateScans = new ScanNode[largeScanIndexList.size()];
         long[] intermediateScanStats = new long[largeScanIndexList.size()];
-        FileFragment[] intermediateFragments = new FileFragment[largeScanIndexList.size()];
+        Fragment[] intermediateFragments = new Fragment[largeScanIndexList.size()];
         int index = 0;
         for (Integer eachIdx : largeScanIndexList) {
           intermediateScans[index] = scans[eachIdx];
           intermediateScanStats[index] = stats[eachIdx];
           intermediateFragments[index++] = fragments[eachIdx];
         }
-        FileFragment[] broadcastFragments = new FileFragment[broadcastIndexList.size()];
+        Fragment[] broadcastFragments = new Fragment[broadcastIndexList.size()];
         ScanNode[] broadcastScans = new ScanNode[broadcastIndexList.size()];
         index = 0;
         for (Integer eachIdx : broadcastIndexList) {
@@ -309,9 +316,9 @@
                                                        SubQuery subQuery,
                                                        ScanNode[] scans,
                                                        long[] stats,
-                                                       FileFragment[] fragments,
+                                                       Fragment[] fragments,
                                                        ScanNode[] broadcastScans,
-                                                       FileFragment[] broadcastFragments) throws IOException {
+                                                       Fragment[] broadcastFragments) throws IOException {
     MasterPlan masterPlan = subQuery.getMasterPlan();
     ExecutionBlock execBlock = subQuery.getBlock();
     // The hash map is modeling as follows:
@@ -394,7 +401,7 @@
     int joinTaskNum = Math.min(maxTaskNum, hashEntries.size());
     LOG.info("The determined number of join tasks is " + joinTaskNum);
 
-    List<FileFragment> rightFragments = new ArrayList<FileFragment>();
+    List<Fragment> rightFragments = new ArrayList<Fragment>();
     rightFragments.add(fragments[1]);
 
     if (broadcastFragments != null) {
@@ -404,14 +411,19 @@
         Path[] partitionScanPaths = null;
         TableDesc tableDesc = masterContext.getTableDescMap().get(eachScan.getCanonicalName());
         if (eachScan.getType() == NodeType.PARTITIONS_SCAN) {
+          FileStorageManager storageManager =
+              (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+
           PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)eachScan;
           partitionScanPaths = partitionScan.getInputPaths();
           // set null to inputPaths in getFragmentsFromPartitionedTable()
-          getFragmentsFromPartitionedTable(subQuery.getStorageManager(), eachScan, tableDesc);
+          getFragmentsFromPartitionedTable(storageManager, eachScan, tableDesc);
           partitionScan.setInputPaths(partitionScanPaths);
         } else {
-          Collection<FileFragment> scanFragments = subQuery.getStorageManager().getSplits(eachScan.getCanonicalName(),
-              tableDesc.getMeta(), tableDesc.getSchema(), new Path(tableDesc.getPath()));
+          StorageManager storageManager = StorageManager.getStorageManager(subQuery.getContext().getConf(),
+              tableDesc.getMeta().getStoreType());
+          Collection<Fragment> scanFragments = storageManager.getSplits(eachScan.getCanonicalName(),
+              tableDesc, eachScan);
           if (scanFragments != null) {
             rightFragments.addAll(scanFragments);
           }
@@ -480,10 +492,10 @@
   /**
    * It creates a number of fragments for all partitions.
    */
-  public static List<FileFragment> getFragmentsFromPartitionedTable(StorageManager sm,
+  public static List<Fragment> getFragmentsFromPartitionedTable(FileStorageManager sm,
                                                                           ScanNode scan,
                                                                           TableDesc table) throws IOException {
-    List<FileFragment> fragments = Lists.newArrayList();
+    List<Fragment> fragments = Lists.newArrayList();
     PartitionedTableScanNode partitionsScan = (PartitionedTableScanNode) scan;
     fragments.addAll(sm.getSplits(
         scan.getCanonicalName(), table.getMeta(), table.getSchema(), partitionsScan.getInputPaths()));
@@ -492,7 +504,7 @@
   }
 
   private static void scheduleLeafTasksWithBroadcastTable(TaskSchedulerContext schedulerContext, SubQuery subQuery,
-                                                          int baseScanId, FileFragment[] fragments) throws IOException {
+                                                          int baseScanId, Fragment[] fragments) throws IOException {
     ExecutionBlock execBlock = subQuery.getBlock();
     ScanNode[] scans = execBlock.getScanNodes();
 
@@ -511,23 +523,27 @@
     //     . add all partition paths to node's inputPaths variable
     //  -> SCAN
     //     . add all fragments to broadcastFragments
-    Collection<FileFragment> baseFragments = null;
-    List<FileFragment> broadcastFragments = new ArrayList<FileFragment>();
+    Collection<Fragment> baseFragments = null;
+    List<Fragment> broadcastFragments = new ArrayList<Fragment>();
     for (int i = 0; i < scans.length; i++) {
       ScanNode scan = scans[i];
       TableDesc desc = subQuery.getContext().getTableDescMap().get(scan.getCanonicalName());
       TableMeta meta = desc.getMeta();
 
-      Collection<FileFragment> scanFragments;
+      Collection<Fragment> scanFragments;
       Path[] partitionScanPaths = null;
       if (scan.getType() == NodeType.PARTITIONS_SCAN) {
         PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)scan;
         partitionScanPaths = partitionScan.getInputPaths();
         // set null to inputPaths in getFragmentsFromPartitionedTable()
-        scanFragments = getFragmentsFromPartitionedTable(subQuery.getStorageManager(), scan, desc);
+        FileStorageManager storageManager =
+            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+        scanFragments = getFragmentsFromPartitionedTable(storageManager, scan, desc);
       } else {
-        scanFragments = subQuery.getStorageManager().getSplits(scan.getCanonicalName(), meta, desc.getSchema(),
-            new Path(desc.getPath()));
+        StorageManager storageManager =
+            StorageManager.getStorageManager(subQuery.getContext().getConf(), desc.getMeta().getStoreType());
+
+        scanFragments = storageManager.getSplits(scan.getCanonicalName(), desc, scan);
       }
 
       if (scanFragments != null) {
@@ -630,46 +646,66 @@
     ExecutionBlock execBlock = subQuery.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
     Path tablePath;
-    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableName());
+    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf()))
+        .getTablePath(scan.getTableName());
 
     ExecutionBlock sampleChildBlock = masterPlan.getChild(subQuery.getId(), 0);
     SortNode sortNode = PlannerUtil.findTopNode(sampleChildBlock.getPlan(), NodeType.SORT);
     SortSpec [] sortSpecs = sortNode.getSortKeys();
     Schema sortSchema = new Schema(channel.getShuffleKeys());
 
+    TupleRange[] ranges;
+    int determinedTaskNum;
+
     // calculate the number of maximum query ranges
     TableStats totalStat = computeChildBlocksStats(subQuery.getContext(), masterPlan, subQuery.getId());
 
     // If there is an empty table in inner join, it should return zero rows.
-    if (totalStat.getNumBytes() == 0 && totalStat.getColumnStats().size() == 0 ) {
+    if (totalStat.getNumBytes() == 0 && totalStat.getColumnStats().size() == 0) {
       return;
     }
     TupleRange mergedRange = TupleUtil.columnStatToRange(sortSpecs, sortSchema, totalStat.getColumnStats(), false);
-    RangePartitionAlgorithm partitioner = new UniformRangePartition(mergedRange, sortSpecs);
-    BigInteger card = partitioner.getTotalCardinality();
 
-    // if the number of the range cardinality is less than the desired number of tasks,
-    // we set the the number of tasks to the number of range cardinality.
-    int determinedTaskNum;
-    if (card.compareTo(BigInteger.valueOf(maxNum)) < 0) {
-      LOG.info(subQuery.getId() + ", The range cardinality (" + card
-          + ") is less then the desired number of tasks (" + maxNum + ")");
-      determinedTaskNum = card.intValue();
+    if (sortNode.getSortPurpose() == SortPurpose.STORAGE_SPECIFIED) {
+      StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
+      CatalogService catalog = subQuery.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
+      LogicalRootNode rootNode = masterPlan.getLogicalPlan().getRootBlock().getRoot();
+      TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+      if (tableDesc == null) {
+        throw new IOException("Can't get table meta data from catalog: " +
+            PlannerUtil.getStoreTableName(masterPlan.getLogicalPlan()));
+      }
+      ranges = StorageManager.getStorageManager(subQuery.getContext().getConf(), storeType)
+          .getInsertSortRanges(subQuery.getContext().getQueryContext(), tableDesc,
+              sortNode.getInSchema(), sortSpecs,
+              mergedRange);
+      determinedTaskNum = ranges.length;
     } else {
-      determinedTaskNum = maxNum;
-    }
+      RangePartitionAlgorithm partitioner = new UniformRangePartition(mergedRange, sortSpecs);
+      BigInteger card = partitioner.getTotalCardinality();
 
-    LOG.info(subQuery.getId() + ", Try to divide " + mergedRange + " into " + determinedTaskNum +
-        " sub ranges (total units: " + determinedTaskNum + ")");
-    TupleRange [] ranges = partitioner.partition(determinedTaskNum);
-    if (ranges == null || ranges.length == 0) {
-      LOG.warn(subQuery.getId() + " no range infos.");
-    }
-    TupleUtil.setMaxRangeIfNull(sortSpecs, sortSchema, totalStat.getColumnStats(), ranges);
-    if (LOG.isDebugEnabled()) {
-      if (ranges != null) {
-        for (TupleRange eachRange : ranges) {
-          LOG.debug(subQuery.getId() + " range: " + eachRange.getStart() + " ~ " + eachRange.getEnd());
+      // if the number of the range cardinality is less than the desired number of tasks,
+      // we set the the number of tasks to the number of range cardinality.
+      if (card.compareTo(BigInteger.valueOf(maxNum)) < 0) {
+        LOG.info(subQuery.getId() + ", The range cardinality (" + card
+            + ") is less then the desired number of tasks (" + maxNum + ")");
+        determinedTaskNum = card.intValue();
+      } else {
+        determinedTaskNum = maxNum;
+      }
+
+      LOG.info(subQuery.getId() + ", Try to divide " + mergedRange + " into " + determinedTaskNum +
+          " sub ranges (total units: " + determinedTaskNum + ")");
+      ranges = partitioner.partition(determinedTaskNum);
+      if (ranges == null || ranges.length == 0) {
+        LOG.warn(subQuery.getId() + " no range infos.");
+      }
+      TupleUtil.setMaxRangeIfNull(sortSpecs, sortSchema, totalStat.getColumnStats(), ranges);
+      if (LOG.isDebugEnabled()) {
+        if (ranges != null) {
+          for (TupleRange eachRange : ranges) {
+            LOG.debug(subQuery.getId() + " range: " + eachRange.getStart() + " ~ " + eachRange.getEnd());
+          }
         }
       }
     }
@@ -772,14 +808,15 @@
 
   public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
                                                  SubQuery subQuery, DataChannel channel,
-                                                 int maxNum) {
+                                                 int maxNum) throws IOException {
     ExecutionBlock execBlock = subQuery.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
     Path tablePath;
-    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableName());
+    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf()))
+        .getTablePath(scan.getTableName());
 
-    FileFragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
-    List<FileFragment> fragments = new ArrayList<FileFragment>();
+    Fragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+    List<Fragment> fragments = new ArrayList<Fragment>();
     fragments.add(frag);
     SubQuery.scheduleFragments(subQuery, fragments);
 
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 39bb7ed..7f05fa4 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -36,7 +36,7 @@
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.ColumnStats;
 import org.apache.tajo.catalog.statistics.StatisticsUtil;
 import org.apache.tajo.catalog.statistics.TableStats;
@@ -59,10 +59,11 @@
 import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
 import org.apache.tajo.master.container.TajoContainer;
 import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.storage.FileStorageManager;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.history.QueryUnitHistory;
@@ -96,7 +97,6 @@
   private TableStats resultStatistics;
   private TableStats inputStatistics;
   private EventHandler<Event> eventHandler;
-  private final StorageManager sm;
   private AbstractTaskScheduler taskScheduler;
   private QueryMasterTask.QueryMasterTaskContext context;
   private final List<String> diagnostics = new ArrayList<String>();
@@ -286,12 +286,10 @@
   private AtomicInteger completeReportReceived = new AtomicInteger(0);
   private SubQueryHistory finalSubQueryHistory;
 
-  public SubQuery(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan,
-                  ExecutionBlock block, StorageManager sm) {
+  public SubQuery(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block) {
     this.context = context;
     this.masterPlan = masterPlan;
     this.block = block;
-    this.sm = sm;
     this.eventHandler = context.getEventHandler();
 
     ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
@@ -509,10 +507,6 @@
     return this.priority;
   }
 
-  public StorageManager getStorageManager() {
-    return sm;
-  }
-  
   public ExecutionBlockId getId() {
     return block.getId();
   }
@@ -677,14 +671,14 @@
     }
 
     DataChannel channel = masterPlan.getOutgoingChannels(getId()).get(0);
-    // get default or store type
-    CatalogProtos.StoreType storeType = CatalogProtos.StoreType.CSV; // default setting
 
     // if store plan (i.e., CREATE or INSERT OVERWRITE)
-    StoreTableNode storeTableNode = PlannerUtil.findTopNode(getBlock().getPlan(), NodeType.STORE);
-    if (storeTableNode != null) {
-      storeType = storeTableNode.getStorageType();
+    StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
+    if (storeType == null) {
+      // get default or store type
+      storeType = StoreType.CSV;
     }
+
     schema = channel.getSchema();
     meta = CatalogUtil.newTableMeta(storeType, new KeyValueSet());
     inputStatistics = statsArray[0];
@@ -1043,7 +1037,7 @@
       ScanNode scan = scans[0];
       TableDesc table = subQuery.context.getTableDescMap().get(scan.getCanonicalName());
 
-      Collection<FileFragment> fragments;
+      Collection<Fragment> fragments;
       TableMeta meta = table.getMeta();
 
       // Depending on scanner node's type, it creates fragments. If scan is for
@@ -1052,10 +1046,13 @@
       // span a number of blocks or possibly consists of a number of files.
       if (scan.getType() == NodeType.PARTITIONS_SCAN) {
         // After calling this method, partition paths are removed from the physical plan.
-        fragments = Repartitioner.getFragmentsFromPartitionedTable(subQuery.getStorageManager(), scan, table);
+        FileStorageManager storageManager =
+            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+        fragments = Repartitioner.getFragmentsFromPartitionedTable(storageManager, scan, table);
       } else {
-        Path inputPath = new Path(table.getPath());
-        fragments = subQuery.getStorageManager().getSplits(scan.getCanonicalName(), meta, table.getSchema(), inputPath);
+        StorageManager storageManager =
+            StorageManager.getStorageManager(subQuery.getContext().getConf(), meta.getStoreType());
+        fragments = storageManager.getSplits(scan.getCanonicalName(), table, scan);
       }
 
       SubQuery.scheduleFragments(subQuery, fragments);
@@ -1073,27 +1070,27 @@
     }
   }
 
-  public static void scheduleFragment(SubQuery subQuery, FileFragment fragment) {
+  public static void scheduleFragment(SubQuery subQuery, Fragment fragment) {
     subQuery.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
         subQuery.getId(), fragment));
   }
 
 
-  public static void scheduleFragments(SubQuery subQuery, Collection<FileFragment> fragments) {
-    for (FileFragment eachFragment : fragments) {
+  public static void scheduleFragments(SubQuery subQuery, Collection<Fragment> fragments) {
+    for (Fragment eachFragment : fragments) {
       scheduleFragment(subQuery, eachFragment);
     }
   }
 
-  public static void scheduleFragments(SubQuery subQuery, Collection<FileFragment> leftFragments,
-                                       Collection<FileFragment> broadcastFragments) {
-    for (FileFragment eachLeafFragment : leftFragments) {
+  public static void scheduleFragments(SubQuery subQuery, Collection<Fragment> leftFragments,
+                                       Collection<Fragment> broadcastFragments) {
+    for (Fragment eachLeafFragment : leftFragments) {
       scheduleFragment(subQuery, eachLeafFragment, broadcastFragments);
     }
   }
 
   public static void scheduleFragment(SubQuery subQuery,
-                                      FileFragment leftFragment, Collection<FileFragment> rightFragments) {
+                                      Fragment leftFragment, Collection<Fragment> rightFragments) {
     subQuery.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
         subQuery.getId(), leftFragment, rightFragments));
   }
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
index 0cc87fc..f1a9224 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -39,6 +39,7 @@
 import org.apache.tajo.master.querymaster.QueryInProgress;
 import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.util.ApplicationIdUtils;
+import org.apache.tajo.util.StringUtils;
 
 import java.io.IOException;
 import java.util.*;
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java
index 0de1b2b..3147bb6 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/IndexUtil.java
@@ -38,7 +38,7 @@
   public static String getIndexNameOfFrag(FileFragment fragment, SortSpec[] keys) {
     StringBuilder builder = new StringBuilder(); 
     builder.append(fragment.getPath().getName() + "_");
-    builder.append(fragment.getStartKey() + "_" + fragment.getEndKey() + "_");
+    builder.append(fragment.getStartKey() + "_" + fragment.getLength() + "_");
     for(int i = 0 ; i < keys.length ; i ++) {
       builder.append(keys[i].getSortKey().getSimpleName()+"_");
     }
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
index 7a81b4b..aaf5754 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryHistory.java
@@ -22,7 +22,7 @@
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.ipc.ClientProtos.QueryHistoryProto;
 import org.apache.tajo.ipc.ClientProtos.SubQueryHistoryProto;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto;
 
 import java.util.ArrayList;
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java
index 556a971..126e3fe 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/QueryUnitHistory.java
@@ -20,7 +20,7 @@
 
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 
 public class QueryUnitHistory implements GsonObject {
   @Expose private String id;
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java b/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
index b3ac4d2..17b9ec7 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/SubQueryHistory.java
@@ -22,7 +22,7 @@
 import com.google.gson.reflect.TypeToken;
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.ipc.ClientProtos.SubQueryHistoryProto;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 
 import java.util.ArrayList;
 import java.util.List;
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
index c11db6f..925c047 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoQueryEngine.java
@@ -30,12 +30,10 @@
 
 public class TajoQueryEngine {
 
-  private final StorageManager storageManager;
   private final PhysicalPlanner phyPlanner;
 
   public TajoQueryEngine(TajoConf conf) throws IOException {
-    this.storageManager = StorageManager.getStorageManager(conf);
-    this.phyPlanner = new PhysicalPlannerImpl(conf, storageManager);
+    this.phyPlanner = new PhysicalPlannerImpl(conf);
   }
   
   public PhysicalExec createPlan(TaskAttemptContext ctx, LogicalNode plan)
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
index cb038df..00eabcc 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
@@ -31,7 +31,6 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.TajoProtos.TaskAttemptState;
 import org.apache.tajo.catalog.Schema;
@@ -53,10 +52,7 @@
 import org.apache.tajo.pullserver.TajoPullServerService;
 import org.apache.tajo.pullserver.retriever.FileChunk;
 import org.apache.tajo.rpc.NullCallback;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.HashShuffleAppenderManager;
-import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.NetUtils;
 import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
@@ -108,39 +104,6 @@
   private Schema finalSchema = null;
   private TupleComparator sortComp = null;
 
-  static final String OUTPUT_FILE_PREFIX="part-";
-  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SUBQUERY =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(2);
-          return fmt;
-        }
-      };
-  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_TASK =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(6);
-          return fmt;
-        }
-      };
-
-  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SEQ =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(3);
-          return fmt;
-        }
-      };
-
   public Task(String taskRunnerId,
               Path baseDir,
               QueryUnitAttemptId taskId,
@@ -190,13 +153,8 @@
         this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys());
       }
     } else {
-      // The final result of a task will be written in a file named part-ss-nnnnnnn,
-      // where ss is the subquery id associated with this task, and nnnnnn is the task id.
-      Path outFilePath = StorageUtil.concatPath(queryContext.getStagingDir(), TajoConstants.RESULT_DIR_NAME,
-          OUTPUT_FILE_PREFIX +
-          OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskId.getQueryUnitId().getExecutionBlockId().getId()) + "-" +
-          OUTPUT_FILE_FORMAT_TASK.get().format(taskId.getQueryUnitId().getId()) + "-" +
-          OUTPUT_FILE_FORMAT_SEQ.get().format(0));
+      Path outFilePath = ((FileStorageManager)StorageManager.getFileStorageManager(systemConf))
+          .getAppenderFilePath(taskId, queryContext.getStagingDir());
       LOG.info("Output File Path: " + outFilePath);
       context.setOutputPath(outFilePath);
     }
diff --git a/tajo-core/src/main/resources/webapps/worker/queryunit.jsp b/tajo-core/src/main/resources/webapps/worker/queryunit.jsp
index 18a67d8..49635d1 100644
--- a/tajo-core/src/main/resources/webapps/worker/queryunit.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/queryunit.jsp
@@ -41,6 +41,7 @@
 <%@ page import="java.text.SimpleDateFormat" %>
 <%@ page import="java.util.Map" %>
 <%@ page import="java.util.Set" %>
+<%@ page import="org.apache.tajo.storage.fragment.Fragment" %>
 
 <%
     String paramQueryId = request.getParameter("queryId");
@@ -102,8 +103,8 @@
     String fragmentInfo = "";
     String delim = "";
     for (CatalogProtos.FragmentProto eachFragment : queryUnit.getAllFragments()) {
-        FileFragment fileFragment = FragmentConvertor.convert(FileFragment.class, eachFragment);
-        fragmentInfo += delim + fileFragment.toString();
+        Fragment fragment = FragmentConvertor.convert(tajoWorker.getConfig(), eachFragment);
+        fragmentInfo += delim + fragment.toString();
         delim = "<br/>";
     }
 
diff --git a/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java b/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
index 45d3c51..fb98be2 100644
--- a/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
@@ -47,7 +47,7 @@
 
   public static void writeTmpTable(TajoConf conf, Path tablePath)
       throws IOException {
-    StorageManager sm = StorageManager.getStorageManager(conf, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, tablePath);
     FileSystem fs = sm.getFileSystem();
 
     Appender appender;
diff --git a/tajo-core/src/test/java/org/apache/tajo/HBaseTestClusterUtil.java b/tajo-core/src/test/java/org/apache/tajo/HBaseTestClusterUtil.java
new file mode 100644
index 0000000..a8e4a5c
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/HBaseTestClusterUtil.java
@@ -0,0 +1,182 @@
+/**
+ * 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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.tajo.util.Bytes;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
+
+public class HBaseTestClusterUtil {
+  private static final Log LOG = LogFactory.getLog(HBaseTestClusterUtil.class);
+  private Configuration conf;
+  private MiniHBaseCluster hbaseCluster;
+  private MiniZooKeeperCluster zkCluster;
+  private File testBaseDir;
+  public HBaseTestClusterUtil(Configuration conf, File testBaseDir) {
+    this.conf = conf;
+    this.testBaseDir = testBaseDir;
+  }
+  /**
+   * Returns the path to the default root dir the minicluster uses.
+   * Note: this does not cause the root dir to be created.
+   * @return Fully qualified path for the default hbase root dir
+   * @throws java.io.IOException
+   */
+  public Path getDefaultRootDirPath() throws IOException {
+    FileSystem fs = FileSystem.get(this.conf);
+    return new Path(fs.makeQualified(fs.getHomeDirectory()),"hbase");
+  }
+
+  /**
+   * Creates an hbase rootdir in user home directory.  Also creates hbase
+   * version file.  Normally you won't make use of this method.  Root hbasedir
+   * is created for you as part of mini cluster startup.  You'd only use this
+   * method if you were doing manual operation.
+   * @return Fully qualified path to hbase root dir
+   * @throws java.io.IOException
+   */
+  public Path createRootDir() throws IOException {
+    FileSystem fs = FileSystem.get(this.conf);
+    Path hbaseRootdir = getDefaultRootDirPath();
+    FSUtils.setRootDir(this.conf, hbaseRootdir);
+    fs.mkdirs(hbaseRootdir);
+    FSUtils.setVersion(fs, hbaseRootdir);
+    return hbaseRootdir;
+  }
+
+  public void stopHBaseCluster() throws IOException {
+    if (hbaseCluster != null) {
+      LOG.info("MiniHBaseCluster stopped");
+      hbaseCluster.shutdown();
+      hbaseCluster.waitUntilShutDown();
+      hbaseCluster = null;
+    }
+  }
+
+  public void startHBaseCluster() throws Exception {
+    if (zkCluster == null) {
+      startMiniZKCluster();
+    }
+    if (hbaseCluster != null) {
+      return;
+    }
+
+    System.setProperty("HBASE_ZNODE_FILE", testBaseDir + "/hbase_znode_file");
+    if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1) == -1) {
+      conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
+    }
+    if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1) == -1) {
+      conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 1);
+    }
+    conf.setBoolean(REPLICATION_ENABLE_KEY, false);
+    createRootDir();
+
+    Configuration c = HBaseConfiguration.create(this.conf);
+
+    hbaseCluster = new MiniHBaseCluster(c, 1);
+
+    // Don't leave here till we've done a successful scan of the hbase:meta
+    HTable t = new HTable(c, TableName.META_TABLE_NAME);
+    ResultScanner s = t.getScanner(new Scan());
+    while (s.next() != null) {
+      continue;
+    }
+    s.close();
+    t.close();
+    LOG.info("MiniHBaseCluster started");
+
+  }
+
+  /**
+   * Start a mini ZK cluster. If the property "test.hbase.zookeeper.property.clientPort" is set
+   *  the port mentionned is used as the default port for ZooKeeper.
+   */
+  public MiniZooKeeperCluster startMiniZKCluster()
+      throws Exception {
+    File zkDataPath = new File(testBaseDir, "zk");
+    if (this.zkCluster != null) {
+      throw new IOException("Cluster already running at " + zkDataPath);
+    }
+    this.zkCluster = new MiniZooKeeperCluster(conf);
+    final int defPort = this.conf.getInt("test.hbase.zookeeper.property.clientPort", 0);
+    if (defPort > 0){
+      // If there is a port in the config file, we use it.
+      this.zkCluster.setDefaultClientPort(defPort);
+    }
+    int clientPort =  this.zkCluster.startup(zkDataPath, 1);
+    this.conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.toString(clientPort));
+    LOG.info("MiniZooKeeperCluster started");
+
+    return this.zkCluster;
+  }
+
+  public void stopZooKeeperCluster() throws IOException {
+    if (zkCluster != null) {
+      LOG.info("MiniZooKeeperCluster stopped");
+      zkCluster.shutdown();
+      zkCluster = null;
+    }
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public MiniZooKeeperCluster getMiniZooKeeperCluster() {
+    return zkCluster;
+  }
+
+  public MiniHBaseCluster getMiniHBaseCluster() {
+    return hbaseCluster;
+  }
+
+  public HTableDescriptor getTableDescriptor(String tableName) throws IOException {
+    HBaseAdmin admin = new HBaseAdmin(conf);
+    try {
+      return admin.getTableDescriptor(Bytes.toBytes(tableName));
+    } finally {
+      admin.close();
+    }
+  }
+
+  public void createTable(HTableDescriptor hTableDesc) throws IOException {
+    HBaseAdmin admin = new HBaseAdmin(conf);
+    try {
+      admin.createTable(hTableDesc);
+    } finally {
+      admin.close();
+    }
+  }
+}
diff --git a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
index 4e4964e..c13960e 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -169,9 +169,9 @@
   private static String currentDatabase;
   private static Set<String> createdTableGlobalSet = new HashSet<String>();
   // queries and results directory corresponding to subclass class.
-  private Path currentQueryPath;
-  private Path currentResultPath;
-  private Path currentDatasetPath;
+  protected Path currentQueryPath;
+  protected Path currentResultPath;
+  protected Path currentDatasetPath;
 
   // for getting a method name
   @Rule public TestName name = new TestName();
@@ -303,7 +303,7 @@
     return executeFile(getMethodName() + ".sql");
   }
 
-  private String getMethodName() {
+  protected String getMethodName() {
     String methodName = name.getMethodName();
     // In the case of parameter execution name's pattern is methodName[0]
     if (methodName.endsWith("]")) {
diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
index 603da0c..804efd8 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -69,6 +69,7 @@
   private FileSystem defaultFS;
   private MiniDFSCluster dfsCluster;
 	private MiniCatalogServer catalogServer;
+  private HBaseTestClusterUtil hbaseUtil;
 
   private TajoMaster tajoMaster;
   private List<TajoWorker> tajoWorkers = new ArrayList<TajoWorker>();
@@ -279,6 +280,10 @@
     return this.defaultFS;
   }
 
+  public HBaseTestClusterUtil getHBaseUtil() {
+    return hbaseUtil;
+  }
+
   ////////////////////////////////////////////////////////
   // Catalog Section
   ////////////////////////////////////////////////////////
@@ -502,6 +507,8 @@
     startMiniDFSCluster(numDataNodes, this.clusterTestBuildDir, dataNodeHosts);
     this.dfsCluster.waitClusterUp();
 
+    hbaseUtil = new HBaseTestClusterUtil(conf, clusterTestBuildDir);
+
     if(!standbyWorkerMode) {
       startMiniYarnCluster();
     }
@@ -589,7 +596,6 @@
     }
 
     if(this.dfsCluster != null) {
-
       try {
         FileSystem fs = this.dfsCluster.getFileSystem();
         if (fs != null) fs.close();
@@ -608,6 +614,10 @@
       }
       this.clusterTestBuildDir = null;
     }
+
+    hbaseUtil.stopZooKeeperCluster();
+    hbaseUtil.stopHBaseCluster();
+
     LOG.info("Minicluster is down");
   }
 
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
index 7f402a1..e331599 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestStringOperatorsAndFunctions.java
@@ -25,6 +25,7 @@
 import org.junit.Test;
 
 import java.io.IOException;
+import java.text.DecimalFormat;
 
 import static org.apache.tajo.common.TajoDataTypes.Type.*;
 
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
index 3437e3a..9ce7b5b 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
@@ -40,10 +40,10 @@
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleComparator;
 import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
@@ -380,7 +380,7 @@
     int index = 0;
 
     for (int i = startIndex; i < startIndex + expectedSize; i++, index++) {
-      FileFragment fragment = FragmentConvertor.convert(util.getConfiguration(), StoreType.CSV, fragments[index]);
+      FileFragment fragment = FragmentConvertor.convert(util.getConfiguration(), fragments[index]);
       assertEquals(expectedFiles.get(i), fragment.getPath());
     }
   }
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
index 0e7f3e6..3803c7a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
@@ -42,10 +42,7 @@
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.Appender;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -140,8 +137,8 @@
         contentsData += j;
       }
     }
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(tableMeta, schema,
-        dataPath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(tableMeta, schema, dataPath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     int writtenSize = 0;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
index aef8064..6a6aafb 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
@@ -79,7 +79,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -89,7 +89,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < OUTER_TUPLE_NUM; i++) {
@@ -110,7 +111,8 @@
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < INNER_TUPLE_NUM; i += 2) {
@@ -150,10 +152,10 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN);
 
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(),
         new Path(employee.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(),
         new Path(people.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -162,7 +164,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -183,9 +185,9 @@
     LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf),
         context).getRootBlock().getRoot();
 
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(), 
         new Path(employee.getPath()), Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(), 
         new Path(people.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
@@ -201,7 +203,7 @@
     ctx.setEnforcer(enforcer);
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
index 60ae849..dc3c28d 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
@@ -66,7 +66,7 @@
   private SQLAnalyzer analyzer;
   private LogicalPlanner planner;
   private LogicalOptimizer optimizer;
-  private StorageManager sm;
+  private FileStorageManager sm;
   private Schema idxSchema;
   private BaseTupleComparator comp;
   private BSTIndex.BSTIndexWriter writer;
@@ -91,7 +91,7 @@
     Path workDir = CommonTestingUtil.getTestDir();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, workDir.toUri().toString());
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
-    sm = StorageManager.getStorageManager(conf, workDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, workDir);
 
     idxPath = new Path(workDir, "test.idx");
 
@@ -117,8 +117,7 @@
     fs = tablePath.getFileSystem(conf);
     fs.mkdirs(tablePath.getParent());
 
-    FileAppender appender = (FileAppender)StorageManager.getStorageManager(conf).getAppender(meta, schema,
-        tablePath);
+    FileAppender appender = (FileAppender)sm.getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < 10000; i++) {
@@ -164,7 +163,7 @@
     this.rndKey = rnd.nextInt(250);
     final String QUERY = "select * from employee where managerId = " + rndKey;
     
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE);
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEqual");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir);
@@ -172,7 +171,7 @@
     LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    TmpPlanner phyPlanner = new TmpPlanner(conf, sm);
+    TmpPlanner phyPlanner = new TmpPlanner(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     int tupleCount = this.randomValues.get(rndKey);
@@ -186,8 +185,8 @@
   }
 
   private class TmpPlanner extends PhysicalPlannerImpl {
-    public TmpPlanner(TajoConf conf, StorageManager sm) {
-      super(conf, sm);
+    public TmpPlanner(TajoConf conf) {
+      super(conf);
     }
 
     @Override
@@ -196,12 +195,11 @@
       Preconditions.checkNotNull(ctx.getTable(scanNode.getTableName()),
           "Error: There is no table matched to %s", scanNode.getTableName());
 
-      List<FileFragment> fragments = FragmentConvertor.convert(ctx.getConf(), meta.getStoreType(),
-          ctx.getTables(scanNode.getTableName()));
+      List<FileFragment> fragments = FragmentConvertor.convert(ctx.getConf(), ctx.getTables(scanNode.getTableName()));
       
       Datum[] datum = new Datum[]{DatumFactory.createInt4(rndKey)};
 
-      return new BSTIndexScanExec(ctx, sm, scanNode, fragments.get(0), idxPath, idxSchema, comp , datum);
+      return new BSTIndexScanExec(ctx, scanNode, fragments.get(0), idxPath, idxSchema, comp , datum);
 
     }
   }
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
index bb40b28..c0bf6ce 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
@@ -59,7 +59,7 @@
   private CatalogService catalog;
   private SQLAnalyzer analyzer;
   private LogicalPlanner planner;
-  private StorageManager sm;
+  private FileStorageManager sm;
   private Path testDir;
 
   private final int numTuple = 100000;
@@ -76,7 +76,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf.setVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR, testDir.toString());
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, testDir);
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -85,7 +85,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
     Tuple tuple = new VTuple(schema.size());
@@ -121,7 +122,7 @@
 
   @Test
   public final void testNext() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
     Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -131,7 +132,7 @@
     LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     
     ProjectionExec proj = (ProjectionExec) exec;
@@ -141,8 +142,7 @@
       UnaryPhysicalExec sortExec = proj.getChild();
       SeqScanExec scan = sortExec.getChild();
 
-      ExternalSortExec extSort = new ExternalSortExec(ctx, sm,
-          ((MemSortExec)sortExec).getPlan(), scan);
+      ExternalSortExec extSort = new ExternalSortExec(ctx, ((MemSortExec)sortExec).getPlan(), scan);
       proj.setChild(extSort);
     }
 
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
index c4ce43b..ecd1c23 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
@@ -61,7 +61,7 @@
   private CatalogService catalog;
   private SQLAnalyzer analyzer;
   private LogicalPlanner planner;
-  private StorageManager sm;
+  private FileStorageManager sm;
   private Path testDir;
   private QueryContext defaultContext;
 
@@ -84,7 +84,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, testDir);
 
     //----------------- dep3 ------------------------------
     // dep_id | dep_name  | loc_id
@@ -107,7 +107,8 @@
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -136,7 +137,8 @@
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -175,7 +177,8 @@
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -227,8 +230,8 @@
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
     appender5.flush();
@@ -266,9 +269,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
+    FileFragment[] dep3Frags = FileStorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
 
@@ -277,7 +280,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -305,9 +308,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
@@ -316,7 +319,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -343,9 +346,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
 
@@ -354,7 +357,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -382,9 +385,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+    FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
@@ -394,7 +397,7 @@
         workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
index a82de92..a81979f 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
@@ -40,10 +40,7 @@
 import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.storage.Appender;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.TUtil;
@@ -92,7 +89,7 @@
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
 
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     //----------------- dep3 ------------------------------
     // dep_id | dep_name  | loc_id
@@ -115,7 +112,8 @@
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -153,7 +151,8 @@
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = StorageManager.getStorageManager(conf).getAppender(dep4Meta, dep4Schema, dep4Path);
+    Appender appender4 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     Tuple tuple4 = new VTuple(dep4Schema.size());
     for (int i = 0; i < 11; i++) {
@@ -184,7 +183,8 @@
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -223,7 +223,8 @@
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -275,8 +276,8 @@
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     appender5.flush();
     appender5.close();
@@ -318,9 +319,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] dep3Frags =
-        StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin0");
@@ -328,7 +329,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -355,9 +356,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin1");
@@ -365,7 +366,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -392,9 +393,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin2");
@@ -402,7 +403,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -430,9 +431,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] dep4Frags =
-        StorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testFullOuterMergeJoin3");
@@ -440,7 +441,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -470,9 +471,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags =
-        StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
@@ -481,7 +482,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -509,9 +510,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags =
-        StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags);
 
@@ -520,7 +521,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -537,9 +538,4 @@
     exec.close();
     assertEquals(7, count);
   }
-
-
-
-
-
 }
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
index 20d4651..4fe6ff2 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
@@ -76,7 +76,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
     employeeSchema.addColumn("managerid", Type.INT4);
@@ -86,8 +86,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
 
@@ -112,7 +112,8 @@
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 10; i += 2) {
@@ -150,9 +151,9 @@
 
   @Test
   public final void testHashAntiJoin() throws IOException, PlanningException {
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(),
         new Path(people.getPath()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -166,7 +167,7 @@
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     // replace an equal join with an hash anti join.
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
index d1fa28a..55e87d4 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
@@ -78,7 +78,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
     employeeSchema.addColumn("managerid", Type.INT4);
@@ -88,8 +88,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
     for (int i = 0; i < 10; i++) {
@@ -111,7 +111,8 @@
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 10; i += 2) {
@@ -152,9 +153,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(),
         new Path(people.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
@@ -163,7 +164,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -195,9 +196,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(),
         new Path(people.getPath()), Integer.MAX_VALUE);
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
@@ -207,7 +208,7 @@
     ctx.setEnforcer(enforcer);
 
     ctx.getQueryContext().setLong(SessionVars.HASH_JOIN_SIZE_LIMIT.keyname(), 100l);
-    PhysicalPlannerImpl phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlannerImpl phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
index 7a43a55..a2f1155 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
@@ -38,10 +38,7 @@
 import org.apache.tajo.plan.LogicalPlanner;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.logical.LogicalNode;
-import org.apache.tajo.storage.Appender;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.TUtil;
@@ -80,7 +77,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
     employeeSchema.addColumn("managerid", Type.INT4);
@@ -90,8 +87,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
 
@@ -116,7 +113,8 @@
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     // make 27 tuples
@@ -158,9 +156,9 @@
 
   @Test
   public final void testHashSemiJoin() throws IOException, PlanningException {
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(),
         new Path(people.getPath()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -174,7 +172,7 @@
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     // replace an equal join with an hash anti join.
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
index ec9daa7..0477771 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
@@ -85,7 +85,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     //----------------- dep3 ------------------------------
     // dep_id | dep_name  | loc_id
@@ -108,7 +108,8 @@
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -137,7 +138,8 @@
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -176,7 +178,8 @@
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -228,8 +231,8 @@
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     
     appender5.flush();
@@ -270,9 +273,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(),
+    FileFragment[] dep3Frags = FileStorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(),
         new Path(dep3.getPath()), Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
         new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
 
@@ -281,7 +284,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -300,9 +303,9 @@
 
   @Test
   public final void testLeftOuter_HashJoinExec1() throws IOException, PlanningException {
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(),
         new Path(job3.getPath()), Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
         new Path(emp3.getPath()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
@@ -314,7 +317,7 @@
     Expr expr = analyzer.parse(QUERIES[1]);
     LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -341,9 +344,9 @@
     @Test
   public final void testLeftOuter_HashJoinExec2() throws IOException, PlanningException {
     
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
         new Path(emp3.getPath()), Integer.MAX_VALUE);
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(),
         new Path(job3.getPath()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
@@ -355,7 +358,7 @@
     Expr expr = analyzer.parse(QUERIES[2]);
     LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -383,9 +386,9 @@
    @Test
   public final void testLeftOuter_HashJoinExec3() throws IOException, PlanningException {
     
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(),
         new Path(emp3.getPath()), Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(),
+    FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(),
         new Path(phone3.getPath()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
@@ -397,7 +400,7 @@
     Expr expr = analyzer.parse(QUERIES[3]);
     LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -425,9 +428,9 @@
    @Test
   public final void testLeftOuter_HashJoinExec4() throws IOException, PlanningException {
     
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, "default.emp3", emp3.getMeta(),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, "default.emp3", emp3.getMeta(),
         new Path(emp3.getPath()), Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = StorageManager.splitNG(conf, "default.phone3", phone3.getMeta(),
+    FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, "default.phone3", phone3.getMeta(),
         new Path(phone3.getPath()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags);
@@ -439,7 +442,7 @@
     Expr expr = analyzer.parse(QUERIES[4]);
     LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
index b3d1f33..36dd77e 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
@@ -81,7 +81,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     //----------------- dep3 ------------------------------
     // dep_id | dep_name  | loc_id
@@ -104,7 +104,8 @@
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -133,7 +134,8 @@
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -172,7 +174,8 @@
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -224,8 +227,8 @@
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     
     appender5.flush();
@@ -254,9 +257,9 @@
 
   @Test
   public final void testLeftOuterNLJoinExec0() throws IOException, PlanningException {
-    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
+    FileFragment[] dep3Frags = FileStorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
@@ -269,7 +272,7 @@
     LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
@@ -295,9 +298,9 @@
 
   @Test
   public final void testLeftOuterNLJoinExec1() throws IOException, PlanningException {
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
@@ -311,7 +314,7 @@
     LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     
     //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
@@ -340,9 +343,9 @@
 
   @Test
   public final void testLeftOuter_NLJoinExec2() throws IOException, PlanningException {
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
@@ -355,7 +358,7 @@
     LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     
     //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
@@ -385,9 +388,9 @@
 
   @Test
   public final void testLeftOuter_NLJoinExec3() throws IOException, PlanningException {
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+    FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
@@ -400,7 +403,7 @@
     LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     
     //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
@@ -429,9 +432,9 @@
 
     @Test
   public final void testLeftOuter_NLJoinExec4() throws IOException, PlanningException {
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+    FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags);
@@ -444,7 +447,7 @@
     LogicalNode plan = planner.createPlan(defaultContext, context).getRootBlock().getRoot();
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     
     //maybe plan results with hash join exec algorithm usage. Must convert from HashLeftOuterJoinExec into NLLeftOuterJoinExec
@@ -470,7 +473,4 @@
     exec.close();
     assertEquals(0, count);
   }
-
-
-
 }
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
index cae5de5..10d4d33 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
@@ -79,7 +79,7 @@
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
     FileSystem fs = testDir.getFileSystem(conf);
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     Schema employeeSchema = new Schema();
     employeeSchema.addColumn("managerid", Type.INT4);
@@ -89,8 +89,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
     for (int i = 0; i < 10; i++) {
@@ -118,7 +118,8 @@
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 10; i += 2) {
@@ -165,9 +166,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
-    FileFragment[] empFrags = sm.splitNG(conf, "default.e", employee.getMeta(), new Path(employee.getPath()),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(), new Path(employee.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = sm.splitNG(conf, "default.p", people.getMeta(), new Path(people.getPath()),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(), new Path(people.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
@@ -176,7 +177,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, root);
     ProjectionExec proj = (ProjectionExec) exec;
     assertTrue(proj.getChild() instanceof MergeJoinExec);
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
index a5d1dc5..e976456 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
@@ -76,7 +76,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -86,7 +86,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < 50; i++) {
@@ -109,7 +110,8 @@
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 50; i += 2) {
@@ -144,9 +146,9 @@
   
   @Test
   public final void testNLCrossJoin() throws IOException, PlanningException {
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(),
         new Path(people.getPath()), Integer.MAX_VALUE);
     
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -159,7 +161,7 @@
     LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf),
         expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     int i = 0;
@@ -173,9 +175,9 @@
 
   @Test
   public final void testNLInnerJoin() throws IOException, PlanningException {
-    FileFragment[] empFrags = StorageManager.splitNG(conf, "default.e", employee.getMeta(),
+    FileFragment[] empFrags = FileStorageManager.splitNG(conf, "default.e", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = StorageManager.splitNG(conf, "default.p", people.getMeta(),
+    FileFragment[] peopleFrags = FileStorageManager.splitNG(conf, "default.p", people.getMeta(),
         new Path(people.getPath()), Integer.MAX_VALUE);
     
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
@@ -188,7 +190,7 @@
     LogicalNode plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf),
         expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     
     Tuple tuple;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index d507b97..cce4ba7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -51,10 +51,12 @@
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.expr.AggregationFunctionCallEval;
 import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.plan.serder.PlanProto.ShuffleType;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.index.bst.BSTIndex;
 import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.CommonTestingUtil;
@@ -78,7 +80,6 @@
 import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
 import static org.apache.tajo.ipc.TajoWorkerProtocol.ColumnPartitionEnforcer.ColumnPartitionAlgorithm;
 import static org.apache.tajo.ipc.TajoWorkerProtocol.SortEnforce.SortAlgorithm;
-import static org.apache.tajo.plan.serder.PlanProto.ShuffleType;
 import static org.junit.Assert.*;
 
 public class TestPhysicalPlanner {
@@ -88,7 +89,7 @@
   private static SQLAnalyzer analyzer;
   private static LogicalPlanner planner;
   private static LogicalOptimizer optimizer;
-  private static StorageManager sm;
+  private static FileStorageManager sm;
   private static Path testDir;
   private static Session session = LocalTajoTestingUtility.createDummySession();
   private static QueryContext defaultContext;
@@ -106,7 +107,7 @@
     util.startCatalogCluster();
     conf = util.getConfiguration();
     testDir = CommonTestingUtil.getTestDir("target/test-data/TestPhysicalPlanner");
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, testDir);
     catalog = util.getMiniCatalogCluster().getCatalog();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
@@ -129,8 +130,7 @@
 
 
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = sm.getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
     for (int i = 0; i < 100; i++) {
@@ -148,7 +148,7 @@
 
     Path scorePath = new Path(testDir, "score");
     TableMeta scoreMeta = CatalogUtil.newTableMeta(StoreType.CSV, new KeyValueSet());
-    appender = StorageManager.getStorageManager(conf).getAppender(scoreMeta, scoreSchema, scorePath);
+    appender = sm.getAppender(scoreMeta, scoreSchema, scorePath);
     appender.init();
     score = new TableDesc(
         CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "score"), scoreSchema, scoreMeta,
@@ -189,8 +189,8 @@
 
     Schema scoreSchmea = score.getSchema();
     TableMeta scoreLargeMeta = CatalogUtil.newTableMeta(StoreType.RAW, new KeyValueSet());
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(scoreLargeMeta, scoreSchmea,
-        scoreLargePath);
+    Appender appender =  ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(scoreLargeMeta, scoreSchmea, scoreLargePath);
     appender.enableStats();
     appender.init();
     largeScore = new TableDesc(
@@ -246,7 +246,7 @@
 
   @Test
   public final void testCreateScanPlan() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -259,7 +259,7 @@
     optimizer.optimize(plan);
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     Tuple tuple;
@@ -277,7 +277,7 @@
 
   @Test
   public final void testCreateScanWithFilterPlan() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanWithFilterPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -290,7 +290,7 @@
     optimizer.optimize(plan);
 
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     Tuple tuple;
@@ -306,7 +306,7 @@
 
   @Test
   public final void testGroupByPlan() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -318,7 +318,7 @@
     optimizer.optimize(plan);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     int i = 0;
@@ -337,7 +337,7 @@
   @Test
   public final void testHashGroupByPlanWithALLField() throws IOException, PlanningException {
     // TODO - currently, this query does not use hash-based group operator.
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(
         "target/test-data/testHashGroupByPlanWithALLField");
@@ -349,7 +349,7 @@
     LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     int i = 0;
@@ -367,7 +367,7 @@
 
   @Test
   public final void testSortGroupByPlan() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortGroupByPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -378,7 +378,7 @@
     LogicalPlan plan = planner.createPlan(defaultContext, context);
     optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan.getRootBlock().getRoot());
 
     /*HashAggregateExec hashAgg = (HashAggregateExec) exec;
@@ -430,7 +430,7 @@
 
   @Test
   public final void testStorePlan() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -443,17 +443,16 @@
     LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-
     TableMeta outputMeta = CatalogUtil.newTableMeta(StoreType.CSV);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
     exec.close();
 
-    Scanner scanner = StorageManager.getStorageManager(conf).getFileScanner(outputMeta, rootNode.getOutSchema(),
-        ctx.getOutputPath());
+    Scanner scanner =  ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getFileScanner(outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;
     int i = 0;
@@ -477,7 +476,7 @@
     TableStats stats = largeScore.getStats();
     assertTrue("Checking meaningfulness of test", stats.getNumBytes() > StorageUnit.MB);
 
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score_large", largeScore.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score_large", largeScore.getMeta(),
         new Path(largeScore.getPath()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlanWithMaxOutputFileSize");
 
@@ -497,7 +496,7 @@
     LogicalNode rootNode = optimizer.optimize(plan);
 
     // executing StoreTableExec
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
@@ -512,7 +511,7 @@
     // checking the file contents
     long totalNum = 0;
     for (FileStatus status : fs.listStatus(ctx.getOutputPath().getParent())) {
-      Scanner scanner = StorageManager.getStorageManager(conf).getFileScanner(
+      Scanner scanner =  ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getFileScanner(
           CatalogUtil.newTableMeta(StoreType.CSV),
           rootNode.getOutSchema(),
           status.getPath());
@@ -528,7 +527,7 @@
 
   @Test
   public final void testStorePlanWithRCFile() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlanWithRCFile");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -543,14 +542,14 @@
 
     TableMeta outputMeta = CatalogUtil.newTableMeta(StoreType.RCFILE);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
     exec.close();
 
-    Scanner scanner = StorageManager.getStorageManager(conf).getFileScanner(outputMeta, rootNode.getOutSchema(),
-        ctx.getOutputPath());
+    Scanner scanner = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getFileScanner(
+        outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;
     int i = 0;
@@ -569,7 +568,7 @@
 
   @Test
   public final void testEnforceForDefaultColumnPartitionStorePlan() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -581,7 +580,7 @@
     Expr context = analyzer.parse(CreateTableAsStmts[2]);
     LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     assertTrue(exec instanceof SortBasedColPartitionStoreExec);
   }
@@ -596,7 +595,7 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.HASH_PARTITION);
 
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -605,7 +604,7 @@
     ctx.setEnforcer(enforcer);
     ctx.setOutputPath(new Path(workDir, "grouped4"));
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     assertTrue(exec instanceof HashBasedColPartitionStoreExec);
   }
@@ -620,7 +619,7 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.SORT_PARTITION);
 
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -629,14 +628,14 @@
     ctx.setEnforcer(enforcer);
     ctx.setOutputPath(new Path(workDir, "grouped5"));
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     assertTrue(exec instanceof SortBasedColPartitionStoreExec);
   }
 
   @Test
   public final void testPartitionedStorePlan() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), id, new FileFragment[] { frags[0] },
@@ -660,7 +659,7 @@
     QueryId queryId = id.getQueryUnitId().getExecutionBlockId().getQueryId();
     ExecutionBlockId ebId = id.getQueryUnitId().getExecutionBlockId();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
@@ -671,7 +670,7 @@
     Path queryLocalTmpDir = new Path(conf.getVar(ConfVars.WORKER_TEMPORAL_DIR) + "/" + executionBlockBaseDir);
     FileStatus [] list = fs.listStatus(queryLocalTmpDir);
 
-    List<FileFragment> fragments = new ArrayList<FileFragment>();
+    List<Fragment> fragments = new ArrayList<Fragment>();
     for (FileStatus status : list) {
       assertTrue(status.isDirectory());
       FileStatus [] files = fs.listStatus(status.getPath());
@@ -705,7 +704,7 @@
   public final void testPartitionedStorePlanWithMaxFileSize() throws IOException, PlanningException {
 
     // Preparing working dir and input fragments
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score_large", largeScore.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score_large", largeScore.getMeta(),
         new Path(largeScore.getPath()), Integer.MAX_VALUE);
     QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlanWithMaxFileSize");
@@ -724,7 +723,7 @@
     LogicalNode rootNode = optimizer.optimize(plan);
 
     // Executing CREATE TABLE PARTITION BY
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
@@ -735,7 +734,7 @@
     // checking the number of partitions
     assertEquals(2, list.length);
 
-    List<FileFragment> fragments = Lists.newArrayList();
+    List<Fragment> fragments = Lists.newArrayList();
     int i = 0;
     for (FileStatus status : list) {
       assertTrue(status.isDirectory());
@@ -769,7 +768,7 @@
   @Test
   public final void testPartitionedStorePlanWithEmptyGroupingSet()
       throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
 
@@ -794,7 +793,7 @@
     QueryId queryId = id.getQueryUnitId().getExecutionBlockId().getQueryId();
     ExecutionBlockId ebId = id.getQueryUnitId().getExecutionBlockId();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
@@ -805,7 +804,7 @@
     Path queryLocalTmpDir = new Path(conf.getVar(ConfVars.WORKER_TEMPORAL_DIR) + "/" + executionBlockBaseDir);
     FileStatus [] list = fs.listStatus(queryLocalTmpDir);
 
-    List<FileFragment> fragments = new ArrayList<FileFragment>();
+    List<Fragment> fragments = new ArrayList<Fragment>();
     for (FileStatus status : list) {
       assertTrue(status.isDirectory());
       FileStatus [] files = fs.listStatus(status.getPath());
@@ -836,7 +835,7 @@
 
   @Test
   public final void testAggregationFunction() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testAggregationFunction");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -853,7 +852,7 @@
       function.setFirstPhase();
     }
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     exec.init();
@@ -867,7 +866,7 @@
 
   @Test
   public final void testCountFunction() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCountFunction");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -884,7 +883,7 @@
       function.setFirstPhase();
     }
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     Tuple tuple = exec.next();
@@ -895,7 +894,7 @@
 
   @Test
   public final void testGroupByWithNullValue() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByWithNullValue");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -906,7 +905,7 @@
     LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     int count = 0;
@@ -920,7 +919,7 @@
 
   @Test
   public final void testUnionPlan() throws IOException, PlanningException, CloneNotSupportedException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testUnionPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -936,7 +935,7 @@
     union.setRightChild((LogicalNode) root.getChild().clone());
     root.setChild(union);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, root);
 
     int count = 0;
@@ -958,7 +957,7 @@
     LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     Tuple tuple;
     exec.init();
@@ -971,7 +970,7 @@
     plan = planner.createPlan(defaultContext, expr);
     rootNode = optimizer.optimize(plan);
 
-    phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    phyPlanner = new PhysicalPlannerImpl(conf);
     exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     tuple = exec.next();
@@ -985,7 +984,7 @@
 
   //@Test
   public final void testCreateIndex() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateIndex");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -995,7 +994,7 @@
     LogicalPlan plan = planner.createPlan(defaultContext, context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     while (exec.next() != null) {
@@ -1012,7 +1011,7 @@
 
   @Test
   public final void testDuplicateEliminate() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(),
         new Path(score.getPath()), Integer.MAX_VALUE);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testDuplicateEliminate");
@@ -1024,7 +1023,7 @@
     LogicalPlan plan = planner.createPlan(defaultContext, expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     Tuple tuple;
 
@@ -1046,7 +1045,7 @@
 
   @Test
   public final void testIndexedStoreExec() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testIndexedStoreExec");
@@ -1064,7 +1063,7 @@
     channel.setShuffleKeys(PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()).toArray());
     ctx.setDataChannel(channel);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     Tuple tuple;
@@ -1084,7 +1083,7 @@
     Path outputPath = StorageUtil.concatPath(workDir, "output", "output");
     TableMeta meta = CatalogUtil.newTableMeta(channel.getStoreType(), new KeyValueSet());
     SeekableScanner scanner =
-        StorageManager.getSeekableScanner(conf, meta, exec.getSchema(), outputPath);
+        FileStorageManager.getSeekableScanner(conf, meta, exec.getSchema(), outputPath);
     scanner.init();
 
     int cnt = 0;
@@ -1139,7 +1138,7 @@
 
   @Test
   public final void testSortEnforcer() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortEnforcer");
@@ -1157,7 +1156,7 @@
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
@@ -1179,7 +1178,7 @@
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);
 
-    phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    phyPlanner = new PhysicalPlannerImpl(conf);
     exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
@@ -1190,7 +1189,7 @@
 
   @Test
   public final void testGroupByEnforcer() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByEnforcer");
     Expr context = analyzer.parse(QUERIES[7]);
@@ -1207,7 +1206,7 @@
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
@@ -1229,7 +1228,7 @@
         new FileFragment[] {frags[0]}, workDir);
     ctx.setEnforcer(enforcer);
 
-    phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    phyPlanner = new PhysicalPlannerImpl(conf);
     exec = phyPlanner.createPlan(ctx, rootNode);
     exec.init();
     exec.next();
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
index a23a2d1..3c78b12 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
@@ -80,7 +80,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf.setVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR, testDir.toString());
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     Schema schema = new Schema();
     schema.addColumn("managerid", TajoDataTypes.Type.INT4);
@@ -89,7 +89,8 @@
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.RAW);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
     Tuple tuple = new VTuple(schema.size());
@@ -136,7 +137,7 @@
   }
 
   private void testProgress(int sortBufferBytesNum) throws Exception {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(),
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employee.getMeta(),
         new Path(employee.getPath()), Integer.MAX_VALUE);
     Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -146,7 +147,7 @@
     LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = plan.getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -156,8 +157,7 @@
       UnaryPhysicalExec sortExec = proj.getChild();
       SeqScanExec scan = sortExec.getChild();
 
-      ExternalSortExec extSort = new ExternalSortExec(ctx, sm,
-          ((MemSortExec)sortExec).getPlan(), scan);
+      ExternalSortExec extSort = new ExternalSortExec(ctx, ((MemSortExec)sortExec).getPlan(), scan);
 
       extSort.setSortBufferBytesNum(sortBufferBytesNum);
       proj.setChild(extSort);
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
index 9ebe871..879ca21 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
@@ -36,10 +36,7 @@
 import org.apache.tajo.plan.LogicalPlanner;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.logical.LogicalNode;
-import org.apache.tajo.storage.Appender;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.TUtil;
@@ -83,7 +80,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     //----------------- dep3 ------------------------------
     // dep_id | dep_name  | loc_id
@@ -106,7 +103,8 @@
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -135,7 +133,8 @@
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -174,7 +173,8 @@
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -232,9 +232,9 @@
 
   @Test
   public final void testRightOuter_HashJoinExec0() throws IOException, PlanningException {
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
+    FileFragment[] dep3Frags = FileStorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
@@ -246,7 +246,7 @@
     Expr expr = analyzer.parse(QUERIES[0]);
     LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -273,9 +273,9 @@
 
   @Test
   public final void testRightOuter_HashJoinExec1() throws IOException, PlanningException {
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
@@ -287,7 +287,7 @@
     Expr expr = analyzer.parse(QUERIES[1]);
     LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -314,9 +314,9 @@
     @Test
   public final void testRightOuter_HashJoinExec2() throws IOException, PlanningException {
     
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
@@ -328,7 +328,7 @@
     Expr expr = analyzer.parse(QUERIES[2]);
     LogicalNode plan = planner.createPlan(defaultContext, expr).getRootBlock().getRoot();
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
index 4956b7f..8bc00cc 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
@@ -87,7 +87,7 @@
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     //----------------- dep3 ------------------------------
     // dep_id | dep_name  | loc_id
@@ -110,7 +110,8 @@
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -148,7 +149,8 @@
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = StorageManager.getStorageManager(conf).getAppender(dep4Meta, dep4Schema, dep4Path);
+    Appender appender4 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     Tuple tuple4 = new VTuple(dep4Schema.size());
     for (int i = 0; i < 11; i++) {
@@ -179,7 +181,8 @@
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -218,7 +221,8 @@
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -270,8 +274,8 @@
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
     appender5.flush();
@@ -313,9 +317,9 @@
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
-    FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
         Integer.MAX_VALUE);
-    FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
+    FileFragment[] dep3Frags = FileStorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
 
@@ -324,7 +328,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -351,9 +355,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin1");
@@ -361,7 +365,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -387,9 +391,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin2");
@@ -397,7 +401,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     ProjectionExec proj = (ProjectionExec) exec;
     assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
@@ -423,9 +427,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] dep4Frags =
-        StorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getPath()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
 
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuter_MergeJoin3");
@@ -433,7 +437,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
 
     ProjectionExec proj = (ProjectionExec) exec;
@@ -460,9 +464,9 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags =
-        StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
@@ -471,7 +475,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     ProjectionExec proj = (ProjectionExec) exec;
     assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
@@ -497,8 +501,8 @@
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileStorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = FileStorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags);
 
@@ -508,7 +512,7 @@
         LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
     ctx.setEnforcer(enforcer);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, plan);
     ProjectionExec proj = (ProjectionExec) exec;
     assertTrue(proj.getChild() instanceof RightOuterMergeJoinExec);
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
index afa7430..8a61cab 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
@@ -57,7 +57,7 @@
   private static SQLAnalyzer analyzer;
   private static LogicalPlanner planner;
   private static LogicalOptimizer optimizer;
-  private static StorageManager sm;
+  private static FileStorageManager sm;
   private static Path workDir;
   private static Path tablePath;
   private static TableMeta employeeMeta;
@@ -70,7 +70,7 @@
     util = TpchTestBase.getInstance().getTestingCluster();
     catalog = util.getMaster().getCatalog();
     workDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    sm = StorageManager.getStorageManager(conf, workDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, workDir);
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -82,7 +82,8 @@
     tablePath = StorageUtil.concatPath(workDir, "employee", "table1");
     sm.getFileSystem().mkdirs(tablePath.getParent());
 
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(employeeMeta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, tablePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < 100; i++) {
@@ -110,7 +111,7 @@
 
   @Test
   public final void testNext() throws IOException, PlanningException {
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employeeMeta, tablePath, Integer.MAX_VALUE);
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employeeMeta, tablePath, Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestSortExec");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility
@@ -120,7 +121,7 @@
     LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), context);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     Tuple tuple;
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
new file mode 100644
index 0000000..db8eb84
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
@@ -0,0 +1,1469 @@
+/**
+ * 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.query;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.plan.expr.*;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.hbase.*;
+import org.apache.tajo.util.Bytes;
+import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.util.TUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.net.InetAddress;
+import java.sql.ResultSet;
+import java.text.DecimalFormat;
+import java.util.*;
+
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
+@Category(IntegrationTest.class)
+public class TestHBaseTable extends QueryTestCaseBase {
+  private static final Log LOG = LogFactory.getLog(TestHBaseTable.class);
+
+  @BeforeClass
+  public static void beforeClass() {
+    try {
+      testingCluster.getHBaseUtil().startHBaseCluster();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    try {
+      testingCluster.getHBaseUtil().stopHBaseCluster();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  @Test
+  public void testVerifyCreateHBaseTableRequiredMeta() throws Exception {
+    try {
+      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
+          "USING hbase").close();
+
+      fail("hbase table must have 'table' meta");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().indexOf("HBase mapped table") >= 0);
+    }
+
+    try {
+      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
+          "USING hbase " +
+          "WITH ('table'='hbase_table')").close();
+
+      fail("hbase table must have 'columns' meta");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().indexOf("'columns' property is required") >= 0);
+    }
+
+    try {
+      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
+          "USING hbase " +
+          "WITH ('table'='hbase_table', 'columns'='col1:,col2:')").close();
+
+      fail("hbase table must have 'hbase.zookeeper.quorum' meta");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().indexOf("HBase mapped table") >= 0);
+    }
+  }
+
+  @Test
+  public void testCreateHBaseTable() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:a,col3:,col2:b', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table1");
+
+    HTableDescriptor hTableDesc = testingCluster.getHBaseUtil().getTableDescriptor("hbase_table");
+    assertNotNull(hTableDesc);
+    assertEquals("hbase_table", hTableDesc.getNameAsString());
+
+    HColumnDescriptor[] hColumns = hTableDesc.getColumnFamilies();
+    // col1 is mapped to rowkey
+    assertEquals(2, hColumns.length);
+    assertEquals("col2", hColumns[0].getNameAsString());
+    assertEquals("col3", hColumns[1].getNameAsString());
+
+    executeString("DROP TABLE hbase_mapped_table1 PURGE").close();
+
+    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
+    try {
+      assertFalse(hAdmin.tableExists("hbase_table"));
+    } finally {
+      hAdmin.close();
+    }
+  }
+
+  @Test
+  public void testCreateNotExistsExternalHBaseTable() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    try {
+      executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) " +
+          "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:a,col3:,col2:b', " +
+          "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+          "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+      fail("External table should be a existed table.");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().indexOf("External table should be a existed table.") >= 0);
+    }
+  }
+
+  @Test
+  public void testCreateRowFieldWithNonText() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    try {
+      executeString("CREATE TABLE hbase_mapped_table2 (rk1 int4, rk2 text, col3 text, col4 text) " +
+          "USING hbase WITH ('table'='hbase_table', 'columns'='0:key#b,1:key,col3:,col2:b', " +
+          "'hbase.rowkey.delimiter'='_', " +
+          "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+          "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+      fail("Key field type should be TEXT type");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().indexOf("Key field type should be TEXT type") >= 0);
+    }
+  }
+
+  @Test
+  public void testCreateExternalHBaseTable() throws Exception {
+    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table_not_purge"));
+    hTableDesc.addFamily(new HColumnDescriptor("col1"));
+    hTableDesc.addFamily(new HColumnDescriptor("col2"));
+    hTableDesc.addFamily(new HColumnDescriptor("col3"));
+    testingCluster.getHBaseUtil().createTable(hTableDesc);
+
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " +
+        "USING hbase WITH ('table'='external_hbase_table_not_purge', 'columns'=':key,col1:a,col2:,col3:b', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("external_hbase_mapped_table");
+
+    executeString("DROP TABLE external_hbase_mapped_table").close();
+
+    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
+    try {
+      assertTrue(hAdmin.tableExists("external_hbase_table_not_purge"));
+      hAdmin.disableTable("external_hbase_table_not_purge");
+      hAdmin.deleteTable("external_hbase_table_not_purge");
+    } finally {
+      hAdmin.close();
+    }
+  }
+
+  @Test
+  public void testSimpleSelectQuery() throws Exception {
+    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
+    hTableDesc.addFamily(new HColumnDescriptor("col1"));
+    hTableDesc.addFamily(new HColumnDescriptor("col2"));
+    hTableDesc.addFamily(new HColumnDescriptor("col3"));
+    testingCluster.getHBaseUtil().createTable(hTableDesc);
+
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " +
+        "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col1:a,col2:,col3:b', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("external_hbase_mapped_table");
+
+    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
+        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HTableInterface htable = hconn.getTable("external_hbase_table");
+
+    try {
+      for (int i = 0; i < 100; i++) {
+        Put put = new Put(String.valueOf(i).getBytes());
+        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
+        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
+        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
+        put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        htable.put(put);
+      }
+
+      ResultSet res = executeString("select * from external_hbase_mapped_table where rk > '20'");
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
+      htable.close();
+    }
+  }
+
+  @Test
+  public void testBinaryMappedQuery() throws Exception {
+    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
+    hTableDesc.addFamily(new HColumnDescriptor("col1"));
+    hTableDesc.addFamily(new HColumnDescriptor("col2"));
+    hTableDesc.addFamily(new HColumnDescriptor("col3"));
+    testingCluster.getHBaseUtil().createTable(hTableDesc);
+
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk int8, col1 text, col2 text, col3 int4)\n " +
+        "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key#b,col1:a,col2:,col3:b#b', \n" +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "', \n" +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("external_hbase_mapped_table");
+
+    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
+        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HTableInterface htable = hconn.getTable("external_hbase_table");
+
+    try {
+      for (int i = 0; i < 100; i++) {
+        Put put = new Put(Bytes.toBytes((long) i));
+        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
+        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
+        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
+        put.add("col3".getBytes(), "b".getBytes(), Bytes.toBytes(i));
+        htable.put(put);
+      }
+
+      ResultSet res = executeString("select * from external_hbase_mapped_table where rk > 20");
+      assertResultSet(res);
+      res.close();
+
+      //Projection
+      res = executeString("select col3, col2, rk from external_hbase_mapped_table where rk > 95");
+
+      String expected = "col3,col2,rk\n" +
+          "-------------------------------\n" +
+          "96,{\"k1\":\"k1-96\", \"k2\":\"k2-96\"},96\n" +
+          "97,{\"k1\":\"k1-97\", \"k2\":\"k2-97\"},97\n" +
+          "98,{\"k1\":\"k1-98\", \"k2\":\"k2-98\"},98\n" +
+          "99,{\"k1\":\"k1-99\", \"k2\":\"k2-99\"},99\n";
+
+      assertEquals(expected, resultSetToString(res));
+      res.close();
+
+    } finally {
+      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
+      htable.close();
+    }
+  }
+
+  @Test
+  public void testColumnKeyValueSelectQuery() throws Exception {
+    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
+    hTableDesc.addFamily(new HColumnDescriptor("col2"));
+    hTableDesc.addFamily(new HColumnDescriptor("col3"));
+    testingCluster.getHBaseUtil().createTable(hTableDesc);
+
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, col2_key text, col2_value text, col3 text) " +
+        "USING hbase WITH ('table'='external_hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " +
+        "'hbase.rowkey.delimiter'='_', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("external_hbase_mapped_table");
+
+    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
+        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HTableInterface htable = hconn.getTable("external_hbase_table");
+
+    try {
+      for (int i = 0; i < 10; i++) {
+        Put put = new Put(Bytes.toBytes("rk-" + i));
+        for (int j = 0; j < 5; j++) {
+          put.add("col2".getBytes(), ("key-" + j).getBytes(), Bytes.toBytes("value-" + j));
+        }
+        put.add("col3".getBytes(), "".getBytes(), ("col3-value-" + i).getBytes());
+        htable.put(put);
+      }
+
+      ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 >= 'rk-0'");
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
+      htable.close();
+    }
+  }
+
+  @Test
+  public void testRowFieldSelectQuery() throws Exception {
+    HTableDescriptor hTableDesc = new HTableDescriptor(TableName.valueOf("external_hbase_table"));
+    hTableDesc.addFamily(new HColumnDescriptor("col3"));
+    testingCluster.getHBaseUtil().createTable(hTableDesc);
+
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, rk2 text, col3 text) " +
+        "USING hbase WITH ('table'='external_hbase_table', 'columns'='0:key,1:key,col3:a', " +
+        "'hbase.rowkey.delimiter'='_', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("external_hbase_mapped_table");
+
+    HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
+        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HTableInterface htable = hconn.getTable("external_hbase_table");
+
+    try {
+      for (int i = 0; i < 100; i++) {
+        Put put = new Put(("field1-" + i + "_field2-" + i).getBytes());
+        put.add("col3".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
+        htable.put(put);
+      }
+
+      ResultSet res = executeString("select * from external_hbase_mapped_table where rk1 > 'field1-20'");
+      assertResultSet(res);
+      cleanupQuery(res);
+    } finally {
+      executeString("DROP TABLE external_hbase_mapped_table PURGE").close();
+      htable.close();
+    }
+  }
+
+  @Test
+  public void testIndexPredication() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b', " +
+        "'hbase.split.rowkeys'='010,040,060,080', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+
+    assertTableExists("hbase_mapped_table");
+    HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
+    hAdmin.tableExists("hbase_table");
+
+    HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+    try {
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      assertEquals(5, keys.getFirst().length);
+
+      DecimalFormat df = new DecimalFormat("000");
+      for (int i = 0; i < 100; i++) {
+        Put put = new Put(String.valueOf(df.format(i)).getBytes());
+        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
+        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
+        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
+        put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        htable.put(put);
+      }
+      assertIndexPredication(false);
+
+      ResultSet res = executeString("select * from hbase_mapped_table where rk >= '020' and rk <= '055'");
+      assertResultSet(res);
+      res.close();
+
+      res = executeString("select * from hbase_mapped_table where rk = '021'");
+      String expected = "rk,col1,col2,col3\n" +
+          "-------------------------------\n" +
+          "021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n";
+
+      assertEquals(expected, resultSetToString(res));
+      res.close();
+    } finally {
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+      htable.close();
+      hAdmin.close();
+    }
+  }
+
+  @Test
+  public void testCompositeRowIndexPredication() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, rk2 text, col1 text, col2 text, col3 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " +
+        "'hbase.split.rowkeys'='010,040,060,080', " +
+        "'hbase.rowkey.delimiter'='_', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+
+    assertTableExists("hbase_mapped_table");
+    HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
+    hAdmin.tableExists("hbase_table");
+
+    HTable htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+    try {
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      assertEquals(5, keys.getFirst().length);
+
+      DecimalFormat df = new DecimalFormat("000");
+      for (int i = 0; i < 100; i++) {
+        Put put = new Put((df.format(i) + "_" + df.format(i)).getBytes());
+        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
+        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
+        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
+        put.add("col3".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        htable.put(put);
+      }
+
+      Scan scan = new Scan();
+      scan.setStartRow("021".getBytes());
+      scan.setStopRow(("021_" + new String(new char[]{Character.MAX_VALUE})).getBytes());
+      Filter filter = new InclusiveStopFilter(scan.getStopRow());
+      scan.setFilter(filter);
+
+      ResultScanner scanner = htable.getScanner(scan);
+      Result result = scanner.next();
+      assertNotNull(result);
+      assertEquals("021_021", new String(result.getRow()));
+      scanner.close();
+
+      assertIndexPredication(true);
+
+      ResultSet res = executeString("select * from hbase_mapped_table where rk = '021'");
+      String expected = "rk,rk2,col1,col2,col3\n" +
+          "-------------------------------\n" +
+          "021,021,a-21,{\"k1\":\"k1-21\", \"k2\":\"k2-21\"},b-21\n";
+
+      assertEquals(expected, resultSetToString(res));
+      res.close();
+    } finally {
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+      htable.close();
+      hAdmin.close();
+    }
+  }
+
+  private void assertIndexPredication(boolean isCompositeRowKey) throws Exception {
+    String postFix = isCompositeRowKey ? "_" + new String(new char[]{Character.MAX_VALUE}) : "";
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    ScanNode scanNode = new ScanNode(1);
+
+    // where rk = '021'
+    EvalNode evalNodeEq = new BinaryEval(EvalType.EQUAL, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("021")));
+    scanNode.setQual(evalNodeEq);
+    StorageManager storageManager = StorageManager.getStorageManager(conf, StoreType.HBASE);
+    List<Fragment> fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
+    assertEquals(1, fragments.size());
+    assertEquals("021", new String(((HBaseFragment)fragments.get(0)).getStartRow()));
+    assertEquals("021" + postFix, new String(((HBaseFragment)fragments.get(0)).getStopRow()));
+
+    // where rk >= '020' and rk <= '055'
+    EvalNode evalNode1 = new BinaryEval(EvalType.GEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("020")));
+    EvalNode evalNode2 = new BinaryEval(EvalType.LEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("055")));
+    EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2);
+    scanNode.setQual(evalNodeA);
+
+    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
+    assertEquals(2, fragments.size());
+    HBaseFragment fragment1 = (HBaseFragment) fragments.get(0);
+    assertEquals("020", new String(fragment1.getStartRow()));
+    assertEquals("040", new String(fragment1.getStopRow()));
+
+    HBaseFragment fragment2 = (HBaseFragment) fragments.get(1);
+    assertEquals("040", new String(fragment2.getStartRow()));
+    assertEquals("055" + postFix, new String(fragment2.getStopRow()));
+
+    // where (rk >= '020' and rk <= '055') or rk = '075'
+    EvalNode evalNode3 = new BinaryEval(EvalType.EQUAL, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("075")));
+    EvalNode evalNodeB = new BinaryEval(EvalType.OR, evalNodeA, evalNode3);
+    scanNode.setQual(evalNodeB);
+    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
+    assertEquals(3, fragments.size());
+    fragment1 = (HBaseFragment) fragments.get(0);
+    assertEquals("020", new String(fragment1.getStartRow()));
+    assertEquals("040", new String(fragment1.getStopRow()));
+
+    fragment2 = (HBaseFragment) fragments.get(1);
+    assertEquals("040", new String(fragment2.getStartRow()));
+    assertEquals("055" + postFix, new String(fragment2.getStopRow()));
+
+    HBaseFragment fragment3 = (HBaseFragment) fragments.get(2);
+    assertEquals("075", new String(fragment3.getStartRow()));
+    assertEquals("075" + postFix, new String(fragment3.getStopRow()));
+
+
+    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078')
+    EvalNode evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("072")));
+    EvalNode evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("078")));
+    EvalNode evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
+    EvalNode evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC);
+    scanNode.setQual(evalNodeD);
+    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
+    assertEquals(3, fragments.size());
+
+    fragment1 = (HBaseFragment) fragments.get(0);
+    assertEquals("020", new String(fragment1.getStartRow()));
+    assertEquals("040", new String(fragment1.getStopRow()));
+
+    fragment2 = (HBaseFragment) fragments.get(1);
+    assertEquals("040", new String(fragment2.getStartRow()));
+    assertEquals("055" + postFix, new String(fragment2.getStopRow()));
+
+    fragment3 = (HBaseFragment) fragments.get(2);
+    assertEquals("072", new String(fragment3.getStartRow()));
+    assertEquals("078" + postFix, new String(fragment3.getStopRow()));
+
+    // where (rk >= '020' and rk <= '055') or (rk >= '057' and rk <= '059')
+    evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("057")));
+    evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
+        new ConstEval(new TextDatum("059")));
+    evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
+    evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC);
+    scanNode.setQual(evalNodeD);
+    fragments = storageManager.getSplits("hbase_mapped_table", tableDesc, scanNode);
+    assertEquals(2, fragments.size());
+
+    fragment1 = (HBaseFragment) fragments.get(0);
+    assertEquals("020", new String(fragment1.getStartRow()));
+    assertEquals("040", new String(fragment1.getStopRow()));
+
+    fragment2 = (HBaseFragment) fragments.get(1);
+    assertEquals("040", new String(fragment2.getStartRow()));
+    assertEquals("059" + postFix, new String(fragment2.getStopRow()));
+  }
+
+  @Test
+  public void testNonForwardQuery() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:#b', " +
+        "'hbase.split.rowkeys'='010,040,060,080', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+
+    assertTableExists("hbase_mapped_table");
+    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
+    HTable htable = null;
+    try {
+      hAdmin.tableExists("hbase_table");
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      assertEquals(5, keys.getFirst().length);
+
+      DecimalFormat df = new DecimalFormat("000");
+      for (int i = 0; i < 100; i++) {
+        Put put = new Put(String.valueOf(df.format(i)).getBytes());
+        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
+        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
+        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
+        put.add("col3".getBytes(), "".getBytes(), Bytes.toBytes(i));
+        htable.put(put);
+      }
+
+      ResultSet res = executeString("select * from hbase_mapped_table");
+      assertResultSet(res);
+      res.close();
+    } finally {
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+      hAdmin.close();
+      if (htable == null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testJoin() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int8) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " +
+        "'hbase.split.rowkeys'='010,040,060,080', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+
+    assertTableExists("hbase_mapped_table");
+    HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
+    HTable htable = null;
+    try {
+      hAdmin.tableExists("hbase_table");
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      assertEquals(5, keys.getFirst().length);
+
+      DecimalFormat df = new DecimalFormat("000");
+      for (int i = 0; i < 100; i++) {
+        Put put = new Put(String.valueOf(df.format(i)).getBytes());
+        put.add("col1".getBytes(), "a".getBytes(), ("a-" + i).getBytes());
+        put.add("col1".getBytes(), "b".getBytes(), ("b-" + i).getBytes());
+        put.add("col2".getBytes(), "k1".getBytes(), ("k1-" + i).getBytes());
+        put.add("col2".getBytes(), "k2".getBytes(), ("k2-" + i).getBytes());
+        put.add("col3".getBytes(), "b".getBytes(), Bytes.toBytes((long) i));
+        htable.put(put);
+      }
+
+      ResultSet res = executeString("select a.rk, a.col1, a.col2, a.col3, b.l_orderkey, b.l_linestatus " +
+          "from hbase_mapped_table a " +
+          "join default.lineitem b on a.col3 = b.l_orderkey");
+      assertResultSet(res);
+      res.close();
+    } finally {
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+      hAdmin.close();
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertInto() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    executeString("insert into hbase_mapped_table " +
+        "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem ").close();
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scan.addFamily(Bytes.toBytes("col2"));
+      scan.addFamily(Bytes.toBytes("col3"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1"), Bytes.toBytes("col2"), Bytes.toBytes("col3")},
+          new byte[][]{null, Bytes.toBytes("a"), null, Bytes.toBytes("b")},
+          new boolean[]{false, false, false, true}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoMultiRegion() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
+        "'hbase.split.rowkeys'='010,040,060,080', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.TEXT);
+    schema.addColumn("name", Type.TEXT);
+    List<String> datas = new ArrayList<String>();
+    DecimalFormat df = new DecimalFormat("000");
+    for (int i = 99; i >= 0; i--) {
+      datas.add(df.format(i) + "|value" + i);
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    executeString("insert into hbase_mapped_table " +
+        "select id, name from base_table ").close();
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1")},
+          new byte[][]{null, Bytes.toBytes("a")},
+          new boolean[]{false, false}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoMultiRegion2() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
+        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.TEXT);
+    schema.addColumn("name", Type.TEXT);
+    List<String> datas = new ArrayList<String>();
+    for (int i = 99; i >= 0; i--) {
+      datas.add(i + "|value" + i);
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    executeString("insert into hbase_mapped_table " +
+        "select id, name from base_table ").close();
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1")},
+          new byte[][]{null, Bytes.toBytes("a")},
+          new boolean[]{false, false}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoMultiRegionWithSplitFile() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    String splitFilePath = currentDatasetPath + "/splits.data";
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
+        "'hbase.split.rowkeys.file'='" + splitFilePath + "', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.TEXT);
+    schema.addColumn("name", Type.TEXT);
+    List<String> datas = new ArrayList<String>();
+    DecimalFormat df = new DecimalFormat("000");
+    for (int i = 99; i >= 0; i--) {
+      datas.add(df.format(i) + "|value" + i);
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    executeString("insert into hbase_mapped_table " +
+        "select id, name from base_table ").close();
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1")},
+          new byte[][]{null, Bytes.toBytes("a")},
+          new boolean[]{false, false}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoMultiRegionMultiRowFields() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a', " +
+        "'hbase.split.rowkeys'='001,002,003,004,005,006,007,008,009', " +
+        "'hbase.rowkey.delimiter'='_', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("id1", Type.TEXT);
+    schema.addColumn("id2", Type.TEXT);
+    schema.addColumn("name", Type.TEXT);
+    DecimalFormat df = new DecimalFormat("000");
+    List<String> datas = new ArrayList<String>();
+    for (int i = 99; i >= 0; i--) {
+      datas.add(df.format(i) + "|" + (i + 100) + "|value" + i);
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    executeString("insert into hbase_mapped_table " +
+        "select id1, id2, name from base_table ").close();
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, null, Bytes.toBytes("col1")},
+          new byte[][]{null, null, Bytes.toBytes("a")},
+          new boolean[]{false, false, false}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoBinaryMultiRegion() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk int4, col1 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key#b,col1:a', " +
+        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("name", Type.TEXT);
+    List<String> datas = new ArrayList<String>();
+    for (int i = 99; i >= 0; i--) {
+      datas.add(i + "|value" + i);
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    executeString("insert into hbase_mapped_table " +
+        "select id, name from base_table ").close();
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1")},
+          new byte[][]{null, Bytes.toBytes("a")},
+          new boolean[]{true, false}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoColumnKeyValue() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col2_key text, col2_value text, col3 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:key:,col2:value:,col3:', " +
+        "'hbase.rowkey.delimiter'='_', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("rk", Type.TEXT);
+    schema.addColumn("col2_key", Type.TEXT);
+    schema.addColumn("col2_value", Type.TEXT);
+    schema.addColumn("col3", Type.TEXT);
+    List<String> datas = new ArrayList<String>();
+    for (int i = 20; i >= 0; i--) {
+      for (int j = 0; j < 3; j++) {
+        datas.add(i + "|ck-" + j + "|value-" + j + "|col3-" + i);
+      }
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    executeString("insert into hbase_mapped_table " +
+        "select rk, col2_key, col2_value, col3 from base_table ").close();
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col2"));
+      scan.addFamily(Bytes.toBytes("col3"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col2"), Bytes.toBytes("col3")},
+          new byte[][]{null, null, null},
+          new boolean[]{false, false, false}, tableDesc.getSchema()));
+
+      ResultSet res = executeString("select * from hbase_mapped_table");
+
+      String expected = "rk,col2_key,col2_value,col3\n" +
+          "-------------------------------\n" +
+          "0,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-0\n" +
+          "1,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-1\n" +
+          "10,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-10\n" +
+          "11,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-11\n" +
+          "12,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-12\n" +
+          "13,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-13\n" +
+          "14,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-14\n" +
+          "15,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-15\n" +
+          "16,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-16\n" +
+          "17,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-17\n" +
+          "18,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-18\n" +
+          "19,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-19\n" +
+          "2,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-2\n" +
+          "20,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-20\n" +
+          "3,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-3\n" +
+          "4,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-4\n" +
+          "5,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-5\n" +
+          "6,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-6\n" +
+          "7,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-7\n" +
+          "8,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-8\n" +
+          "9,[\"ck-0\", \"ck-1\", \"ck-2\"],[\"value-0\", \"value-1\", \"value-2\"],col3-9\n";
+
+      assertEquals(expected, resultSetToString(res));
+      res.close();
+
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoDifferentType() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
+        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("name", Type.TEXT);
+    List<String> datas = new ArrayList<String>();
+    for (int i = 99; i >= 0; i--) {
+      datas.add(i + "|value" + i);
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    try {
+      executeString("insert into hbase_mapped_table " +
+          "select id, name from base_table ").close();
+      fail("If inserting data type different with target table data type, should throw exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().indexOf("VerifyException") >= 0);
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+    }
+  }
+
+  @Test
+  public void testInsertIntoRowField() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text, col2 text, col3 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'='0:key,1:key,col1:a,col2:,col3:b', " +
+        "'hbase.rowkey.delimiter'='_', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    executeString("insert into hbase_mapped_table " +
+        "select l_orderkey::text, l_partkey::text, l_shipdate, l_returnflag, l_suppkey::text from default.lineitem ");
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scan.addFamily(Bytes.toBytes("col2"));
+      scan.addFamily(Bytes.toBytes("col3"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1"), Bytes.toBytes("col2"), Bytes.toBytes("col3")},
+          new byte[][]{null, Bytes.toBytes("a"), Bytes.toBytes(""), Bytes.toBytes("b")},
+          new boolean[]{false, false, false, false}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testCATS() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    // create test table
+    KeyValueSet tableOptions = new KeyValueSet();
+    tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.TEXT);
+    schema.addColumn("name", Type.TEXT);
+    List<String> datas = new ArrayList<String>();
+    DecimalFormat df = new DecimalFormat("000");
+    for (int i = 99; i >= 0; i--) {
+      datas.add(df.format(i) + "|value" + i);
+    }
+    TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+        schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
+        "'hbase.split.rowkeys'='010,040,060,080', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')" +
+        " as " +
+        "select id, name from base_table"
+    ).close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scanner = htable.getScanner(scan);
+
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1")},
+          new byte[][]{null, Bytes.toBytes("a")},
+          new boolean[]{false, false}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoUsingPut() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    Map<String, String> sessions = new HashMap<String, String>();
+    sessions.put(HBaseStorageConstants.INSERT_PUT_MODE, "true");
+    client.updateSessionVariables(sessions);
+
+    HTable htable = null;
+    ResultScanner scanner = null;
+    try {
+      executeString("insert into hbase_mapped_table " +
+          "select l_orderkey::text, l_shipdate, l_returnflag, l_suppkey from default.lineitem ").close();
+
+      htable = new HTable(testingCluster.getHBaseUtil().getConf(), "hbase_table");
+
+      Scan scan = new Scan();
+      scan.addFamily(Bytes.toBytes("col1"));
+      scan.addFamily(Bytes.toBytes("col2"));
+      scan.addFamily(Bytes.toBytes("col3"));
+      scanner = htable.getScanner(scan);
+
+      // result is dirrerent with testInsertInto because l_orderkey is not unique.
+      assertStrings(resultSetToString(scanner,
+          new byte[][]{null, Bytes.toBytes("col1"), Bytes.toBytes("col2"), Bytes.toBytes("col3")},
+          new byte[][]{null, Bytes.toBytes("a"), null, Bytes.toBytes("b")},
+          new boolean[]{false, false, false, true}, tableDesc.getSchema()));
+
+    } finally {
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+
+      client.unsetSessionVariables(TUtil.newList(HBaseStorageConstants.INSERT_PUT_MODE));
+
+      if (scanner != null) {
+        scanner.close();
+      }
+
+      if (htable != null) {
+        htable.close();
+      }
+    }
+  }
+
+  @Test
+  public void testInsertIntoLocation() throws Exception {
+    String hostName = InetAddress.getLocalHost().getHostName();
+    String zkPort = testingCluster.getHBaseUtil().getConf().get(HConstants.ZOOKEEPER_CLIENT_PORT);
+    assertNotNull(zkPort);
+
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text) " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:', " +
+        "'hbase.split.rowkeys'='010,040,060,080', " +
+        "'" + HConstants.ZOOKEEPER_QUORUM + "'='" + hostName + "'," +
+        "'" + HConstants.ZOOKEEPER_CLIENT_PORT + "'='" + zkPort + "')").close();
+
+    assertTableExists("hbase_mapped_table");
+
+    try {
+      // create test table
+      KeyValueSet tableOptions = new KeyValueSet();
+      tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+      tableOptions.set(StorageConstants.CSVFILE_NULL, "\\\\N");
+
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.TEXT);
+      schema.addColumn("name", Type.TEXT);
+      schema.addColumn("comment", Type.TEXT);
+      List<String> datas = new ArrayList<String>();
+      DecimalFormat df = new DecimalFormat("000");
+      for (int i = 99; i >= 0; i--) {
+        datas.add(df.format(i) + "|value" + i + "|comment-" + i);
+      }
+      TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
+          schema, tableOptions, datas.toArray(new String[]{}), 2);
+
+      executeString("insert into location '/tmp/hfile_test' " +
+          "select id, name, comment from base_table ").close();
+
+      FileSystem fs = testingCluster.getDefaultFileSystem();
+      Path path = new Path("/tmp/hfile_test");
+      assertTrue(fs.exists(path));
+
+      FileStatus[] files = fs.listStatus(path);
+      assertNotNull(files);
+      assertEquals(2, files.length);
+
+      int index = 0;
+      for (FileStatus eachFile: files) {
+        assertEquals("/tmp/hfile_test/part-01-00000" + index + "-00" + index, eachFile.getPath().toUri().getPath());
+        for (FileStatus subFile: fs.listStatus(eachFile.getPath())) {
+          assertTrue(subFile.isFile());
+          assertTrue(subFile.getLen() > 0);
+        }
+        index++;
+      }
+    } finally {
+      executeString("DROP TABLE base_table PURGE").close();
+      executeString("DROP TABLE hbase_mapped_table PURGE").close();
+    }
+  }
+
+  private String resultSetToString(ResultScanner scanner,
+                                   byte[][] cfNames, byte[][] qualifiers,
+                                   boolean[] binaries,
+                                   Schema schema) throws Exception {
+    StringBuilder sb = new StringBuilder();
+    Result result = null;
+    while ( (result = scanner.next()) != null ) {
+      if (binaries[0]) {
+        sb.append(HBaseBinarySerializerDeserializer.deserialize(schema.getColumn(0), result.getRow()).asChar());
+      } else {
+        sb.append(new String(result.getRow()));
+      }
+
+      for (int i = 0; i < cfNames.length; i++) {
+        if (cfNames[i] == null) {
+          //rowkey
+          continue;
+        }
+        if (qualifiers[i] == null) {
+          Map<byte[], byte[]> values = result.getFamilyMap(cfNames[i]);
+          if (values == null) {
+            sb.append(", null");
+          } else {
+            sb.append(", {");
+            String delim = "";
+            for (Map.Entry<byte[], byte[]> valueEntry: values.entrySet()) {
+              byte[] keyBytes = valueEntry.getKey();
+              byte[] valueBytes = valueEntry.getValue();
+
+              if (binaries[i]) {
+                sb.append(delim).append("\"").append(keyBytes == null ? "" : Bytes.toLong(keyBytes)).append("\"");
+                sb.append(": \"").append(HBaseBinarySerializerDeserializer.deserialize(schema.getColumn(i), valueBytes)).append("\"");
+              } else {
+                sb.append(delim).append("\"").append(keyBytes == null ? "" : new String(keyBytes)).append("\"");
+                sb.append(": \"").append(HBaseTextSerializerDeserializer.deserialize(schema.getColumn(i), valueBytes)).append("\"");
+              }
+              delim = ", ";
+            }
+            sb.append("}");
+          }
+        } else {
+          byte[] value = result.getValue(cfNames[i], qualifiers[i]);
+          if (value == null) {
+            sb.append(", null");
+          } else {
+            if (binaries[i]) {
+              sb.append(", ").append(HBaseBinarySerializerDeserializer.deserialize(schema.getColumn(i), value));
+            } else {
+              sb.append(", ").append(HBaseTextSerializerDeserializer.deserialize(schema.getColumn(i), value));
+            }
+          }
+        }
+      }
+      sb.append("\n");
+    }
+
+    return sb.toString();
+  }
+}
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
index c1cec2b..68b3fb3 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
@@ -565,8 +565,8 @@
         }
         Path dataPath = new Path(table.getPath().toString(), fileIndex + ".csv");
         fileIndex++;
-        appender = StorageManager.getStorageManager(conf).getAppender(tableMeta, schema,
-            dataPath);
+        appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+            .getAppender(tableMeta, schema, dataPath);
         appender.init();
       }
       String[] columnDatas = rows[i].split("\\|");
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
index 223ea8e..c249b61 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -1026,41 +1026,42 @@
     cleanupQuery(res);
   }
 
-  @Test
-  public void testComplexJoinCondition3() throws Exception {
-    // select n1.n_nationkey, n1.n_name, n2.n_name from nation n1 join nation n2 on lower(n1.n_name) = lower(n2.n_name);
-    ResultSet res = executeQuery();
-    assertResultSet(res);
-    cleanupQuery(res);
-  }
-
-  @Test
-  public void testComplexJoinCondition4() throws Exception {
-    ResultSet res = executeQuery();
-    assertResultSet(res);
-    cleanupQuery(res);
-  }
-
-  @Test
-  public void testComplexJoinCondition5() throws Exception {
-    ResultSet res = executeQuery();
-    assertResultSet(res);
-    cleanupQuery(res);
-  }
-
-  @Test
-  public void testComplexJoinCondition6() throws Exception {
-    ResultSet res = executeQuery();
-    assertResultSet(res);
-    cleanupQuery(res);
-  }
-
-  @Test
-  public void testComplexJoinCondition7() throws Exception {
-    ResultSet res = executeQuery();
-    assertResultSet(res);
-    cleanupQuery(res);
-  }
+  //FIXME The following testcase should be uncommented after resolving the issue TAJO-1126.
+//  @Test
+//  public void testComplexJoinCondition3() throws Exception {
+//    // select n1.n_nationkey, n1.n_name, n2.n_name from nation n1 join nation n2 on lower(n1.n_name) = lower(n2.n_name);
+//    ResultSet res = executeQuery();
+//    assertResultSet(res);
+//    cleanupQuery(res);
+//  }
+//
+//  @Test
+//  public void testComplexJoinCondition4() throws Exception {
+//    ResultSet res = executeQuery();
+//    assertResultSet(res);
+//    cleanupQuery(res);
+//  }
+//
+//  @Test
+//  public void testComplexJoinCondition5() throws Exception {
+//    ResultSet res = executeQuery();
+//    assertResultSet(res);
+//    cleanupQuery(res);
+//  }
+//
+//  @Test
+//  public void testComplexJoinCondition6() throws Exception {
+//    ResultSet res = executeQuery();
+//    assertResultSet(res);
+//    cleanupQuery(res);
+//  }
+//
+//  @Test
+//  public void testComplexJoinCondition7() throws Exception {
+//    ResultSet res = executeQuery();
+//    assertResultSet(res);
+//    cleanupQuery(res);
+//  }
 
   @Test
   public void testFullOuterJoinWithEmptyIntermediateData() throws Exception {
diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
index f491c63..b8f3ef7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
+++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
@@ -55,7 +55,7 @@
   private static TajoTestingCluster util;
   private static TajoConf conf;
   private static TableDesc desc;
-  private static StorageManager sm;
+  private static FileStorageManager sm;
   private static TableMeta scoreMeta;
   private static Schema scoreSchema;
 
@@ -63,7 +63,7 @@
   public static void setup() throws Exception {
     util = TpchTestBase.getInstance().getTestingCluster();
     conf = util.getConfiguration();
-    sm = StorageManager.getStorageManager(conf);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
 
     scoreSchema = new Schema();
     scoreSchema.addColumn("deptname", Type.TEXT);
@@ -73,8 +73,7 @@
 
     Path p = sm.getTablePath("score");
     sm.getFileSystem().mkdirs(p);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(scoreMeta, scoreSchema,
-        new Path(p, "score"));
+    Appender appender = sm.getAppender(scoreMeta, scoreSchema, new Path(p, "score"));
     appender.init();
     int deptSize = 100;
     int tupleNum = 10000;
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
index 067c6c8..712243b 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
@@ -82,7 +82,7 @@
     logicalPlanner = new LogicalPlanner(catalog);
     optimizer = new LogicalOptimizer(conf);
 
-    StorageManager sm  = StorageManager.getStorageManager(conf);
+    StorageManager sm  = StorageManager.getFileStorageManager(conf);
     dispatcher = new AsyncDispatcher();
     dispatcher.init(conf);
     dispatcher.start();
diff --git a/tajo-core/src/test/java/org/apache/tajo/storage/TestFileFragment.java b/tajo-core/src/test/java/org/apache/tajo/storage/TestFileFragment.java
index e6ff7a3..d0ab1c0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/storage/TestFileFragment.java
+++ b/tajo-core/src/test/java/org/apache/tajo/storage/TestFileFragment.java
@@ -47,7 +47,7 @@
     assertEquals("table1_1", fragment1.getTableName());
     assertEquals(new Path(path, "table0"), fragment1.getPath());
     assertTrue(0 == fragment1.getStartKey());
-    assertTrue(500 == fragment1.getEndKey());
+    assertTrue(500 == fragment1.getLength());
   }
 
   @Test
@@ -58,7 +58,7 @@
     assertEquals("table1_1", fragment1.getTableName());
     assertEquals(new Path(path, "table0"), fragment1.getPath());
     assertTrue(0 == fragment1.getStartKey());
-    assertTrue(500 == fragment1.getEndKey());
+    assertTrue(500 == fragment1.getLength());
   }
 
   @Test
diff --git a/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java b/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java
index 742b07f..f36ff24 100644
--- a/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java
+++ b/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java
@@ -69,7 +69,8 @@
 
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.ROWFILE);
 
-    StorageManager sm = StorageManager.getStorageManager(conf, new Path(conf.getVar(ConfVars.ROOT_DIR)));
+    FileStorageManager sm =
+        (FileStorageManager)StorageManager.getFileStorageManager(conf, new Path(conf.getVar(ConfVars.ROOT_DIR)));
 
     Path tablePath = new Path("/test");
     Path metaPath = new Path(tablePath, ".meta");
@@ -79,7 +80,7 @@
 
     FileUtil.writeProto(fs, metaPath, meta.getProto());
 
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, dataPath);
+    Appender appender = sm.getAppender(meta, schema, dataPath);
     appender.enableStats();
     appender.init();
 
@@ -110,7 +111,7 @@
 
     int tupleCnt = 0;
     start = System.currentTimeMillis();
-    Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
     while ((tuple=scanner.next()) != null) {
       tupleCnt++;
diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
index 186a7f5..5a93538 100644
--- a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
@@ -90,7 +90,7 @@
     catalog = util.getMiniCatalogCluster().getCatalog();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 
     analyzer = new SQLAnalyzer();
     planner = new LogicalPlanner(catalog);
@@ -120,7 +120,7 @@
 
     Path tableDir = StorageUtil.concatPath(testDir, "testGet", "table.csv");
     fs.mkdirs(tableDir.getParent());
-    Appender appender = sm.getAppender(employeeMeta, schema, tableDir);
+    Appender appender = ((FileStorageManager)sm).getAppender(employeeMeta, schema, tableDir);
     appender.init();
 
     Tuple tuple = new VTuple(schema.size());
@@ -145,7 +145,7 @@
         tableDir.toUri());
     catalog.createTable(employee);
 
-    FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employeeMeta, tableDir, Integer.MAX_VALUE);
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", employeeMeta, tableDir, Integer.MAX_VALUE);
 
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newQueryUnitAttemptId(),
@@ -155,7 +155,7 @@
     LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     ExternalSortExec sort = null;
@@ -169,7 +169,7 @@
     }
 
     SortSpec[] sortSpecs = sort.getPlan().getSortKeys();
-    RangeShuffleFileWriteExec idxStoreExec = new RangeShuffleFileWriteExec(ctx, sm, sort, sort.getSchema(),
+    RangeShuffleFileWriteExec idxStoreExec = new RangeShuffleFileWriteExec(ctx, sort, sort.getSchema(),
         sort.getSchema(), sortSpecs);
 
     exec = idxStoreExec;
@@ -185,7 +185,7 @@
     reader.open();
 
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.RAW, new KeyValueSet());
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema,
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema,
         StorageUtil.concatPath(testDir, "output", "output"));
 
     scanner.init();
@@ -245,7 +245,7 @@
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path tablePath = StorageUtil.concatPath(testDir, "testGetFromDescendingOrder", "table.csv");
     fs.mkdirs(tablePath.getParent());
-    Appender appender = sm.getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)sm).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = (TEST_TUPLE - 1); i >= 0 ; i--) {
@@ -268,7 +268,7 @@
         CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta, tablePath.toUri());
     catalog.createTable(employee);
 
-    FileFragment[] frags = sm.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE);
+    FileFragment[] frags = FileStorageManager.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE);
 
     TaskAttemptContext
         ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -279,7 +279,7 @@
     LogicalPlan plan = planner.createPlan(LocalTajoTestingUtility.createDummyContext(conf), expr);
     LogicalNode rootNode = optimizer.optimize(plan);
 
-    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
+    PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
     ExternalSortExec sort = null;
@@ -293,7 +293,7 @@
     }
 
     SortSpec[] sortSpecs = sort.getPlan().getSortKeys();
-    RangeShuffleFileWriteExec idxStoreExec = new RangeShuffleFileWriteExec(ctx, sm, sort,
+    RangeShuffleFileWriteExec idxStoreExec = new RangeShuffleFileWriteExec(ctx, sort,
         sort.getSchema(), sort.getSchema(), sortSpecs);
 
     exec = idxStoreExec;
@@ -308,7 +308,7 @@
         new Path(testDir, "output/index"), keySchema, comp);
     reader.open();
     TableMeta outputMeta = CatalogUtil.newTableMeta(StoreType.RAW, new KeyValueSet());
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, outputMeta, schema,
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, outputMeta, schema,
         StorageUtil.concatPath(testDir, "output", "output"));
     scanner.init();
     int cnt = 0;
diff --git a/tajo-core/src/test/resources/dataset/TestHBaseTable/splits.data b/tajo-core/src/test/resources/dataset/TestHBaseTable/splits.data
new file mode 100644
index 0000000..417d480
--- /dev/null
+++ b/tajo-core/src/test/resources/dataset/TestHBaseTable/splits.data
@@ -0,0 +1,4 @@
+010
+040
+060
+080
\ No newline at end of file
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testBinaryMappedQuery.result b/tajo-core/src/test/resources/results/TestHBaseTable/testBinaryMappedQuery.result
new file mode 100644
index 0000000..8d50bf1
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testBinaryMappedQuery.result
@@ -0,0 +1,81 @@
+rk,col1,col2,col3
+-------------------------------
+21,a-21,{"k1":"k1-21", "k2":"k2-21"},21
+22,a-22,{"k1":"k1-22", "k2":"k2-22"},22
+23,a-23,{"k1":"k1-23", "k2":"k2-23"},23
+24,a-24,{"k1":"k1-24", "k2":"k2-24"},24
+25,a-25,{"k1":"k1-25", "k2":"k2-25"},25
+26,a-26,{"k1":"k1-26", "k2":"k2-26"},26
+27,a-27,{"k1":"k1-27", "k2":"k2-27"},27
+28,a-28,{"k1":"k1-28", "k2":"k2-28"},28
+29,a-29,{"k1":"k1-29", "k2":"k2-29"},29
+30,a-30,{"k1":"k1-30", "k2":"k2-30"},30
+31,a-31,{"k1":"k1-31", "k2":"k2-31"},31
+32,a-32,{"k1":"k1-32", "k2":"k2-32"},32
+33,a-33,{"k1":"k1-33", "k2":"k2-33"},33
+34,a-34,{"k1":"k1-34", "k2":"k2-34"},34
+35,a-35,{"k1":"k1-35", "k2":"k2-35"},35
+36,a-36,{"k1":"k1-36", "k2":"k2-36"},36
+37,a-37,{"k1":"k1-37", "k2":"k2-37"},37
+38,a-38,{"k1":"k1-38", "k2":"k2-38"},38
+39,a-39,{"k1":"k1-39", "k2":"k2-39"},39
+40,a-40,{"k1":"k1-40", "k2":"k2-40"},40
+41,a-41,{"k1":"k1-41", "k2":"k2-41"},41
+42,a-42,{"k1":"k1-42", "k2":"k2-42"},42
+43,a-43,{"k1":"k1-43", "k2":"k2-43"},43
+44,a-44,{"k1":"k1-44", "k2":"k2-44"},44
+45,a-45,{"k1":"k1-45", "k2":"k2-45"},45
+46,a-46,{"k1":"k1-46", "k2":"k2-46"},46
+47,a-47,{"k1":"k1-47", "k2":"k2-47"},47
+48,a-48,{"k1":"k1-48", "k2":"k2-48"},48
+49,a-49,{"k1":"k1-49", "k2":"k2-49"},49
+50,a-50,{"k1":"k1-50", "k2":"k2-50"},50
+51,a-51,{"k1":"k1-51", "k2":"k2-51"},51
+52,a-52,{"k1":"k1-52", "k2":"k2-52"},52
+53,a-53,{"k1":"k1-53", "k2":"k2-53"},53
+54,a-54,{"k1":"k1-54", "k2":"k2-54"},54
+55,a-55,{"k1":"k1-55", "k2":"k2-55"},55
+56,a-56,{"k1":"k1-56", "k2":"k2-56"},56
+57,a-57,{"k1":"k1-57", "k2":"k2-57"},57
+58,a-58,{"k1":"k1-58", "k2":"k2-58"},58
+59,a-59,{"k1":"k1-59", "k2":"k2-59"},59
+60,a-60,{"k1":"k1-60", "k2":"k2-60"},60
+61,a-61,{"k1":"k1-61", "k2":"k2-61"},61
+62,a-62,{"k1":"k1-62", "k2":"k2-62"},62
+63,a-63,{"k1":"k1-63", "k2":"k2-63"},63
+64,a-64,{"k1":"k1-64", "k2":"k2-64"},64
+65,a-65,{"k1":"k1-65", "k2":"k2-65"},65
+66,a-66,{"k1":"k1-66", "k2":"k2-66"},66
+67,a-67,{"k1":"k1-67", "k2":"k2-67"},67
+68,a-68,{"k1":"k1-68", "k2":"k2-68"},68
+69,a-69,{"k1":"k1-69", "k2":"k2-69"},69
+70,a-70,{"k1":"k1-70", "k2":"k2-70"},70
+71,a-71,{"k1":"k1-71", "k2":"k2-71"},71
+72,a-72,{"k1":"k1-72", "k2":"k2-72"},72
+73,a-73,{"k1":"k1-73", "k2":"k2-73"},73
+74,a-74,{"k1":"k1-74", "k2":"k2-74"},74
+75,a-75,{"k1":"k1-75", "k2":"k2-75"},75
+76,a-76,{"k1":"k1-76", "k2":"k2-76"},76
+77,a-77,{"k1":"k1-77", "k2":"k2-77"},77
+78,a-78,{"k1":"k1-78", "k2":"k2-78"},78
+79,a-79,{"k1":"k1-79", "k2":"k2-79"},79
+80,a-80,{"k1":"k1-80", "k2":"k2-80"},80
+81,a-81,{"k1":"k1-81", "k2":"k2-81"},81
+82,a-82,{"k1":"k1-82", "k2":"k2-82"},82
+83,a-83,{"k1":"k1-83", "k2":"k2-83"},83
+84,a-84,{"k1":"k1-84", "k2":"k2-84"},84
+85,a-85,{"k1":"k1-85", "k2":"k2-85"},85
+86,a-86,{"k1":"k1-86", "k2":"k2-86"},86
+87,a-87,{"k1":"k1-87", "k2":"k2-87"},87
+88,a-88,{"k1":"k1-88", "k2":"k2-88"},88
+89,a-89,{"k1":"k1-89", "k2":"k2-89"},89
+90,a-90,{"k1":"k1-90", "k2":"k2-90"},90
+91,a-91,{"k1":"k1-91", "k2":"k2-91"},91
+92,a-92,{"k1":"k1-92", "k2":"k2-92"},92
+93,a-93,{"k1":"k1-93", "k2":"k2-93"},93
+94,a-94,{"k1":"k1-94", "k2":"k2-94"},94
+95,a-95,{"k1":"k1-95", "k2":"k2-95"},95
+96,a-96,{"k1":"k1-96", "k2":"k2-96"},96
+97,a-97,{"k1":"k1-97", "k2":"k2-97"},97
+98,a-98,{"k1":"k1-98", "k2":"k2-98"},98
+99,a-99,{"k1":"k1-99", "k2":"k2-99"},99
\ No newline at end of file
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testCATS.result b/tajo-core/src/test/resources/results/TestHBaseTable/testCATS.result
new file mode 100644
index 0000000..72013f2
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testCATS.result
@@ -0,0 +1,100 @@
+000, value0
+001, value1
+002, value2
+003, value3
+004, value4
+005, value5
+006, value6
+007, value7
+008, value8
+009, value9
+010, value10
+011, value11
+012, value12
+013, value13
+014, value14
+015, value15
+016, value16
+017, value17
+018, value18
+019, value19
+020, value20
+021, value21
+022, value22
+023, value23
+024, value24
+025, value25
+026, value26
+027, value27
+028, value28
+029, value29
+030, value30
+031, value31
+032, value32
+033, value33
+034, value34
+035, value35
+036, value36
+037, value37
+038, value38
+039, value39
+040, value40
+041, value41
+042, value42
+043, value43
+044, value44
+045, value45
+046, value46
+047, value47
+048, value48
+049, value49
+050, value50
+051, value51
+052, value52
+053, value53
+054, value54
+055, value55
+056, value56
+057, value57
+058, value58
+059, value59
+060, value60
+061, value61
+062, value62
+063, value63
+064, value64
+065, value65
+066, value66
+067, value67
+068, value68
+069, value69
+070, value70
+071, value71
+072, value72
+073, value73
+074, value74
+075, value75
+076, value76
+077, value77
+078, value78
+079, value79
+080, value80
+081, value81
+082, value82
+083, value83
+084, value84
+085, value85
+086, value86
+087, value87
+088, value88
+089, value89
+090, value90
+091, value91
+092, value92
+093, value93
+094, value94
+095, value95
+096, value96
+097, value97
+098, value98
+099, value99
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testColumnKeyValueSelectQuery.result b/tajo-core/src/test/resources/results/TestHBaseTable/testColumnKeyValueSelectQuery.result
new file mode 100644
index 0000000..82d4fd2
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testColumnKeyValueSelectQuery.result
@@ -0,0 +1,12 @@
+rk1,col2_key,col2_value,col3
+-------------------------------
+rk-0,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-0
+rk-1,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-1
+rk-2,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-2
+rk-3,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-3
+rk-4,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-4
+rk-5,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-5
+rk-6,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-6
+rk-7,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-7
+rk-8,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-8
+rk-9,["key-0", "key-1", "key-2", "key-3", "key-4"],["value-0", "value-1", "value-2", "value-3", "value-4"],col3-value-9
\ No newline at end of file
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testIndexPredication.result b/tajo-core/src/test/resources/results/TestHBaseTable/testIndexPredication.result
new file mode 100644
index 0000000..f38d238
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testIndexPredication.result
@@ -0,0 +1,38 @@
+rk,col1,col2,col3
+-------------------------------
+020,a-20,{"k1":"k1-20", "k2":"k2-20"},b-20
+021,a-21,{"k1":"k1-21", "k2":"k2-21"},b-21
+022,a-22,{"k1":"k1-22", "k2":"k2-22"},b-22
+023,a-23,{"k1":"k1-23", "k2":"k2-23"},b-23
+024,a-24,{"k1":"k1-24", "k2":"k2-24"},b-24
+025,a-25,{"k1":"k1-25", "k2":"k2-25"},b-25
+026,a-26,{"k1":"k1-26", "k2":"k2-26"},b-26
+027,a-27,{"k1":"k1-27", "k2":"k2-27"},b-27
+028,a-28,{"k1":"k1-28", "k2":"k2-28"},b-28
+029,a-29,{"k1":"k1-29", "k2":"k2-29"},b-29
+030,a-30,{"k1":"k1-30", "k2":"k2-30"},b-30
+031,a-31,{"k1":"k1-31", "k2":"k2-31"},b-31
+032,a-32,{"k1":"k1-32", "k2":"k2-32"},b-32
+033,a-33,{"k1":"k1-33", "k2":"k2-33"},b-33
+034,a-34,{"k1":"k1-34", "k2":"k2-34"},b-34
+035,a-35,{"k1":"k1-35", "k2":"k2-35"},b-35
+036,a-36,{"k1":"k1-36", "k2":"k2-36"},b-36
+037,a-37,{"k1":"k1-37", "k2":"k2-37"},b-37
+038,a-38,{"k1":"k1-38", "k2":"k2-38"},b-38
+039,a-39,{"k1":"k1-39", "k2":"k2-39"},b-39
+040,a-40,{"k1":"k1-40", "k2":"k2-40"},b-40
+041,a-41,{"k1":"k1-41", "k2":"k2-41"},b-41
+042,a-42,{"k1":"k1-42", "k2":"k2-42"},b-42
+043,a-43,{"k1":"k1-43", "k2":"k2-43"},b-43
+044,a-44,{"k1":"k1-44", "k2":"k2-44"},b-44
+045,a-45,{"k1":"k1-45", "k2":"k2-45"},b-45
+046,a-46,{"k1":"k1-46", "k2":"k2-46"},b-46
+047,a-47,{"k1":"k1-47", "k2":"k2-47"},b-47
+048,a-48,{"k1":"k1-48", "k2":"k2-48"},b-48
+049,a-49,{"k1":"k1-49", "k2":"k2-49"},b-49
+050,a-50,{"k1":"k1-50", "k2":"k2-50"},b-50
+051,a-51,{"k1":"k1-51", "k2":"k2-51"},b-51
+052,a-52,{"k1":"k1-52", "k2":"k2-52"},b-52
+053,a-53,{"k1":"k1-53", "k2":"k2-53"},b-53
+054,a-54,{"k1":"k1-54", "k2":"k2-54"},b-54
+055,a-55,{"k1":"k1-55", "k2":"k2-55"},b-55
\ No newline at end of file
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertInto.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertInto.result
new file mode 100644
index 0000000..e0c97ef
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertInto.result
@@ -0,0 +1,3 @@
+1, 1996-03-13, {"": "N"}, 7706
+2, 1997-01-28, {"": "N"}, 1191
+3, 1994-02-02, {"": "R"}, 1798
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoBinaryMultiRegion.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoBinaryMultiRegion.result
new file mode 100644
index 0000000..c55873a
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoBinaryMultiRegion.result
@@ -0,0 +1,100 @@
+0, value0
+1, value1
+2, value2
+3, value3
+4, value4
+5, value5
+6, value6
+7, value7
+8, value8
+9, value9
+1, value10
+1, value11
+1, value12
+1, value13
+1, value14
+1, value15
+1, value16
+1, value17
+1, value18
+1, value19
+2, value20
+2, value21
+2, value22
+2, value23
+2, value24
+2, value25
+2, value26
+2, value27
+2, value28
+2, value29
+3, value30
+3, value31
+3, value32
+3, value33
+3, value34
+3, value35
+3, value36
+3, value37
+3, value38
+3, value39
+4, value40
+4, value41
+4, value42
+4, value43
+4, value44
+4, value45
+4, value46
+4, value47
+4, value48
+4, value49
+5, value50
+5, value51
+5, value52
+5, value53
+5, value54
+5, value55
+5, value56
+5, value57
+5, value58
+5, value59
+6, value60
+6, value61
+6, value62
+6, value63
+6, value64
+6, value65
+6, value66
+6, value67
+6, value68
+6, value69
+7, value70
+7, value71
+7, value72
+7, value73
+7, value74
+7, value75
+7, value76
+7, value77
+7, value78
+7, value79
+8, value80
+8, value81
+8, value82
+8, value83
+8, value84
+8, value85
+8, value86
+8, value87
+8, value88
+8, value89
+9, value90
+9, value91
+9, value92
+9, value93
+9, value94
+9, value95
+9, value96
+9, value97
+9, value98
+9, value99
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoColumnKeyValue.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoColumnKeyValue.result
new file mode 100644
index 0000000..a37593b
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoColumnKeyValue.result
@@ -0,0 +1,21 @@
+0, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-0"}
+1, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-1"}
+10, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-10"}
+11, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-11"}
+12, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-12"}
+13, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-13"}
+14, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-14"}
+15, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-15"}
+16, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-16"}
+17, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-17"}
+18, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-18"}
+19, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-19"}
+2, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-2"}
+20, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-20"}
+3, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-3"}
+4, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-4"}
+5, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-5"}
+6, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-6"}
+7, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-7"}
+8, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-8"}
+9, {"ck-0": "value-0", "ck-1": "value-1", "ck-2": "value-2"}, {"": "col3-9"}
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegion.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegion.result
new file mode 100644
index 0000000..72013f2
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegion.result
@@ -0,0 +1,100 @@
+000, value0
+001, value1
+002, value2
+003, value3
+004, value4
+005, value5
+006, value6
+007, value7
+008, value8
+009, value9
+010, value10
+011, value11
+012, value12
+013, value13
+014, value14
+015, value15
+016, value16
+017, value17
+018, value18
+019, value19
+020, value20
+021, value21
+022, value22
+023, value23
+024, value24
+025, value25
+026, value26
+027, value27
+028, value28
+029, value29
+030, value30
+031, value31
+032, value32
+033, value33
+034, value34
+035, value35
+036, value36
+037, value37
+038, value38
+039, value39
+040, value40
+041, value41
+042, value42
+043, value43
+044, value44
+045, value45
+046, value46
+047, value47
+048, value48
+049, value49
+050, value50
+051, value51
+052, value52
+053, value53
+054, value54
+055, value55
+056, value56
+057, value57
+058, value58
+059, value59
+060, value60
+061, value61
+062, value62
+063, value63
+064, value64
+065, value65
+066, value66
+067, value67
+068, value68
+069, value69
+070, value70
+071, value71
+072, value72
+073, value73
+074, value74
+075, value75
+076, value76
+077, value77
+078, value78
+079, value79
+080, value80
+081, value81
+082, value82
+083, value83
+084, value84
+085, value85
+086, value86
+087, value87
+088, value88
+089, value89
+090, value90
+091, value91
+092, value92
+093, value93
+094, value94
+095, value95
+096, value96
+097, value97
+098, value98
+099, value99
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegion2.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegion2.result
new file mode 100644
index 0000000..be11042
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegion2.result
@@ -0,0 +1,100 @@
+0, value0
+1, value1
+10, value10
+11, value11
+12, value12
+13, value13
+14, value14
+15, value15
+16, value16
+17, value17
+18, value18
+19, value19
+2, value2
+20, value20
+21, value21
+22, value22
+23, value23
+24, value24
+25, value25
+26, value26
+27, value27
+28, value28
+29, value29
+3, value3
+30, value30
+31, value31
+32, value32
+33, value33
+34, value34
+35, value35
+36, value36
+37, value37
+38, value38
+39, value39
+4, value4
+40, value40
+41, value41
+42, value42
+43, value43
+44, value44
+45, value45
+46, value46
+47, value47
+48, value48
+49, value49
+5, value5
+50, value50
+51, value51
+52, value52
+53, value53
+54, value54
+55, value55
+56, value56
+57, value57
+58, value58
+59, value59
+6, value6
+60, value60
+61, value61
+62, value62
+63, value63
+64, value64
+65, value65
+66, value66
+67, value67
+68, value68
+69, value69
+7, value7
+70, value70
+71, value71
+72, value72
+73, value73
+74, value74
+75, value75
+76, value76
+77, value77
+78, value78
+79, value79
+8, value8
+80, value80
+81, value81
+82, value82
+83, value83
+84, value84
+85, value85
+86, value86
+87, value87
+88, value88
+89, value89
+9, value9
+90, value90
+91, value91
+92, value92
+93, value93
+94, value94
+95, value95
+96, value96
+97, value97
+98, value98
+99, value99
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegionMultiRowFields.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegionMultiRowFields.result
new file mode 100644
index 0000000..0b740c8
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegionMultiRowFields.result
@@ -0,0 +1,100 @@
+000_100, value0
+001_101, value1
+002_102, value2
+003_103, value3
+004_104, value4
+005_105, value5
+006_106, value6
+007_107, value7
+008_108, value8
+009_109, value9
+010_110, value10
+011_111, value11
+012_112, value12
+013_113, value13
+014_114, value14
+015_115, value15
+016_116, value16
+017_117, value17
+018_118, value18
+019_119, value19
+020_120, value20
+021_121, value21
+022_122, value22
+023_123, value23
+024_124, value24
+025_125, value25
+026_126, value26
+027_127, value27
+028_128, value28
+029_129, value29
+030_130, value30
+031_131, value31
+032_132, value32
+033_133, value33
+034_134, value34
+035_135, value35
+036_136, value36
+037_137, value37
+038_138, value38
+039_139, value39
+040_140, value40
+041_141, value41
+042_142, value42
+043_143, value43
+044_144, value44
+045_145, value45
+046_146, value46
+047_147, value47
+048_148, value48
+049_149, value49
+050_150, value50
+051_151, value51
+052_152, value52
+053_153, value53
+054_154, value54
+055_155, value55
+056_156, value56
+057_157, value57
+058_158, value58
+059_159, value59
+060_160, value60
+061_161, value61
+062_162, value62
+063_163, value63
+064_164, value64
+065_165, value65
+066_166, value66
+067_167, value67
+068_168, value68
+069_169, value69
+070_170, value70
+071_171, value71
+072_172, value72
+073_173, value73
+074_174, value74
+075_175, value75
+076_176, value76
+077_177, value77
+078_178, value78
+079_179, value79
+080_180, value80
+081_181, value81
+082_182, value82
+083_183, value83
+084_184, value84
+085_185, value85
+086_186, value86
+087_187, value87
+088_188, value88
+089_189, value89
+090_190, value90
+091_191, value91
+092_192, value92
+093_193, value93
+094_194, value94
+095_195, value95
+096_196, value96
+097_197, value97
+098_198, value98
+099_199, value99
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegionWithSplitFile.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegionWithSplitFile.result
new file mode 100644
index 0000000..72013f2
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoMultiRegionWithSplitFile.result
@@ -0,0 +1,100 @@
+000, value0
+001, value1
+002, value2
+003, value3
+004, value4
+005, value5
+006, value6
+007, value7
+008, value8
+009, value9
+010, value10
+011, value11
+012, value12
+013, value13
+014, value14
+015, value15
+016, value16
+017, value17
+018, value18
+019, value19
+020, value20
+021, value21
+022, value22
+023, value23
+024, value24
+025, value25
+026, value26
+027, value27
+028, value28
+029, value29
+030, value30
+031, value31
+032, value32
+033, value33
+034, value34
+035, value35
+036, value36
+037, value37
+038, value38
+039, value39
+040, value40
+041, value41
+042, value42
+043, value43
+044, value44
+045, value45
+046, value46
+047, value47
+048, value48
+049, value49
+050, value50
+051, value51
+052, value52
+053, value53
+054, value54
+055, value55
+056, value56
+057, value57
+058, value58
+059, value59
+060, value60
+061, value61
+062, value62
+063, value63
+064, value64
+065, value65
+066, value66
+067, value67
+068, value68
+069, value69
+070, value70
+071, value71
+072, value72
+073, value73
+074, value74
+075, value75
+076, value76
+077, value77
+078, value78
+079, value79
+080, value80
+081, value81
+082, value82
+083, value83
+084, value84
+085, value85
+086, value86
+087, value87
+088, value88
+089, value89
+090, value90
+091, value91
+092, value92
+093, value93
+094, value94
+095, value95
+096, value96
+097, value97
+098, value98
+099, value99
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoRowField.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoRowField.result
new file mode 100644
index 0000000..152b91b
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoRowField.result
@@ -0,0 +1,4 @@
+1_1, 1996-03-13, N, 7706
+2_2, 1997-01-28, N, 1191
+3_2, 1994-02-02, R, 1798
+3_3, 1993-11-09, R, 6540
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result
new file mode 100644
index 0000000..1c22960
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result
@@ -0,0 +1,3 @@
+1, 1996-04-12, {"": "N"}, 7311
+2, 1997-01-28, {"": "N"}, 1191
+3, 1993-11-09, {"": "R"}, 6540
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testJoin.result b/tajo-core/src/test/resources/results/TestHBaseTable/testJoin.result
new file mode 100644
index 0000000..fe4fcae
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testJoin.result
@@ -0,0 +1,7 @@
+rk,col1,col2,col3,l_orderkey,l_linestatus
+-------------------------------
+001,a-1,{"k1":"k1-1", "k2":"k2-1"},1,1,O
+001,a-1,{"k1":"k1-1", "k2":"k2-1"},1,1,O
+002,a-2,{"k1":"k1-2", "k2":"k2-2"},2,2,O
+003,a-3,{"k1":"k1-3", "k2":"k2-3"},3,3,F
+003,a-3,{"k1":"k1-3", "k2":"k2-3"},3,3,F
\ No newline at end of file
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testNonForwardQuery.result b/tajo-core/src/test/resources/results/TestHBaseTable/testNonForwardQuery.result
new file mode 100644
index 0000000..4f5fd8f
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testNonForwardQuery.result
@@ -0,0 +1,102 @@
+rk,col1,col2,col3
+-------------------------------
+000,a-0,{"k1":"k1-0", "k2":"k2-0"},0
+001,a-1,{"k1":"k1-1", "k2":"k2-1"},1
+002,a-2,{"k1":"k1-2", "k2":"k2-2"},2
+003,a-3,{"k1":"k1-3", "k2":"k2-3"},3
+004,a-4,{"k1":"k1-4", "k2":"k2-4"},4
+005,a-5,{"k1":"k1-5", "k2":"k2-5"},5
+006,a-6,{"k1":"k1-6", "k2":"k2-6"},6
+007,a-7,{"k1":"k1-7", "k2":"k2-7"},7
+008,a-8,{"k1":"k1-8", "k2":"k2-8"},8
+009,a-9,{"k1":"k1-9", "k2":"k2-9"},9
+010,a-10,{"k1":"k1-10", "k2":"k2-10"},10
+011,a-11,{"k1":"k1-11", "k2":"k2-11"},11
+012,a-12,{"k1":"k1-12", "k2":"k2-12"},12
+013,a-13,{"k1":"k1-13", "k2":"k2-13"},13
+014,a-14,{"k1":"k1-14", "k2":"k2-14"},14
+015,a-15,{"k1":"k1-15", "k2":"k2-15"},15
+016,a-16,{"k1":"k1-16", "k2":"k2-16"},16
+017,a-17,{"k1":"k1-17", "k2":"k2-17"},17
+018,a-18,{"k1":"k1-18", "k2":"k2-18"},18
+019,a-19,{"k1":"k1-19", "k2":"k2-19"},19
+020,a-20,{"k1":"k1-20", "k2":"k2-20"},20
+021,a-21,{"k1":"k1-21", "k2":"k2-21"},21
+022,a-22,{"k1":"k1-22", "k2":"k2-22"},22
+023,a-23,{"k1":"k1-23", "k2":"k2-23"},23
+024,a-24,{"k1":"k1-24", "k2":"k2-24"},24
+025,a-25,{"k1":"k1-25", "k2":"k2-25"},25
+026,a-26,{"k1":"k1-26", "k2":"k2-26"},26
+027,a-27,{"k1":"k1-27", "k2":"k2-27"},27
+028,a-28,{"k1":"k1-28", "k2":"k2-28"},28
+029,a-29,{"k1":"k1-29", "k2":"k2-29"},29
+030,a-30,{"k1":"k1-30", "k2":"k2-30"},30
+031,a-31,{"k1":"k1-31", "k2":"k2-31"},31
+032,a-32,{"k1":"k1-32", "k2":"k2-32"},32
+033,a-33,{"k1":"k1-33", "k2":"k2-33"},33
+034,a-34,{"k1":"k1-34", "k2":"k2-34"},34
+035,a-35,{"k1":"k1-35", "k2":"k2-35"},35
+036,a-36,{"k1":"k1-36", "k2":"k2-36"},36
+037,a-37,{"k1":"k1-37", "k2":"k2-37"},37
+038,a-38,{"k1":"k1-38", "k2":"k2-38"},38
+039,a-39,{"k1":"k1-39", "k2":"k2-39"},39
+040,a-40,{"k1":"k1-40", "k2":"k2-40"},40
+041,a-41,{"k1":"k1-41", "k2":"k2-41"},41
+042,a-42,{"k1":"k1-42", "k2":"k2-42"},42
+043,a-43,{"k1":"k1-43", "k2":"k2-43"},43
+044,a-44,{"k1":"k1-44", "k2":"k2-44"},44
+045,a-45,{"k1":"k1-45", "k2":"k2-45"},45
+046,a-46,{"k1":"k1-46", "k2":"k2-46"},46
+047,a-47,{"k1":"k1-47", "k2":"k2-47"},47
+048,a-48,{"k1":"k1-48", "k2":"k2-48"},48
+049,a-49,{"k1":"k1-49", "k2":"k2-49"},49
+050,a-50,{"k1":"k1-50", "k2":"k2-50"},50
+051,a-51,{"k1":"k1-51", "k2":"k2-51"},51
+052,a-52,{"k1":"k1-52", "k2":"k2-52"},52
+053,a-53,{"k1":"k1-53", "k2":"k2-53"},53
+054,a-54,{"k1":"k1-54", "k2":"k2-54"},54
+055,a-55,{"k1":"k1-55", "k2":"k2-55"},55
+056,a-56,{"k1":"k1-56", "k2":"k2-56"},56
+057,a-57,{"k1":"k1-57", "k2":"k2-57"},57
+058,a-58,{"k1":"k1-58", "k2":"k2-58"},58
+059,a-59,{"k1":"k1-59", "k2":"k2-59"},59
+060,a-60,{"k1":"k1-60", "k2":"k2-60"},60
+061,a-61,{"k1":"k1-61", "k2":"k2-61"},61
+062,a-62,{"k1":"k1-62", "k2":"k2-62"},62
+063,a-63,{"k1":"k1-63", "k2":"k2-63"},63
+064,a-64,{"k1":"k1-64", "k2":"k2-64"},64
+065,a-65,{"k1":"k1-65", "k2":"k2-65"},65
+066,a-66,{"k1":"k1-66", "k2":"k2-66"},66
+067,a-67,{"k1":"k1-67", "k2":"k2-67"},67
+068,a-68,{"k1":"k1-68", "k2":"k2-68"},68
+069,a-69,{"k1":"k1-69", "k2":"k2-69"},69
+070,a-70,{"k1":"k1-70", "k2":"k2-70"},70
+071,a-71,{"k1":"k1-71", "k2":"k2-71"},71
+072,a-72,{"k1":"k1-72", "k2":"k2-72"},72
+073,a-73,{"k1":"k1-73", "k2":"k2-73"},73
+074,a-74,{"k1":"k1-74", "k2":"k2-74"},74
+075,a-75,{"k1":"k1-75", "k2":"k2-75"},75
+076,a-76,{"k1":"k1-76", "k2":"k2-76"},76
+077,a-77,{"k1":"k1-77", "k2":"k2-77"},77
+078,a-78,{"k1":"k1-78", "k2":"k2-78"},78
+079,a-79,{"k1":"k1-79", "k2":"k2-79"},79
+080,a-80,{"k1":"k1-80", "k2":"k2-80"},80
+081,a-81,{"k1":"k1-81", "k2":"k2-81"},81
+082,a-82,{"k1":"k1-82", "k2":"k2-82"},82
+083,a-83,{"k1":"k1-83", "k2":"k2-83"},83
+084,a-84,{"k1":"k1-84", "k2":"k2-84"},84
+085,a-85,{"k1":"k1-85", "k2":"k2-85"},85
+086,a-86,{"k1":"k1-86", "k2":"k2-86"},86
+087,a-87,{"k1":"k1-87", "k2":"k2-87"},87
+088,a-88,{"k1":"k1-88", "k2":"k2-88"},88
+089,a-89,{"k1":"k1-89", "k2":"k2-89"},89
+090,a-90,{"k1":"k1-90", "k2":"k2-90"},90
+091,a-91,{"k1":"k1-91", "k2":"k2-91"},91
+092,a-92,{"k1":"k1-92", "k2":"k2-92"},92
+093,a-93,{"k1":"k1-93", "k2":"k2-93"},93
+094,a-94,{"k1":"k1-94", "k2":"k2-94"},94
+095,a-95,{"k1":"k1-95", "k2":"k2-95"},95
+096,a-96,{"k1":"k1-96", "k2":"k2-96"},96
+097,a-97,{"k1":"k1-97", "k2":"k2-97"},97
+098,a-98,{"k1":"k1-98", "k2":"k2-98"},98
+099,a-99,{"k1":"k1-99", "k2":"k2-99"},99
\ No newline at end of file
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testRowFieldSelectQuery.result b/tajo-core/src/test/resources/results/TestHBaseTable/testRowFieldSelectQuery.result
new file mode 100644
index 0000000..34369df
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testRowFieldSelectQuery.result
@@ -0,0 +1,88 @@
+rk1,rk2,col3
+-------------------------------
+field1-21,field2-21,a-21
+field1-22,field2-22,a-22
+field1-23,field2-23,a-23
+field1-24,field2-24,a-24
+field1-25,field2-25,a-25
+field1-26,field2-26,a-26
+field1-27,field2-27,a-27
+field1-28,field2-28,a-28
+field1-29,field2-29,a-29
+field1-30,field2-30,a-30
+field1-31,field2-31,a-31
+field1-32,field2-32,a-32
+field1-33,field2-33,a-33
+field1-34,field2-34,a-34
+field1-35,field2-35,a-35
+field1-36,field2-36,a-36
+field1-37,field2-37,a-37
+field1-38,field2-38,a-38
+field1-39,field2-39,a-39
+field1-3,field2-3,a-3
+field1-40,field2-40,a-40
+field1-41,field2-41,a-41
+field1-42,field2-42,a-42
+field1-43,field2-43,a-43
+field1-44,field2-44,a-44
+field1-45,field2-45,a-45
+field1-46,field2-46,a-46
+field1-47,field2-47,a-47
+field1-48,field2-48,a-48
+field1-49,field2-49,a-49
+field1-4,field2-4,a-4
+field1-50,field2-50,a-50
+field1-51,field2-51,a-51
+field1-52,field2-52,a-52
+field1-53,field2-53,a-53
+field1-54,field2-54,a-54
+field1-55,field2-55,a-55
+field1-56,field2-56,a-56
+field1-57,field2-57,a-57
+field1-58,field2-58,a-58
+field1-59,field2-59,a-59
+field1-5,field2-5,a-5
+field1-60,field2-60,a-60
+field1-61,field2-61,a-61
+field1-62,field2-62,a-62
+field1-63,field2-63,a-63
+field1-64,field2-64,a-64
+field1-65,field2-65,a-65
+field1-66,field2-66,a-66
+field1-67,field2-67,a-67
+field1-68,field2-68,a-68
+field1-69,field2-69,a-69
+field1-6,field2-6,a-6
+field1-70,field2-70,a-70
+field1-71,field2-71,a-71
+field1-72,field2-72,a-72
+field1-73,field2-73,a-73
+field1-74,field2-74,a-74
+field1-75,field2-75,a-75
+field1-76,field2-76,a-76
+field1-77,field2-77,a-77
+field1-78,field2-78,a-78
+field1-79,field2-79,a-79
+field1-7,field2-7,a-7
+field1-80,field2-80,a-80
+field1-81,field2-81,a-81
+field1-82,field2-82,a-82
+field1-83,field2-83,a-83
+field1-84,field2-84,a-84
+field1-85,field2-85,a-85
+field1-86,field2-86,a-86
+field1-87,field2-87,a-87
+field1-88,field2-88,a-88
+field1-89,field2-89,a-89
+field1-8,field2-8,a-8
+field1-90,field2-90,a-90
+field1-91,field2-91,a-91
+field1-92,field2-92,a-92
+field1-93,field2-93,a-93
+field1-94,field2-94,a-94
+field1-95,field2-95,a-95
+field1-96,field2-96,a-96
+field1-97,field2-97,a-97
+field1-98,field2-98,a-98
+field1-99,field2-99,a-99
+field1-9,field2-9,a-9
\ No newline at end of file
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testSimpleSelectQuery.result b/tajo-core/src/test/resources/results/TestHBaseTable/testSimpleSelectQuery.result
new file mode 100644
index 0000000..ef2f27a
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testSimpleSelectQuery.result
@@ -0,0 +1,88 @@
+rk,col1,col2,col3
+-------------------------------
+21,a-21,{"k1":"k1-21", "k2":"k2-21"},b-21
+22,a-22,{"k1":"k1-22", "k2":"k2-22"},b-22
+23,a-23,{"k1":"k1-23", "k2":"k2-23"},b-23
+24,a-24,{"k1":"k1-24", "k2":"k2-24"},b-24
+25,a-25,{"k1":"k1-25", "k2":"k2-25"},b-25
+26,a-26,{"k1":"k1-26", "k2":"k2-26"},b-26
+27,a-27,{"k1":"k1-27", "k2":"k2-27"},b-27
+28,a-28,{"k1":"k1-28", "k2":"k2-28"},b-28
+29,a-29,{"k1":"k1-29", "k2":"k2-29"},b-29
+3,a-3,{"k1":"k1-3", "k2":"k2-3"},b-3
+30,a-30,{"k1":"k1-30", "k2":"k2-30"},b-30
+31,a-31,{"k1":"k1-31", "k2":"k2-31"},b-31
+32,a-32,{"k1":"k1-32", "k2":"k2-32"},b-32
+33,a-33,{"k1":"k1-33", "k2":"k2-33"},b-33
+34,a-34,{"k1":"k1-34", "k2":"k2-34"},b-34
+35,a-35,{"k1":"k1-35", "k2":"k2-35"},b-35
+36,a-36,{"k1":"k1-36", "k2":"k2-36"},b-36
+37,a-37,{"k1":"k1-37", "k2":"k2-37"},b-37
+38,a-38,{"k1":"k1-38", "k2":"k2-38"},b-38
+39,a-39,{"k1":"k1-39", "k2":"k2-39"},b-39
+4,a-4,{"k1":"k1-4", "k2":"k2-4"},b-4
+40,a-40,{"k1":"k1-40", "k2":"k2-40"},b-40
+41,a-41,{"k1":"k1-41", "k2":"k2-41"},b-41
+42,a-42,{"k1":"k1-42", "k2":"k2-42"},b-42
+43,a-43,{"k1":"k1-43", "k2":"k2-43"},b-43
+44,a-44,{"k1":"k1-44", "k2":"k2-44"},b-44
+45,a-45,{"k1":"k1-45", "k2":"k2-45"},b-45
+46,a-46,{"k1":"k1-46", "k2":"k2-46"},b-46
+47,a-47,{"k1":"k1-47", "k2":"k2-47"},b-47
+48,a-48,{"k1":"k1-48", "k2":"k2-48"},b-48
+49,a-49,{"k1":"k1-49", "k2":"k2-49"},b-49
+5,a-5,{"k1":"k1-5", "k2":"k2-5"},b-5
+50,a-50,{"k1":"k1-50", "k2":"k2-50"},b-50
+51,a-51,{"k1":"k1-51", "k2":"k2-51"},b-51
+52,a-52,{"k1":"k1-52", "k2":"k2-52"},b-52
+53,a-53,{"k1":"k1-53", "k2":"k2-53"},b-53
+54,a-54,{"k1":"k1-54", "k2":"k2-54"},b-54
+55,a-55,{"k1":"k1-55", "k2":"k2-55"},b-55
+56,a-56,{"k1":"k1-56", "k2":"k2-56"},b-56
+57,a-57,{"k1":"k1-57", "k2":"k2-57"},b-57
+58,a-58,{"k1":"k1-58", "k2":"k2-58"},b-58
+59,a-59,{"k1":"k1-59", "k2":"k2-59"},b-59
+6,a-6,{"k1":"k1-6", "k2":"k2-6"},b-6
+60,a-60,{"k1":"k1-60", "k2":"k2-60"},b-60
+61,a-61,{"k1":"k1-61", "k2":"k2-61"},b-61
+62,a-62,{"k1":"k1-62", "k2":"k2-62"},b-62
+63,a-63,{"k1":"k1-63", "k2":"k2-63"},b-63
+64,a-64,{"k1":"k1-64", "k2":"k2-64"},b-64
+65,a-65,{"k1":"k1-65", "k2":"k2-65"},b-65
+66,a-66,{"k1":"k1-66", "k2":"k2-66"},b-66
+67,a-67,{"k1":"k1-67", "k2":"k2-67"},b-67
+68,a-68,{"k1":"k1-68", "k2":"k2-68"},b-68
+69,a-69,{"k1":"k1-69", "k2":"k2-69"},b-69
+7,a-7,{"k1":"k1-7", "k2":"k2-7"},b-7
+70,a-70,{"k1":"k1-70", "k2":"k2-70"},b-70
+71,a-71,{"k1":"k1-71", "k2":"k2-71"},b-71
+72,a-72,{"k1":"k1-72", "k2":"k2-72"},b-72
+73,a-73,{"k1":"k1-73", "k2":"k2-73"},b-73
+74,a-74,{"k1":"k1-74", "k2":"k2-74"},b-74
+75,a-75,{"k1":"k1-75", "k2":"k2-75"},b-75
+76,a-76,{"k1":"k1-76", "k2":"k2-76"},b-76
+77,a-77,{"k1":"k1-77", "k2":"k2-77"},b-77
+78,a-78,{"k1":"k1-78", "k2":"k2-78"},b-78
+79,a-79,{"k1":"k1-79", "k2":"k2-79"},b-79
+8,a-8,{"k1":"k1-8", "k2":"k2-8"},b-8
+80,a-80,{"k1":"k1-80", "k2":"k2-80"},b-80
+81,a-81,{"k1":"k1-81", "k2":"k2-81"},b-81
+82,a-82,{"k1":"k1-82", "k2":"k2-82"},b-82
+83,a-83,{"k1":"k1-83", "k2":"k2-83"},b-83
+84,a-84,{"k1":"k1-84", "k2":"k2-84"},b-84
+85,a-85,{"k1":"k1-85", "k2":"k2-85"},b-85
+86,a-86,{"k1":"k1-86", "k2":"k2-86"},b-86
+87,a-87,{"k1":"k1-87", "k2":"k2-87"},b-87
+88,a-88,{"k1":"k1-88", "k2":"k2-88"},b-88
+89,a-89,{"k1":"k1-89", "k2":"k2-89"},b-89
+9,a-9,{"k1":"k1-9", "k2":"k2-9"},b-9
+90,a-90,{"k1":"k1-90", "k2":"k2-90"},b-90
+91,a-91,{"k1":"k1-91", "k2":"k2-91"},b-91
+92,a-92,{"k1":"k1-92", "k2":"k2-92"},b-92
+93,a-93,{"k1":"k1-93", "k2":"k2-93"},b-93
+94,a-94,{"k1":"k1-94", "k2":"k2-94"},b-94
+95,a-95,{"k1":"k1-95", "k2":"k2-95"},b-95
+96,a-96,{"k1":"k1-96", "k2":"k2-96"},b-96
+97,a-97,{"k1":"k1-97", "k2":"k2-97"},b-97
+98,a-98,{"k1":"k1-98", "k2":"k2-98"},b-98
+99,a-99,{"k1":"k1-99", "k2":"k2-99"},b-99
\ No newline at end of file
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index d350889..eb8ada9 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -75,7 +75,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
       <scope>provided</scope>
     </dependency>
   </dependencies>
diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo
index d4e99f9..8a9c17a 100755
--- a/tajo-dist/src/main/bin/tajo
+++ b/tajo-dist/src/main/bin/tajo
@@ -226,6 +226,21 @@
 ##############################################################################
 
 ##############################################################################
+# Set HBase CLASSPATH
+##############################################################################
+if [ "$HBASE_HOME" != "" ]; then
+  for f in ${HBASE_HOME}/lib/hbase-*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+  for f in ${HBASE_HOME}/lib/htrace-*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+fi
+##############################################################################
+# HBase Configuration End
+##############################################################################
+
+##############################################################################
 # Find and Set Hive CLASSPATH
 ##############################################################################
 
diff --git a/tajo-dist/src/main/conf/tajo-env.sh b/tajo-dist/src/main/conf/tajo-env.sh
index 92d00bd..064a82f 100755
--- a/tajo-dist/src/main/conf/tajo-env.sh
+++ b/tajo-dist/src/main/conf/tajo-env.sh
@@ -24,6 +24,9 @@
 # Hadoop home. Required
 # export HADOOP_HOME=
 
+# HBase home. optional
+# export HBASE_HOME=
+
 # The java implementation to use.  Required.
 # export JAVA_HOME=/usr/java/default
 
diff --git a/tajo-jdbc/pom.xml b/tajo-jdbc/pom.xml
index 20cdf16..1c3c410 100644
--- a/tajo-jdbc/pom.xml
+++ b/tajo-jdbc/pom.xml
@@ -102,7 +102,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalOptimizer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalOptimizer.java
index bcc3bce..750e64e 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalOptimizer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalOptimizer.java
@@ -44,6 +44,7 @@
 import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor;
 
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Set;
 import java.util.Stack;
 
@@ -86,6 +87,12 @@
     }
   }
 
+  public void addRuleAfterToJoinOpt(RewriteRule rewriteRule) {
+    if (rewriteRule != null) {
+      rulesAfterToJoinOpt.addRewriteRule(rewriteRule);
+    }
+  }
+
   public LogicalNode optimize(LogicalPlan plan) throws PlanningException {
     return optimize(null, plan);
   }
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/Target.java b/tajo-plan/src/main/java/org/apache/tajo/plan/Target.java
index f49a93d..0524240 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/Target.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/Target.java
@@ -21,7 +21,7 @@
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.plan.expr.FieldEval;
 import org.apache.tajo.plan.serder.PlanGsonHelper;
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java
index 4321d02..24101e5 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CaseWhenEval.java
@@ -27,7 +27,7 @@
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.plan.serder.PlanGsonHelper;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.TUtil;
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java
index 638383a..e028481 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java
@@ -22,7 +22,7 @@
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.Datum;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.plan.serder.PlanGsonHelper;
 import org.apache.tajo.storage.Tuple;
 
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
index 006449f..5a83f48 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/GeneralFunction.java
@@ -23,7 +23,7 @@
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.function.Function;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.plan.expr.FunctionEval;
 import org.apache.tajo.storage.Tuple;
 
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java
index 709ef34..55b6871 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/BinaryNode.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.plan.logical;
 
 import com.google.gson.annotations.Expose;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 
 public abstract class BinaryNode extends LogicalNode implements Cloneable, GsonObject {
 	@Expose LogicalNode leftChild = null;
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/InsertNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/InsertNode.java
index df55b63..d1d8582 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/InsertNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/InsertNode.java
@@ -46,7 +46,9 @@
     } else {
       tableSchema = desc.getSchema();
     }
-    setPath(new Path(desc.getPath()));
+    if (desc.getPath() != null) {
+      setPath(new Path(desc.getPath()));
+    }
     setOptions(desc.getMeta().getOptions());
     setStorageType(desc.getMeta().getStoreType());
 
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java
index c42a05e..beb2a8f 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/LogicalNode.java
@@ -23,7 +23,7 @@
 
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.storage.json.GsonObject;
 import org.apache.tajo.plan.PlanString;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.serder.PlanGsonHelper;
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/SortNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/SortNode.java
index a697f9f..2e5e159 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/SortNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/SortNode.java
@@ -25,10 +25,16 @@
 import org.apache.tajo.util.TUtil;
 
 public final class SortNode extends UnaryNode implements Cloneable {
+  public static enum SortPurpose {
+    NORMAL,
+    STORAGE_SPECIFIED
+  }
 	@Expose private SortSpec [] sortKeys;
+  @Expose private SortPurpose sortPurpose;
 
   public SortNode(int pid) {
     super(pid, NodeType.SORT);
+    sortPurpose = SortPurpose.NORMAL;
   }
 
   public void setSortSpecs(SortSpec[] sortSpecs) {
@@ -65,8 +71,7 @@
     PlanString planStr = new PlanString(this);
     StringBuilder sb = new StringBuilder("Sort Keys: ");
     for (int i = 0; i < sortKeys.length; i++) {
-      sb.append(sortKeys[i].getSortKey().getSimpleName()).append(" ")
-          .append(sortKeys[i].isAscending() ? "asc" : "desc");
+      sb.append(sortKeys[i].toString());
       if( i < sortKeys.length - 1) {
         sb.append(",");
       }
@@ -78,8 +83,7 @@
   public String toString() {
     StringBuilder sb = new StringBuilder("Sort [key= ");
     for (int i = 0; i < sortKeys.length; i++) {    
-      sb.append(sortKeys[i].getSortKey().getQualifiedName()).append(" ")
-          .append(sortKeys[i].isAscending() ? "asc" : "desc");
+      sb.append(sortKeys[i].toString());
       if(i < sortKeys.length - 1) {
         sb.append(",");
       }
@@ -91,4 +95,12 @@
     return sb.toString()+"\n"
         + getChild().toString();
   }
+
+  public SortPurpose getSortPurpose() {
+    return sortPurpose;
+  }
+
+  public void setSortPurpose(SortPurpose sortPurpose) {
+    this.sortPurpose = sortPurpose;
+  }
 }
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java
index a7d0ce2..ea58437 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java
@@ -306,7 +306,7 @@
 
       boolean sameVariable =
           EvalTreeUtil.findUniqueColumns(orEval.getLeftExpr())
-          .equals(EvalTreeUtil.findUniqueColumns(orEval.getRightExpr()));
+              .equals(EvalTreeUtil.findUniqueColumns(orEval.getRightExpr()));
 
       return indexable && sameVariable;
     } else {
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java
index 5a75e58..8003af5 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeAdapter.java
@@ -22,8 +22,8 @@
 package org.apache.tajo.plan.serder;
 
 import com.google.gson.*;
-import org.apache.tajo.json.CommonGsonHelper;
-import org.apache.tajo.json.GsonSerDerAdapter;
+import org.apache.tajo.storage.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.GsonSerDerAdapter;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.plan.expr.EvalType;
 
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java
index f86ba5f..20c16bb 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeAdapter.java
@@ -22,10 +22,10 @@
 package org.apache.tajo.plan.serder;
 
 import com.google.gson.*;
-import org.apache.tajo.json.CommonGsonHelper;
+import org.apache.tajo.storage.json.CommonGsonHelper;
 import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.plan.logical.NodeType;
-import org.apache.tajo.json.GsonSerDerAdapter;
+import org.apache.tajo.storage.json.GsonSerDerAdapter;
 
 import java.lang.reflect.Type;
 
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
index 8cafbd0..e0a7ad5 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/PlanGsonHelper.java
@@ -27,7 +27,7 @@
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.function.Function;
-import org.apache.tajo.json.*;
+import org.apache.tajo.storage.json.*;
 import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.plan.function.AggFunction;
 import org.apache.tajo.plan.function.GeneralFunction;
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
index c55c203..4e61de4 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
@@ -25,11 +25,9 @@
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.annotation.Nullable;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SchemaUtil;
-import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.plan.*;
 import org.apache.tajo.plan.expr.*;
@@ -41,6 +39,7 @@
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
 
+import java.io.IOException;
 import java.util.*;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.CSV;
@@ -793,4 +792,109 @@
       }
     }
   }
+
+  public static boolean isFileStorageType(String storageType) {
+    if (storageType.equalsIgnoreCase("hbase")) {
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  public static boolean isFileStorageType(StoreType storageType) {
+    if (storageType== StoreType.HBASE) {
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  public static StoreType getStoreType(LogicalPlan plan) {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+    NodeType nodeType = rootNode.getChild().getType();
+    if (nodeType == NodeType.CREATE_TABLE) {
+      return ((CreateTableNode)rootNode.getChild()).getStorageType();
+    } else if (nodeType == NodeType.INSERT) {
+      return ((InsertNode)rootNode.getChild()).getStorageType();
+    } else {
+      return null;
+    }
+  }
+
+  public static String getStoreTableName(LogicalPlan plan) {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+    NodeType nodeType = rootNode.getChild().getType();
+    if (nodeType == NodeType.CREATE_TABLE) {
+      return ((CreateTableNode)rootNode.getChild()).getTableName();
+    } else if (nodeType == NodeType.INSERT) {
+      return ((InsertNode)rootNode.getChild()).getTableName();
+    } else {
+      return null;
+    }
+  }
+
+  public static TableDesc getTableDesc(CatalogService catalog, LogicalNode node) throws IOException {
+    if (node.getType() == NodeType.CREATE_TABLE) {
+      return createTableDesc((CreateTableNode)node);
+    }
+    String tableName = null;
+    InsertNode insertNode = null;
+    if (node.getType() == NodeType.INSERT) {
+      insertNode = (InsertNode)node;
+      tableName = insertNode.getTableName();
+    } else {
+      return null;
+    }
+
+    if (tableName != null) {
+      String[] tableTokens = tableName.split("\\.");
+      if (tableTokens.length >= 2) {
+        if (catalog.existsTable(tableTokens[0], tableTokens[1])) {
+          return catalog.getTableDesc(tableTokens[0], tableTokens[1]);
+        }
+      }
+    } else {
+      if (insertNode.getPath() != null) {
+        //insert ... location
+        return createTableDesc(insertNode);
+      }
+    }
+    return null;
+  }
+
+  private static TableDesc createTableDesc(CreateTableNode createTableNode) {
+    TableMeta meta = new TableMeta(createTableNode.getStorageType(), createTableNode.getOptions());
+
+    TableDesc tableDescTobeCreated =
+        new TableDesc(
+            createTableNode.getTableName(),
+            createTableNode.getTableSchema(),
+            meta,
+            createTableNode.getPath() != null ? createTableNode.getPath().toUri() : null);
+
+    tableDescTobeCreated.setExternal(createTableNode.isExternal());
+
+    if (createTableNode.hasPartition()) {
+      tableDescTobeCreated.setPartitionMethod(createTableNode.getPartitionMethod());
+    }
+
+    return tableDescTobeCreated;
+  }
+
+  private static TableDesc createTableDesc(InsertNode insertNode) {
+    TableMeta meta = new TableMeta(insertNode.getStorageType(), insertNode.getOptions());
+
+    TableDesc tableDescTobeCreated =
+        new TableDesc(
+            insertNode.getTableName(),
+            insertNode.getTableSchema(),
+            meta,
+            insertNode.getPath() != null ? insertNode.getPath().toUri() : null);
+
+    if (insertNode.hasPartition()) {
+      tableDescTobeCreated.setPartitionMethod(insertNode.getPartitionMethod());
+    }
+
+    return tableDescTobeCreated;
+  }
 }
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index 14a81c3..82ccbdc 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -36,6 +36,7 @@
     <hadoop.version>2.5.1</hadoop.version>
     <protobuf.version>2.5.0</protobuf.version>
     <tajo.version>0.9.1-SNAPSHOT</tajo.version>
+    <hbase.version>0.98.7-hadoop2</hbase.version>
     <tajo.root>${project.parent.relativePath}/..</tajo.root>
     <extra.source.path>src/main/hadoop-${hadoop.version}</extra.source.path>
   </properties>
@@ -743,6 +744,22 @@
         <groupId>org.apache.tajo</groupId>
         <artifactId>tajo-storage</artifactId>
         <version>${tajo.version}</version>
+        <type>pom</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-storage-common</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-storage-hdfs</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-storage-hbase</artifactId>
+        <version>${tajo.version}</version>
       </dependency>
       <dependency>
         <groupId>org.apache.tajo</groupId>
diff --git a/tajo-storage/pom.xml b/tajo-storage/pom.xml
index 7ede2e1..8acb1a9 100644
--- a/tajo-storage/pom.xml
+++ b/tajo-storage/pom.xml
@@ -1,5 +1,5 @@
-<!--
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
   Copyright 2012 Database Lab., Korea Univ.
 
   Licensed under the Apache License, Version 2.0 (the "License");
@@ -16,310 +16,47 @@
   -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
   <parent>
     <artifactId>tajo-project</artifactId>
     <groupId>org.apache.tajo</groupId>
     <version>0.9.1-SNAPSHOT</version>
     <relativePath>../tajo-project</relativePath>
   </parent>
-
+  <modelVersion>4.0.0</modelVersion>
   <artifactId>tajo-storage</artifactId>
-  <packaging>jar</packaging>
+  <packaging>pom</packaging>
   <name>Tajo Storage</name>
-  <description>Tajo Storage Package</description>
-
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-    <parquet.version>1.5.0</parquet.version>
-    <parquet.format.version>2.1.0</parquet.format.version>
   </properties>
 
-  <repositories>
-    <repository>
-      <id>repository.jboss.org</id>
-      <url>https://repository.jboss.org/nexus/content/repositories/releases/
-            </url>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </repository>
-  </repositories>
+  <modules>
+    <module>tajo-storage-common</module>
+    <module>tajo-storage-hdfs</module>
+    <module>tajo-storage-hbase</module>
+  </modules>
 
   <build>
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
-          <encoding>${project.build.sourceEncoding}</encoding>
-        </configuration>
-      </plugin>
-      <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>verify</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-          </execution>
-        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <artifactId>maven-deploy-plugin</artifactId>
         <configuration>
-          <excludes>
-            <exclude>src/test/resources/dataset/**</exclude>
-          </excludes>
+          <skip>true</skip>
         </configuration>
       </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <systemProperties>
-            <tajo.test>TRUE</tajo.test>
-          </systemProperties>
-          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <version>2.4</version>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>create-protobuf-generated-sources-directory</id>
-            <phase>initialize</phase>
-            <configuration>
-              <target>
-                <mkdir dir="target/generated-sources/proto" />
-              </target>
-            </configuration>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <version>1.2</version>
-        <executions>
-          <execution>
-            <id>generate-sources</id>
-            <phase>generate-sources</phase>
-            <configuration>
-              <executable>protoc</executable>
-              <arguments>
-                <argument>-Isrc/main/proto/</argument>
-                <argument>--proto_path=../tajo-common/src/main/proto</argument>
-                <argument>--proto_path=../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
-                <argument>--java_out=target/generated-sources/proto</argument>
-                <argument>src/main/proto/IndexProtos.proto</argument>
-              </arguments>
-            </configuration>
-            <goals>
-              <goal>exec</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>build-helper-maven-plugin</artifactId>
-        <version>1.5</version>
-        <executions>
-          <execution>
-            <id>add-source</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>add-source</goal>
-            </goals>
-            <configuration>
-              <sources>
-                <source>target/generated-sources/proto</source>
-              </sources>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-pmd-plugin</artifactId>
-        <version>2.7.1</version>
-      </plugin>
     </plugins>
   </build>
 
 
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-catalog-common</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-mapred</artifactId>
-      <version>1.7.7</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-core</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>slf4j-api</artifactId>
-          <groupId>org.slf4j</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-json</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <scope>provided</scope>
-      <exclusions>
-      <exclusion>
-        <groupId>commons-el</groupId>
-        <artifactId>commons-el</artifactId>
-      </exclusion>
-      <exclusion>
-        <groupId>tomcat</groupId>
-        <artifactId>jasper-runtime</artifactId>
-      </exclusion>
-      <exclusion>
-        <groupId>tomcat</groupId>
-        <artifactId>jasper-compiler</artifactId>
-      </exclusion>
-      <exclusion>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jsp-2.1-jetty</artifactId>
-      </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.jersey-test-framework</groupId>
-          <artifactId>jersey-test-framework-grizzly2</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-minicluster</artifactId>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>commons-el</groupId>
-          <artifactId>commons-el</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-runtime</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-compiler</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jsp-2.1-jetty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.jersey-test-framework</groupId>
-          <artifactId>jersey-test-framework-grizzly2</artifactId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-server-tests</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-app</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-api</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-hs</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-core</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>parquet-column</artifactId>
-      <version>${parquet.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>parquet-hadoop</artifactId>
-      <version>${parquet.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>parquet-format</artifactId>
-      <version>${parquet.format.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-buffer</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>net.minidev</groupId>
-      <artifactId>json-smart</artifactId>
-      <version>2.0</version>
-    </dependency>
-  </dependencies>
-
   <profiles>
     <profile>
       <id>docs</id>
@@ -364,7 +101,7 @@
             <executions>
               <execution>
                 <id>dist</id>
-                <phase>package</phase>
+                <phase>prepare-package</phase>
                 <goals>
                   <goal>run</goal>
                 </goals>
@@ -387,12 +124,15 @@
                       echo
                       echo "Current directory `pwd`"
                       echo
-                      run rm -rf ${project.artifactId}-${project.version}
-                      run mkdir ${project.artifactId}-${project.version}
-                      run cd ${project.artifactId}-${project.version}
-                      run cp -r ${basedir}/target/${project.artifactId}-${project.version}*.jar .
+                      run rm -rf tajo-storage-${project.version}
+                      run mkdir tajo-storage-${project.version}
+                      run cd tajo-storage-${project.version}
+                      run cp -r ${basedir}/tajo-storage-common/target/tajo-storage-common-${project.version}*.jar .
+                      run cp -r ${basedir}/tajo-storage-hdfs/target/tajo-storage-hdfs-${project.version}*.jar .
+                      run cp -r ${basedir}/tajo-storage-hbase/target/tajo-storage-hbase-${project.version}*.jar .
+
                       echo
-                      echo "Tajo Storage dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo "Tajo Storage dist layout available at: ${project.build.directory}/tajo-storage-${project.version}"
                       echo
                     </echo>
                     <exec executable="sh" dir="${project.build.directory}" failonerror="true">
@@ -412,11 +152,7 @@
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-project-info-reports-plugin</artifactId>
-        <version>2.4</version>
-        <configuration>
-          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
-        </configuration>
+        <artifactId>maven-surefire-report-plugin</artifactId>
       </plugin>
     </plugins>
   </reporting>
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/NullScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/NullScanner.java
deleted file mode 100644
index 4cec67d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/NullScanner.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package org.apache.tajo.storage; /**
- * 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.
- */
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.fragment.FileFragment;
-
-import java.io.IOException;
-
-public class NullScanner extends FileScanner {
-  public NullScanner(Configuration conf, Schema schema, TableMeta meta, FileFragment fragment) {
-    super(conf, schema, meta, fragment);
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    progress = 1.0f;
-
-    return null;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    progress = 0.0f;
-  }
-
-  @Override
-  public void close() throws IOException {
-    progress = 0.0f;
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return false;
-  }
-
-  @Override
-  public boolean isSelectable() {
-    return true;
-  }
-
-  @Override
-  public boolean isSplittable() {
-    return true;
-  }
-}
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.java b/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.java
deleted file mode 100644
index cc85c1d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.java
+++ /dev/null
@@ -1,45 +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.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.fragment.FileFragment;
-
-import java.io.IOException;
-
-public abstract class Storage {
-  protected final Configuration conf;
-  
-  public Storage(final Configuration conf) {
-    this.conf = conf;
-  }
-  
-  public Configuration getConf() {
-    return this.conf;
-  }
-  
-  public abstract Appender getAppender(TableMeta meta, Path path)
-    throws IOException;
-
-  public abstract Scanner openScanner(Schema schema, FileFragment[] tablets)
-    throws IOException;
-}
diff --git a/tajo-storage/tajo-storage-common/pom.xml b/tajo-storage/tajo-storage-common/pom.xml
new file mode 100644
index 0000000..c600b4b
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/pom.xml
@@ -0,0 +1,337 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.9.1-SNAPSHOT</version>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>tajo-storage-common</artifactId>
+  <packaging>jar</packaging>
+  <name>Tajo Storage Common</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/IndexProtos.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-pmd-plugin</artifactId>
+        <version>2.7.1</version>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-plan</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>slf4j-api</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jersey-json</artifactId>
+          <groupId>com.sun.jersey</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-server-tests</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-hs</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-buffer</artifactId>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
\ No newline at end of file
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Appender.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Appender.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/BufferPool.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BufferPool.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/BufferPool.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BufferPool.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DataLocation.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DataLocation.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/DataLocation.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DataLocation.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskInfo.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskInfo.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/DiskInfo.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskInfo.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskMountInfo.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/DiskUtil.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/MemoryUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/MemoryUtil.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
similarity index 88%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
index 4122c76..66b3667 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.storage;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
@@ -26,7 +25,7 @@
 import org.apache.tajo.catalog.statistics.ColumnStats;
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -37,9 +36,9 @@
   private Configuration conf;
   private TableMeta meta;
   private Schema schema;
-  private List<FileFragment> fragments;
-  private Iterator<FileFragment> iterator;
-  private FileFragment currentFragment;
+  private List<Fragment> fragments;
+  private Iterator<Fragment> iterator;
+  private Fragment currentFragment;
   private Scanner currentScanner;
   private Tuple tuple;
   private boolean projectable = false;
@@ -48,12 +47,12 @@
   private float progress;
   protected TableStats tableStats;
 
-  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<FileFragment> rawFragmentList)
+  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<Fragment> rawFragmentList)
       throws IOException {
     this(conf, schema, meta, rawFragmentList, schema);
   }
 
-  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<FileFragment> rawFragmentList,
+  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<Fragment> rawFragmentList,
                       Schema target)
       throws IOException {
     this.conf = conf;
@@ -61,12 +60,13 @@
     this.meta = meta;
     this.target = target;
 
-    this.fragments = new ArrayList<FileFragment>();
+    this.fragments = new ArrayList<Fragment>();
 
     long numBytes = 0;
-    for (FileFragment eachFileFragment: rawFragmentList) {
-      numBytes += eachFileFragment.getEndKey();
-      if (eachFileFragment.getEndKey() > 0) {
+    for (Fragment eachFileFragment: rawFragmentList) {
+      long fragmentLength = StorageManager.getFragmentLength((TajoConf)conf, eachFileFragment);
+      if (fragmentLength > 0) {
+        numBytes += fragmentLength;
         fragments.add(eachFileFragment);
       }
     }
@@ -122,13 +122,16 @@
   @Override
   public void reset() throws IOException {
     this.iterator = fragments.iterator();
+    if (currentScanner != null) {
+      currentScanner.close();
+    }
     this.currentScanner = getNextScanner();
   }
 
   private Scanner getNextScanner() throws IOException {
     if (iterator.hasNext()) {
       currentFragment = iterator.next();
-      currentScanner = StorageManager.getStorageManager((TajoConf)conf).getScanner(meta, schema,
+      currentScanner = StorageManager.getStorageManager((TajoConf)conf, meta.getStoreType()).getScanner(meta, schema,
           currentFragment, target);
       currentScanner.init();
       return currentScanner;
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java
new file mode 100644
index 0000000..4272228
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java
@@ -0,0 +1,109 @@
+package org.apache.tajo.storage; /**
+ * 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.
+ */
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+
+public class NullScanner implements Scanner {
+  protected final Configuration conf;
+  protected final TableMeta meta;
+  protected final Schema schema;
+  protected final Fragment fragment;
+  protected final int columnNum;
+  protected Column [] targets;
+  protected float progress;
+  protected TableStats tableStats;
+
+  public NullScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) {
+    this.conf = conf;
+    this.meta = meta;
+    this.schema = schema;
+    this.fragment = fragment;
+    this.tableStats = new TableStats();
+    this.columnNum = this.schema.size();
+  }
+
+  @Override
+  public void init() throws IOException {
+    progress = 0.0f;
+    tableStats.setNumBytes(0);
+    tableStats.setNumBlocks(0);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    progress = 1.0f;
+    return null;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    progress = 0.0f;
+  }
+
+  @Override
+  public void close() throws IOException {
+    progress = 1.0f;
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return false;
+  }
+
+  @Override
+  public void setTarget(Column[] targets) {
+    this.targets = targets;
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return true;
+  }
+
+  @Override
+  public void setSearchCondition(Object expr) {
+
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return true;
+  }
+
+  @Override
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    return tableStats;
+  }
+
+  @Override
+  public Schema getSchema() {
+    return schema;
+  }
+}
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Scanner.java
similarity index 87%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Scanner.java
index 16c4faa..0356b19 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Scanner.java
@@ -38,22 +38,22 @@
    * @return retrieve null if the scanner has no more tuples. 
    * Otherwise it returns one tuple.
    * 
-   * @throws IOException if internal I/O error occurs during next method
+   * @throws java.io.IOException if internal I/O error occurs during next method
    */
   Tuple next() throws IOException;
-  
+
   /**
-   * Reset the cursor. After executed, the scanner 
+   * Reset the cursor. After executed, the scanner
    * will retrieve the first tuple.
-   * 
-   * @throws IOException if internal I/O error occurs during reset method
+   *
+   * @throws java.io.IOException if internal I/O error occurs during reset method
    */
   void reset() throws IOException;
-  
+
   /**
    * Close scanner
-   * 
-   * @throws IOException if internal I/O error occurs during close method
+   *
+   * @throws java.io.IOException if internal I/O error occurs during close method
    */
   void close() throws IOException;
 
@@ -69,7 +69,7 @@
    * Set target columns
    * @param targets columns to be projected
    */
-  void setTarget(Column [] targets);
+  void setTarget(Column[] targets);
 
   /**
    * It returns if the selection is executed in the underlying scanner layer.
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SeekableScanner.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SeekableScanner.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
new file mode 100644
index 0000000..e2d89d6
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
@@ -0,0 +1,979 @@
+/**
+ * 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.storage;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.NodeType;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.plan.rewrite.RewriteRule;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.URI;
+import java.text.NumberFormat;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * StorageManager manages the functions of storing and reading data.
+ * StorageManager is a abstract class.
+ * For supporting such as HDFS, HBASE, a specific StorageManager should be implemented by inheriting this class.
+ *
+ */
+public abstract class StorageManager {
+  private final Log LOG = LogFactory.getLog(StorageManager.class);
+
+  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
+      Configuration.class,
+      Schema.class,
+      TableMeta.class,
+      Fragment.class
+  };
+
+  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
+      Configuration.class,
+      QueryUnitAttemptId.class,
+      Schema.class,
+      TableMeta.class,
+      Path.class
+  };
+
+  public static final PathFilter hiddenFileFilter = new PathFilter() {
+    public boolean accept(Path p) {
+      String name = p.getName();
+      return !name.startsWith("_") && !name.startsWith(".");
+    }
+  };
+
+  protected TajoConf conf;
+  protected StoreType storeType;
+
+  /**
+   * Cache of StorageManager.
+   * Key is manager key(warehouse path) + store type
+   */
+  private static final Map<String, StorageManager> storageManagers = Maps.newHashMap();
+
+  /**
+   * Cache of scanner handlers for each storage type.
+   */
+  protected static final Map<String, Class<? extends Scanner>> SCANNER_HANDLER_CACHE
+      = new ConcurrentHashMap<String, Class<? extends Scanner>>();
+
+  /**
+   * Cache of appender handlers for each storage type.
+   */
+  protected static final Map<String, Class<? extends Appender>> APPENDER_HANDLER_CACHE
+      = new ConcurrentHashMap<String, Class<? extends Appender>>();
+
+  /**
+   * Cache of constructors for each class. Pins the classes so they
+   * can't be garbage collected until ReflectionUtils can be collected.
+   */
+  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
+      new ConcurrentHashMap<Class<?>, Constructor<?>>();
+
+  public StorageManager(StoreType storeType) {
+    this.storeType = storeType;
+  }
+
+  /**
+   * Initialize storage manager.
+   * @throws java.io.IOException
+   */
+  protected abstract void storageInit() throws IOException;
+
+  /**
+   * This method is called after executing "CREATE TABLE" statement.
+   * If a storage is a file based storage, a storage manager may create directory.
+   *
+   * @param tableDesc Table description which is created.
+   * @param ifNotExists Creates the table only when the table does not exist.
+   * @throws java.io.IOException
+   */
+  public abstract void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException;
+
+  /**
+   * This method is called after executing "DROP TABLE" statement with the 'PURGE' option
+   * which is the option to delete all the data.
+   *
+   * @param tableDesc
+   * @throws java.io.IOException
+   */
+  public abstract void purgeTable(TableDesc tableDesc) throws IOException;
+
+  /**
+   * Returns the splits that will serve as input for the scan tasks. The
+   * number of splits matches the number of regions in a table.
+   * @param fragmentId The table name or previous ExecutionBlockId
+   * @param tableDesc The table description for the target data.
+   * @param scanNode The logical node for scanning.
+   * @return The list of input fragments.
+   * @throws java.io.IOException
+   */
+  public abstract List<Fragment> getSplits(String fragmentId, TableDesc tableDesc,
+                                           ScanNode scanNode) throws IOException;
+
+  /**
+   * It returns the splits that will serve as input for the non-forward query scanner such as 'select * from table1'.
+   * The result list should be small. If there is many fragments for scanning, TajoMaster uses the paging navigation.
+   * @param tableDesc The table description for the target data.
+   * @param currentPage The current page number within the entire list.
+   * @param numFragments The number of fragments in the result.
+   * @return The list of input fragments.
+   * @throws java.io.IOException
+   */
+  public abstract List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments)
+      throws IOException;
+
+  /**
+   * It returns the storage property.
+   * @return The storage property
+   */
+  public abstract StorageProperty getStorageProperty();
+
+  /**
+   * Release storage manager resource
+   */
+  public abstract void closeStorageManager();
+
+  /**
+   * It is called by a Repartitioner for range shuffling when the SortRangeType of SortNode is USING_STORAGE_MANAGER.
+   * In general Repartitioner determines the partition range using previous output statistics data.
+   * In the special cases, such as HBase Repartitioner uses the result of this method.
+   *
+   * @param queryContext The current query context which contains query properties.
+   * @param tableDesc The table description for the target data.
+   * @param inputSchema The input schema
+   * @param sortSpecs The sort specification that contains the sort column and sort order.
+   * @return The list of sort ranges.
+   * @throws java.io.IOException
+   */
+  public abstract TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
+                                                   Schema inputSchema, SortSpec[] sortSpecs,
+                                                   TupleRange dataRange) throws IOException;
+
+  /**
+   * This method is called before executing 'INSERT' or 'CREATE TABLE as SELECT'.
+   * In general Tajo creates the target table after finishing the final sub-query of CATS.
+   * But In the special cases, such as HBase INSERT or CAST query uses the target table information.
+   * That kind of the storage should implements the logic related to creating table in this method.
+   *
+   * @param node The child node of the root node.
+   * @throws java.io.IOException
+   */
+  public abstract void beforeInsertOrCATS(LogicalNode node) throws IOException;
+
+  /**
+   * It is called when the query failed.
+   * Each storage manager should implement to be processed when the query fails in this method.
+   *
+   * @param node The child node of the root node.
+   * @throws java.io.IOException
+   */
+  public abstract void rollbackOutputCommit(LogicalNode node) throws IOException;
+
+  /**
+   * Returns the current storage type.
+   * @return
+   */
+  public StoreType getStoreType() {
+    return storeType;
+  }
+
+  /**
+   * Initialize StorageManager instance. It should be called before using.
+   *
+   * @param tajoConf
+   * @throws java.io.IOException
+   */
+  public void init(TajoConf tajoConf) throws IOException {
+    this.conf = tajoConf;
+    storageInit();
+  }
+
+  /**
+   * Close StorageManager
+   * @throws java.io.IOException
+   */
+  public void close() throws IOException {
+    synchronized(storageManagers) {
+      for (StorageManager eachStorageManager: storageManagers.values()) {
+        eachStorageManager.closeStorageManager();
+      }
+    }
+  }
+
+  /**
+   * Returns the splits that will serve as input for the scan tasks. The
+   * number of splits matches the number of regions in a table.
+   *
+   * @param fragmentId The table name or previous ExecutionBlockId
+   * @param tableDesc The table description for the target data.
+   * @return The list of input fragments.
+   * @throws java.io.IOException
+   */
+  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc) throws IOException {
+    return getSplits(fragmentId, tableDesc, null);
+  }
+
+  /**
+   * Returns FileStorageManager instance.
+   *
+   * @param tajoConf Tajo system property.
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getFileStorageManager(TajoConf tajoConf) throws IOException {
+    return getFileStorageManager(tajoConf, null);
+  }
+
+  /**
+   * Returns FileStorageManager instance and sets WAREHOUSE_DIR property in tajoConf with warehousePath parameter.
+   *
+   * @param tajoConf Tajo system property.
+   * @param warehousePath The warehouse directory to be set in the tajoConf.
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getFileStorageManager(TajoConf tajoConf, Path warehousePath) throws IOException {
+    URI uri;
+    TajoConf copiedConf = new TajoConf(tajoConf);
+    if (warehousePath != null) {
+      copiedConf.setVar(ConfVars.WAREHOUSE_DIR, warehousePath.toUri().toString());
+    }
+    uri = TajoConf.getWarehouseDir(copiedConf).toUri();
+    String key = "file".equals(uri.getScheme()) ? "file" : uri.toString();
+    return getStorageManager(copiedConf, StoreType.CSV, key);
+  }
+
+  /**
+   * Returns the proper StorageManager instance according to the storeType.
+   *
+   * @param tajoConf Tajo system property.
+   * @param storeType Storage type
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getStorageManager(TajoConf tajoConf, String storeType) throws IOException {
+    if ("HBASE".equals(storeType)) {
+      return getStorageManager(tajoConf, StoreType.HBASE);
+    } else {
+      return getStorageManager(tajoConf, StoreType.CSV);
+    }
+  }
+
+  /**
+   * Returns the proper StorageManager instance according to the storeType.
+   *
+   * @param tajoConf Tajo system property.
+   * @param storeType Storage type
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getStorageManager(TajoConf tajoConf, StoreType storeType) throws IOException {
+    return getStorageManager(tajoConf, storeType, null);
+  }
+
+  /**
+   * Returns the proper StorageManager instance according to the storeType
+   *
+   * @param tajoConf Tajo system property.
+   * @param storeType Storage type
+   * @param managerKey Key that can identify each storage manager(may be a path)
+   * @return
+   * @throws java.io.IOException
+   */
+  public static synchronized StorageManager getStorageManager (
+      TajoConf tajoConf, StoreType storeType, String managerKey) throws IOException {
+    synchronized (storageManagers) {
+      String storeKey = storeType + managerKey;
+      StorageManager manager = storageManagers.get(storeKey);
+      if (manager == null) {
+        String typeName = "hdfs";
+
+        switch (storeType) {
+          case HBASE:
+            typeName = "hbase";
+            break;
+          default:
+            typeName = "hdfs";
+        }
+
+        Class<? extends StorageManager> storageManagerClass =
+            tajoConf.getClass(String.format("tajo.storage.manager.%s.class", typeName), null, StorageManager.class);
+
+        if (storageManagerClass == null) {
+          throw new IOException("Unknown Storage Type: " + typeName);
+        }
+
+        try {
+          Constructor<? extends StorageManager> constructor =
+              (Constructor<? extends StorageManager>) CONSTRUCTOR_CACHE.get(storageManagerClass);
+          if (constructor == null) {
+            constructor = storageManagerClass.getDeclaredConstructor(new Class<?>[]{StoreType.class});
+            constructor.setAccessible(true);
+            CONSTRUCTOR_CACHE.put(storageManagerClass, constructor);
+          }
+          manager = constructor.newInstance(new Object[]{storeType});
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+        manager.init(tajoConf);
+        storageManagers.put(storeKey, manager);
+      }
+
+      return manager;
+    }
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @param target Columns which are selected.
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment, Schema target) throws IOException {
+    return getScanner(meta, schema, FragmentConvertor.convert(conf, fragment), target);
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment) throws IOException {
+    return getScanner(meta, schema, fragment, schema);
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @param target The output schema
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
+    if (fragment.isEmpty()) {
+      Scanner scanner = new NullScanner(conf, schema, meta, fragment);
+      scanner.setTarget(target.toArray());
+
+      return scanner;
+    }
+
+    Scanner scanner;
+
+    Class<? extends Scanner> scannerClass = getScannerClass(meta.getStoreType());
+    scanner = newScannerInstance(scannerClass, conf, schema, meta, fragment);
+    if (scanner.isProjectable()) {
+      scanner.setTarget(target.toArray());
+    }
+
+    return scanner;
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param conf The system property
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @param target The output schema
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public static synchronized SeekableScanner getSeekableScanner(
+      TajoConf conf, TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
+    return (SeekableScanner)getStorageManager(conf, meta.getStoreType()).getScanner(meta, schema, fragment, target);
+  }
+
+  /**
+   * Returns Appender instance.
+   * @param queryContext Query property.
+   * @param taskAttemptId Task id.
+   * @param meta Table meta data.
+   * @param schema Output schema.
+   * @param workDir Working directory
+   * @return Appender instance
+   * @throws java.io.IOException
+   */
+  public Appender getAppender(OverridableConf queryContext,
+                              QueryUnitAttemptId taskAttemptId, TableMeta meta, Schema schema, Path workDir)
+      throws IOException {
+    Appender appender;
+
+    Class<? extends Appender> appenderClass;
+
+    String handlerName = meta.getStoreType().name().toLowerCase();
+    appenderClass = APPENDER_HANDLER_CACHE.get(handlerName);
+    if (appenderClass == null) {
+      appenderClass = conf.getClass(
+          String.format("tajo.storage.appender-handler.%s.class",
+              meta.getStoreType().name().toLowerCase()), null, Appender.class);
+      APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
+    }
+
+    if (appenderClass == null) {
+      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
+    }
+
+    appender = newAppenderInstance(appenderClass, conf, taskAttemptId, meta, schema, workDir);
+
+    return appender;
+  }
+
+  /**
+   * Creates a scanner instance.
+   *
+   * @param theClass Concrete class of scanner
+   * @param conf System property
+   * @param schema Input schema
+   * @param meta Table meta data
+   * @param fragment The fragment for scanning
+   * @param <T>
+   * @return The scanner instance
+   */
+  public static <T> T newScannerInstance(Class<T> theClass, Configuration conf, Schema schema, TableMeta meta,
+                                         Fragment fragment) {
+    T result;
+    try {
+      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
+      if (meth == null) {
+        meth = theClass.getDeclaredConstructor(DEFAULT_SCANNER_PARAMS);
+        meth.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(theClass, meth);
+      }
+      result = meth.newInstance(new Object[]{conf, schema, meta, fragment});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result;
+  }
+
+  /**
+   * Creates a scanner instance.
+   *
+   * @param theClass Concrete class of scanner
+   * @param conf System property
+   * @param taskAttemptId Task id
+   * @param meta Table meta data
+   * @param schema Input schema
+   * @param workDir Working directory
+   * @param <T>
+   * @return The scanner instance
+   */
+  public static <T> T newAppenderInstance(Class<T> theClass, Configuration conf, QueryUnitAttemptId taskAttemptId,
+                                          TableMeta meta, Schema schema, Path workDir) {
+    T result;
+    try {
+      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
+      if (meth == null) {
+        meth = theClass.getDeclaredConstructor(DEFAULT_APPENDER_PARAMS);
+        meth.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(theClass, meth);
+      }
+      result = meth.newInstance(new Object[]{conf, taskAttemptId, schema, meta, workDir});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result;
+  }
+
+  /**
+   * Return the Scanner class for the StoreType that is defined in storage-default.xml.
+   *
+   * @param storeType store type
+   * @return The Scanner class
+   * @throws java.io.IOException
+   */
+  public Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException {
+    String handlerName = storeType.name().toLowerCase();
+    Class<? extends Scanner> scannerClass = SCANNER_HANDLER_CACHE.get(handlerName);
+    if (scannerClass == null) {
+      scannerClass = conf.getClass(
+          String.format("tajo.storage.scanner-handler.%s.class",storeType.name().toLowerCase()), null, Scanner.class);
+      SCANNER_HANDLER_CACHE.put(handlerName, scannerClass);
+    }
+
+    if (scannerClass == null) {
+      throw new IOException("Unknown Storage Type: " + storeType.name());
+    }
+
+    return scannerClass;
+  }
+
+  /**
+   * Return length of the fragment.
+   * In the UNKNOWN_LENGTH case get FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH from the configuration.
+   *
+   * @param conf Tajo system property
+   * @param fragment Fragment
+   * @return
+   */
+  public static long getFragmentLength(TajoConf conf, Fragment fragment) {
+    if (fragment.getLength() == TajoConstants.UNKNOWN_LENGTH) {
+      return conf.getLongVar(ConfVars.FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH);
+    } else {
+      return fragment.getLength();
+    }
+  }
+
+  /**
+   * It is called after making logical plan. Storage manager should verify the schema for inserting.
+   *
+   * @param tableDesc The table description of insert target.
+   * @param outSchema  The output schema of select query for inserting.
+   * @throws java.io.IOException
+   */
+  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException {
+    // nothing to do
+  }
+
+  /**
+   * Returns the list of storage specified rewrite rules.
+   * This values are used by LogicalOptimizer.
+   *
+   * @param queryContext The query property
+   * @param tableDesc The description of the target table.
+   * @return The list of storage specified rewrite rules
+   * @throws java.io.IOException
+   */
+  public List<RewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
+    return null;
+  }
+
+  /**
+   * Finalizes result data. Tajo stores result data in the staging directory.
+   * If the query fails, clean up the staging directory.
+   * Otherwise the query is successful, move to the final directory from the staging directory.
+   *
+   * @param queryContext The query property
+   * @param finalEbId The final execution block id
+   * @param plan The query plan
+   * @param schema The final output schema
+   * @param tableDesc The description of the target table
+   * @return Saved path
+   * @throws java.io.IOException
+   */
+  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
+                               LogicalPlan plan, Schema schema,
+                               TableDesc tableDesc) throws IOException {
+    return commitOutputData(queryContext, finalEbId, plan, schema, tableDesc, true);
+  }
+
+  /**
+   * Finalizes result data. Tajo stores result data in the staging directory.
+   * If the query fails, clean up the staging directory.
+   * Otherwise the query is successful, move to the final directory from the staging directory.
+   *
+   * @param queryContext The query property
+   * @param finalEbId The final execution block id
+   * @param plan The query plan
+   * @param schema The final output schema
+   * @param tableDesc The description of the target table
+   * @param changeFileSeq If true change result file name with max sequence.
+   * @return Saved path
+   * @throws java.io.IOException
+   */
+  protected Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
+                               LogicalPlan plan, Schema schema,
+                               TableDesc tableDesc, boolean changeFileSeq) throws IOException {
+    Path stagingDir = new Path(queryContext.get(QueryVars.STAGING_DIR));
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    Path finalOutputDir;
+    if (!queryContext.get(QueryVars.OUTPUT_TABLE_PATH, "").isEmpty()) {
+      finalOutputDir = new Path(queryContext.get(QueryVars.OUTPUT_TABLE_PATH));
+      try {
+        FileSystem fs = stagingResultDir.getFileSystem(conf);
+
+        if (queryContext.getBool(QueryVars.OUTPUT_OVERWRITE, false)) { // INSERT OVERWRITE INTO
+
+          // It moves the original table into the temporary location.
+          // Then it moves the new result table into the original table location.
+          // Upon failed, it recovers the original table if possible.
+          boolean movedToOldTable = false;
+          boolean committed = false;
+          Path oldTableDir = new Path(stagingDir, TajoConstants.INSERT_OVERWIRTE_OLD_TABLE_NAME);
+          ContentSummary summary = fs.getContentSummary(stagingResultDir);
+
+          if (!queryContext.get(QueryVars.OUTPUT_PARTITIONS, "").isEmpty() && summary.getFileCount() > 0L) {
+            // This is a map for existing non-leaf directory to rename. A key is current directory and a value is
+            // renaming directory.
+            Map<Path, Path> renameDirs = TUtil.newHashMap();
+            // This is a map for recovering existing partition directory. A key is current directory and a value is
+            // temporary directory to back up.
+            Map<Path, Path> recoveryDirs = TUtil.newHashMap();
+
+            try {
+              if (!fs.exists(finalOutputDir)) {
+                fs.mkdirs(finalOutputDir);
+              }
+
+              visitPartitionedDirectory(fs, stagingResultDir, finalOutputDir, stagingResultDir.toString(),
+                  renameDirs, oldTableDir);
+
+              // Rename target partition directories
+              for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
+                // Backup existing data files for recovering
+                if (fs.exists(entry.getValue())) {
+                  String recoveryPathString = entry.getValue().toString().replaceAll(finalOutputDir.toString(),
+                      oldTableDir.toString());
+                  Path recoveryPath = new Path(recoveryPathString);
+                  fs.rename(entry.getValue(), recoveryPath);
+                  fs.exists(recoveryPath);
+                  recoveryDirs.put(entry.getValue(), recoveryPath);
+                }
+                // Delete existing directory
+                fs.delete(entry.getValue(), true);
+                // Rename staging directory to final output directory
+                fs.rename(entry.getKey(), entry.getValue());
+              }
+
+            } catch (IOException ioe) {
+              // Remove created dirs
+              for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
+                fs.delete(entry.getValue(), true);
+              }
+
+              // Recovery renamed dirs
+              for(Map.Entry<Path, Path> entry : recoveryDirs.entrySet()) {
+                fs.delete(entry.getValue(), true);
+                fs.rename(entry.getValue(), entry.getKey());
+              }
+
+              throw new IOException(ioe.getMessage());
+            }
+          } else { // no partition
+            try {
+
+              // if the final output dir exists, move all contents to the temporary table dir.
+              // Otherwise, just make the final output dir. As a result, the final output dir will be empty.
+              if (fs.exists(finalOutputDir)) {
+                fs.mkdirs(oldTableDir);
+
+                for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
+                  fs.rename(status.getPath(), oldTableDir);
+                }
+
+                movedToOldTable = fs.exists(oldTableDir);
+              } else { // if the parent does not exist, make its parent directory.
+                fs.mkdirs(finalOutputDir);
+              }
+
+              // Move the results to the final output dir.
+              for (FileStatus status : fs.listStatus(stagingResultDir)) {
+                fs.rename(status.getPath(), finalOutputDir);
+              }
+
+              // Check the final output dir
+              committed = fs.exists(finalOutputDir);
+
+            } catch (IOException ioe) {
+              // recover the old table
+              if (movedToOldTable && !committed) {
+
+                // if commit is failed, recover the old data
+                for (FileStatus status : fs.listStatus(finalOutputDir, StorageManager.hiddenFileFilter)) {
+                  fs.delete(status.getPath(), true);
+                }
+
+                for (FileStatus status : fs.listStatus(oldTableDir)) {
+                  fs.rename(status.getPath(), finalOutputDir);
+                }
+              }
+
+              throw new IOException(ioe.getMessage());
+            }
+          }
+        } else {
+          String queryType = queryContext.get(QueryVars.COMMAND_TYPE);
+
+          if (queryType != null && queryType.equals(NodeType.INSERT.name())) { // INSERT INTO an existing table
+
+            NumberFormat fmt = NumberFormat.getInstance();
+            fmt.setGroupingUsed(false);
+            fmt.setMinimumIntegerDigits(3);
+
+            if (!queryContext.get(QueryVars.OUTPUT_PARTITIONS, "").isEmpty()) {
+              for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
+                if (eachFile.isFile()) {
+                  LOG.warn("Partition table can't have file in a staging dir: " + eachFile.getPath());
+                  continue;
+                }
+                moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, -1, changeFileSeq);
+              }
+            } else {
+              int maxSeq = StorageUtil.getMaxFileSequence(fs, finalOutputDir, false) + 1;
+              for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
+                if (eachFile.getPath().getName().startsWith("_")) {
+                  continue;
+                }
+                moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, maxSeq++, changeFileSeq);
+              }
+            }
+            // checking all file moved and remove empty dir
+            verifyAllFileMoved(fs, stagingResultDir);
+            FileStatus[] files = fs.listStatus(stagingResultDir);
+            if (files != null && files.length != 0) {
+              for (FileStatus eachFile: files) {
+                LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
+              }
+            }
+          } else { // CREATE TABLE AS SELECT (CTAS)
+            if (fs.exists(finalOutputDir)) {
+              for (FileStatus status : fs.listStatus(stagingResultDir)) {
+                fs.rename(status.getPath(), finalOutputDir);
+              }
+            } else {
+              fs.rename(stagingResultDir, finalOutputDir);
+            }
+            LOG.info("Moved from the staging dir to the output directory '" + finalOutputDir);
+          }
+        }
+
+        // remove the staging directory if the final output dir is given.
+        Path stagingDirRoot = stagingDir.getParent();
+        fs.delete(stagingDirRoot, true);
+      } catch (Throwable t) {
+        LOG.error(t);
+        throw new IOException(t);
+      }
+    } else {
+      finalOutputDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    }
+
+    return finalOutputDir;
+  }
+
+  /**
+   * Attach the sequence number to the output file name and than move the file into the final result path.
+   *
+   * @param fs FileSystem
+   * @param stagingResultDir The staging result dir
+   * @param fileStatus The file status
+   * @param finalOutputPath Final output path
+   * @param nf Number format
+   * @param fileSeq The sequence number
+   * @throws java.io.IOException
+   */
+  private void moveResultFromStageToFinal(FileSystem fs, Path stagingResultDir,
+                                          FileStatus fileStatus, Path finalOutputPath,
+                                          NumberFormat nf,
+                                          int fileSeq, boolean changeFileSeq) throws IOException {
+    if (fileStatus.isDirectory()) {
+      String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
+      if (subPath != null) {
+        Path finalSubPath = new Path(finalOutputPath, subPath);
+        if (!fs.exists(finalSubPath)) {
+          fs.mkdirs(finalSubPath);
+        }
+        int maxSeq = StorageUtil.getMaxFileSequence(fs, finalSubPath, false);
+        for (FileStatus eachFile : fs.listStatus(fileStatus.getPath())) {
+          if (eachFile.getPath().getName().startsWith("_")) {
+            continue;
+          }
+          moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputPath, nf, ++maxSeq, changeFileSeq);
+        }
+      } else {
+        throw new IOException("Wrong staging dir:" + stagingResultDir + "," + fileStatus.getPath());
+      }
+    } else {
+      String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
+      if (subPath != null) {
+        Path finalSubPath = new Path(finalOutputPath, subPath);
+        if (changeFileSeq) {
+          finalSubPath = new Path(finalSubPath.getParent(), replaceFileNameSeq(finalSubPath, fileSeq, nf));
+        }
+        if (!fs.exists(finalSubPath.getParent())) {
+          fs.mkdirs(finalSubPath.getParent());
+        }
+        if (fs.exists(finalSubPath)) {
+          throw new IOException("Already exists data file:" + finalSubPath);
+        }
+        boolean success = fs.rename(fileStatus.getPath(), finalSubPath);
+        if (success) {
+          LOG.info("Moving staging file[" + fileStatus.getPath() + "] + " +
+              "to final output[" + finalSubPath + "]");
+        } else {
+          LOG.error("Can't move staging file[" + fileStatus.getPath() + "] + " +
+              "to final output[" + finalSubPath + "]");
+        }
+      }
+    }
+  }
+
+  /**
+   * Removes the path of the parent.
+   * @param parentPath
+   * @param childPath
+   * @return
+   */
+  private String extractSubPath(Path parentPath, Path childPath) {
+    String parentPathStr = parentPath.toUri().getPath();
+    String childPathStr = childPath.toUri().getPath();
+
+    if (parentPathStr.length() > childPathStr.length()) {
+      return null;
+    }
+
+    int index = childPathStr.indexOf(parentPathStr);
+    if (index != 0) {
+      return null;
+    }
+
+    return childPathStr.substring(parentPathStr.length() + 1);
+  }
+
+  /**
+   * Attach the sequence number to a path.
+   *
+   * @param path Path
+   * @param seq sequence number
+   * @param nf Number format
+   * @return New path attached with sequence number
+   * @throws java.io.IOException
+   */
+  private String replaceFileNameSeq(Path path, int seq, NumberFormat nf) throws IOException {
+    String[] tokens = path.getName().split("-");
+    if (tokens.length != 4) {
+      throw new IOException("Wrong result file name:" + path);
+    }
+    return tokens[0] + "-" + tokens[1] + "-" + tokens[2] + "-" + nf.format(seq);
+  }
+
+  /**
+   * Make sure all files are moved.
+   * @param fs FileSystem
+   * @param stagingPath The stagind directory
+   * @return
+   * @throws java.io.IOException
+   */
+  private boolean verifyAllFileMoved(FileSystem fs, Path stagingPath) throws IOException {
+    FileStatus[] files = fs.listStatus(stagingPath);
+    if (files != null && files.length != 0) {
+      for (FileStatus eachFile: files) {
+        if (eachFile.isFile()) {
+          LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
+          return false;
+        } else {
+          if (verifyAllFileMoved(fs, eachFile.getPath())) {
+            fs.delete(eachFile.getPath(), false);
+          } else {
+            return false;
+          }
+        }
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * This method sets a rename map which includes renamed staging directory to final output directory recursively.
+   * If there exists some data files, this delete it for duplicate data.
+   *
+   *
+   * @param fs
+   * @param stagingPath
+   * @param outputPath
+   * @param stagingParentPathString
+   * @throws java.io.IOException
+   */
+  private void visitPartitionedDirectory(FileSystem fs, Path stagingPath, Path outputPath,
+                                         String stagingParentPathString,
+                                         Map<Path, Path> renameDirs, Path oldTableDir) throws IOException {
+    FileStatus[] files = fs.listStatus(stagingPath);
+
+    for(FileStatus eachFile : files) {
+      if (eachFile.isDirectory()) {
+        Path oldPath = eachFile.getPath();
+
+        // Make recover directory.
+        String recoverPathString = oldPath.toString().replaceAll(stagingParentPathString,
+            oldTableDir.toString());
+        Path recoveryPath = new Path(recoverPathString);
+        if (!fs.exists(recoveryPath)) {
+          fs.mkdirs(recoveryPath);
+        }
+
+        visitPartitionedDirectory(fs, eachFile.getPath(), outputPath, stagingParentPathString,
+            renameDirs, oldTableDir);
+        // Find last order partition for renaming
+        String newPathString = oldPath.toString().replaceAll(stagingParentPathString,
+            outputPath.toString());
+        Path newPath = new Path(newPathString);
+        if (!isLeafDirectory(fs, eachFile.getPath())) {
+          renameDirs.put(eachFile.getPath(), newPath);
+        } else {
+          if (!fs.exists(newPath)) {
+            fs.mkdirs(newPath);
+          }
+        }
+      }
+    }
+  }
+
+  private boolean isLeafDirectory(FileSystem fs, Path path) throws IOException {
+    boolean retValue = false;
+
+    FileStatus[] files = fs.listStatus(path);
+    for (FileStatus file : files) {
+      if (fs.isDirectory(file.getPath())) {
+        retValue = true;
+        break;
+      }
+    }
+
+    return retValue;
+  }
+}
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
similarity index 61%
copy from tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
copy to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
index 7baf7aa..6816d08 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
@@ -16,20 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.storage.index;
+package org.apache.tajo.storage;
 
-import org.apache.tajo.storage.Tuple;
+public class StorageProperty {
+  private boolean supportsInsertInto;
+  private boolean sortedInsert;
 
-import java.io.IOException;
+  public boolean isSupportsInsertInto() {
+    return supportsInsertInto;
+  }
 
-public interface IndexReader {
-  
-  /**
-   * Find the offset corresponding to key which is equal to a given key.
-   * 
-   * @param key
-   * @return
-   * @throws IOException 
-   */
-  public long find(Tuple key) throws IOException;
+  public void setSupportsInsertInto(boolean supportsInsertInto) {
+    this.supportsInsertInto = supportsInsertInto;
+  }
+
+  public boolean isSortedInsert() {
+    return sortedInsert;
+  }
+
+  public void setSortedInsert(boolean sortedInsert) {
+    this.sortedInsert = sortedInsert;
+  }
 }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
index 1789cc9..54fdb69 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
@@ -26,10 +26,8 @@
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.KeyValueSet;
-import parquet.hadoop.ParquetOutputFormat;
 import sun.nio.ch.DirectBuffer;
 
 import java.io.DataInput;
@@ -124,7 +122,7 @@
    * @param path
    * @param recursive
    * @return The maximum sequence number
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public static int getMaxFileSequence(FileSystem fs, Path path, boolean recursive) throws IOException {
     if (!fs.isDirectory(path)) {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableStatistics.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableStatistics.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleComparator.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleComparator.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TupleRange.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleRange.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/TupleRange.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleRange.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
similarity index 88%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
index 3f9c160..ac43197 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
@@ -28,4 +28,12 @@
 
   @Override
   public abstract FragmentProto getProto();
+
+  public abstract long getLength();
+
+  public abstract String getKey();
+
+  public String[] getHosts();
+
+  public abstract boolean isEmpty();
 }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
similarity index 84%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
index 0315a8d..07720c7 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
@@ -30,7 +30,6 @@
 import java.util.Map;
 
 import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 
 @ThreadSafe
 public class FragmentConvertor {
@@ -47,18 +46,17 @@
    */
   private static final Class<?>[] DEFAULT_FRAGMENT_PARAMS = { ByteString.class };
 
-  public static Class<? extends Fragment> getFragmentClass(Configuration conf, StoreType storeType)
-      throws IOException {
-    String handlerName = storeType.name().toLowerCase();
-    Class<? extends Fragment> fragmentClass = CACHED_FRAGMENT_CLASSES.get(handlerName);
+  public static Class<? extends Fragment> getFragmentClass(Configuration conf, String storeType)
+  throws IOException {
+    Class<? extends Fragment> fragmentClass = CACHED_FRAGMENT_CLASSES.get(storeType.toLowerCase());
     if (fragmentClass == null) {
       fragmentClass = conf.getClass(
-          String.format("tajo.storage.fragment.%s.class", storeType.name().toLowerCase()), null, Fragment.class);
-      CACHED_FRAGMENT_CLASSES.put(handlerName, fragmentClass);
+          String.format("tajo.storage.fragment.%s.class", storeType.toLowerCase()), null, Fragment.class);
+      CACHED_FRAGMENT_CLASSES.put(storeType.toLowerCase(), fragmentClass);
     }
 
     if (fragmentClass == null) {
-      throw new IOException("No such a fragment for " + storeType.name());
+      throw new IOException("No such a fragment for " + storeType.toLowerCase());
     }
 
     return fragmentClass;
@@ -81,11 +79,11 @@
     return result;
   }
 
-  public static <T extends Fragment> T convert(Configuration conf, StoreType storeType, FragmentProto fragment)
+  public static <T extends Fragment> T convert(Configuration conf, FragmentProto fragment)
       throws IOException {
-    Class<T> fragmentClass = (Class<T>) getFragmentClass(conf, storeType);
+    Class<T> fragmentClass = (Class<T>) getFragmentClass(conf, fragment.getStoreType().toLowerCase());
     if (fragmentClass == null) {
-      throw new IOException("No such a fragment class for " + storeType.name());
+      throw new IOException("No such a fragment class for " + fragment.getStoreType());
     }
     return convert(fragmentClass, fragment);
   }
@@ -102,14 +100,13 @@
     return list;
   }
 
-  public static <T extends Fragment> List<T> convert(Configuration conf, StoreType storeType,
-                                                           FragmentProto...fragments) throws IOException {
+  public static <T extends Fragment> List<T> convert(Configuration conf, FragmentProto...fragments) throws IOException {
     List<T> list = Lists.newArrayList();
     if (fragments == null) {
       return list;
     }
     for (FragmentProto proto : fragments) {
-      list.add((T) convert(conf, storeType, proto));
+      list.add((T) convert(conf, proto));
     }
     return list;
   }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/RowBlockReader.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/TupleBuilder.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
rename to tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
diff --git a/tajo-storage/src/main/proto/IndexProtos.proto b/tajo-storage/tajo-storage-common/src/main/proto/IndexProtos.proto
similarity index 100%
rename from tajo-storage/src/main/proto/IndexProtos.proto
rename to tajo-storage/tajo-storage-common/src/main/proto/IndexProtos.proto
diff --git a/tajo-storage/src/main/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
similarity index 85%
rename from tajo-storage/src/main/resources/storage-default.xml
rename to tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
index e861b7d..67033ed 100644
--- a/tajo-storage/src/main/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
@@ -20,10 +20,14 @@
   -->
 
 <configuration>
+  <!-- Storage Manager Configuration -->
   <property>
-    <name>tajo.storage.manager.maxReadBytes</name>
-    <value>8388608</value>
-    <description></description>
+    <name>tajo.storage.manager.hdfs.class</name>
+    <value>org.apache.tajo.storage.FileStorageManager</value>
+  </property>
+  <property>
+    <name>tajo.storage.manager.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
   </property>
 
   <property>
@@ -35,7 +39,7 @@
   <!--- Registered Scanner Handler -->
   <property>
     <name>tajo.storage.scanner-handler</name>
-    <value>textfile,csv,json,raw,rcfile,row,parquet,sequencefile,avro</value>
+    <value>textfile,csv,json,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
   </property>
 
   <!--- Fragment Class Configurations -->
@@ -75,6 +79,10 @@
     <name>tajo.storage.fragment.avro.class</name>
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
+  <property>
+    <name>tajo.storage.fragment.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseFragment</value>
+  </property>
 
   <!--- Scanner Handler -->
   <property>
@@ -122,10 +130,15 @@
     <value>org.apache.tajo.storage.avro.AvroScanner</value>
   </property>
 
+  <property>
+    <name>tajo.storage.scanner-handler.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseScanner</value>
+  </property>
+  
   <!--- Appender Handler -->
   <property>
     <name>tajo.storage.appender-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
+    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
   </property>
 
   <property>
@@ -172,4 +185,14 @@
     <name>tajo.storage.appender-handler.avro.class</name>
     <value>org.apache.tajo.storage.avro.AvroAppender</value>
   </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.hfile.class</name>
+    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
+  </property>
 </configuration>
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
similarity index 99%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
index 387fed5..0251dc7 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
@@ -18,11 +18,11 @@
 
 package org.apache.tajo.storage;
 
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestVTuple.java
similarity index 98%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestVTuple.java
index 9837fd1..1bbd9ec 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestVTuple.java
@@ -19,16 +19,16 @@
 package org.apache.tajo.storage;
 
 
+import org.apache.tajo.datum.DatumFactory;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.tajo.datum.DatumFactory;
 
 import static org.junit.Assert.*;
 
 public class TestVTuple {
 
 	/**
-	 * @throws java.lang.Exception
+	 * @throws Exception
 	 */
 	@Before
 	public void setUp() throws Exception {
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
rename to tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
diff --git a/tajo-storage/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
similarity index 81%
copy from tajo-storage/src/test/resources/storage-default.xml
copy to tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
index f4c81c7..d1c561b 100644
--- a/tajo-storage/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
@@ -25,26 +25,28 @@
     <value>org.apache.tajo.storage.s3.SmallBlockS3FileSystem</value>
   </property>
 
+  <!-- Storage Manager Configuration -->
+  <property>
+    <name>tajo.storage.manager.hdfs.class</name>
+    <value>org.apache.tajo.storage.FileStorageManager</value>
+  </property>
+  <property>
+    <name>tajo.storage.manager.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
+  </property>
+
   <!--- Registered Scanner Handler -->
   <property>
     <name>tajo.storage.scanner-handler</name>
-    <value>textfile,csv,json,raw,rcfile,row,parquet,sequencefile,avro</value>
+    <value>csv,raw,rcfile,row,trevni,parquet,sequencefile,avro</value>
   </property>
 
   <!--- Fragment Class Configurations -->
   <property>
-    <name>tajo.storage.fragment.textfile.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
     <name>tajo.storage.fragment.csv.class</name>
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
   <property>
-    <name>tajo.storage.fragment.json.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
     <name>tajo.storage.fragment.raw.class</name>
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
@@ -57,10 +59,14 @@
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
   <property>
-    <name>tajo.storage.fragment.parquet.class</name>
+    <name>tajo.storage.fragment.trevni.class</name>
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
   <property>
+    <name>tajo.storage.fragment.parquet.class</name>
+    <value>org.apache.tajo.storage.FileFragment</value>
+  </property>
+  <property>
     <name>tajo.storage.fragment.sequencefile.class</name>
     <value>org.apache.tajo.storage.fragment.FileFragment</value>
   </property>
@@ -71,21 +77,11 @@
 
   <!--- Scanner Handler -->
   <property>
-    <name>tajo.storage.scanner-handler.textfile.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
-  </property>
-
-  <property>
     <name>tajo.storage.scanner-handler.csv.class</name>
     <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
   </property>
 
   <property>
-    <name>tajo.storage.scanner-handler.json.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
-  </property>
-
-  <property>
     <name>tajo.storage.scanner-handler.raw.class</name>
     <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
   </property>
@@ -101,6 +97,11 @@
   </property>
 
   <property>
+    <name>tajo.storage.scanner-handler.trevni.class</name>
+    <value>org.apache.tajo.storage.trevni.TrevniScanner</value>
+  </property>
+
+  <property>
     <name>tajo.storage.scanner-handler.parquet.class</name>
     <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
   </property>
@@ -118,12 +119,7 @@
   <!--- Appender Handler -->
   <property>
     <name>tajo.storage.appender-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.textfile.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
+    <value>csv,raw,rcfile,row,trevni,parquet,sequencefile,avro</value>
   </property>
 
   <property>
@@ -132,11 +128,6 @@
   </property>
 
   <property>
-    <name>tajo.storage.appender-handler.json.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
-  </property>
-
-  <property>
     <name>tajo.storage.appender-handler.raw.class</name>
     <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
   </property>
@@ -152,6 +143,11 @@
   </property>
 
   <property>
+    <name>tajo.storage.appender-handler.trevni.class</name>
+    <value>org.apache.tajo.storage.trevni.TrevniAppender</value>
+  </property>
+
+  <property>
     <name>tajo.storage.appender-handler.parquet.class</name>
     <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
   </property>
diff --git a/tajo-storage/tajo-storage-hbase/pom.xml b/tajo-storage/tajo-storage-hbase/pom.xml
new file mode 100644
index 0000000..e37149d
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/pom.xml
@@ -0,0 +1,349 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Copyright 2012 Database Lab., Korea Univ.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.9.1-SNAPSHOT</version>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tajo-storage-hbase</artifactId>
+  <packaging>jar</packaging>
+  <name>Tajo HBase Storage</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/StorageFragmentProtos.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-plan</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>slf4j-api</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jersey-json</artifactId>
+          <groupId>com.sun.jersey</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-server-tests</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-hs</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+      <version>${hbase.version}</version>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
\ No newline at end of file
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
new file mode 100644
index 0000000..8615235
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
@@ -0,0 +1,223 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.TableStatistics;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.TUtil;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An abstract class for HBase appender.
+ */
+public abstract class AbstractHBaseAppender implements Appender {
+  protected Configuration conf;
+  protected Schema schema;
+  protected TableMeta meta;
+  protected QueryUnitAttemptId taskAttemptId;
+  protected Path stagingDir;
+  protected boolean inited = false;
+
+  protected ColumnMapping columnMapping;
+  protected TableStatistics stats;
+  protected boolean enabledStats;
+
+  protected int columnNum;
+
+  protected byte[][][] mappingColumnFamilies;
+  protected boolean[] isBinaryColumns;
+  protected boolean[] isRowKeyMappings;
+  protected boolean[] isColumnKeys;
+  protected boolean[] isColumnValues;
+  protected int[] rowKeyFieldIndexes;
+  protected int[] rowkeyColumnIndexes;
+  protected char rowKeyDelimiter;
+
+  // the following four variables are used for '<cfname>:key:' or '<cfname>:value:' mapping
+  protected int[] columnKeyValueDataIndexes;
+  protected byte[][] columnKeyDatas;
+  protected byte[][] columnValueDatas;
+  protected byte[][] columnKeyCfNames;
+
+  protected KeyValue[] keyValues;
+
+  public AbstractHBaseAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                       Schema schema, TableMeta meta, Path stagingDir) {
+    this.conf = conf;
+    this.schema = schema;
+    this.meta = meta;
+    this.stagingDir = stagingDir;
+    this.taskAttemptId = taskAttemptId;
+  }
+
+  @Override
+  public void init() throws IOException {
+    if (inited) {
+      throw new IllegalStateException("FileAppender is already initialized.");
+    }
+    inited = true;
+    if (enabledStats) {
+      stats = new TableStatistics(this.schema);
+    }
+    columnMapping = new ColumnMapping(schema, meta);
+
+    mappingColumnFamilies = columnMapping.getMappingColumns();
+
+    isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    List<Integer> rowkeyColumnIndexList = new ArrayList<Integer>();
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (isRowKeyMappings[i]) {
+        rowkeyColumnIndexList.add(i);
+      }
+    }
+    rowkeyColumnIndexes = TUtil.toArray(rowkeyColumnIndexList);
+
+    isBinaryColumns = columnMapping.getIsBinaryColumns();
+    isColumnKeys = columnMapping.getIsColumnKeys();
+    isColumnValues = columnMapping.getIsColumnValues();
+    rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
+    rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
+
+    this.columnNum = schema.size();
+
+    // In the case of '<cfname>:key:' or '<cfname>:value:' KeyValue object should be set with the qualifier and value
+    // which are mapped to the same column family.
+    columnKeyValueDataIndexes = new int[isColumnKeys.length];
+    int index = 0;
+    int numKeyValues = 0;
+    Map<String, Integer> cfNameIndexMap = new HashMap<String, Integer>();
+    for (int i = 0; i < isColumnKeys.length; i++) {
+      if (isRowKeyMappings[i]) {
+        continue;
+      }
+      if (isColumnKeys[i] || isColumnValues[i]) {
+        String cfName = new String(mappingColumnFamilies[i][0]);
+        if (!cfNameIndexMap.containsKey(cfName)) {
+          cfNameIndexMap.put(cfName, index);
+          columnKeyValueDataIndexes[i] = index;
+          index++;
+          numKeyValues++;
+        } else {
+          columnKeyValueDataIndexes[i] = cfNameIndexMap.get(cfName);
+        }
+      } else {
+        numKeyValues++;
+      }
+    }
+    columnKeyCfNames = new byte[cfNameIndexMap.size()][];
+    for (Map.Entry<String, Integer> entry: cfNameIndexMap.entrySet()) {
+      columnKeyCfNames[entry.getValue()] = entry.getKey().getBytes();
+    }
+    columnKeyDatas = new byte[cfNameIndexMap.size()][];
+    columnValueDatas = new byte[cfNameIndexMap.size()][];
+
+    keyValues = new KeyValue[numKeyValues];
+  }
+
+  private ByteArrayOutputStream bout = new ByteArrayOutputStream();
+
+  protected byte[] getRowKeyBytes(Tuple tuple) throws IOException {
+    Datum datum;
+    byte[] rowkey;
+    if (rowkeyColumnIndexes.length > 1) {
+      bout.reset();
+      for (int i = 0; i < rowkeyColumnIndexes.length; i++) {
+        datum = tuple.get(rowkeyColumnIndexes[i]);
+        if (isBinaryColumns[rowkeyColumnIndexes[i]]) {
+          rowkey = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(rowkeyColumnIndexes[i]), datum);
+        } else {
+          rowkey = HBaseTextSerializerDeserializer.serialize(schema.getColumn(rowkeyColumnIndexes[i]), datum);
+        }
+        bout.write(rowkey);
+        if (i < rowkeyColumnIndexes.length - 1) {
+          bout.write(rowKeyDelimiter);
+        }
+      }
+      rowkey = bout.toByteArray();
+    } else {
+      int index = rowkeyColumnIndexes[0];
+      datum = tuple.get(index);
+      if (isBinaryColumns[index]) {
+        rowkey = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(index), datum);
+      } else {
+        rowkey = HBaseTextSerializerDeserializer.serialize(schema.getColumn(index), datum);
+      }
+    }
+
+    return rowkey;
+  }
+
+  protected void readKeyValues(Tuple tuple, byte[] rowkey) throws IOException {
+    int keyValIndex = 0;
+    for (int i = 0; i < columnNum; i++) {
+      if (isRowKeyMappings[i]) {
+        continue;
+      }
+      Datum datum = tuple.get(i);
+      byte[] value;
+      if (isBinaryColumns[i]) {
+        value = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(i), datum);
+      } else {
+        value = HBaseTextSerializerDeserializer.serialize(schema.getColumn(i), datum);
+      }
+
+      if (isColumnKeys[i]) {
+        columnKeyDatas[columnKeyValueDataIndexes[i]] = value;
+      } else if (isColumnValues[i]) {
+        columnValueDatas[columnKeyValueDataIndexes[i]] = value;
+      } else {
+        keyValues[keyValIndex] = new KeyValue(rowkey, mappingColumnFamilies[i][0], mappingColumnFamilies[i][1], value);
+        keyValIndex++;
+      }
+    }
+
+    for (int i = 0; i < columnKeyDatas.length; i++) {
+      keyValues[keyValIndex++] = new KeyValue(rowkey, columnKeyCfNames[i], columnKeyDatas[i], columnValueDatas[i]);
+    }
+  }
+
+  @Override
+  public void enableStats() {
+    enabledStats = true;
+  }
+
+  @Override
+  public TableStats getStats() {
+    if (enabledStats) {
+      return stats.getTableStat();
+    } else {
+      return null;
+    }
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
new file mode 100644
index 0000000..79161cc
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.plan.logical.SortNode;
+import org.apache.tajo.plan.logical.SortNode.SortPurpose;
+import org.apache.tajo.plan.logical.UnaryNode;
+import org.apache.tajo.plan.rewrite.RewriteRule;
+import org.apache.tajo.plan.util.PlannerUtil;
+
+public class AddSortForInsertRewriter implements RewriteRule {
+  private int[] sortColumnIndexes;
+  private Column[] sortColumns;
+  public AddSortForInsertRewriter(TableDesc tableDesc, Column[] sortColumns) {
+    this.sortColumns = sortColumns;
+    this.sortColumnIndexes = new int[sortColumns.length];
+
+    Schema tableSchema = tableDesc.getSchema();
+    for (int i = 0; i < sortColumns.length; i++) {
+      sortColumnIndexes[i] = tableSchema.getColumnId(sortColumns[i].getQualifiedName());
+    }
+  }
+
+  @Override
+  public String getName() {
+    return "AddSortForInsertRewriter";
+  }
+
+  @Override
+  public boolean isEligible(LogicalPlan plan) {
+    StoreType storeType = PlannerUtil.getStoreType(plan);
+    return storeType != null;
+  }
+
+  @Override
+  public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+    UnaryNode insertNode = rootNode.getChild();
+    LogicalNode childNode = insertNode.getChild();
+
+    Schema sortSchema = childNode.getOutSchema();
+    SortNode sortNode = plan.createNode(SortNode.class);
+    sortNode.setSortPurpose(SortPurpose.STORAGE_SPECIFIED);
+    sortNode.setInSchema(sortSchema);
+    sortNode.setOutSchema(sortSchema);
+
+    SortSpec[] sortSpecs = new SortSpec[sortColumns.length];
+    int index = 0;
+
+    for (int i = 0; i < sortColumnIndexes.length; i++) {
+      Column sortColumn = sortSchema.getColumn(sortColumnIndexes[i]);
+      if (sortColumn == null) {
+        throw new PlanningException("Can't fine proper sort column:" + sortColumns[i]);
+      }
+      sortSpecs[index++] = new SortSpec(sortColumn, true, true);
+    }
+    sortNode.setSortSpecs(sortSpecs);
+
+    sortNode.setChild(insertNode.getChild());
+    insertNode.setChild(sortNode);
+    plan.getRootBlock().registerNode(sortNode);
+
+    return plan;
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
new file mode 100644
index 0000000..7ddf09a
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
@@ -0,0 +1,236 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.util.BytesUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ColumnMapping {
+  private TableMeta tableMeta;
+  private Schema schema;
+  private char rowKeyDelimiter;
+
+  private String hbaseTableName;
+
+  private int[] rowKeyFieldIndexes;
+  private boolean[] isRowKeyMappings;
+  private boolean[] isBinaryColumns;
+  private boolean[] isColumnKeys;
+  private boolean[] isColumnValues;
+
+  // schema order -> 0: cf name, 1: column name -> name bytes
+  private byte[][][] mappingColumns;
+
+  private int numRowKeys;
+
+  public ColumnMapping(Schema schema, TableMeta tableMeta) throws IOException {
+    this.schema = schema;
+    this.tableMeta = tableMeta;
+
+    init();
+  }
+
+  public void init() throws IOException {
+    hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY);
+    String delim = tableMeta.getOption(HBaseStorageConstants.META_ROWKEY_DELIMITER, "").trim();
+    if (delim.length() > 0) {
+      rowKeyDelimiter = delim.charAt(0);
+    }
+    isRowKeyMappings = new boolean[schema.size()];
+    rowKeyFieldIndexes = new int[schema.size()];
+    isBinaryColumns = new boolean[schema.size()];
+    isColumnKeys = new boolean[schema.size()];
+    isColumnValues = new boolean[schema.size()];
+
+    mappingColumns = new byte[schema.size()][][];
+
+    for (int i = 0; i < schema.size(); i++) {
+      rowKeyFieldIndexes[i] = -1;
+    }
+
+    String columnMapping = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
+    if (columnMapping == null || columnMapping.isEmpty()) {
+      throw new IOException("'columns' property is required.");
+    }
+
+    String[] columnMappingTokens = columnMapping.split(",");
+
+    if (columnMappingTokens.length != schema.getColumns().size()) {
+      throw new IOException("The number of mapped HBase columns is great than the number of Tajo table columns");
+    }
+
+    int index = 0;
+    for (String eachToken: columnMappingTokens) {
+      mappingColumns[index] = new byte[2][];
+
+      byte[][] mappingTokens = BytesUtils.splitPreserveAllTokens(eachToken.trim().getBytes(), ':');
+
+      if (mappingTokens.length == 3) {
+        if (mappingTokens[0].length == 0) {
+          // cfname
+          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
+              "or '<cfname>:value:' or '<cfname>:value:#b'");
+        }
+        //<cfname>:key: or <cfname>:value:
+        if (mappingTokens[2].length != 0) {
+          String binaryOption = new String(mappingTokens[2]);
+          if ("#b".equals(binaryOption)) {
+            isBinaryColumns[index] = true;
+          } else {
+            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
+                "or '<cfname>:value:' or '<cfname>:value:#b'");
+          }
+        }
+        mappingColumns[index][0] = mappingTokens[0];
+        String keyOrValue = new String(mappingTokens[1]);
+        if (HBaseStorageConstants.KEY_COLUMN_MAPPING.equalsIgnoreCase(keyOrValue)) {
+          isColumnKeys[index] = true;
+        } else if (HBaseStorageConstants.VALUE_COLUMN_MAPPING.equalsIgnoreCase(keyOrValue)) {
+          isColumnValues[index] = true;
+        } else {
+          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
+        }
+      } else if (mappingTokens.length == 2) {
+        //<cfname>: or <cfname>:<qualifier> or :key
+        String cfName = new String(mappingTokens[0]);
+        String columnName = new String(mappingTokens[1]);
+        RowKeyMapping rowKeyMapping = getRowKeyMapping(cfName, columnName);
+        if (rowKeyMapping != null) {
+          isRowKeyMappings[index] = true;
+          numRowKeys++;
+          isBinaryColumns[index] = rowKeyMapping.isBinary();
+          if (!cfName.isEmpty()) {
+            if (rowKeyDelimiter == 0) {
+              throw new IOException("hbase.rowkey.delimiter is required.");
+            }
+            rowKeyFieldIndexes[index] = Integer.parseInt(cfName);
+          } else {
+            rowKeyFieldIndexes[index] = -1; //rowkey is mapped a single column.
+          }
+        } else {
+          if (cfName.isEmpty()) {
+            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
+          }
+          if (cfName != null) {
+            mappingColumns[index][0] = Bytes.toBytes(cfName);
+          }
+
+          if (columnName != null && !columnName.isEmpty()) {
+            String[] columnNameTokens = columnName.split("#");
+            if (columnNameTokens[0].isEmpty()) {
+              mappingColumns[index][1] = null;
+            } else {
+              mappingColumns[index][1] = Bytes.toBytes(columnNameTokens[0]);
+            }
+            if (columnNameTokens.length == 2 && "b".equals(columnNameTokens[1])) {
+              isBinaryColumns[index] = true;
+            }
+          }
+        }
+      } else {
+        throw new IOException(eachToken + " 'column' attribute '[cfname]:[qualfier]:'");
+      }
+
+      index++;
+    } // for loop
+  }
+
+  public List<String> getColumnFamilyNames() {
+    List<String> cfNames = new ArrayList<String>();
+
+    for (byte[][] eachCfName: mappingColumns) {
+      if (eachCfName != null && eachCfName.length > 0 && eachCfName[0] != null) {
+        String cfName = new String(eachCfName[0]);
+        if (!cfNames.contains(cfName)) {
+          cfNames.add(cfName);
+        }
+      }
+    }
+
+    return cfNames;
+  }
+
+  private RowKeyMapping getRowKeyMapping(String cfName, String columnName) {
+    if (columnName == null || columnName.isEmpty()) {
+      return null;
+    }
+
+    String[] tokens = columnName.split("#");
+    if (!tokens[0].equalsIgnoreCase(HBaseStorageConstants.KEY_COLUMN_MAPPING)) {
+      return null;
+    }
+
+    RowKeyMapping rowKeyMapping = new RowKeyMapping();
+
+    if (tokens.length == 2 && "b".equals(tokens[1])) {
+      rowKeyMapping.setBinary(true);
+    }
+
+    if (cfName != null && !cfName.isEmpty()) {
+      rowKeyMapping.setKeyFieldIndex(Integer.parseInt(cfName));
+    }
+    return rowKeyMapping;
+  }
+
+  public char getRowKeyDelimiter() {
+    return rowKeyDelimiter;
+  }
+
+  public int[] getRowKeyFieldIndexes() {
+    return rowKeyFieldIndexes;
+  }
+
+  public boolean[] getIsRowKeyMappings() {
+    return isRowKeyMappings;
+  }
+
+  public byte[][][] getMappingColumns() {
+    return mappingColumns;
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  public boolean[] getIsBinaryColumns() {
+    return isBinaryColumns;
+  }
+
+  public String getHbaseTableName() {
+    return hbaseTableName;
+  }
+
+  public boolean[] getIsColumnKeys() {
+    return isColumnKeys;
+  }
+
+  public int getNumRowKeys() {
+    return numRowKeys;
+  }
+
+  public boolean[] getIsColumnValues() {
+    return isColumnValues;
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
new file mode 100644
index 0000000..c05c5bb
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.util.Bytes;
+
+import java.io.IOException;
+
+public class HBaseBinarySerializerDeserializer {
+
+  public static Datum deserialize(Column col, byte[] bytes) throws IOException {
+    Datum datum;
+    switch (col.getDataType().getType()) {
+      case INT1:
+      case INT2:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt2(Bytes.toShort(bytes));
+        break;
+      case INT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt4(Bytes.toInt(bytes));
+        break;
+      case INT8:
+        if (bytes.length == 4) {
+          datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt8(Bytes.toInt(bytes));
+        } else {
+          datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt8(Bytes.toLong(bytes));
+        }
+        break;
+      case FLOAT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createFloat4(Bytes.toFloat(bytes));
+        break;
+      case FLOAT8:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createFloat8(Bytes.toDouble(bytes));
+        break;
+      case TEXT:
+        datum = bytes == null ? NullDatum.get() : DatumFactory.createText(bytes);
+        break;
+      default:
+        datum = NullDatum.get();
+        break;
+    }
+    return datum;
+  }
+
+  public static byte[] serialize(Column col, Datum datum) throws IOException {
+    if (datum == null || datum instanceof NullDatum) {
+      return null;
+    }
+
+    byte[] bytes;
+    switch (col.getDataType().getType()) {
+      case INT1:
+      case INT2:
+        bytes = Bytes.toBytes(datum.asInt2());
+        break;
+      case INT4:
+        bytes = Bytes.toBytes(datum.asInt4());
+        break;
+      case INT8:
+        bytes = Bytes.toBytes(datum.asInt8());
+        break;
+      case FLOAT4:
+        bytes = Bytes.toBytes(datum.asFloat4());
+        break;
+      case FLOAT8:
+        bytes = Bytes.toBytes(datum.asFloat8());
+        break;
+      case TEXT:
+        bytes = Bytes.toBytes(datum.asChars());
+        break;
+      default:
+        bytes = null;
+        break;
+    }
+
+    return bytes;
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java
new file mode 100644
index 0000000..2674511
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java
@@ -0,0 +1,198 @@
+/**
+ * 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.storage.hbase;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.hbase.StorageFragmentProtos.*;
+
+public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Cloneable {
+  @Expose
+  private String tableName;
+  @Expose
+  private String hbaseTableName;
+  @Expose
+  private byte[] startRow;
+  @Expose
+  private byte[] stopRow;
+  @Expose
+  private String regionLocation;
+  @Expose
+  private boolean last;
+  @Expose
+  private long length;
+
+  public HBaseFragment(String tableName, String hbaseTableName, byte[] startRow, byte[] stopRow, String regionLocation) {
+    this.tableName = tableName;
+    this.hbaseTableName = hbaseTableName;
+    this.startRow = startRow;
+    this.stopRow = stopRow;
+    this.regionLocation = regionLocation;
+    this.last = false;
+  }
+
+  public HBaseFragment(ByteString raw) throws InvalidProtocolBufferException {
+    HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder();
+    builder.mergeFrom(raw);
+    builder.build();
+    init(builder.build());
+  }
+
+  private void init(HBaseFragmentProto proto) {
+    this.tableName = proto.getTableName();
+    this.hbaseTableName = proto.getHbaseTableName();
+    this.startRow = proto.getStartRow().toByteArray();
+    this.stopRow = proto.getStopRow().toByteArray();
+    this.regionLocation = proto.getRegionLocation();
+    this.length = proto.getLength();
+    this.last = proto.getLast();
+  }
+
+  @Override
+  public int compareTo(HBaseFragment t) {
+    return Bytes.compareTo(startRow, t.startRow);
+  }
+
+  @Override
+  public String getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public String getKey() {
+    return new String(startRow);
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return startRow == null || stopRow == null;
+  }
+
+  @Override
+  public long getLength() {
+    return length;
+  }
+
+  public void setLength(long length) {
+    this.length = length;
+  }
+
+  @Override
+  public String[] getHosts() {
+    return new String[] {regionLocation};
+  }
+
+  public Object clone() throws CloneNotSupportedException {
+    HBaseFragment frag = (HBaseFragment) super.clone();
+    frag.tableName = tableName;
+    frag.hbaseTableName = hbaseTableName;
+    frag.startRow = startRow;
+    frag.stopRow = stopRow;
+    frag.regionLocation = regionLocation;
+    frag.last = last;
+    frag.length = length;
+    return frag;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof HBaseFragment) {
+      HBaseFragment t = (HBaseFragment) o;
+      if (tableName.equals(t.tableName)
+          && Bytes.equals(startRow, t.startRow)
+          && Bytes.equals(stopRow, t.stopRow)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(tableName, hbaseTableName, startRow, stopRow);
+  }
+
+  @Override
+  public String toString() {
+    return "\"fragment\": {\"tableName\": \""+ tableName + "\", hbaseTableName\": \"" + hbaseTableName + "\"" +
+        ", \"startRow\": \"" + new String(startRow) + "\"" +
+        ", \"stopRow\": \"" + new String(stopRow) + "\"" +
+        ", \"length\": \"" + length + "\"}" ;
+  }
+
+  @Override
+  public FragmentProto getProto() {
+    HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder();
+    builder.setTableName(tableName)
+        .setHbaseTableName(hbaseTableName)
+        .setStartRow(ByteString.copyFrom(startRow))
+        .setStopRow(ByteString.copyFrom(stopRow))
+        .setLast(last)
+        .setLength(length)
+        .setRegionLocation(regionLocation);
+
+    FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder();
+    fragmentBuilder.setId(this.tableName);
+    fragmentBuilder.setContents(builder.buildPartial().toByteString());
+    fragmentBuilder.setStoreType(StoreType.HBASE.name());
+    return fragmentBuilder.build();
+  }
+
+  public byte[] getStartRow() {
+    return startRow;
+  }
+
+  public byte[] getStopRow() {
+    return stopRow;
+  }
+
+  public String getRegionLocation() {
+    return regionLocation;
+  }
+
+  public boolean isLast() {
+    return last;
+  }
+
+  public void setLast(boolean last) {
+    this.last = last;
+  }
+
+  public String getHbaseTableName() {
+    return hbaseTableName;
+  }
+
+  public void setHbaseTableName(String hbaseTableName) {
+    this.hbaseTableName = hbaseTableName;
+  }
+
+  public void setStartRow(byte[] startRow) {
+    this.startRow = startRow;
+  }
+
+  public void setStopRow(byte[] stopRow) {
+    this.stopRow = stopRow;
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
new file mode 100644
index 0000000..50f61a8
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
@@ -0,0 +1,120 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public class HBasePutAppender extends AbstractHBaseAppender {
+  private HTableInterface htable;
+  private long totalNumBytes;
+
+  public HBasePutAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                          Schema schema, TableMeta meta, Path stagingDir) {
+    super(conf, taskAttemptId, schema, meta, stagingDir);
+  }
+
+  @Override
+  public void init() throws IOException {
+    super.init();
+
+    Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(conf, meta);
+    HConnection hconn = ((HBaseStorageManager) StorageManager.getStorageManager((TajoConf)conf, StoreType.HBASE))
+        .getConnection(hbaseConf);
+    htable = hconn.getTable(columnMapping.getHbaseTableName());
+    htable.setAutoFlushTo(false);
+    htable.setWriteBufferSize(5 * 1024 * 1024);
+  }
+
+  @Override
+  public void addTuple(Tuple tuple) throws IOException {
+    byte[] rowkey = getRowKeyBytes(tuple);
+    totalNumBytes += rowkey.length;
+    Put put = new Put(rowkey);
+    readKeyValues(tuple, rowkey);
+
+    for (int i = 0; i < columnNum; i++) {
+      if (isRowKeyMappings[i]) {
+        continue;
+      }
+      Datum datum = tuple.get(i);
+      byte[] value;
+      if (isBinaryColumns[i]) {
+        value = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(i), datum);
+      } else {
+        value = HBaseTextSerializerDeserializer.serialize(schema.getColumn(i), datum);
+      }
+
+      if (isColumnKeys[i]) {
+        columnKeyDatas[columnKeyValueDataIndexes[i]] = value;
+      } else if (isColumnValues[i]) {
+        columnValueDatas[columnKeyValueDataIndexes[i]] = value;
+      } else {
+        put.add(mappingColumnFamilies[i][0], mappingColumnFamilies[i][1], value);
+        totalNumBytes += value.length;
+      }
+    }
+
+    for (int i = 0; i < columnKeyDatas.length; i++) {
+     put.add(columnKeyCfNames[i], columnKeyDatas[i], columnValueDatas[i]);
+      totalNumBytes += columnKeyDatas[i].length + columnValueDatas[i].length;
+    }
+
+    htable.put(put);
+
+    if (enabledStats) {
+      stats.incrementRow();
+      stats.setNumBytes(totalNumBytes);
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    htable.flushCommits();
+  }
+
+  @Override
+  public long getEstimatedOutputSize() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (htable != null) {
+      htable.flushCommits();
+      htable.close();
+    }
+    if (enabledStats) {
+      stats.setNumBytes(totalNumBytes);
+    }
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
new file mode 100644
index 0000000..5cae077
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
@@ -0,0 +1,449 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.BytesUtils;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class HBaseScanner implements Scanner {
+  private static final Log LOG = LogFactory.getLog(HBaseScanner.class);
+  private static final int DEFAULT_FETCH_SIZE = 1000;
+  private static final int MAX_LIST_SIZE = 100;
+
+  protected boolean inited = false;
+  private TajoConf conf;
+  private Schema schema;
+  private TableMeta meta;
+  private HBaseFragment fragment;
+  private Scan scan;
+  private HTableInterface htable;
+  private Configuration hbaseConf;
+  private Column[] targets;
+  private TableStats tableStats;
+  private ResultScanner scanner;
+  private AtomicBoolean finished = new AtomicBoolean(false);
+  private float progress = 0.0f;
+  private int scanFetchSize;
+  private Result[] scanResults;
+  private int scanResultIndex = -1;
+  private Column[] schemaColumns;
+
+  private ColumnMapping columnMapping;
+  private int[] targetIndexes;
+
+  private int numRows = 0;
+
+  private byte[][][] mappingColumnFamilies;
+  private boolean[] isRowKeyMappings;
+  private boolean[] isBinaryColumns;
+  private boolean[] isColumnKeys;
+  private boolean[] isColumnValues;
+
+  private int[] rowKeyFieldIndexes;
+  private char rowKeyDelimiter;
+
+  public HBaseScanner (Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
+    this.conf = (TajoConf)conf;
+    this.schema = schema;
+    this.meta = meta;
+    this.fragment = (HBaseFragment)fragment;
+    this.tableStats = new TableStats();
+  }
+
+  @Override
+  public void init() throws IOException {
+    inited = true;
+    schemaColumns = schema.toArray();
+    if (fragment != null) {
+      tableStats.setNumBytes(0);
+      tableStats.setNumBlocks(1);
+    }
+    if (schema != null) {
+      for(Column eachColumn: schema.getColumns()) {
+        ColumnStats columnStats = new ColumnStats(eachColumn);
+        tableStats.addColumnStat(columnStats);
+      }
+    }
+
+    scanFetchSize = Integer.parseInt(
+        meta.getOption(HBaseStorageConstants.META_FETCH_ROWNUM_KEY, "" + DEFAULT_FETCH_SIZE));
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+
+    columnMapping = new ColumnMapping(schema, meta);
+    targetIndexes = new int[targets.length];
+    int index = 0;
+    for (Column eachTargetColumn: targets) {
+      targetIndexes[index++] = schema.getColumnId(eachTargetColumn.getQualifiedName());
+    }
+
+    mappingColumnFamilies = columnMapping.getMappingColumns();
+    isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    isBinaryColumns = columnMapping.getIsBinaryColumns();
+    isColumnKeys = columnMapping.getIsColumnKeys();
+    isColumnValues = columnMapping.getIsColumnValues();
+
+    rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
+    rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
+
+    hbaseConf = HBaseStorageManager.getHBaseConfiguration(conf, meta);
+
+    initScanner();
+  }
+
+  private void initScanner() throws IOException {
+    scan = new Scan();
+    scan.setBatch(scanFetchSize);
+    scan.setCacheBlocks(false);
+    scan.setCaching(scanFetchSize);
+
+    FilterList filters = null;
+    if (targetIndexes == null || targetIndexes.length == 0) {
+      filters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
+      filters.addFilter(new FirstKeyOnlyFilter());
+      filters.addFilter(new KeyOnlyFilter());
+    } else {
+      boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+      for (int eachIndex : targetIndexes) {
+        if (isRowKeyMappings[eachIndex]) {
+          continue;
+        }
+        byte[][] mappingColumn = columnMapping.getMappingColumns()[eachIndex];
+        if (mappingColumn[1] == null) {
+          scan.addFamily(mappingColumn[0]);
+        } else {
+          scan.addColumn(mappingColumn[0], mappingColumn[1]);
+        }
+      }
+    }
+
+    scan.setStartRow(fragment.getStartRow());
+    if (fragment.isLast() && fragment.getStopRow() != null &&
+        fragment.getStopRow().length > 0) {
+      // last and stopRow is not empty
+      if (filters == null) {
+        filters = new FilterList();
+      }
+      filters.addFilter(new InclusiveStopFilter(fragment.getStopRow()));
+    } else {
+      scan.setStopRow(fragment.getStopRow());
+    }
+
+    if (filters != null) {
+      scan.setFilter(filters);
+    }
+
+    if (htable == null) {
+      HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
+          .getConnection(hbaseConf);
+      htable = hconn.getTable(fragment.getHbaseTableName());
+    }
+    scanner = htable.getScanner(scan);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (finished.get()) {
+      return null;
+    }
+
+    if (scanResults == null || scanResultIndex >= scanResults.length) {
+      scanResults = scanner.next(scanFetchSize);
+      if (scanResults == null || scanResults.length == 0) {
+        finished.set(true);
+        progress = 1.0f;
+        return null;
+      }
+      scanResultIndex = 0;
+    }
+
+    Result result = scanResults[scanResultIndex++];
+    Tuple resultTuple = new VTuple(schema.size());
+    for (int i = 0; i < targetIndexes.length; i++) {
+      resultTuple.put(targetIndexes[i], getDatum(result, targetIndexes[i]));
+    }
+    numRows++;
+    return resultTuple;
+  }
+
+  private Datum getDatum(Result result, int fieldId) throws IOException {
+    byte[] value = null;
+    if (isRowKeyMappings[fieldId]) {
+      value = result.getRow();
+      if (!isBinaryColumns[fieldId] && rowKeyFieldIndexes[fieldId] >= 0) {
+        int rowKeyFieldIndex = rowKeyFieldIndexes[fieldId];
+
+        byte[][] rowKeyFields = BytesUtils.splitPreserveAllTokens(value, rowKeyDelimiter);
+
+        if (rowKeyFields.length < rowKeyFieldIndex) {
+          return NullDatum.get();
+        } else {
+          value = rowKeyFields[rowKeyFieldIndex];
+        }
+      }
+    } else {
+      if (isColumnKeys[fieldId]) {
+        NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
+        if (cfMap != null) {
+          Set<byte[]> keySet = cfMap.keySet();
+          if (keySet.size() == 1) {
+            try {
+              return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], keySet.iterator().next());
+            } catch (Exception e) {
+              LOG.error(e.getMessage(), e);
+              throw new RuntimeException(e.getMessage(), e);
+            }
+          } else {
+            StringBuilder sb = new StringBuilder();
+            sb.append("[");
+            int count = 0;
+            for (byte[] eachKey : keySet) {
+              if (count > 0) {
+                sb.append(", ");
+              }
+              Datum datum = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], eachKey);
+              sb.append("\"").append(datum.asChars()).append("\"");
+              count++;
+              if (count > MAX_LIST_SIZE) {
+                break;
+              }
+            }
+            sb.append("]");
+            return new TextDatum(sb.toString());
+          }
+        }
+      } else if (isColumnValues[fieldId]) {
+        NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
+        if (cfMap != null) {
+          Collection<byte[]> valueList = cfMap.values();
+          if (valueList.size() == 1) {
+            try {
+              return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], valueList.iterator().next());
+            } catch (Exception e) {
+              LOG.error(e.getMessage(), e);
+              throw new RuntimeException(e.getMessage(), e);
+            }
+          } else {
+            StringBuilder sb = new StringBuilder();
+            sb.append("[");
+            int count = 0;
+            for (byte[] eachValue : valueList) {
+              if (count > 0) {
+                sb.append(", ");
+              }
+              Datum datum = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], eachValue);
+              sb.append("\"").append(datum.asChars()).append("\"");
+              count++;
+              if (count > MAX_LIST_SIZE) {
+                break;
+              }
+            }
+            sb.append("]");
+            return new TextDatum(sb.toString());
+          }
+        }
+      } else {
+        if (mappingColumnFamilies[fieldId][1] == null) {
+          NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
+          if (cfMap != null && !cfMap.isEmpty()) {
+            int count = 0;
+            String delim = "";
+
+            if (cfMap.size() == 0) {
+              return NullDatum.get();
+            } else if (cfMap.size() == 1) {
+              // If a column family is mapped without column name like "cf1:" and the number of cells is one,
+              // return value is flat format not json format.
+              NavigableMap.Entry<byte[], byte[]> entry = cfMap.entrySet().iterator().next();
+              byte[] entryKey = entry.getKey();
+              byte[] entryValue = entry.getValue();
+              if (entryKey == null || entryKey.length == 0) {
+                try {
+                  if (isBinaryColumns[fieldId]) {
+                    return HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue);
+                  } else {
+                    return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue);
+                  }
+                } catch (Exception e) {
+                  LOG.error(e.getMessage(), e);
+                  throw new RuntimeException(e.getMessage(), e);
+                }
+              }
+            }
+            StringBuilder sb = new StringBuilder();
+            sb.append("{");
+            for (NavigableMap.Entry<byte[], byte[]> entry : cfMap.entrySet()) {
+              byte[] entryKey = entry.getKey();
+              byte[] entryValue = entry.getValue();
+
+              String keyText = new String(entryKey);
+              String valueText = null;
+              if (entryValue != null) {
+                try {
+                  if (isBinaryColumns[fieldId]) {
+                    valueText = HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue).asChars();
+                  } else {
+                    valueText = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue).asChars();
+                  }
+                } catch (Exception e) {
+                  LOG.error(e.getMessage(), e);
+                  throw new RuntimeException(e.getMessage(), e);
+                }
+              }
+              sb.append(delim).append("\"").append(keyText).append("\":\"").append(valueText).append("\"");
+              delim = ", ";
+              count++;
+              if (count > MAX_LIST_SIZE) {
+                break;
+              }
+            } //end of for
+            sb.append("}");
+            return new TextDatum(sb.toString());
+          } else {
+            value = null;
+          }
+        } else {
+          value = result.getValue(mappingColumnFamilies[fieldId][0], mappingColumnFamilies[fieldId][1]);
+        }
+      }
+    }
+
+    if (value == null) {
+      return NullDatum.get();
+    } else {
+      try {
+        if (isBinaryColumns[fieldId]) {
+          return HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], value);
+        } else {
+          return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], value);
+        }
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new RuntimeException(e.getMessage(), e);
+      }
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    progress = 0.0f;
+    scanResultIndex = -1;
+    scanResults = null;
+    finished.set(false);
+    tableStats = new TableStats();
+
+    if (scanner != null) {
+      scanner.close();
+      scanner = null;
+    }
+
+    initScanner();
+  }
+
+  @Override
+  public void close() throws IOException {
+    progress = 1.0f;
+    finished.set(true);
+    if (scanner != null) {
+      try {
+        scanner.close();
+        scanner = null;
+      } catch (Exception e) {
+        LOG.warn("Error while closing hbase scanner: " + e.getMessage(), e);
+      }
+    }
+    if (htable != null) {
+      htable.close();
+      htable = null;
+    }
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  @Override
+  public void setTarget(Column[] targets) {
+    if (inited) {
+      throw new IllegalStateException("Should be called before init()");
+    }
+    this.targets = targets;
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return false;
+  }
+
+  @Override
+  public void setSearchCondition(Object expr) {
+    // TODO implements adding column filter to scanner.
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return true;
+  }
+
+  @Override
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    tableStats.setNumRows(numRows);
+    return tableStats;
+  }
+
+  @Override
+  public Schema getSchema() {
+    return schema;
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
new file mode 100644
index 0000000..2c525a1
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
@@ -0,0 +1,33 @@
+/**
+ * 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.storage.hbase;
+
+public interface HBaseStorageConstants {
+  public static final String KEY_COLUMN_MAPPING = "key";
+  public static final String VALUE_COLUMN_MAPPING = "value";
+  public static final String META_FETCH_ROWNUM_KEY = "fetch.rownum";
+  public static final String META_TABLE_KEY = "table";
+  public static final String META_COLUMNS_KEY = "columns";
+  public static final String META_SPLIT_ROW_KEYS_KEY = "hbase.split.rowkeys";
+  public static final String META_SPLIT_ROW_KEYS_FILE_KEY = "hbase.split.rowkeys.file";
+  public static final String META_ZK_QUORUM_KEY = "hbase.zookeeper.quorum";
+  public static final String META_ROWKEY_DELIMITER = "hbase.rowkey.delimiter";
+
+  public static final String INSERT_PUT_MODE = "tajo.hbase.insert.put.mode";
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
new file mode 100644
index 0000000..a6e7a81
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
@@ -0,0 +1,1135 @@
+/**
+ * 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.storage.hbase;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.expr.*;
+import org.apache.tajo.plan.logical.CreateTableNode;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.NodeType;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.plan.rewrite.RewriteRule;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.Bytes;
+import org.apache.tajo.util.BytesUtils;
+import org.apache.tajo.util.Pair;
+import org.apache.tajo.util.TUtil;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.*;
+
+/**
+ * StorageManager for HBase table.
+ */
+public class HBaseStorageManager extends StorageManager {
+  private static final Log LOG = LogFactory.getLog(HBaseStorageManager.class);
+
+  private Map<HConnectionKey, HConnection> connMap = new HashMap<HConnectionKey, HConnection>();
+
+  public HBaseStorageManager (StoreType storeType) {
+    super(storeType);
+  }
+
+  @Override
+  public void storageInit() throws IOException {
+  }
+
+  @Override
+  public void closeStorageManager() {
+    synchronized (connMap) {
+      for (HConnection eachConn: connMap.values()) {
+        try {
+          eachConn.close();
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
+    createTable(tableDesc.getMeta(), tableDesc.getSchema(), tableDesc.isExternal(), ifNotExists);
+    TableStats stats = new TableStats();
+    stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
+    tableDesc.setStats(stats);
+  }
+
+  private void createTable(TableMeta tableMeta, Schema schema,
+                           boolean isExternal, boolean ifNotExists) throws IOException {
+    String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
+    if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
+      throw new IOException("HBase mapped table is required a '" +
+          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
+    }
+    TableName hTableName = TableName.valueOf(hbaseTableName);
+
+    String mappedColumns = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
+    if (mappedColumns != null && mappedColumns.split(",").length > schema.size()) {
+      throw new IOException("Columns property has more entry than Tajo table columns");
+    }
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+    int numRowKeys = 0;
+    boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (isRowKeyMappings[i]) {
+        numRowKeys++;
+      }
+    }
+    if (numRowKeys > 1) {
+      for (int i = 0; i < isRowKeyMappings.length; i++) {
+        if (isRowKeyMappings[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
+          throw new IOException("Key field type should be TEXT type.");
+        }
+      }
+    }
+
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (columnMapping.getIsColumnKeys()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
+        throw new IOException("Column key field('<cfname>:key:') type should be TEXT type.");
+      }
+      if (columnMapping.getIsColumnValues()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
+        throw new IOException("Column value field(('<cfname>:value:') type should be TEXT type.");
+      }
+    }
+
+    Configuration hConf = getHBaseConfiguration(conf, tableMeta);
+    HBaseAdmin hAdmin =  new HBaseAdmin(hConf);
+
+    try {
+      if (isExternal) {
+        // If tajo table is external table, only check validation.
+        if (mappedColumns == null || mappedColumns.isEmpty()) {
+          throw new IOException("HBase mapped table is required a '" +
+              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
+        }
+        if (!hAdmin.tableExists(hTableName)) {
+          throw new IOException("HBase table [" + hbaseTableName + "] not exists. " +
+              "External table should be a existed table.");
+        }
+        HTableDescriptor hTableDescriptor = hAdmin.getTableDescriptor(hTableName);
+        Set<String> tableColumnFamilies = new HashSet<String>();
+        for (HColumnDescriptor eachColumn : hTableDescriptor.getColumnFamilies()) {
+          tableColumnFamilies.add(eachColumn.getNameAsString());
+        }
+
+        Collection<String> mappingColumnFamilies =columnMapping.getColumnFamilyNames();
+        if (mappingColumnFamilies.isEmpty()) {
+          throw new IOException("HBase mapped table is required a '" +
+              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
+        }
+
+        for (String eachMappingColumnFamily : mappingColumnFamilies) {
+          if (!tableColumnFamilies.contains(eachMappingColumnFamily)) {
+            throw new IOException("There is no " + eachMappingColumnFamily + " column family in " + hbaseTableName);
+          }
+        }
+      } else {
+        if (hAdmin.tableExists(hbaseTableName)) {
+          if (ifNotExists) {
+            return;
+          } else {
+            throw new IOException("HBase table [" + hbaseTableName + "] already exists.");
+          }
+        }
+        // Creating hbase table
+        HTableDescriptor hTableDescriptor = parseHTableDescriptor(tableMeta, schema);
+
+        byte[][] splitKeys = getSplitKeys(conf, schema, tableMeta);
+        if (splitKeys == null) {
+          hAdmin.createTable(hTableDescriptor);
+        } else {
+          hAdmin.createTable(hTableDescriptor, splitKeys);
+        }
+      }
+    } finally {
+      hAdmin.close();
+    }
+  }
+
+  /**
+   * Returns initial region split keys.
+   *
+   * @param conf
+   * @param schema
+   * @param meta
+   * @return
+   * @throws java.io.IOException
+   */
+  private byte[][] getSplitKeys(TajoConf conf, Schema schema, TableMeta meta) throws IOException {
+    String splitRowKeys = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_KEY, "");
+    String splitRowKeysFile = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_FILE_KEY, "");
+
+    if ((splitRowKeys == null || splitRowKeys.isEmpty()) &&
+        (splitRowKeysFile == null || splitRowKeysFile.isEmpty())) {
+      return null;
+    }
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, meta);
+    boolean[] isBinaryColumns = columnMapping.getIsBinaryColumns();
+    boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
+
+    boolean rowkeyBinary = false;
+    int numRowKeys = 0;
+    Column rowKeyColumn = null;
+    for (int i = 0; i < isBinaryColumns.length; i++) {
+      if (isBinaryColumns[i] && isRowKeys[i]) {
+        rowkeyBinary = true;
+      }
+      if (isRowKeys[i]) {
+        numRowKeys++;
+        rowKeyColumn = schema.getColumn(i);
+      }
+    }
+
+    if (rowkeyBinary && numRowKeys > 1) {
+      throw new IOException("If rowkey is mapped to multi column and a rowkey is binary, " +
+          "Multiple region for creation is not support.");
+    }
+
+    if (splitRowKeys != null && !splitRowKeys.isEmpty()) {
+      String[] splitKeyTokens = splitRowKeys.split(",");
+      byte[][] splitKeys = new byte[splitKeyTokens.length][];
+      for (int i = 0; i < splitKeyTokens.length; i++) {
+        if (numRowKeys == 1 && rowkeyBinary) {
+          splitKeys[i] = HBaseBinarySerializerDeserializer.serialize(rowKeyColumn, new TextDatum(splitKeyTokens[i]));
+        } else {
+          splitKeys[i] = HBaseTextSerializerDeserializer.serialize(rowKeyColumn, new TextDatum(splitKeyTokens[i]));
+        }
+      }
+      return splitKeys;
+    }
+
+    if (splitRowKeysFile != null && !splitRowKeysFile.isEmpty()) {
+      // If there is many split keys, Tajo allows to define in the file.
+      Path path = new Path(splitRowKeysFile);
+      FileSystem fs = path.getFileSystem(conf);
+      if (!fs.exists(path)) {
+        throw new IOException("hbase.split.rowkeys.file=" + path.toString() + " not exists.");
+      }
+
+      SortedSet<String> splitKeySet = new TreeSet<String>();
+      BufferedReader reader = null;
+      try {
+        reader = new BufferedReader(new InputStreamReader(fs.open(path)));
+        String line = null;
+        while ( (line = reader.readLine()) != null ) {
+          if (line.isEmpty()) {
+            continue;
+          }
+          splitKeySet.add(line);
+        }
+      } finally {
+        if (reader != null) {
+          reader.close();
+        }
+      }
+
+      if (splitKeySet.isEmpty()) {
+        return null;
+      }
+
+      byte[][] splitKeys = new byte[splitKeySet.size()][];
+      int index = 0;
+      for (String eachKey: splitKeySet) {
+        if (numRowKeys == 1 && rowkeyBinary) {
+          splitKeys[index++] = HBaseBinarySerializerDeserializer.serialize(rowKeyColumn, new TextDatum(eachKey));
+        } else {
+          splitKeys[index++] = HBaseTextSerializerDeserializer.serialize(rowKeyColumn, new TextDatum(eachKey));
+        }
+      }
+
+      return splitKeys;
+    }
+
+    return null;
+  }
+
+  /**
+   * Creates Configuration instance and sets with hbase connection options.
+   *
+   * @param conf
+   * @param tableMeta
+   * @return
+   * @throws java.io.IOException
+   */
+  public static Configuration getHBaseConfiguration(Configuration conf, TableMeta tableMeta) throws IOException {
+    String zkQuorum = tableMeta.getOption(HBaseStorageConstants.META_ZK_QUORUM_KEY, "");
+    if (zkQuorum == null || zkQuorum.trim().isEmpty()) {
+      throw new IOException("HBase mapped table is required a '" +
+          HBaseStorageConstants.META_ZK_QUORUM_KEY + "' attribute.");
+    }
+
+    Configuration hbaseConf = (conf == null) ? HBaseConfiguration.create() : HBaseConfiguration.create(conf);
+    hbaseConf.set(HConstants.ZOOKEEPER_QUORUM, zkQuorum);
+
+    for (Map.Entry<String, String> eachOption: tableMeta.getOptions().getAllKeyValus().entrySet()) {
+      String key = eachOption.getKey();
+      if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) {
+        hbaseConf.set(key, eachOption.getValue());
+      }
+    }
+    return hbaseConf;
+  }
+
+  /**
+   * Creates HTableDescription using table meta data.
+   *
+   * @param tableMeta
+   * @param schema
+   * @return
+   * @throws java.io.IOException
+   */
+  public static HTableDescriptor parseHTableDescriptor(TableMeta tableMeta, Schema schema) throws IOException {
+    String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
+    if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
+      throw new IOException("HBase mapped table is required a '" +
+          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
+    }
+    TableName hTableName = TableName.valueOf(hbaseTableName);
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+
+    HTableDescriptor hTableDescriptor = new HTableDescriptor(hTableName);
+
+    Collection<String> columnFamilies = columnMapping.getColumnFamilyNames();
+    //If 'columns' attribute is empty, Tajo table columns are mapped to all HBase table column.
+    if (columnFamilies.isEmpty()) {
+      for (Column eachColumn: schema.getColumns()) {
+        columnFamilies.add(eachColumn.getSimpleName());
+      }
+    }
+
+    for (String eachColumnFamily: columnFamilies) {
+      hTableDescriptor.addFamily(new HColumnDescriptor(eachColumnFamily));
+    }
+
+    return hTableDescriptor;
+  }
+
+  @Override
+  public void purgeTable(TableDesc tableDesc) throws IOException {
+    HBaseAdmin hAdmin =  new HBaseAdmin(getHBaseConfiguration(conf, tableDesc.getMeta()));
+
+    try {
+      HTableDescriptor hTableDesc = parseHTableDescriptor(tableDesc.getMeta(), tableDesc.getSchema());
+      LOG.info("Deleting hbase table: " + new String(hTableDesc.getName()));
+      hAdmin.disableTable(hTableDesc.getName());
+      hAdmin.deleteTable(hTableDesc.getName());
+    } finally {
+      hAdmin.close();
+    }
+  }
+
+  /**
+   * Returns columns which are mapped to the rowkey of the hbase table.
+   *
+   * @param tableDesc
+   * @return
+   * @throws java.io.IOException
+   */
+  private Column[] getIndexableColumns(TableDesc tableDesc) throws IOException {
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+    boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    int[] rowKeyIndexes = columnMapping.getRowKeyFieldIndexes();
+
+    Column indexColumn = null;
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (isRowKeyMappings[i]) {
+        if (columnMapping.getNumRowKeys() == 1 ||
+            rowKeyIndexes[i] == 0) {
+          indexColumn = tableDesc.getSchema().getColumn(i);
+        }
+      }
+    }
+    return new Column[]{indexColumn};
+  }
+
+  @Override
+  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc, ScanNode scanNode) throws IOException {
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+
+    List<IndexPredication> indexPredications = getIndexPredications(columnMapping, tableDesc, scanNode);
+    Configuration hconf = getHBaseConfiguration(conf, tableDesc.getMeta());
+    HTable htable = null;
+    HBaseAdmin hAdmin = null;
+
+    try {
+      htable = new HTable(hconf, tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY));
+
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
+        HRegionLocation regLoc = htable.getRegionLocation(HConstants.EMPTY_BYTE_ARRAY, false);
+        if (null == regLoc) {
+          throw new IOException("Expecting at least one region.");
+        }
+        List<Fragment> fragments = new ArrayList<Fragment>(1);
+        Fragment fragment = new HBaseFragment(fragmentId, htable.getName().getNameAsString(),
+            HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, regLoc.getHostname());
+        fragments.add(fragment);
+        return fragments;
+      }
+
+      List<byte[]> startRows;
+      List<byte[]> stopRows;
+
+      if (indexPredications != null && !indexPredications.isEmpty()) {
+        // indexPredications is Disjunctive set
+        startRows = new ArrayList<byte[]>();
+        stopRows = new ArrayList<byte[]>();
+        for (IndexPredication indexPredication: indexPredications) {
+          byte[] startRow;
+          byte[] stopRow;
+          if (indexPredication.getStartValue() != null) {
+            startRow = serialize(columnMapping, indexPredication, indexPredication.getStartValue());
+          } else {
+            startRow = HConstants.EMPTY_START_ROW;
+          }
+          if (indexPredication.getStopValue() != null) {
+            stopRow = serialize(columnMapping, indexPredication, indexPredication.getStopValue());
+          } else {
+            stopRow = HConstants.EMPTY_END_ROW;
+          }
+          startRows.add(startRow);
+          stopRows.add(stopRow);
+        }
+      } else {
+        startRows = TUtil.newList(HConstants.EMPTY_START_ROW);
+        stopRows = TUtil.newList(HConstants.EMPTY_END_ROW);
+      }
+
+      hAdmin =  new HBaseAdmin(hconf);
+      Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
+
+      // region startkey -> HBaseFragment
+      Map<byte[], HBaseFragment> fragmentMap = new HashMap<byte[], HBaseFragment>();
+      for (int i = 0; i < keys.getFirst().length; i++) {
+        HRegionLocation location = htable.getRegionLocation(keys.getFirst()[i], false);
+
+        byte[] regionStartKey = keys.getFirst()[i];
+        byte[] regionStopKey = keys.getSecond()[i];
+
+        int startRowsSize = startRows.size();
+        for (int j = 0; j < startRowsSize; j++) {
+          byte[] startRow = startRows.get(j);
+          byte[] stopRow = stopRows.get(j);
+          // determine if the given start an stop key fall into the region
+          if ((startRow.length == 0 || regionStopKey.length == 0 || Bytes.compareTo(startRow, regionStopKey) < 0)
+              && (stopRow.length == 0 || Bytes.compareTo(stopRow, regionStartKey) > 0)) {
+            byte[] fragmentStart = (startRow.length == 0 || Bytes.compareTo(regionStartKey, startRow) >= 0) ?
+                regionStartKey : startRow;
+
+            byte[] fragmentStop = (stopRow.length == 0 || Bytes.compareTo(regionStopKey, stopRow) <= 0) &&
+                regionStopKey.length > 0 ? regionStopKey : stopRow;
+
+            String regionName = location.getRegionInfo().getRegionNameAsString();
+
+            ServerLoad serverLoad = serverLoadMap.get(location.getServerName());
+            if (serverLoad == null) {
+              serverLoad = hAdmin.getClusterStatus().getLoad(location.getServerName());
+              serverLoadMap.put(location.getServerName(), serverLoad);
+            }
+
+            if (fragmentMap.containsKey(regionStartKey)) {
+              HBaseFragment prevFragment = fragmentMap.get(regionStartKey);
+              if (Bytes.compareTo(fragmentStart, prevFragment.getStartRow()) < 0) {
+                prevFragment.setStartRow(fragmentStart);
+              }
+              if (Bytes.compareTo(fragmentStop, prevFragment.getStopRow()) > 0) {
+                prevFragment.setStopRow(fragmentStop);
+              }
+            } else {
+              HBaseFragment fragment = new HBaseFragment(fragmentId, htable.getName().getNameAsString(),
+                  fragmentStart, fragmentStop, location.getHostname());
+
+              // get region size
+              boolean foundLength = false;
+              for (Map.Entry<byte[], RegionLoad> entry : serverLoad.getRegionsLoad().entrySet()) {
+                if (regionName.equals(Bytes.toString(entry.getKey()))) {
+                  RegionLoad regionLoad = entry.getValue();
+                  long storeFileSize = (regionLoad.getStorefileSizeMB() + regionLoad.getMemStoreSizeMB()) * 1024L * 1024L;
+                  fragment.setLength(storeFileSize);
+                  foundLength = true;
+                  break;
+                }
+              }
+
+              if (!foundLength) {
+                fragment.setLength(TajoConstants.UNKNOWN_LENGTH);
+              }
+
+              fragmentMap.put(regionStartKey, fragment);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("getFragments: fragment -> " + i + " -> " + fragment);
+              }
+            }
+          }
+        }
+      }
+
+      List<HBaseFragment> fragments = new ArrayList<HBaseFragment>(fragmentMap.values());
+      Collections.sort(fragments);
+      if (!fragments.isEmpty()) {
+        fragments.get(fragments.size() - 1).setLast(true);
+      }
+      return (ArrayList<Fragment>) (ArrayList) fragments;
+    } finally {
+      if (htable != null) {
+        htable.close();
+      }
+      if (hAdmin != null) {
+        hAdmin.close();
+      }
+    }
+  }
+
+  private byte[] serialize(ColumnMapping columnMapping,
+                           IndexPredication indexPredication, Datum datum) throws IOException {
+    if (columnMapping.getIsBinaryColumns()[indexPredication.getColumnId()]) {
+      return HBaseBinarySerializerDeserializer.serialize(indexPredication.getColumn(), datum);
+    } else {
+      return HBaseTextSerializerDeserializer.serialize(indexPredication.getColumn(), datum);
+    }
+  }
+
+  @Override
+  public Appender getAppender(OverridableConf queryContext,
+                              QueryUnitAttemptId taskAttemptId, TableMeta meta, Schema schema, Path workDir)
+      throws IOException {
+    if ("true".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
+      return new HBasePutAppender(conf, taskAttemptId, schema, meta, workDir);
+    } else {
+      return super.getAppender(queryContext, taskAttemptId, meta, schema, workDir);
+    }
+  }
+
+  @Override
+  public List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments)
+      throws IOException {
+    Configuration hconf = getHBaseConfiguration(conf, tableDesc.getMeta());
+    HTable htable = null;
+    HBaseAdmin hAdmin = null;
+    try {
+      htable = new HTable(hconf, tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY));
+
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
+        return new ArrayList<Fragment>(1);
+      }
+      hAdmin =  new HBaseAdmin(hconf);
+      Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
+
+      List<Fragment> fragments = new ArrayList<Fragment>(keys.getFirst().length);
+
+      int start = currentPage * numFragments;
+      if (start >= keys.getFirst().length) {
+        return new ArrayList<Fragment>(1);
+      }
+      int end = (currentPage + 1) * numFragments;
+      if (end > keys.getFirst().length) {
+        end = keys.getFirst().length;
+      }
+      for (int i = start; i < end; i++) {
+        HRegionLocation location = htable.getRegionLocation(keys.getFirst()[i], false);
+
+        String regionName = location.getRegionInfo().getRegionNameAsString();
+        ServerLoad serverLoad = serverLoadMap.get(location.getServerName());
+        if (serverLoad == null) {
+          serverLoad = hAdmin.getClusterStatus().getLoad(location.getServerName());
+          serverLoadMap.put(location.getServerName(), serverLoad);
+        }
+
+        HBaseFragment fragment = new HBaseFragment(tableDesc.getName(), htable.getName().getNameAsString(),
+            location.getRegionInfo().getStartKey(), location.getRegionInfo().getEndKey(), location.getHostname());
+
+        // get region size
+        boolean foundLength = false;
+        for (Map.Entry<byte[], RegionLoad> entry : serverLoad.getRegionsLoad().entrySet()) {
+          if (regionName.equals(Bytes.toString(entry.getKey()))) {
+            RegionLoad regionLoad = entry.getValue();
+            long storeLength = (regionLoad.getStorefileSizeMB() + regionLoad.getMemStoreSizeMB()) * 1024L * 1024L;
+            if (storeLength == 0) {
+              // If store size is smaller than 1 MB, storeLength is zero
+              storeLength = 1 * 1024 * 1024;  //default 1MB
+            }
+            fragment.setLength(storeLength);
+            foundLength = true;
+            break;
+          }
+        }
+
+        if (!foundLength) {
+          fragment.setLength(TajoConstants.UNKNOWN_LENGTH);
+        }
+
+        fragments.add(fragment);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("getFragments: fragment -> " + i + " -> " + fragment);
+        }
+      }
+
+      if (!fragments.isEmpty()) {
+        ((HBaseFragment) fragments.get(fragments.size() - 1)).setLast(true);
+      }
+      return fragments;
+    } finally {
+      if (htable != null) {
+        htable.close();
+      }
+      if (hAdmin != null) {
+        hAdmin.close();
+      }
+    }
+  }
+
+  public HConnection getConnection(Configuration hbaseConf) throws IOException {
+    synchronized(connMap) {
+      HConnectionKey key = new HConnectionKey(hbaseConf);
+      HConnection conn = connMap.get(key);
+      if (conn == null) {
+        conn = HConnectionManager.createConnection(hbaseConf);
+        connMap.put(key, conn);
+      }
+
+      return conn;
+    }
+  }
+
+  static class HConnectionKey {
+    final static String[] CONNECTION_PROPERTIES = new String[] {
+        HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT,
+        HConstants.ZOOKEEPER_CLIENT_PORT,
+        HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
+        HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+        HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
+        HConstants.HBASE_META_SCANNER_CACHING,
+        HConstants.HBASE_CLIENT_INSTANCE_ID,
+        HConstants.RPC_CODEC_CONF_KEY };
+
+    private Map<String, String> properties;
+    private String username;
+
+    HConnectionKey(Configuration conf) {
+      Map<String, String> m = new HashMap<String, String>();
+      if (conf != null) {
+        for (String property : CONNECTION_PROPERTIES) {
+          String value = conf.get(property);
+          if (value != null) {
+            m.put(property, value);
+          }
+        }
+      }
+      this.properties = Collections.unmodifiableMap(m);
+
+      try {
+        UserProvider provider = UserProvider.instantiate(conf);
+        User currentUser = provider.getCurrent();
+        if (currentUser != null) {
+          username = currentUser.getName();
+        }
+      } catch (IOException ioe) {
+        LOG.warn("Error obtaining current user, skipping username in HConnectionKey", ioe);
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      if (username != null) {
+        result = username.hashCode();
+      }
+      for (String property : CONNECTION_PROPERTIES) {
+        String value = properties.get(property);
+        if (value != null) {
+          result = prime * result + value.hashCode();
+        }
+      }
+
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      HConnectionKey that = (HConnectionKey) obj;
+      if (this.username != null && !this.username.equals(that.username)) {
+        return false;
+      } else if (this.username == null && that.username != null) {
+        return false;
+      }
+      if (this.properties == null) {
+        if (that.properties != null) {
+          return false;
+        }
+      } else {
+        if (that.properties == null) {
+          return false;
+        }
+        for (String property : CONNECTION_PROPERTIES) {
+          String thisValue = this.properties.get(property);
+          String thatValue = that.properties.get(property);
+          //noinspection StringEquality
+          if (thisValue == thatValue) {
+            continue;
+          }
+          if (thisValue == null || !thisValue.equals(thatValue)) {
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "HConnectionKey{" +
+          "properties=" + properties +
+          ", username='" + username + '\'' +
+          '}';
+    }
+  }
+
+  public List<IndexPredication> getIndexPredications(ColumnMapping columnMapping,
+                                                     TableDesc tableDesc, ScanNode scanNode) throws IOException {
+    List<IndexPredication> indexPredications = new ArrayList<IndexPredication>();
+    Column[] indexableColumns = getIndexableColumns(tableDesc);
+    if (indexableColumns != null && indexableColumns.length == 1) {
+      // Currently supports only single index column.
+      List<Set<EvalNode>> indexablePredicateList = findIndexablePredicateSet(scanNode, indexableColumns);
+      for (Set<EvalNode> eachEvalSet: indexablePredicateList) {
+        Pair<Datum, Datum> indexPredicationValues = getIndexablePredicateValue(columnMapping, eachEvalSet);
+        if (indexPredicationValues != null) {
+          IndexPredication indexPredication = new IndexPredication();
+          indexPredication.setColumn(indexableColumns[0]);
+          indexPredication.setColumnId(tableDesc.getLogicalSchema().getColumnId(indexableColumns[0].getQualifiedName()));
+          indexPredication.setStartValue(indexPredicationValues.getFirst());
+          indexPredication.setStopValue(indexPredicationValues.getSecond());
+
+          indexPredications.add(indexPredication);
+        }
+      }
+    }
+    return indexPredications;
+  }
+
+  public List<Set<EvalNode>> findIndexablePredicateSet(ScanNode scanNode, Column[] indexableColumns) throws IOException {
+    List<Set<EvalNode>> indexablePredicateList = new ArrayList<Set<EvalNode>>();
+
+    // if a query statement has a search condition, try to find indexable predicates
+    if (indexableColumns != null && scanNode.getQual() != null) {
+      EvalNode[] disjunctiveForms = AlgebraicUtil.toDisjunctiveNormalFormArray(scanNode.getQual());
+
+      // add qualifier to schema for qual
+      for (Column column : indexableColumns) {
+        for (EvalNode disjunctiveExpr : disjunctiveForms) {
+          EvalNode[] conjunctiveForms = AlgebraicUtil.toConjunctiveNormalFormArray(disjunctiveExpr);
+          Set<EvalNode> indexablePredicateSet = Sets.newHashSet();
+          for (EvalNode conjunctiveExpr : conjunctiveForms) {
+            if (checkIfIndexablePredicateOnTargetColumn(conjunctiveExpr, column)) {
+              indexablePredicateSet.add(conjunctiveExpr);
+            }
+          }
+          if (!indexablePredicateSet.isEmpty()) {
+            indexablePredicateList.add(indexablePredicateSet);
+          }
+        }
+      }
+    }
+
+    return indexablePredicateList;
+  }
+
+  private boolean checkIfIndexablePredicateOnTargetColumn(EvalNode evalNode, Column targetColumn) {
+    if (checkIfIndexablePredicate(evalNode) || checkIfConjunctiveButOneVariable(evalNode)) {
+      Set<Column> variables = EvalTreeUtil.findUniqueColumns(evalNode);
+      // if it contains only single variable matched to a target column
+      return variables.size() == 1 && variables.contains(targetColumn);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   *
+   * @param evalNode The expression to be checked
+   * @return true if an conjunctive expression, consisting of indexable expressions
+   */
+  private boolean checkIfConjunctiveButOneVariable(EvalNode evalNode) {
+    if (evalNode.getType() == EvalType.AND) {
+      BinaryEval orEval = (BinaryEval) evalNode;
+      boolean indexable =
+          checkIfIndexablePredicate(orEval.getLeftExpr()) &&
+              checkIfIndexablePredicate(orEval.getRightExpr());
+
+      boolean sameVariable =
+          EvalTreeUtil.findUniqueColumns(orEval.getLeftExpr())
+              .equals(EvalTreeUtil.findUniqueColumns(orEval.getRightExpr()));
+
+      return indexable && sameVariable;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Check if an expression consists of one variable and one constant and
+   * the expression is a comparison operator.
+   *
+   * @param evalNode The expression to be checked
+   * @return true if an expression consists of one variable and one constant
+   * and the expression is a comparison operator. Other, false.
+   */
+  private boolean checkIfIndexablePredicate(EvalNode evalNode) {
+    return AlgebraicUtil.containSingleVar(evalNode) && isIndexableOperator(evalNode);
+  }
+
+  public static boolean isIndexableOperator(EvalNode expr) {
+    return expr.getType() == EvalType.EQUAL ||
+        expr.getType() == EvalType.LEQ ||
+        expr.getType() == EvalType.LTH ||
+        expr.getType() == EvalType.GEQ ||
+        expr.getType() == EvalType.GTH ||
+        expr.getType() == EvalType.BETWEEN;
+  }
+
+  public Pair<Datum, Datum> getIndexablePredicateValue(ColumnMapping columnMapping,
+                                                       Set<EvalNode> evalNodes) {
+    Datum startDatum = null;
+    Datum endDatum = null;
+    for (EvalNode evalNode: evalNodes) {
+      if (evalNode instanceof BinaryEval) {
+        BinaryEval binaryEval = (BinaryEval) evalNode;
+        EvalNode left = binaryEval.getLeftExpr();
+        EvalNode right = binaryEval.getRightExpr();
+
+        Datum constValue = null;
+        if (left.getType() == EvalType.CONST) {
+          constValue = ((ConstEval) left).getValue();
+        } else if (right.getType() == EvalType.CONST) {
+          constValue = ((ConstEval) right).getValue();
+        }
+
+        if (constValue != null) {
+          if (evalNode.getType() == EvalType.EQUAL ||
+              evalNode.getType() == EvalType.GEQ ||
+              evalNode.getType() == EvalType.GTH) {
+            if (startDatum != null) {
+              if (constValue.compareTo(startDatum) > 0) {
+                startDatum = constValue;
+              }
+            } else {
+              startDatum = constValue;
+            }
+          }
+
+          if (evalNode.getType() == EvalType.EQUAL ||
+              evalNode.getType() == EvalType.LEQ ||
+              evalNode.getType() == EvalType.LTH) {
+            if (endDatum != null) {
+              if (constValue.compareTo(endDatum) < 0) {
+                endDatum = constValue;
+              }
+            } else {
+              endDatum = constValue;
+            }
+          }
+        }
+      } else if (evalNode instanceof BetweenPredicateEval) {
+        BetweenPredicateEval betweenEval = (BetweenPredicateEval) evalNode;
+        if (betweenEval.getBegin().getType() == EvalType.CONST && betweenEval.getEnd().getType() == EvalType.CONST) {
+          Datum value = ((ConstEval) betweenEval.getBegin()).getValue();
+          if (startDatum != null) {
+            if (value.compareTo(startDatum) > 0) {
+              startDatum = value;
+            }
+          } else {
+            startDatum = value;
+          }
+
+          value = ((ConstEval) betweenEval.getEnd()).getValue();
+          if (endDatum != null) {
+            if (value.compareTo(endDatum) < 0) {
+              endDatum = value;
+            }
+          } else {
+            endDatum = value;
+          }
+        }
+      }
+    }
+
+    if (endDatum != null && columnMapping != null && columnMapping.getNumRowKeys() > 1) {
+      endDatum = new TextDatum(endDatum.asChars() +
+          new String(new char[]{columnMapping.getRowKeyDelimiter(), Character.MAX_VALUE}));
+    }
+    if (startDatum != null || endDatum != null) {
+      return new Pair<Datum, Datum>(startDatum, endDatum);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
+                               LogicalPlan plan, Schema schema,
+                               TableDesc tableDesc) throws IOException {
+    if (tableDesc == null) {
+      throw new IOException("TableDesc is null while calling loadIncrementalHFiles: " + finalEbId);
+    }
+    Path stagingDir = new Path(queryContext.get(QueryVars.STAGING_DIR));
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+
+    Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
+    hbaseConf.set("hbase.loadincremental.threads.max", "2");
+
+    JobContextImpl jobContext = new JobContextImpl(hbaseConf,
+        new JobID(finalEbId.getQueryId().toString(), finalEbId.getId()));
+
+    FileOutputCommitter committer = new FileOutputCommitter(stagingResultDir, jobContext);
+    Path jobAttemptPath = committer.getJobAttemptPath(jobContext);
+    FileSystem fs = jobAttemptPath.getFileSystem(queryContext.getConf());
+    if (!fs.exists(jobAttemptPath) || fs.listStatus(jobAttemptPath) == null) {
+      LOG.warn("No query attempt file in " + jobAttemptPath);
+      return stagingResultDir;
+    }
+    committer.commitJob(jobContext);
+
+    if (tableDesc.getName() == null && tableDesc.getPath() != null) {
+
+      // insert into location
+      return super.commitOutputData(queryContext, finalEbId, plan, schema, tableDesc, false);
+    } else {
+      // insert into table
+      String tableName = tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY);
+
+      HTable htable = new HTable(hbaseConf, tableName);
+      try {
+        LoadIncrementalHFiles loadIncrementalHFiles = null;
+        try {
+          loadIncrementalHFiles = new LoadIncrementalHFiles(hbaseConf);
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+          throw new IOException(e.getMessage(), e);
+        }
+        loadIncrementalHFiles.doBulkLoad(stagingResultDir, htable);
+
+        return stagingResultDir;
+      } finally {
+        htable.close();
+      }
+    }
+  }
+
+  @Override
+  public TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
+                                          Schema inputSchema, SortSpec[] sortSpecs, TupleRange dataRange)
+      throws IOException {
+    try {
+      int[] sortKeyIndexes = new int[sortSpecs.length];
+      for (int i = 0; i < sortSpecs.length; i++) {
+        sortKeyIndexes[i] = inputSchema.getColumnId(sortSpecs[i].getSortKey().getQualifiedName());
+      }
+
+      ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+      Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
+
+      HTable htable = new HTable(hbaseConf, columnMapping.getHbaseTableName());
+      try {
+        byte[][] endKeys = htable.getEndKeys();
+        if (endKeys.length == 1) {
+          return new TupleRange[]{dataRange};
+        }
+        List<TupleRange> tupleRanges = new ArrayList<TupleRange>(endKeys.length);
+
+        TupleComparator comparator = new BaseTupleComparator(inputSchema, sortSpecs);
+        Tuple previousTuple = dataRange.getStart();
+
+        for (byte[] eachEndKey : endKeys) {
+          Tuple endTuple = new VTuple(sortSpecs.length);
+          byte[][] rowKeyFields;
+          if (sortSpecs.length > 1) {
+            byte[][] splitValues = BytesUtils.splitPreserveAllTokens(eachEndKey, columnMapping.getRowKeyDelimiter());
+            if (splitValues.length == sortSpecs.length) {
+              rowKeyFields = splitValues;
+            } else {
+              rowKeyFields = new byte[sortSpecs.length][];
+              for (int j = 0; j < sortSpecs.length; j++) {
+                if (j < splitValues.length) {
+                  rowKeyFields[j] = splitValues[j];
+                } else {
+                  rowKeyFields[j] = null;
+                }
+              }
+            }
+
+          } else {
+            rowKeyFields = new byte[1][];
+            rowKeyFields[0] = eachEndKey;
+          }
+
+          for (int i = 0; i < sortSpecs.length; i++) {
+            if (columnMapping.getIsBinaryColumns()[sortKeyIndexes[i]]) {
+              endTuple.put(i,
+                  HBaseBinarySerializerDeserializer.deserialize(inputSchema.getColumn(sortKeyIndexes[i]),
+                      rowKeyFields[i]));
+            } else {
+              endTuple.put(i,
+                  HBaseTextSerializerDeserializer.deserialize(inputSchema.getColumn(sortKeyIndexes[i]),
+                      rowKeyFields[i]));
+            }
+          }
+          tupleRanges.add(new TupleRange(sortSpecs, previousTuple, endTuple));
+          previousTuple = endTuple;
+        }
+
+        // Last region endkey is empty. Tajo ignores empty key, so endkey is replaced with max data value.
+        if (comparator.compare(dataRange.getEnd(), tupleRanges.get(tupleRanges.size() - 1).getStart()) >= 0) {
+          tupleRanges.get(tupleRanges.size() - 1).setEnd(dataRange.getEnd());
+        } else {
+          tupleRanges.remove(tupleRanges.size() - 1);
+        }
+        return tupleRanges.toArray(new TupleRange[]{});
+      } finally {
+        htable.close();
+      }
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+      throw new IOException(t.getMessage(), t);
+    }
+  }
+
+  public List<RewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
+    if ("false".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
+      List<RewriteRule> rules = new ArrayList<RewriteRule>();
+      rules.add(new AddSortForInsertRewriter(tableDesc, getIndexColumns(tableDesc)));
+      return rules;
+    } else {
+      return null;
+    }
+  }
+
+  private Column[] getIndexColumns(TableDesc tableDesc) throws IOException {
+    List<Column> indexColumns = new ArrayList<Column>();
+
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+
+    boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
+    for (int i = 0; i < isRowKeys.length; i++) {
+      if (isRowKeys[i]) {
+        indexColumns.add(tableDesc.getSchema().getColumn(i));
+      }
+    }
+
+    return indexColumns.toArray(new Column[]{});
+  }
+
+  @Override
+  public StorageProperty getStorageProperty() {
+    StorageProperty storageProperty = new StorageProperty();
+    storageProperty.setSortedInsert(true);
+    storageProperty.setSupportsInsertInto(true);
+    return storageProperty;
+  }
+
+  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
+    if (node.getType() == NodeType.CREATE_TABLE) {
+      CreateTableNode cNode = (CreateTableNode)node;
+      if (!cNode.isExternal()) {
+        TableMeta tableMeta = new TableMeta(cNode.getStorageType(), cNode.getOptions());
+        createTable(tableMeta, cNode.getTableSchema(), cNode.isExternal(), cNode.isIfNotExists());
+      }
+    }
+  }
+
+  @Override
+  public void rollbackOutputCommit(LogicalNode node) throws IOException {
+    if (node.getType() == NodeType.CREATE_TABLE) {
+      CreateTableNode cNode = (CreateTableNode)node;
+      if (cNode.isExternal()) {
+        return;
+      }
+      TableMeta tableMeta = new TableMeta(cNode.getStorageType(), cNode.getOptions());
+      HBaseAdmin hAdmin =  new HBaseAdmin(getHBaseConfiguration(conf, tableMeta));
+
+      try {
+        HTableDescriptor hTableDesc = parseHTableDescriptor(tableMeta, cNode.getTableSchema());
+        LOG.info("Delete table cause query failed:" + hTableDesc.getName());
+        hAdmin.disableTable(hTableDesc.getName());
+        hAdmin.deleteTable(hTableDesc.getName());
+      } finally {
+        hAdmin.close();
+      }
+    }
+  }
+
+  @Override
+  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException  {
+    if (tableDesc != null) {
+      Schema tableSchema = tableDesc.getSchema();
+      if (tableSchema.size() != outSchema.size()) {
+        throw new IOException("The number of table columns is different from SELECT columns");
+      }
+
+      for (int i = 0; i < tableSchema.size(); i++) {
+        if (!tableSchema.getColumn(i).getDataType().equals(outSchema.getColumn(i).getDataType())) {
+          throw new IOException(outSchema.getColumn(i).getQualifiedName() +
+              "(" + outSchema.getColumn(i).getDataType().getType() + ")" +
+              " is different column type with " + tableSchema.getColumn(i).getSimpleName() +
+              "(" + tableSchema.getColumn(i).getDataType().getType() + ")");
+        }
+      }
+    }
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
new file mode 100644
index 0000000..a0ad492
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
@@ -0,0 +1,71 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.util.NumberUtil;
+
+import java.io.IOException;
+
+public class HBaseTextSerializerDeserializer {
+  public static Datum deserialize(Column col, byte[] bytes) throws IOException {
+    Datum datum;
+    switch (col.getDataType().getType()) {
+      case INT1:
+      case INT2:
+        datum = bytes == null  || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createInt2((short)NumberUtil.parseInt(bytes, 0, bytes.length));
+        break;
+      case INT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createInt4(NumberUtil.parseInt(bytes, 0, bytes.length));
+        break;
+      case INT8:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createInt8(new String(bytes, 0, bytes.length));
+        break;
+      case FLOAT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createFloat4(new String(bytes, 0, bytes.length));
+        break;
+      case FLOAT8:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createFloat8(NumberUtil.parseDouble(bytes, 0, bytes.length));
+        break;
+      case TEXT:
+        datum = bytes == null ? NullDatum.get() : DatumFactory.createText(bytes);
+        break;
+      default:
+        datum = NullDatum.get();
+        break;
+    }
+    return datum;
+  }
+
+  public static byte[] serialize(Column col, Datum datum) throws IOException {
+    if (datum == null || datum instanceof NullDatum) {
+      return null;
+    }
+
+    return datum.asChars().getBytes();
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
new file mode 100644
index 0000000..07f7988
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
@@ -0,0 +1,167 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.Bytes;
+
+import java.io.IOException;
+import java.util.TreeSet;
+
+public class HFileAppender extends AbstractHBaseAppender {
+  private static final Log LOG = LogFactory.getLog(HFileAppender.class);
+
+  private RecordWriter<ImmutableBytesWritable, Cell> writer;
+  private TaskAttemptContext writerContext;
+  private Path workingFilePath;
+  private FileOutputCommitter committer;
+
+  public HFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                       Schema schema, TableMeta meta, Path stagingDir) {
+    super(conf, taskAttemptId, schema, meta, stagingDir);
+  }
+
+  @Override
+  public void init() throws IOException {
+    super.init();
+
+    Configuration taskConf = new Configuration();
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    taskConf.set(FileOutputFormat.OUTDIR, stagingResultDir.toString());
+
+    ExecutionBlockId ebId = taskAttemptId.getQueryUnitId().getExecutionBlockId();
+    writerContext = new TaskAttemptContextImpl(taskConf,
+        new TaskAttemptID(ebId.getQueryId().toString(), ebId.getId(), TaskType.MAP,
+            taskAttemptId.getQueryUnitId().getId(), taskAttemptId.getId()));
+
+    HFileOutputFormat2 hFileOutputFormat2 = new HFileOutputFormat2();
+    try {
+      writer = hFileOutputFormat2.getRecordWriter(writerContext);
+
+      committer = new FileOutputCommitter(FileOutputFormat.getOutputPath(writerContext), writerContext);
+      workingFilePath = committer.getWorkPath();
+    } catch (InterruptedException e) {
+      throw new IOException(e.getMessage(), e);
+    }
+
+    LOG.info("Created hbase file writer: " + workingFilePath);
+  }
+
+  long totalNumBytes = 0;
+  ImmutableBytesWritable keyWritable = new ImmutableBytesWritable();
+  boolean first = true;
+  TreeSet<KeyValue> kvSet = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
+
+
+  @Override
+  public void addTuple(Tuple tuple) throws IOException {
+    Datum datum;
+
+    byte[] rowkey = getRowKeyBytes(tuple);
+
+    if (!first && !Bytes.equals(keyWritable.get(), 0, keyWritable.getLength(), rowkey, 0, rowkey.length)) {
+      try {
+        for (KeyValue kv : kvSet) {
+          writer.write(keyWritable, kv);
+          totalNumBytes += keyWritable.getLength() + kv.getLength();
+        }
+        kvSet.clear();
+        // Statistical section
+        if (enabledStats) {
+          stats.incrementRow();
+        }
+      } catch (InterruptedException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+
+    first = false;
+
+    keyWritable.set(rowkey);
+
+    readKeyValues(tuple, rowkey);
+    if (keyValues != null) {
+      for (KeyValue eachKeyVal: keyValues) {
+        kvSet.add(eachKeyVal);
+      }
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+  }
+
+  @Override
+  public long getEstimatedOutputSize() throws IOException {
+    // StoreTableExec uses this value as rolling file length
+    // Not rolling
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!kvSet.isEmpty()) {
+      try {
+        for (KeyValue kv : kvSet) {
+          writer.write(keyWritable, kv);
+          totalNumBytes += keyWritable.getLength() + keyWritable.getLength();
+        }
+        kvSet.clear();
+        // Statistical section
+        if (enabledStats) {
+          stats.incrementRow();
+        }
+      } catch (InterruptedException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+
+    if (enabledStats) {
+      stats.setNumBytes(totalNumBytes);
+    }
+    if (writer != null) {
+      try {
+        writer.close(writerContext);
+        committer.commitTask(writerContext);
+      } catch (InterruptedException e) {
+      }
+    }
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
new file mode 100644
index 0000000..3a58e50
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
@@ -0,0 +1,61 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+
+public class IndexPredication {
+  private Column column;
+  private int columnId;
+  private Datum startValue;
+  private Datum stopValue;
+
+  public Column getColumn() {
+    return column;
+  }
+
+  public void setColumn(Column column) {
+    this.column = column;
+  }
+
+  public int getColumnId() {
+    return columnId;
+  }
+
+  public void setColumnId(int columnId) {
+    this.columnId = columnId;
+  }
+
+  public Datum getStartValue() {
+    return startValue;
+  }
+
+  public void setStartValue(Datum startValue) {
+    this.startValue = startValue;
+  }
+
+  public Datum getStopValue() {
+    return stopValue;
+  }
+
+  public void setStopValue(Datum stopValue) {
+    this.stopValue = stopValue;
+  }
+}
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
similarity index 65%
copy from tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
copy to tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
index 7baf7aa..4577703 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
@@ -16,20 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.storage.index;
+package org.apache.tajo.storage.hbase;
 
-import org.apache.tajo.storage.Tuple;
+public class RowKeyMapping {
+  private boolean isBinary;
+  private int keyFieldIndex;
 
-import java.io.IOException;
+  public boolean isBinary() {
+    return isBinary;
+  }
 
-public interface IndexReader {
-  
-  /**
-   * Find the offset corresponding to key which is equal to a given key.
-   * 
-   * @param key
-   * @return
-   * @throws IOException 
-   */
-  public long find(Tuple key) throws IOException;
+  public void setBinary(boolean isBinary) {
+    this.isBinary = isBinary;
+  }
+
+  public int getKeyFieldIndex() {
+    return keyFieldIndex;
+  }
+
+  public void setKeyFieldIndex(int keyFieldIndex) {
+    this.keyFieldIndex = keyFieldIndex;
+  }
 }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
similarity index 61%
copy from tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
copy to tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
index 7baf7aa..668b116 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
@@ -16,20 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.storage.index;
+option java_package = "org.apache.tajo.storage.hbase";
+option java_outer_classname = "StorageFragmentProtos";
+option optimize_for = SPEED;
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
 
-import org.apache.tajo.storage.Tuple;
+import "CatalogProtos.proto";
 
-import java.io.IOException;
-
-public interface IndexReader {
-  
-  /**
-   * Find the offset corresponding to key which is equal to a given key.
-   * 
-   * @param key
-   * @return
-   * @throws IOException 
-   */
-  public long find(Tuple key) throws IOException;
-}
+message HBaseFragmentProto {
+  required string tableName = 1;
+  required string hbaseTableName = 2;
+  required bytes startRow = 3;
+  required bytes stopRow = 4;
+  required bool last = 5;
+  required int64 length = 6;
+  optional string regionLocation = 7;
+}
\ No newline at end of file
diff --git a/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java
new file mode 100644
index 0000000..68939d6
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java
@@ -0,0 +1,93 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.util.KeyValueSet;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class TestColumnMapping {
+  @Test
+  public void testColumnKeyValueMapping() throws Exception {
+    KeyValueSet keyValueSet = new KeyValueSet();
+    keyValueSet.set(HBaseStorageConstants.META_TABLE_KEY, "test");
+    keyValueSet.set(HBaseStorageConstants.META_COLUMNS_KEY, ":key,col2:key:,col2:value:#b,col3:");
+
+    Schema schema = new Schema();
+    schema.addColumn("c1", Type.TEXT);
+    schema.addColumn("c2", Type.TEXT);
+    schema.addColumn("c3", Type.TEXT);
+    schema.addColumn("c4", Type.TEXT);
+
+    TableMeta tableMeta = new TableMeta(StoreType.HBASE, keyValueSet);
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+
+    List<String> cfNames = columnMapping.getColumnFamilyNames();
+    assertEquals(2, cfNames.size());
+    assertEquals("col2", cfNames.get(0));
+    assertEquals("col3", cfNames.get(1));
+
+    for (int i = 0; i < columnMapping.getIsBinaryColumns().length; i++) {
+      if (i == 2) {
+        assertTrue(columnMapping.getIsBinaryColumns()[i]);
+      } else {
+        assertFalse(columnMapping.getIsBinaryColumns()[i]);
+      }
+    }
+
+    for (int i = 0; i < columnMapping.getIsRowKeyMappings().length; i++) {
+      if (i == 0) {
+        assertTrue(columnMapping.getIsRowKeyMappings()[i]);
+      } else {
+        assertFalse(columnMapping.getIsRowKeyMappings()[i]);
+      }
+    }
+
+    String[] expectedColumnNames = { null, null, null, null};
+    for (int i = 0; i < schema.size(); i++) {
+      String columnName = columnMapping.getMappingColumns()[i][1] == null ? null :
+          new String(columnMapping.getMappingColumns()[i][1]);
+      assertEquals(expectedColumnNames[i], columnName);
+    }
+
+    for (int i = 0; i < schema.size(); i++) {
+      if (i == 1) {
+        assertTrue(columnMapping.getIsColumnKeys()[i]);
+      } else {
+        assertFalse(columnMapping.getIsColumnKeys()[i]);
+      }
+    }
+
+    for (int i = 0; i < schema.size(); i++) {
+      if (i == 2) {
+        assertTrue(columnMapping.getIsColumnValues()[i]);
+      } else {
+        assertFalse(columnMapping.getIsColumnValues()[i]);
+      }
+    }
+  }
+}
diff --git a/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
new file mode 100644
index 0000000..1fc4065
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
@@ -0,0 +1,109 @@
+/**
+ * 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.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.plan.expr.*;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.util.Pair;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestHBaseStorageManager {
+  @Test
+  public void testGetIndexPredications() throws Exception {
+    Column rowkeyColumn = new Column("rk", Type.TEXT);
+    // where rk >= '020' and rk <= '055'
+    ScanNode scanNode = new ScanNode(1);
+    EvalNode evalNode1 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("020")));
+    EvalNode evalNode2 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("055")));
+    EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2);
+    scanNode.setQual(evalNodeA);
+
+    HBaseStorageManager storageManager =
+        (HBaseStorageManager) StorageManager.getStorageManager(new TajoConf(), StoreType.HBASE);
+    List<Set<EvalNode>> indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertNotNull(indexEvals);
+    assertEquals(1, indexEvals.size());
+    Pair<Datum, Datum> indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    // where (rk >= '020' and rk <= '055') or rk = '075'
+    EvalNode evalNode3 = new BinaryEval(EvalType.EQUAL, new FieldEval(rowkeyColumn),new ConstEval(new TextDatum("075")));
+    EvalNode evalNodeB = new BinaryEval(EvalType.OR, evalNodeA, evalNode3);
+    scanNode.setQual(evalNodeB);
+    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertEquals(2, indexEvals.size());
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
+    assertEquals("075", indexPredicateValue.getFirst().asChars());
+    assertEquals("075", indexPredicateValue.getSecond().asChars());
+
+    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078')
+    EvalNode evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("072")));
+    EvalNode evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("078")));
+    EvalNode evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
+    EvalNode evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC);
+    scanNode.setQual(evalNodeD);
+    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertEquals(2, indexEvals.size());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
+    assertEquals("072", indexPredicateValue.getFirst().asChars());
+    assertEquals("078", indexPredicateValue.getSecond().asChars());
+
+    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078' and rk >= '073')
+    evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("072")));
+    evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("078")));
+    evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
+    EvalNode evalNode6 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("073")));
+    evalNodeD = new BinaryEval(EvalType.AND, evalNodeC, evalNode6);
+    EvalNode evalNodeE = new BinaryEval(EvalType.OR, evalNodeA, evalNodeD);
+    scanNode.setQual(evalNodeE);
+    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertEquals(2, indexEvals.size());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
+    assertEquals("073", indexPredicateValue.getFirst().asChars());
+    assertEquals("078", indexPredicateValue.getSecond().asChars());
+  }
+}
diff --git a/tajo-storage/tajo-storage-hdfs/pom.xml b/tajo-storage/tajo-storage-hdfs/pom.xml
new file mode 100644
index 0000000..ef8e9c2
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/pom.xml
@@ -0,0 +1,385 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Copyright 2012 Database Lab., Korea Univ.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.9.1-SNAPSHOT</version>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tajo-storage-hdfs</artifactId>
+  <packaging>jar</packaging>
+  <name>Tajo HDFS Storage</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    <parquet.version>1.5.0</parquet.version>
+    <parquet.format.version>2.1.0</parquet.format.version>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <excludes>
+            <exclude>src/test/resources/dataset/**</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/StorageFragmentProtos.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-plan</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>trevni-core</artifactId>
+      <version>1.7.3</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>trevni-avro</artifactId>
+      <version>1.7.3</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>slf4j-api</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jersey-json</artifactId>
+          <groupId>com.sun.jersey</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-server-tests</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-hs</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-format</artifactId>
+      <version>${parquet.format.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-buffer</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>net.minidev</groupId>
+      <artifactId>json-smart</artifactId>
+      <version>2.0</version>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
\ No newline at end of file
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
similarity index 96%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
index 5ddc3fb..4bf4c99 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -24,10 +24,10 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.*;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -38,7 +38,7 @@
 import org.apache.tajo.exception.UnsupportedException;
 import org.apache.tajo.storage.compress.CodecPool;
 import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
 import org.apache.tajo.util.BytesUtils;
 
@@ -76,9 +76,10 @@
     private NonSyncByteArrayOutputStream os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
     private SerializerDeserializer serde;
 
-    public CSVAppender(Configuration conf, final Schema schema, final TableMeta meta, final Path path) throws IOException {
-      super(conf, schema, meta, path);
-      this.fs = path.getFileSystem(conf);
+    public CSVAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
+                       final Schema schema, final TableMeta meta, final Path workDir) throws IOException {
+      super(conf, taskAttemptId, schema, meta, workDir);
+      this.fs = workDir.getFileSystem(conf);
       this.meta = meta;
       this.schema = schema;
       this.delimiter = StringEscapeUtils.unescapeJava(
@@ -99,7 +100,7 @@
     @Override
     public void init() throws IOException {
       if (!fs.exists(path.getParent())) {
-        throw new FileNotFoundException(path.toString());
+        throw new FileNotFoundException(path.getParent().toString());
       }
 
       //determine the intermediate file type
@@ -254,11 +255,11 @@
   }
 
   public static class CSVScanner extends FileScanner implements SeekableScanner {
-    public CSVScanner(Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment)
+    public CSVScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment)
         throws IOException {
       super(conf, schema, meta, fragment);
       factory = new CompressionCodecFactory(conf);
-      codec = factory.getCodec(fragment.getPath());
+      codec = factory.getCodec(this.fragment.getPath());
       if (codec == null || codec instanceof SplittableCompressionCodec) {
         splittable = true;
       }
@@ -316,7 +317,7 @@
 
       recordCount = 0;
       pos = startOffset = fragment.getStartKey();
-      end = startOffset + fragment.getEndKey();
+      end = startOffset + fragment.getLength();
 
       if (codec != null) {
         decompressor = CodecPool.getDecompressor(codec);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
similarity index 65%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
index 04278e9..47f67c6 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
@@ -18,28 +18,50 @@
 
 package org.apache.tajo.storage;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.conf.TajoConf;
 
 import java.io.IOException;
 
 public abstract class FileAppender implements Appender {
+  private static final Log LOG = LogFactory.getLog(FileAppender.class);
+
   protected boolean inited = false;
 
   protected final Configuration conf;
   protected final TableMeta meta;
   protected final Schema schema;
-  protected final Path path;
+  protected final Path workDir;
+  protected final QueryUnitAttemptId taskAttemptId;
 
   protected boolean enabledStats;
-  
-  public FileAppender(Configuration conf, Schema schema, TableMeta meta, Path path) {
+  protected Path path;
+
+  public FileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId, Schema schema,
+                      TableMeta meta, Path workDir) {
     this.conf = conf;
     this.meta = meta;
     this.schema = schema;
-    this.path = path;
+    this.workDir = workDir;
+    this.taskAttemptId = taskAttemptId;
+
+    try {
+      if (taskAttemptId != null) {
+        this.path = ((FileStorageManager)StorageManager.getFileStorageManager((TajoConf) conf))
+            .getAppenderFilePath(taskAttemptId, workDir);
+      } else {
+        this.path = workDir;
+      }
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+      throw new IllegalStateException("Error while opeining FileAppender: " + e.getMessage(), e);
+    }
   }
 
   public void init() throws IOException {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java
similarity index 94%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java
index f15c4c9..038f0f4 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java
@@ -30,6 +30,7 @@
 import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 import java.io.IOException;
 
@@ -49,11 +50,11 @@
 
   protected TableStats tableStats;
 
-  public FileScanner(Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment) {
+  public FileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment) {
     this.conf = conf;
     this.meta = meta;
     this.schema = schema;
-    this.fragment = fragment;
+    this.fragment = (FileFragment)fragment;
     this.tableStats = new TableStats();
     this.columnNum = this.schema.size();
   }
@@ -63,7 +64,7 @@
     progress = 0.0f;
 
     if (fragment != null) {
-      tableStats.setNumBytes(fragment.getEndKey());
+      tableStats.setNumBytes(fragment.getLength());
       tableStats.setNumBlocks(1);
     }
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
similarity index 65%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
index 220eb6c..060bf16 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
@@ -18,68 +18,80 @@
 
 package org.apache.tajo.storage;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.ScanNode;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.util.Bytes;
-import org.apache.tajo.util.FileUtil;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.net.URI;
+import java.text.NumberFormat;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
 
-/**
- * StorageManager
- */
-public class StorageManager {
-  private final Log LOG = LogFactory.getLog(StorageManager.class);
+public class FileStorageManager extends StorageManager {
+  private final Log LOG = LogFactory.getLog(FileStorageManager.class);
 
-  protected final TajoConf conf;
-  protected final FileSystem fs;
-  protected final Path tableBaseDir;
-  protected final boolean blocksMetadataEnabled;
+  static final String OUTPUT_FILE_PREFIX="part-";
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SUBQUERY =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(2);
+          return fmt;
+        }
+      };
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_TASK =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(6);
+          return fmt;
+        }
+      };
+
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SEQ =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(3);
+          return fmt;
+        }
+      };
+
+  protected FileSystem fs;
+  protected Path tableBaseDir;
+  protected boolean blocksMetadataEnabled;
   private static final HdfsVolumeId zeroVolumeId = new HdfsVolumeId(Bytes.toBytes(0));
 
-  private static final Map<String, StorageManager> storageManagers = Maps.newHashMap();
+  public FileStorageManager(StoreType storeType) {
+    super(storeType);
+  }
 
-  /**
-   * Cache of scanner handlers for each storage type.
-   */
-  protected static final Map<String, Class<? extends Scanner>> SCANNER_HANDLER_CACHE
-      = new ConcurrentHashMap<String, Class<? extends Scanner>>();
-
-  /**
-   * Cache of appender handlers for each storage type.
-   */
-  protected static final Map<String, Class<? extends FileAppender>> APPENDER_HANDLER_CACHE
-      = new ConcurrentHashMap<String, Class<? extends FileAppender>>();
-
-  /**
-   * Cache of constructors for each class. Pins the classes so they
-   * can't be garbage collected until ReflectionUtils can be collected.
-   */
-  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
-      new ConcurrentHashMap<Class<?>, Constructor<?>>();
-
-  private StorageManager(TajoConf conf) throws IOException {
-    this.conf = conf;
+  @Override
+  protected void storageInit() throws IOException {
     this.tableBaseDir = TajoConf.getWarehouseDir(conf);
     this.fs = tableBaseDir.getFileSystem(conf);
     this.blocksMetadataEnabled = conf.getBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
@@ -88,33 +100,6 @@
       LOG.warn("does not support block metadata. ('dfs.datanode.hdfs-blocks-metadata.enabled')");
   }
 
-  public static StorageManager getStorageManager(TajoConf conf) throws IOException {
-    return getStorageManager(conf, null);
-  }
-
-  public static synchronized StorageManager getStorageManager (
-      TajoConf conf, Path warehouseDir) throws IOException {
-
-    URI uri;
-    TajoConf localConf = new TajoConf(conf);
-    if (warehouseDir != null) {
-      localConf.setVar(ConfVars.WAREHOUSE_DIR, warehouseDir.toUri().toString());
-    }
-
-    uri = TajoConf.getWarehouseDir(localConf).toUri();
-
-    String key = "file".equals(uri.getScheme()) ? "file" : uri.toString();
-
-    if(storageManagers.containsKey(key)) {
-      StorageManager sm = storageManagers.get(key);
-      return sm;
-    } else {
-      StorageManager storageManager = new StorageManager(localConf);
-      storageManagers.put(key, storageManager);
-      return storageManager;
-    }
-  }
-
   public Scanner getFileScanner(TableMeta meta, Schema schema, Path path)
       throws IOException {
     FileSystem fs = path.getFileSystem(conf);
@@ -124,22 +109,10 @@
 
   public Scanner getFileScanner(TableMeta meta, Schema schema, Path path, FileStatus status)
       throws IOException {
-    FileFragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
+    Fragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
     return getScanner(meta, schema, fragment);
   }
 
-  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment) throws IOException {
-    return getScanner(meta, schema, FragmentConvertor.convert(conf, meta.getStoreType(), fragment), schema);
-  }
-
-  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment, Schema target) throws IOException {
-    return getScanner(meta, schema, FragmentConvertor.convert(conf, meta.getStoreType(), fragment), target);
-  }
-
-  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment) throws IOException {
-    return getScanner(meta, schema, fragment, schema);
-  }
-
   public FileSystem getFileSystem() {
     return this.fs;
   }
@@ -176,47 +149,10 @@
     return new Path(tableBaseDir, tableName);
   }
 
-  public Appender getAppender(TableMeta meta, Schema schema, Path path)
+  @VisibleForTesting
+  public Appender getAppender(TableMeta meta, Schema schema, Path filePath)
       throws IOException {
-    Appender appender;
-
-    Class<? extends FileAppender> appenderClass;
-
-    String handlerName = meta.getStoreType().name().toLowerCase();
-    appenderClass = APPENDER_HANDLER_CACHE.get(handlerName);
-    if (appenderClass == null) {
-      appenderClass = conf.getClass(
-          String.format("tajo.storage.appender-handler.%s.class",
-              meta.getStoreType().name().toLowerCase()), null,
-          FileAppender.class);
-      APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
-    }
-
-    if (appenderClass == null) {
-      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
-    }
-
-    appender = newAppenderInstance(appenderClass, conf, meta, schema, path);
-
-    return appender;
-  }
-
-  public TableMeta getTableMeta(Path tablePath) throws IOException {
-    TableMeta meta;
-
-    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);
-
-    CatalogProtos.TableProto tableProto = (CatalogProtos.TableProto) FileUtil.loadProto(tableMetaIn,
-        CatalogProtos.TableProto.getDefaultInstance());
-    meta = new TableMeta(tableProto);
-
-    return meta;
+    return getAppender(null, null, meta, schema, filePath);
   }
 
   public FileFragment[] split(String tableName) throws IOException {
@@ -332,13 +268,22 @@
   /////////////////////////////////////////////////////////////////////////////
   // FileInputFormat Area
   /////////////////////////////////////////////////////////////////////////////
-
-  public static final PathFilter hiddenFileFilter = new PathFilter() {
-    public boolean accept(Path p) {
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
+  public Path getAppenderFilePath(QueryUnitAttemptId taskAttemptId, Path workDir) {
+    if (taskAttemptId == null) {
+      // For testcase
+      return workDir;
     }
-  };
+    // The final result of a task will be written in a file named part-ss-nnnnnnn,
+    // where ss is the subquery id associated with this task, and nnnnnn is the task id.
+    Path outFilePath = StorageUtil.concatPath(workDir, TajoConstants.RESULT_DIR_NAME,
+        OUTPUT_FILE_PREFIX +
+            OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskAttemptId.getQueryUnitId().getExecutionBlockId().getId()) + "-" +
+            OUTPUT_FILE_FORMAT_TASK.get().format(taskAttemptId.getQueryUnitId().getId()) + "-" +
+            OUTPUT_FILE_FORMAT_SEQ.get().format(0));
+    LOG.info("Output File Path: " + outFilePath);
+
+    return outFilePath;
+  }
 
   /**
    * Proxy PathFilter that accepts a path only if all filters given in the
@@ -557,12 +502,12 @@
    *
    * @throws IOException
    */
-  public List<FileFragment> getSplits(String tableName, TableMeta meta, Schema schema, Path... inputs)
+  public List<Fragment> getSplits(String tableName, TableMeta meta, Schema schema, Path... inputs)
       throws IOException {
     // generate splits'
 
-    List<FileFragment> splits = Lists.newArrayList();
-    List<FileFragment> volumeSplits = Lists.newArrayList();
+    List<Fragment> splits = Lists.newArrayList();
+    List<Fragment> volumeSplits = Lists.newArrayList();
     List<BlockLocation> blockLocations = Lists.newArrayList();
 
     for (Path p : inputs) {
@@ -645,7 +590,7 @@
     return splits;
   }
 
-  private void setVolumeMeta(List<FileFragment> splits, final List<BlockLocation> blockLocations)
+  private void setVolumeMeta(List<Fragment> splits, final List<BlockLocation> blockLocations)
       throws IOException {
 
     int locationSize = blockLocations.size();
@@ -664,7 +609,7 @@
     int lsLimit = conf.getInt(DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     int blockLocationIdx = 0;
 
-    Iterator<FileFragment> iter = splits.iterator();
+    Iterator<Fragment> iter = splits.iterator();
     while (locationSize > blockLocationIdx) {
 
       int subSize = Math.min(locationSize - blockLocationIdx, lsLimit);
@@ -673,7 +618,7 @@
       BlockStorageLocation[] blockStorageLocations = fs.getFileBlockStorageLocations(locations);
 
       for (BlockStorageLocation blockStorageLocation : blockStorageLocations) {
-        iter.next().setDiskIds(getDiskIds(blockStorageLocation.getVolumeIds()));
+        ((FileFragment)iter.next()).setDiskIds(getDiskIds(blockStorageLocation.getVolumeIds()));
         blockLocationIdx++;
       }
     }
@@ -701,105 +646,230 @@
     }
   }
 
-  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
-      Configuration.class,
-      Schema.class,
-      TableMeta.class,
-      FileFragment.class
-  };
+  @Override
+  public List<Fragment> getSplits(String tableName, TableDesc table, ScanNode scanNode) throws IOException {
+    return getSplits(tableName, table.getMeta(), table.getSchema(), new Path(table.getPath()));
+  }
 
-  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
-      Configuration.class,
-      Schema.class,
-      TableMeta.class,
-      Path.class
-  };
+  @Override
+  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
+    if (!tableDesc.isExternal()) {
+      String [] splitted = CatalogUtil.splitFQTableName(tableDesc.getName());
+      String databaseName = splitted[0];
+      String simpleTableName = splitted[1];
 
-  /**
-   * create a scanner instance.
-   */
-  public static <T> T newScannerInstance(Class<T> theClass, Configuration conf, Schema schema, TableMeta meta,
-                                         Fragment fragment) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(DEFAULT_SCANNER_PARAMS);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(new Object[]{conf, schema, meta, fragment});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+      // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} )
+      Path tablePath = StorageUtil.concatPath(tableBaseDir, databaseName, simpleTableName);
+      tableDesc.setPath(tablePath.toUri());
+    } else {
+      Preconditions.checkState(tableDesc.getPath() != null, "ERROR: LOCATION must be given.");
     }
 
-    return result;
+    Path path = new Path(tableDesc.getPath());
+
+    FileSystem fs = path.getFileSystem(conf);
+    TableStats stats = new TableStats();
+    if (tableDesc.isExternal()) {
+      if (!fs.exists(path)) {
+        LOG.error(path.toUri() + " does not exist");
+        throw new IOException("ERROR: " + path.toUri() + " does not exist");
+      }
+    } else {
+      fs.mkdirs(path);
+    }
+
+    long totalSize = 0;
+
+    try {
+      totalSize = calculateSize(path);
+    } catch (IOException e) {
+      LOG.warn("Cannot calculate the size of the relation", e);
+    }
+
+    stats.setNumBytes(totalSize);
+
+    if (tableDesc.isExternal()) { // if it is an external table, there is no way to know the exact row number without processing.
+      stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
+    }
+
+    tableDesc.setStats(stats);
+  }
+
+  @Override
+  public void purgeTable(TableDesc tableDesc) throws IOException {
+    try {
+      Path path = new Path(tableDesc.getPath());
+      FileSystem fs = path.getFileSystem(conf);
+      LOG.info("Delete table data dir: " + path);
+      fs.delete(path, true);
+    } catch (IOException e) {
+      throw new InternalError(e.getMessage());
+    }
+  }
+
+  @Override
+  public List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numResultFragments) throws IOException {
+    // Listing table data file which is not empty.
+    // If the table is a partitioned table, return file list which has same partition key.
+    Path tablePath = new Path(tableDesc.getPath());
+    FileSystem fs = tablePath.getFileSystem(conf);
+
+    //In the case of partitioned table, we should return same partition key data files.
+    int partitionDepth = 0;
+    if (tableDesc.hasPartition()) {
+      partitionDepth = tableDesc.getPartitionMethod().getExpressionSchema().getColumns().size();
+    }
+
+    List<FileStatus> nonZeroLengthFiles = new ArrayList<FileStatus>();
+    if (fs.exists(tablePath)) {
+      getNonZeroLengthDataFiles(fs, tablePath, nonZeroLengthFiles, currentPage, numResultFragments,
+          new AtomicInteger(0), tableDesc.hasPartition(), 0, partitionDepth);
+    }
+
+    List<Fragment> fragments = new ArrayList<Fragment>();
+
+    String[] previousPartitionPathNames = null;
+    for (FileStatus eachFile: nonZeroLengthFiles) {
+      FileFragment fileFragment = new FileFragment(tableDesc.getName(), eachFile.getPath(), 0, eachFile.getLen(), null);
+
+      if (partitionDepth > 0) {
+        // finding partition key;
+        Path filePath = fileFragment.getPath();
+        Path parentPath = filePath;
+        String[] parentPathNames = new String[partitionDepth];
+        for (int i = 0; i < partitionDepth; i++) {
+          parentPath = parentPath.getParent();
+          parentPathNames[partitionDepth - i - 1] = parentPath.getName();
+        }
+
+        // If current partitionKey == previousPartitionKey, add to result.
+        if (previousPartitionPathNames == null) {
+          fragments.add(fileFragment);
+        } else if (previousPartitionPathNames != null && Arrays.equals(previousPartitionPathNames, parentPathNames)) {
+          fragments.add(fileFragment);
+        } else {
+          break;
+        }
+        previousPartitionPathNames = parentPathNames;
+      } else {
+        fragments.add(fileFragment);
+      }
+    }
+
+    return fragments;
   }
 
   /**
-   * create a scanner instance.
+   *
+   * @param fs
+   * @param path The table path
+   * @param result The final result files to be used
+   * @param startFileIndex
+   * @param numResultFiles
+   * @param currentFileIndex
+   * @param partitioned A flag to indicate if this table is partitioned
+   * @param currentDepth Current visiting depth of partition directories
+   * @param maxDepth The partition depth of this table
+   * @throws IOException
    */
-  public static <T> T newAppenderInstance(Class<T> theClass, Configuration conf, TableMeta meta, Schema schema,
-                                          Path path) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(DEFAULT_APPENDER_PARAMS);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
+  private void getNonZeroLengthDataFiles(FileSystem fs, Path path, List<FileStatus> result,
+                                                int startFileIndex, int numResultFiles,
+                                                AtomicInteger currentFileIndex, boolean partitioned,
+                                                int currentDepth, int maxDepth) throws IOException {
+    // Intermediate directory
+    if (fs.isDirectory(path)) {
+
+      FileStatus[] files = fs.listStatus(path, StorageManager.hiddenFileFilter);
+
+      if (files != null && files.length > 0) {
+
+        for (FileStatus eachFile : files) {
+
+          // checking if the enough number of files are found
+          if (result.size() >= numResultFiles) {
+            return;
+          }
+          if (eachFile.isDirectory()) {
+
+            getNonZeroLengthDataFiles(
+                fs,
+                eachFile.getPath(),
+                result,
+                startFileIndex,
+                numResultFiles,
+                currentFileIndex,
+                partitioned,
+                currentDepth + 1, // increment a visiting depth
+                maxDepth);
+
+            // if partitioned table, we should ignore files located in the intermediate directory.
+            // we can ensure that this file is in leaf directory if currentDepth == maxDepth.
+          } else if (eachFile.isFile() && eachFile.getLen() > 0 && (!partitioned || currentDepth == maxDepth)) {
+            if (currentFileIndex.get() >= startFileIndex) {
+              result.add(eachFile);
+            }
+            currentFileIndex.incrementAndGet();
+          }
+        }
       }
-      result = meth.newInstance(new Object[]{conf, schema, meta, path});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
 
-    return result;
-  }
-
-  public Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException {
-    String handlerName = storeType.name().toLowerCase();
-    Class<? extends Scanner> scannerClass = SCANNER_HANDLER_CACHE.get(handlerName);
-    if (scannerClass == null) {
-      scannerClass = conf.getClass(
-          String.format("tajo.storage.scanner-handler.%s.class",storeType.name().toLowerCase()), null, Scanner.class);
-      SCANNER_HANDLER_CACHE.put(handlerName, scannerClass);
-    }
-
-    if (scannerClass == null) {
-      throw new IOException("Unknown Storage Type: " + storeType.name());
-    }
-
-    return scannerClass;
-  }
-
-  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
-    if (fragment instanceof FileFragment) {
-      FileFragment fileFragment = (FileFragment)fragment;
-      if (fileFragment.getEndKey() == 0) {
-        Scanner scanner = new NullScanner(conf, schema, meta, fileFragment);
-        scanner.setTarget(target.toArray());
-
-        return scanner;
+      // Files located in leaf directory
+    } else {
+      FileStatus fileStatus = fs.getFileStatus(path);
+      if (fileStatus != null && fileStatus.getLen() > 0) {
+        if (currentFileIndex.get() >= startFileIndex) {
+          result.add(fileStatus);
+        }
+        currentFileIndex.incrementAndGet();
+        if (result.size() >= numResultFiles) {
+          return;
+        }
       }
     }
+  }
 
-    Scanner scanner;
-
-    Class<? extends Scanner> scannerClass = getScannerClass(meta.getStoreType());
-    scanner = newScannerInstance(scannerClass, conf, schema, meta, fragment);
-    if (scanner.isProjectable()) {
-      scanner.setTarget(target.toArray());
+  @Override
+  public StorageProperty getStorageProperty() {
+    StorageProperty storageProperty = new StorageProperty();
+    storageProperty.setSortedInsert(false);
+    if (storeType == StoreType.RAW) {
+      storageProperty.setSupportsInsertInto(false);
+    } else {
+      storageProperty.setSupportsInsertInto(true);
     }
 
-    return scanner;
+    return storageProperty;
   }
 
-  public static synchronized SeekableScanner getSeekableScanner(
-      TajoConf conf, TableMeta meta, Schema schema, FileFragment fragment, Schema target) throws IOException {
-    return (SeekableScanner)getStorageManager(conf, null).getScanner(meta, schema, fragment, target);
+  @Override
+  public void closeStorageManager() {
   }
 
+  @Override
+  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
+  }
+
+  @Override
+  public void rollbackOutputCommit(LogicalNode node) throws IOException {
+  }
+
+  @Override
+  public TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
+                                          Schema inputSchema, SortSpec[] sortSpecs, TupleRange dataRange)
+      throws IOException {
+    return null;
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param conf The system property
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param path The data file path
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
   public static synchronized SeekableScanner getSeekableScanner(
       TajoConf conf, TableMeta meta, Schema schema, Path path) throws IOException {
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
similarity index 99%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
index 40cad32..33b2750 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
@@ -79,7 +79,7 @@
    * @param taskId
    * @param tuples
    * @return written bytes
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public int addTuples(QueryUnitAttemptId taskId, List<Tuple> tuples) throws IOException {
     synchronized(appender) {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
index 84d81d5..636ae0f 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
@@ -31,7 +31,6 @@
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.util.Pair;
-import org.apache.tajo.storage.StorageManager;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -85,8 +84,8 @@
         if (!fs.exists(dataFile.getParent())) {
           fs.mkdirs(dataFile.getParent());
         }
-        FileAppender appender = (FileAppender) StorageManager.getStorageManager(
-            tajoConf).getAppender(meta, outSchema, dataFile);
+        FileAppender appender = (FileAppender)((FileStorageManager)StorageManager.getFileStorageManager(
+            tajoConf, null)).getAppender(meta, outSchema, dataFile);
         appender.enableStats();
         appender.init();
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/LineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/LineReader.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/LineReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/LineReader.java
index 66c610a..0f31baf 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/LineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/LineReader.java
@@ -60,7 +60,7 @@
    * default buffer-size (64k).
    *
    * @param in The input stream
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public LineReader(InputStream in) {
     this(in, DEFAULT_BUFFER_SIZE);
@@ -72,7 +72,7 @@
    *
    * @param in         The input stream
    * @param bufferSize Size of the read buffer
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public LineReader(InputStream in, int bufferSize) {
     this.in = in;
@@ -88,7 +88,7 @@
    *
    * @param in   input stream
    * @param conf configuration
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public LineReader(InputStream in, Configuration conf) throws IOException {
     this(in, conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE));
@@ -117,7 +117,7 @@
    * @param in                   The input stream
    * @param bufferSize           Size of the read buffer
    * @param recordDelimiterBytes The delimiter
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public LineReader(InputStream in, int bufferSize,
                     byte[] recordDelimiterBytes) {
@@ -136,7 +136,7 @@
    * @param in                   input stream
    * @param conf                 configuration
    * @param recordDelimiterBytes The delimiter
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public LineReader(InputStream in, Configuration conf,
                     byte[] recordDelimiterBytes) throws IOException {
@@ -150,7 +150,7 @@
   /**
    * Close the underlying stream.
    *
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public void close() throws IOException {
     in.close();
@@ -174,7 +174,7 @@
    *                          potentially by as much as one buffer length.
    * @return the number of bytes read including the (longest) newline
    *         found.
-   * @throws IOException if the underlying stream throws
+   * @throws java.io.IOException if the underlying stream throws
    */
   public int readLine(Text str, int maxLineLength,
                       int maxBytesToConsume) throws IOException {
@@ -540,7 +540,7 @@
    * @param str           the object to store the given line
    * @param maxLineLength the maximum number of bytes to store into str.
    * @return the number of bytes read including the newline
-   * @throws IOException if the underlying stream throws
+   * @throws java.io.IOException if the underlying stream throws
    */
   public int readLine(Text str, int maxLineLength) throws IOException {
     return readLine(str, maxLineLength, Integer.MAX_VALUE);
@@ -551,7 +551,7 @@
    *
    * @param str the object to store the given line
    * @return the number of bytes read including the newline
-   * @throws IOException if the underlying stream throws
+   * @throws java.io.IOException if the underlying stream throws
    */
   public int readLine(Text str) throws IOException {
     return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
similarity index 96%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
index 2fae243..cb86f35 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
@@ -25,6 +25,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.statistics.TableStats;
@@ -32,7 +33,7 @@
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.datum.ProtobufDatumFactory;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.BitArray;
 
@@ -66,7 +67,7 @@
     private long filePosition;
     private boolean forceFillBuffer;
 
-    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, FileFragment fragment) throws IOException {
+    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
       super(conf, schema, meta, fragment);
     }
 
@@ -81,15 +82,14 @@
       } catch (IllegalArgumentException iae) {
         throw new IOException(iae);
       }
-
       fis = new FileInputStream(file);
       channel = fis.getChannel();
       filePosition = startOffset = fragment.getStartKey();
-      endOffset = fragment.getStartKey() + fragment.getEndKey();
+      endOffset = fragment.getStartKey() + fragment.getLength();
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("RawFileScanner open:" + fragment + "," + channel.position() + ", file size :" + channel.size()
-            + ", fragment length :" + fragment.getEndKey());
+            + ", fragment length :" + fragment.getLength());
       }
 
       buf = BufferPool.directBuffer(64 * StorageUnit.KB);
@@ -127,9 +127,9 @@
       if(!forceFillBuffer && filePosition > offset && offset > filePosition - buffer.limit()){
         buffer.position((int)(offset - (filePosition - buffer.limit())));
       } else {
-        if(offset < startOffset || offset > startOffset + fragment.getEndKey()){
+        if(offset < startOffset || offset > startOffset + fragment.getLength()){
           throw new IndexOutOfBoundsException(String.format("range(%d, %d), offset: %d",
-              startOffset, startOffset + fragment.getEndKey(), offset));
+              startOffset, startOffset + fragment.getLength(), offset));
         }
         channel.position(offset);
         filePosition = offset;
@@ -432,7 +432,7 @@
       if(tableStats != null){
         tableStats.setNumRows(recordCount);
         tableStats.setReadBytes(totalReadBytes); // actual read bytes (scan + rescan * n)
-        tableStats.setNumBytes(fragment.getEndKey());
+        tableStats.setNumBytes(fragment.getLength());
       }
       return tableStats;
     }
@@ -465,8 +465,9 @@
 
     private TableStatistics stats;
 
-    public RawFileAppender(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
-      super(conf, schema, meta, path);
+    public RawFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                           Schema schema, TableMeta meta, Path workDir) throws IOException {
+      super(conf, taskAttemptId, schema, meta, workDir);
     }
 
     public void init() throws IOException {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java
similarity index 96%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java
index db36771..8da6ada 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java
@@ -25,6 +25,7 @@
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
@@ -33,7 +34,7 @@
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.BitArray;
 
 import java.io.FileNotFoundException;
@@ -66,7 +67,7 @@
     private BitArray nullFlags;
     private long bufferStartPos;
 
-    public RowFileScanner(Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment)
+    public RowFileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment)
         throws IOException {
       super(conf, schema, meta, fragment);
 
@@ -75,8 +76,8 @@
 
       nullFlags = new BitArray(schema.size());
       tupleHeaderSize = nullFlags.bytesLength() + (2 * Short.SIZE / 8);
-      this.start = fragment.getStartKey();
-      this.end = this.start + fragment.getEndKey();
+      this.start = this.fragment.getStartKey();
+      this.end = this.start + this.fragment.getLength();
     }
 
     public void init() throws IOException {
@@ -127,7 +128,7 @@
      * Find the sync from the front of the buffer
      *
      * @return return true if it succeeds to find the sync.
-     * @throws IOException
+     * @throws java.io.IOException
      */
     private boolean checkSync() throws IOException {
       buffer.getInt();                           // escape
@@ -313,9 +314,10 @@
     // statistics
     private TableStatistics stats;
 
-    public RowFileAppender(Configuration conf, final Schema schema, final TableMeta meta, final Path path)
+    public RowFileAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
+                           final Schema schema, final TableMeta meta, final Path workDir)
         throws IOException {
-      super(conf, schema, meta, path);
+      super(conf, taskAttemptId, schema, meta, workDir);
     }
 
     public void init() throws IOException {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/SplitLineReader.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/SplitLineReader.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
similarity index 94%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
index 6af8da0..dbb8bd0 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
@@ -18,30 +18,29 @@
 
 package org.apache.tajo.storage.avro;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.io.DatumWriter;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.FileAppender;
 import org.apache.tajo.storage.TableStatistics;
 import org.apache.tajo.storage.Tuple;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
 
 /**
  * FileAppender for writing to Avro files.
@@ -56,14 +55,16 @@
    * Creates a new AvroAppender.
    *
    * @param conf Configuration properties.
+   * @param taskAttemptId The task attempt id
    * @param schema The table schema.
    * @param meta The table metadata.
-   * @param path The path of the Parquet file to write to.
+   * @param workDir The path of the Parquet file to write to.
    */
   public AvroAppender(Configuration conf,
+                      QueryUnitAttemptId taskAttemptId,
                       org.apache.tajo.catalog.Schema schema,
-                      TableMeta meta, Path path) throws IOException {
-    super(conf, schema, meta, path);
+                      TableMeta meta, Path workDir) throws IOException {
+    super(conf, taskAttemptId, schema, meta, workDir);
   }
 
   /**
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
similarity index 98%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
index 816ae25..51594df 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
@@ -38,7 +38,7 @@
 import org.apache.tajo.storage.FileScanner;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -63,7 +63,7 @@
    */
   public AvroScanner(Configuration conf,
                      final org.apache.tajo.catalog.Schema schema,
-                     final TableMeta meta, final FileFragment fragment) {
+                     final TableMeta meta, final Fragment fragment) {
     super(conf, schema, meta, fragment);
   }
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/package-info.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/package-info.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/avro/package-info.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/package-info.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
similarity index 91%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
index 6fe6841..ac413ca 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
@@ -24,6 +24,8 @@
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.storage.StorageFragmentProtos.*;
 import org.apache.tajo.util.TUtil;
 
 import java.io.IOException;
@@ -31,14 +33,13 @@
 import java.util.Arrays;
 import java.util.List;
 
-import static org.apache.tajo.catalog.proto.CatalogProtos.FileFragmentProto;
 import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
 
 public class FileFragment implements Fragment, Comparable<FileFragment>, Cloneable {
   @Expose private String tableName; // required
   @Expose private Path uri; // required
-  @Expose private Long startOffset; // required
-  @Expose private Long length; // required
+  @Expose public Long startOffset; // required
+  @Expose public Long length; // required
 
   private String[] hosts; // Datanode hostnames
   @Expose private int[] diskIds;
@@ -120,6 +121,7 @@
     this.diskIds = diskIds;
   }
 
+  @Override
   public String getTableName() {
     return this.tableName;
   }
@@ -136,10 +138,20 @@
     return this.startOffset;
   }
 
-  public Long getEndKey() {
+  @Override
+  public String getKey() {
+    return this.uri.toString();
+  }
+
+  @Override
+  public long getLength() {
     return this.length;
   }
 
+  @Override
+  public boolean isEmpty() {
+    return this.length <= 0;
+  }
   /**
    * 
    * The offset range of tablets <b>MUST NOT</b> be overlapped.
@@ -169,7 +181,7 @@
       FileFragment t = (FileFragment) o;
       if (getPath().equals(t.getPath())
           && TUtil.checkEquals(t.getStartKey(), this.getStartKey())
-          && TUtil.checkEquals(t.getEndKey(), this.getEndKey())) {
+          && TUtil.checkEquals(t.getLength(), this.getLength())) {
         return true;
       }
     }
@@ -195,7 +207,7 @@
   public String toString() {
     return "\"fragment\": {\"id\": \""+ tableName +"\", \"path\": "
     		+getPath() + "\", \"start\": " + this.getStartKey() + ",\"length\": "
-        + getEndKey() + "}" ;
+        + getLength() + "}" ;
   }
 
   public FragmentProto getProto() {
@@ -218,6 +230,7 @@
 
     FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder();
     fragmentBuilder.setId(this.tableName);
+    fragmentBuilder.setStoreType(StoreType.CSV.name());
     fragmentBuilder.setContents(builder.buildPartial().toByteString());
     return fragmentBuilder.build();
   }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
similarity index 87%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
index ccba3be..a6af19b 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
@@ -20,14 +20,13 @@
 
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.storage.BaseTupleComparator;
 import org.apache.tajo.storage.TupleComparator;
 
 import java.io.IOException;
 
 public interface IndexMethod {
   IndexWriter getIndexWriter(final Path fileName, int level, Schema keySchema,
-      TupleComparator comparator) throws IOException;
+                             TupleComparator comparator) throws IOException;
   IndexReader getIndexReader(final Path fileName, Schema keySchema,
-      TupleComparator comparator) throws IOException;
+                             TupleComparator comparator) throws IOException;
 }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexReader.java
similarity index 96%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexReader.java
index 7baf7aa..3ae5c9d 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexReader.java
@@ -29,7 +29,7 @@
    * 
    * @param key
    * @return
-   * @throws IOException 
+   * @throws java.io.IOException
    */
   public long find(Tuple key) throws IOException;
 }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
similarity index 94%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
index 688bbc7..0c07b4a 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
@@ -32,14 +32,14 @@
    * 
    * @param key to find
    * @return
-   * @throws IOException 
+   * @throws java.io.IOException
    */
   public long find(Tuple key, boolean nextKey) throws IOException;
-  
+
   /**
    * Return the next offset from the latest find or next offset
    * @return
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public long next() throws IOException;
 }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
similarity index 99%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
index f093f9d..81a1ffd 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
@@ -101,7 +101,7 @@
      *
      * @param level
      *          : IndexCreater.ONE_LEVEL_INDEX or IndexCreater.TWO_LEVEL_INDEX
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public BSTIndexWriter(final Path fileName, int level, Schema keySchema,
         TupleComparator comparator) throws IOException {
@@ -183,7 +183,7 @@
 
       int entryNum = keySet.size();
       writeHeader(entryNum);
-      
+
       int loadCount = this.loadNum - 1;
       for (Tuple key : keySet) {
 
@@ -198,7 +198,7 @@
         byte[] buf = rowStoreEncoder.toBytes(key);
         out.writeInt(buf.length);
         out.write(buf);
-        
+
         /**/
         LinkedList<Long> offsetList = keyOffsetMap.get(key);
         /* offset num writing */
@@ -306,7 +306,7 @@
      * @param fileName
      * @param keySchema
      * @param comparator
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public BSTIndexReader(final Path fileName, Schema keySchema, TupleComparator comparator) throws IOException {
       this.fileName = fileName;
@@ -401,7 +401,7 @@
     /**
      *
      * @return
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public long find(Tuple key) throws IOException {
       return find(key, false);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerDe.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
similarity index 95%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
index c7007d8..cd31ada 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java
@@ -20,22 +20,18 @@
 
 
 import net.minidev.json.JSONObject;
-import org.apache.commons.lang.CharSet;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.common.exception.NotImplementedException;
-import org.apache.tajo.datum.ProtobufDatum;
 import org.apache.tajo.datum.TextDatum;
 import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
 import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.text.TextLineSerDe;
 import org.apache.tajo.storage.text.TextLineSerializer;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.nio.charset.Charset;
 
 public class JsonLineSerializer extends TextLineSerializer {
   private static ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
similarity index 93%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
index 3a3bb57..b10d423 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.storage.parquet;
 
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.storage.StorageConstants;
 import parquet.hadoop.ParquetOutputFormat;
 import parquet.hadoop.metadata.CompressionCodecName;
@@ -51,11 +52,11 @@
    * @param conf Configuration properties.
    * @param schema The table schema.
    * @param meta The table metadata.
-   * @param path The path of the Parquet file to write to.
+   * @param workDir The path of the Parquet file to write to.
    */
-  public ParquetAppender(Configuration conf, Schema schema, TableMeta meta,
-                         Path path) throws IOException {
-    super(conf, schema, meta, path);
+  public ParquetAppender(Configuration conf, QueryUnitAttemptId taskAttemptId, Schema schema, TableMeta meta,
+                         Path workDir) throws IOException {
+    super(conf, taskAttemptId, schema, meta, workDir);
     this.blockSize = Integer.parseInt(
         meta.getOption(ParquetOutputFormat.BLOCK_SIZE, StorageConstants.PARQUET_DEFAULT_BLOCK_SIZE));
     this.pageSize = Integer.parseInt(
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
similarity index 95%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
index 36b89b8..2f8efcf 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
@@ -23,7 +23,7 @@
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.storage.FileScanner;
 import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 import java.io.IOException;
 
@@ -42,7 +42,7 @@
    * @param fragment
    */
   public ParquetScanner(Configuration conf, final Schema schema,
-                        final TableMeta meta, final FileFragment fragment) {
+                        final TableMeta meta, final Fragment fragment) {
     super(conf, schema, meta, fragment);
   }
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
index 69b76c4..5f220c5 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
@@ -42,7 +42,7 @@
    * @param blockSize The block size threshold.
    * @param pageSize See parquet write up. Blocks are subdivided into pages
    *                 for alignment.
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public TajoParquetWriter(Path file,
                            Schema schema,
@@ -68,7 +68,7 @@
    *                 for alignment.
    * @param enableDictionary Whether to use a dictionary to compress columns.
    * @param validating Whether to turn on validation.
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public TajoParquetWriter(Path file,
                            Schema schema,
@@ -92,7 +92,7 @@
    *
    * @param file The Path of the file to write to.
    * @param schema The Tajo schema of the table.
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public TajoParquetWriter(Path file, Schema schema) throws IOException {
     this(file,
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
similarity index 93%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
index 269f782..a64e987 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
@@ -18,22 +18,20 @@
 
 package org.apache.tajo.storage.parquet;
 
-import java.util.Map;
-
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
 import parquet.Log;
 import parquet.hadoop.api.InitContext;
 import parquet.hadoop.api.ReadSupport;
 import parquet.io.api.RecordMaterializer;
 import parquet.schema.MessageType;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.json.CatalogGsonHelper;
-import org.apache.tajo.storage.Tuple;
+import java.util.Map;
 
 /**
- * Tajo implementation of {@link ReadSupport} for {@link Tuple}s.
- * Users should use {@link ParquetScanner} and not this class directly.
+ * Tajo implementation of {@link parquet.hadoop.api.ReadSupport} for {@link org.apache.tajo.storage.Tuple}s.
+ * Users should use {@link org.apache.tajo.storage.parquet.ParquetScanner} and not this class directly.
  */
 public class TajoReadSupport extends ReadSupport<Tuple> {
   private static final Log LOG = Log.getLog(TajoReadSupport.class);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
index 7c3d79d..4375fa4 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
@@ -18,29 +18,23 @@
 
 package org.apache.tajo.storage.parquet;
 
-import com.google.protobuf.Message;
 import com.google.protobuf.InvalidProtocolBufferException;
-
-import java.nio.ByteBuffer;
-
-import parquet.io.api.GroupConverter;
-import parquet.io.api.Converter;
-import parquet.io.api.PrimitiveConverter;
-import parquet.io.api.Binary;
-import parquet.schema.Type;
-import parquet.schema.GroupType;
-
+import com.google.protobuf.Message;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.*;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.BlobDatum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.ProtobufDatumFactory;
+import parquet.io.api.Binary;
+import parquet.io.api.Converter;
+import parquet.io.api.GroupConverter;
+import parquet.io.api.PrimitiveConverter;
+import parquet.schema.GroupType;
+import parquet.schema.Type;
+
+import java.nio.ByteBuffer;
 
 /**
  * Converter to convert a Parquet record into a Tajo Tuple.
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
similarity index 99%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
index e31828c..436159c 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
@@ -18,14 +18,13 @@
 
 package org.apache.tajo.storage.parquet;
 
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
 import parquet.io.api.GroupConverter;
 import parquet.io.api.RecordMaterializer;
 import parquet.schema.MessageType;
 
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.storage.Tuple;
-
 /**
  * Materializes a Tajo Tuple from a stream of Parquet data.
  */
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
similarity index 89%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
index 2592231..555b623 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
@@ -157,48 +157,48 @@
     switch (type) {
       case BOOLEAN:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BOOLEAN);
+                         PrimitiveTypeName.BOOLEAN);
       case BIT:
       case INT2:
       case INT4:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.INT32);
+                         PrimitiveTypeName.INT32);
       case INT8:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.INT64);
+                         PrimitiveTypeName.INT64);
       case FLOAT4:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.FLOAT);
+                         PrimitiveTypeName.FLOAT);
       case FLOAT8:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.DOUBLE);
+                         PrimitiveTypeName.DOUBLE);
       case CHAR:
       case TEXT:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY,
+                         PrimitiveTypeName.BINARY,
                          OriginalType.UTF8);
       case PROTOBUF:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY);
+                         PrimitiveTypeName.BINARY);
       case BLOB:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY);
+                         PrimitiveTypeName.BINARY);
       case INET4:
       case INET6:
         return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY);
+                         PrimitiveTypeName.BINARY);
       default:
         throw new RuntimeException("Cannot convert Tajo type: " + type);
     }
   }
 
   private PrimitiveType primitive(String name,
-                                  PrimitiveType.PrimitiveTypeName primitive) {
+                                  PrimitiveTypeName primitive) {
     return new PrimitiveType(Type.Repetition.OPTIONAL, primitive, name, null);
   }
 
   private PrimitiveType primitive(String name,
-                                  PrimitiveType.PrimitiveTypeName primitive,
+                                  PrimitiveTypeName primitive,
                                   OriginalType originalType) {
     return new PrimitiveType(Type.Repetition.OPTIONAL, primitive, name,
                              originalType);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
similarity index 95%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
index 35165de..00aadf4 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
@@ -18,10 +18,12 @@
 
 package org.apache.tajo.storage.parquet;
 
-import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Tuple;
 import parquet.hadoop.api.WriteSupport;
 import parquet.io.api.Binary;
 import parquet.io.api.RecordConsumer;
@@ -29,15 +31,12 @@
 import parquet.schema.MessageType;
 import parquet.schema.Type;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.datum.Datum;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 /**
- * Tajo implementation of {@link WriteSupport} for {@link Tuple}s.
+ * Tajo implementation of {@link parquet.hadoop.api.WriteSupport} for {@link org.apache.tajo.storage.Tuple}s.
  * Users should use {@link ParquetAppender} and not this class directly.
  */
 public class TajoWriteSupport extends WriteSupport<Tuple> {
@@ -70,7 +69,7 @@
   /**
    * Called once per row group.
    *
-   * @param recordConsumer The {@link RecordConsumer} to write to.
+   * @param recordConsumer The {@link parquet.io.api.RecordConsumer} to write to.
    */
   @Override
   public void prepareForWrite(RecordConsumer recordConsumer) {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/package-info.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/parquet/package-info.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
index c83b505..158c740 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
@@ -102,7 +102,7 @@
    * Returns a copy of the underlying bytes referenced by this instance.
    *
    * @return a new copied byte array
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public byte[] getBytesCopy() throws IOException {
     lazyDecompress();
@@ -114,7 +114,7 @@
   /**
    * Returns the underlying bytes.
    *
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public byte[] getData() throws IOException {
     lazyDecompress();
@@ -125,7 +125,7 @@
    * readFields() will corrupt the array. So use the set method whenever
    * possible.
    *
-   * @see #readFields(DataInput)
+   * @see #readFields(java.io.DataInput)
    */
   public void set(byte[] newData, int offset, int len) {
     bytes = newData;
@@ -138,7 +138,7 @@
    * readFields() will corrupt the array. So use the set method whenever
    * possible.
    *
-   * @see #readFields(DataInput)
+   * @see #readFields(java.io.DataInput)
    */
   public void set(LazyDecompressionCallback newData, int offset, int len) {
     bytes = null;
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
similarity index 94%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
index 707d55a..eab2356 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
@@ -23,7 +23,7 @@
 /**
  * Used to call back lazy decompression process.
  *
- * @see BytesRefWritable
+ * @see org.apache.tajo.storage.rcfile.BytesRefWritable
  */
 public interface LazyDecompressionCallback {
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
similarity index 96%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
index e5507ad..d88223b 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
@@ -29,6 +29,7 @@
 import org.apache.hadoop.io.SequenceFile.Metadata;
 import org.apache.hadoop.io.compress.*;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
@@ -38,7 +39,7 @@
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 
 import java.io.Closeable;
 import java.io.*;
@@ -61,7 +62,7 @@
  * which can be set like: <code>conf.setInt(COLUMNS_BUFFER_SIZE_CONF_STR,
  * 4 * 1024 * 1024)</code> .
  * <p>
- * <code>RCFile</code> provides {@link Writer}, {@link Reader} and classes for
+ * <code>RCFile</code> provides {@link java.io.Writer}, {@link java.io.Reader} and classes for
  * writing, reading respectively.
  * </p>
  * <p/>
@@ -77,11 +78,11 @@
  * RCFile compresses values in a more fine-grained manner then record level
  * compression. However, It currently does not support compress the key part
  * yet. The actual compression algorithm used to compress key and/or values can
- * be specified by using the appropriate {@link CompressionCodec}.
+ * be specified by using the appropriate {@link org.apache.hadoop.io.compress.CompressionCodec}.
  * </p>
  * <p/>
  * <p>
- * The {@link Reader} is used to read and explain the bytes of RCFile.
+ * The {@link java.io.Reader} is used to read and explain the bytes of RCFile.
  * </p>
  * <p/>
  * <h4 id="Formats">RCFile Formats</h4>
@@ -95,7 +96,7 @@
  * keys/values in this file.</li>
  * <li>compression codec - <code>CompressionCodec</code> class which is used
  * for compression of keys and/or values (if compression is enabled).</li>
- * <li>metadata - {@link Metadata} for this file.</li>
+ * <li>metadata - {@link org.apache.hadoop.io.SequenceFile.Metadata} for this file.</li>
  * <li>sync - A sync marker to denote end of the header.</li>
  * </ul>
  * <p/>
@@ -709,8 +710,9 @@
       return out.getPos();
     }
 
-    public RCFileAppender(Configuration conf, final Schema schema, final TableMeta meta, final Path path) throws IOException {
-      super(conf, schema, meta, path);
+    public RCFileAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
+                          final Schema schema, final TableMeta meta, final Path workDir) throws IOException {
+      super(conf, taskAttemptId, schema, meta, workDir);
 
       RECORD_INTERVAL = conf.getInt(RECORD_INTERVAL_CONF_STR, RECORD_INTERVAL);
       COLUMNS_BUFFER_SIZE = conf.getInt(COLUMNS_BUFFER_SIZE_CONF_STR, COLUMNS_BUFFER_SIZE);
@@ -878,13 +880,13 @@
 
     /**
      * Append a row of values. Currently it only can accept <
-     * {@link Tuple}. If its <code>size()</code> is less than the
+     * {@link org.apache.tajo.storage.Tuple}. If its <code>size()</code> is less than the
      * column number in the file, zero bytes are appended for the empty columns.
      * If its size() is greater then the column number in the file, the exceeded
      * columns' bytes are ignored.
      *
      * @param tuple a Tuple with the list of serialized columns
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public void append(Tuple tuple) throws IOException {
       int size = schema.size();
@@ -920,7 +922,7 @@
      * get number of bytes to store the keyBuffer.
      *
      * @return number of bytes used to store this KeyBuffer on disk
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public int getKeyBufferSize() throws IOException {
       int ret = 0;
@@ -940,7 +942,7 @@
      * get number of bytes to store the key part.
      *
      * @return number of bytes used to store this Key part on disk
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public int getKeyPartSize() throws IOException {
       int ret = 12; //12 bytes |record count, key length, compressed key length|
@@ -1176,12 +1178,12 @@
     private SerializerDeserializer serde;
 
     public RCFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
-                         final FileFragment fragment) throws IOException {
+                         final Fragment fragment) throws IOException {
       super(conf, schema, meta, fragment);
       conf.setInt("io.file.buffer.size", 4096); //TODO remove
 
-      startOffset = fragment.getStartKey();
-      endOffset = startOffset + fragment.getEndKey();
+      startOffset = this.fragment.getStartKey();
+      endOffset = startOffset + this.fragment.getLength();
       start = 0;
     }
 
@@ -1293,7 +1295,7 @@
 
     /**
      * Override this method to specialize the type of
-     * {@link FSDataInputStream} returned.
+     * {@link org.apache.hadoop.fs.FSDataInputStream} returned.
      */
     protected FSDataInputStream openFile(FileSystem fs, Path file, int bufferSize) throws IOException {
       return fs.open(file, bufferSize);
@@ -1397,10 +1399,10 @@
      * <p/>
      * <p/>
      * The position passed must be a position returned by
-     * {@link RCFile.RCFileAppender#getLength()} when writing this file. To seek to an
-     * arbitrary position, use {@link RCFile.RCFileScanner#sync(long)}. In another
+     * {@link org.apache.tajo.storage.rcfile.RCFile.RCFileAppender#getLength()} when writing this file. To seek to an
+     * arbitrary position, use {@link org.apache.tajo.storage.rcfile.RCFile.RCFileScanner#sync(long)}. In another
      * words, the current seek can only seek to the end of the file. For other
-     * positions, use {@link RCFile.RCFileScanner#sync(long)}.
+     * positions, use {@link org.apache.tajo.storage.rcfile.RCFile.RCFileScanner#sync(long)}.
      */
     public void seek(long position) throws IOException {
       in.seek(position);
@@ -1488,7 +1490,7 @@
      * block.
      *
      * @return the length of the next record or -1 if there is no next record
-     * @throws IOException
+     * @throws java.io.IOException
      */
     private int readRecordLength() throws IOException {
       if (in.getPos() >= end) {
@@ -1651,7 +1653,7 @@
           return 0.0f;
         } else {
           //if scanner read the header, filePos moved to zero
-          return Math.min(1.0f, (float)(Math.max(filePos - startOffset, 0)) / (float)(fragment.getEndKey()));
+          return Math.min(1.0f, (float)(Math.max(filePos - startOffset, 0)) / (float)(fragment.getLength()));
         }
       } catch (IOException e) {
         LOG.error(e.getMessage(), e);
@@ -1666,7 +1668,7 @@
      * value buffer.
      *
      * @return next row number
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public boolean nextBuffer(LongWritable readRows) throws IOException {
       if (readRowsIndexInBuffer < recordsNumInValBuffer) {
@@ -1692,7 +1694,7 @@
      * get the current row used,make sure called {@link #next()}
      * first.
      *
-     * @throws IOException
+     * @throws java.io.IOException
      */
     public void getCurrentRow(Tuple tuple) throws IOException {
       if (!keyInit || rowFetched) {
@@ -1727,7 +1729,7 @@
      * @param selCol - index among selectedColumns
      * @param col    - column object to update the state of.  prvLength will be
      *               set to the new read position
-     * @throws IOException
+     * @throws java.io.IOException
      */
     private void colAdvanceRow(int selCol, SelectedColumn col) throws IOException {
       if (col.runLength > 0) {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
similarity index 96%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
index f5cef62..14e0f26 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
@@ -29,6 +29,7 @@
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -73,8 +74,9 @@
 
   private Writable EMPTY_KEY;
 
-  public SequenceFileAppender(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
-    super(conf, schema, meta, path);
+  public SequenceFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                              Schema schema, TableMeta meta, Path workDir) throws IOException {
+    super(conf, taskAttemptId, schema, meta, workDir);
     this.meta = meta;
     this.schema = schema;
   }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
similarity index 98%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
index b0ef67d..74563ff 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
@@ -32,7 +32,7 @@
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.BytesUtils;
 
 import java.io.IOException;
@@ -71,7 +71,7 @@
 
   private Writable EMPTY_KEY;
 
-  public SequenceFileScanner(Configuration conf, Schema schema, TableMeta meta, FileFragment fragment) throws IOException {
+  public SequenceFileScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
     super(conf, schema, meta, fragment);
   }
 
@@ -96,7 +96,7 @@
     this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0);
 
     this.start = fragment.getStartKey();
-    this.end = start + fragment.getEndKey();
+    this.end = start + fragment.getLength();
 
     if (fragment.getStartKey() > reader.getPosition())
       reader.sync(this.start);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineDeserializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerDe.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
index 7397000..48154eb 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/CSVLineSerializer.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.storage.text;
 
-import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.datum.Datum;
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
similarity index 98%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
index eb1929e..10d86bd 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
@@ -31,9 +31,9 @@
 import org.apache.hadoop.io.compress.SplittableCompressionCodec;
 import org.apache.tajo.common.exception.NotImplementedException;
 import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.BufferPool;
 import org.apache.tajo.storage.ByteBufInputChannel;
 import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.BufferPool;
 import org.apache.tajo.storage.compress.CodecPool;
 import org.apache.tajo.storage.fragment.FileFragment;
 
@@ -77,7 +77,7 @@
     }
     if (fis == null) fis = fs.open(fragment.getPath());
     pos = startOffset = fragment.getStartKey();
-    end = startOffset + fragment.getEndKey();
+    end = startOffset + fragment.getLength();
 
     if (codec != null) {
       decompressor = CodecPool.getDecompressor(codec);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
similarity index 95%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
index ab8a0b5..8824e3e 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.apache.hadoop.io.compress.Compressor;
+import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.statistics.TableStats;
@@ -37,6 +38,7 @@
 import org.apache.tajo.storage.compress.CodecPool;
 import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
 import org.apache.tajo.util.ReflectionUtil;
 
@@ -54,7 +56,6 @@
 public class DelimitedTextFile {
 
   public static final byte LF = '\n';
-  public static int EOF = -1;
 
   private static final Log LOG = LogFactory.getLog(DelimitedTextFile.class);
 
@@ -113,9 +114,10 @@
     private NonSyncByteArrayOutputStream os;
     private TextLineSerializer serializer;
 
-    public DelimitedTextFileAppender(Configuration conf, final Schema schema, final TableMeta meta, final Path path)
+    public DelimitedTextFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                                     final Schema schema, final TableMeta meta, final Path path)
         throws IOException {
-      super(conf, schema, meta, path);
+      super(conf, taskAttemptId, schema, meta, path);
       this.fs = path.getFileSystem(conf);
       this.meta = meta;
       this.schema = schema;
@@ -284,21 +286,22 @@
     private int errorNum;
 
     public DelimitedTextFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
-                                    final FileFragment fragment)
+                                    final Fragment fragment)
         throws IOException {
       super(conf, schema, meta, fragment);
-      reader = new DelimitedLineReader(conf, fragment);
+      reader = new DelimitedLineReader(conf, this.fragment);
       if (!reader.isCompressed()) {
         splittable = true;
       }
 
-      startOffset = fragment.getStartKey();
-      endOffset = startOffset + fragment.getEndKey();
+      startOffset = this.fragment.getStartKey();
+      endOffset = startOffset + fragment.getLength();
 
       errorTorrenceMaxNum =
           Integer.parseInt(meta.getOption(TEXT_ERROR_TOLERANCE_MAXNUM, DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM));
     }
 
+
     @Override
     public void init() throws IOException {
       if (reader != null) {
@@ -470,7 +473,7 @@
       if (tableStats != null && reader != null) {
         tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
         tableStats.setNumRows(recordCount);
-        tableStats.setNumBytes(fragment.getEndKey());
+        tableStats.setNumBytes(fragment.getLength());
       }
       return tableStats;
     }
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineDeserializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineParsingError.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
similarity index 100%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerializer.java
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
similarity index 95%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
index 543336f..f76593e 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
@@ -18,6 +18,13 @@
 
 package org.apache.tajo.storage.thirdparty.parquet;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.*;
+import org.apache.hadoop.util.ReflectionUtils;
+import parquet.bytes.BytesInput;
+import parquet.hadoop.BadConfigurationException;
+import parquet.hadoop.metadata.CompressionCodecName;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -25,19 +32,6 @@
 import java.util.HashMap;
 import java.util.Map;
 
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.util.ReflectionUtils;
-
-import parquet.bytes.BytesInput;
-import parquet.hadoop.BadConfigurationException;
-import parquet.hadoop.metadata.CompressionCodecName;
-
 class CodecFactory {
 
   public class BytesDecompressor {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
similarity index 97%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
index 5f89ead..0dedd9b 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
@@ -18,17 +18,6 @@
 
 package org.apache.tajo.storage.thirdparty.parquet;
 
-import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
-import static parquet.Log.INFO;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import parquet.Log;
 import parquet.bytes.BytesInput;
 import parquet.bytes.CapacityByteArrayOutputStream;
@@ -37,12 +26,18 @@
 import parquet.column.page.DictionaryPage;
 import parquet.column.page.PageWriteStore;
 import parquet.column.page.PageWriter;
-import parquet.column.statistics.Statistics;
 import parquet.column.statistics.BooleanStatistics;
+import parquet.column.statistics.Statistics;
 import parquet.format.converter.ParquetMetadataConverter;
 import parquet.io.ParquetEncodingException;
 import parquet.schema.MessageType;
 
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
+import static parquet.Log.INFO;
+
 class ColumnChunkPageWriteStore implements PageWriteStore {
   private static final Log LOG = Log.getLog(ColumnChunkPageWriteStore.class);
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
similarity index 99%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
index 61567e5..6bbd7b5 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
@@ -18,9 +18,6 @@
 
 package org.apache.tajo.storage.thirdparty.parquet;
 
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import parquet.Log;
@@ -39,6 +36,10 @@
 import parquet.schema.MessageType;
 import parquet.schema.Type;
 
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
 import static java.lang.String.format;
 import static parquet.Log.DEBUG;
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
similarity index 99%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
index 7410d2b..532d9a2 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
@@ -18,16 +18,6 @@
 
 package org.apache.tajo.storage.thirdparty.parquet;
 
-import static java.lang.Math.max;
-import static java.lang.Math.min;
-import static java.lang.String.format;
-import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
-import static parquet.Log.DEBUG;
-import static parquet.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.util.Map;
-
 import parquet.Log;
 import parquet.column.ParquetProperties.WriterVersion;
 import parquet.column.impl.ColumnWriteStoreImpl;
@@ -36,6 +26,16 @@
 import parquet.io.MessageColumnIO;
 import parquet.schema.MessageType;
 
+import java.io.IOException;
+import java.util.Map;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.String.format;
+import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
+import static parquet.Log.DEBUG;
+import static parquet.Preconditions.checkNotNull;
+
 class InternalParquetRecordWriter<T> {
   private static final Log LOG = Log.getLog(InternalParquetRecordWriter.class);
 
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
similarity index 95%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
index 73ce7c2..f1c5368 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
@@ -18,24 +18,10 @@
 
 package org.apache.tajo.storage.thirdparty.parquet;
 
-import static parquet.Log.DEBUG;
-import static parquet.format.Util.writeFileMetaData;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-
 import parquet.Log;
 import parquet.Version;
 import parquet.bytes.BytesInput;
@@ -45,17 +31,19 @@
 import parquet.column.statistics.Statistics;
 import parquet.format.converter.ParquetMetadataConverter;
 import parquet.hadoop.Footer;
-import parquet.hadoop.metadata.BlockMetaData;
-import parquet.hadoop.metadata.ColumnChunkMetaData;
-import parquet.hadoop.metadata.ColumnPath;
-import parquet.hadoop.metadata.CompressionCodecName;
-import parquet.hadoop.metadata.FileMetaData;
-import parquet.hadoop.metadata.GlobalMetaData;
-import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.hadoop.metadata.*;
 import parquet.io.ParquetEncodingException;
 import parquet.schema.MessageType;
 import parquet.schema.PrimitiveType.PrimitiveTypeName;
 
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.*;
+import java.util.Map.Entry;
+
+import static parquet.Log.DEBUG;
+import static parquet.format.Util.writeFileMetaData;
+
 /**
  * Internal implementation of the Parquet file writer as a block container
  *
@@ -148,7 +136,7 @@
    * @param schema the schema of the data
    * @param out the file to write to
    * @param codec the codec to use to compress blocks
-   * @throws IOException if the file can not be created
+   * @throws java.io.IOException if the file can not be created
    */
   public ParquetFileWriter(Configuration configuration, MessageType schema, Path file) throws IOException {
     super();
@@ -159,7 +147,7 @@
 
   /**
    * start the file
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public void start() throws IOException {
     state = state.start();
@@ -170,7 +158,7 @@
   /**
    * start a block
    * @param recordCount the record count in this block
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public void startBlock(long recordCount) throws IOException {
     state = state.startBlock();
@@ -186,7 +174,7 @@
    * @param valueCount the value count in this column
    * @param statistics the statistics in this column
    * @param compressionCodecName
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public void startColumn(ColumnDescriptor descriptor,
                           long valueCount,
@@ -312,7 +300,7 @@
    * @param bytes bytes to be written including page headers
    * @param uncompressedTotalPageSize total uncompressed size (without page headers)
    * @param compressedTotalPageSize total compressed size (without page headers)
-   * @throws IOException
+   * @throws java.io.IOException
    */
   void writeDataPages(BytesInput bytes,
                       long uncompressedTotalPageSize,
@@ -332,7 +320,7 @@
 
   /**
    * end a column (once all rep, def and data have been written)
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public void endColumn() throws IOException {
     state = state.endColumn();
@@ -357,7 +345,7 @@
 
   /**
    * ends a block once all column chunks have been written
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public void endBlock() throws IOException {
     state = state.endBlock();
@@ -371,7 +359,7 @@
    * ends a file once all blocks have been written.
    * closes the file.
    * @param extraMetaData the extra meta data to write in the footer
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public void end(Map<String, String> extraMetaData) throws IOException {
     state = state.end();
@@ -395,7 +383,7 @@
    * @param configuration the configuration to use to get the FileSystem
    * @param outputPath the directory to write the _metadata file to
    * @param footers the list of footers to merge
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public static void writeMetadataFile(Configuration configuration, Path outputPath, List<Footer> footers) throws IOException {
     Path metaDataPath = new Path(outputPath, PARQUET_METADATA_FILE);
@@ -432,7 +420,7 @@
 
   /**
    * @return the current position in the underlying file
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public long getPos() throws IOException {
     return out.getPos();
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
similarity index 93%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
index 0fb2c3a..9c167a0 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
@@ -18,22 +18,13 @@
 
 package org.apache.tajo.storage.thirdparty.parquet;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-
 import parquet.filter.UnboundRecordFilter;
-import parquet.hadoop.*;
+import parquet.hadoop.Footer;
+import parquet.hadoop.ParquetFileReader;
 import parquet.hadoop.api.InitContext;
 import parquet.hadoop.api.ReadSupport;
 import parquet.hadoop.api.ReadSupport.ReadContext;
@@ -41,6 +32,10 @@
 import parquet.hadoop.metadata.GlobalMetaData;
 import parquet.schema.MessageType;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.*;
+
 /**
  * Read records from a Parquet file.
  */
@@ -57,7 +52,7 @@
   /**
    * @param file the file to read
    * @param readSupport to materialize records
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public ParquetReader(Path file, ReadSupport<T> readSupport) throws IOException {
     this(file, readSupport, null);
@@ -67,7 +62,7 @@
    * @param conf the configuration
    * @param file the file to read
    * @param readSupport to materialize records
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport) throws IOException {
     this(conf, file, readSupport, null);
@@ -77,7 +72,7 @@
    * @param file the file to read
    * @param readSupport to materialize records
    * @param filter the filter to use to filter records
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public ParquetReader(Path file, ReadSupport<T> readSupport, UnboundRecordFilter filter) throws IOException {
     this(new Configuration(), file, readSupport, filter);
@@ -88,7 +83,7 @@
    * @param file the file to read
    * @param readSupport to materialize records
    * @param filter the filter to use to filter records
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport, UnboundRecordFilter filter) throws IOException {
     this.readSupport = readSupport;
@@ -113,7 +108,7 @@
 
   /**
    * @return the next record or null if finished
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public T read() throws IOException {
     try {
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
similarity index 90%
rename from tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
rename to tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
index 0447a47..7527437 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
@@ -51,7 +51,7 @@
    * @param blockSize the block size threshold
    * @param pageSize the page size threshold
    * @throws java.io.IOException
-   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, CompressionCodecName, int, int, boolean, boolean)
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, boolean, boolean)
    */
   public ParquetWriter(Path file, WriteSupport<T> writeSupport, CompressionCodecName compressionCodecName, int blockSize, int pageSize) throws IOException {
     this(file, writeSupport, compressionCodecName, blockSize, pageSize,
@@ -68,8 +68,8 @@
    * @param pageSize the page size threshold (both data and dictionary)
    * @param enableDictionary to turn dictionary encoding on
    * @param validating to turn on validation using the schema
-   * @throws IOException
-   * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean)
+   * @throws java.io.IOException
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, int, boolean, boolean)
    */
   public ParquetWriter(
       Path file,
@@ -93,8 +93,8 @@
    * @param dictionaryPageSize the page size threshold for the dictionary pages
    * @param enableDictionary to turn dictionary encoding on
    * @param validating to turn on validation using the schema
-   * @throws IOException
-   * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion)
+   * @throws java.io.IOException
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion)
    */
   public ParquetWriter(
       Path file,
@@ -124,9 +124,9 @@
    * @param dictionaryPageSize the page size threshold for the dictionary pages
    * @param enableDictionary to turn dictionary encoding on
    * @param validating to turn on validation using the schema
-   * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion}
-   * @throws IOException
-   * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion, org.apache.hadoop.conf.Configuration)
+   * @param writerVersion version of parquetWriter from {@link parquet.column.ParquetProperties.WriterVersion}
+   * @throws java.io.IOException
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion, org.apache.hadoop.conf.Configuration)
    */
   public ParquetWriter(
       Path file,
@@ -152,9 +152,9 @@
    * @param dictionaryPageSize the page size threshold for the dictionary pages
    * @param enableDictionary to turn dictionary encoding on
    * @param validating to turn on validation using the schema
-   * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion}
+   * @param writerVersion version of parquetWriter from {@link parquet.column.ParquetProperties.WriterVersion}
    * @param conf Hadoop configuration to use while accessing the filesystem
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public ParquetWriter(
       Path file,
@@ -196,7 +196,7 @@
    *
    * @param file the file to create
    * @param writeSupport the implementation to write a record to a RecordConsumer
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public ParquetWriter(Path file, WriteSupport<T> writeSupport) throws IOException {
     this(file, writeSupport, DEFAULT_COMPRESSION_CODEC_NAME, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-storage/tajo-storage-hdfs/src/main/proto/StorageFragmentProtos.proto
similarity index 63%
copy from tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
copy to tajo-storage/tajo-storage-hdfs/src/main/proto/StorageFragmentProtos.proto
index 7baf7aa..ce9aab6 100644
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/proto/StorageFragmentProtos.proto
@@ -16,20 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.tajo.storage.index;
+option java_package = "org.apache.tajo.storage";
+option java_outer_classname = "StorageFragmentProtos";
+option optimize_for = SPEED;
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
 
-import org.apache.tajo.storage.Tuple;
+import "CatalogProtos.proto";
 
-import java.io.IOException;
-
-public interface IndexReader {
-  
-  /**
-   * Find the offset corresponding to key which is equal to a given key.
-   * 
-   * @param key
-   * @return
-   * @throws IOException 
-   */
-  public long find(Tuple key) throws IOException;
-}
+message FileFragmentProto {
+  required string id = 1;
+  required string path = 2;
+  required int64 startOffset = 3;
+  required int64 length = 4;
+  repeated string hosts = 5;
+  repeated int32 diskIds = 6;
+}
\ No newline at end of file
diff --git a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServer.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServer.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/HttpFileServer.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServer.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerHandler.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerHandler.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/HttpFileServerHandler.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerHandler.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
similarity index 96%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
index fd5a63e..3c78d6b 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
@@ -123,7 +123,7 @@
 
     String fileName = "Compression_" + codec.getSimpleName();
     Path tablePath = new Path(testDir, fileName);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.enableStats();
 
     appender.init();
@@ -155,7 +155,7 @@
     FileFragment[] tablets = new FileFragment[1];
     tablets[0] = new FileFragment(fileName, tablePath, 0, fileLen);
 
-    Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
 
     if (StoreType.CSV == storeType) {
       if (SplittableCompressionCodec.class.isAssignableFrom(codec)) {
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
similarity index 94%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
index 93fb12b..8749925 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
@@ -31,7 +31,6 @@
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.FileUtil;
 import org.junit.Test;
@@ -105,7 +104,7 @@
     TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "-1");
     FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple tuple;
@@ -127,7 +126,7 @@
     TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "1");
     FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     assertNotNull(scanner.next());
@@ -149,7 +148,7 @@
     TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.JSON);
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "0");
     FileFragment fragment =  getFileFragment("testErrorTolerance2.json");
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     try {
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java
similarity index 84%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java
index c3d4992..19a39a2 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java
@@ -32,6 +32,7 @@
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -44,9 +45,9 @@
 
 import static org.junit.Assert.*;
 
-public class TestStorageManager {
+public class TestFileStorageManager {
 	private TajoConf conf;
-	private static String TEST_PATH = "target/test-data/TestStorageManager";
+	private static String TEST_PATH = "target/test-data/TestFileStorageManager";
   StorageManager sm = null;
   private Path testDir;
   private FileSystem fs;
@@ -56,7 +57,7 @@
 		conf = new TajoConf();
     testDir = CommonTestingUtil.getTestDir(TEST_PATH);
     fs = testDir.getFileSystem(conf);
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
 	}
 
 	@After
@@ -83,14 +84,14 @@
 
     Path path = StorageUtil.concatPath(testDir, "testGetScannerAndAppender", "table.csv");
     fs.mkdirs(path.getParent());
-		Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, path);
+		Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, path);
     appender.init();
 		for(Tuple t : tuples) {
 		  appender.addTuple(t);
 		}
 		appender.close();
 
-		Scanner scanner = StorageManager.getStorageManager(conf).getFileScanner(meta, schema, path);
+		Scanner scanner = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getFileScanner(meta, schema, path);
     scanner.init();
 		int i=0;
 		while(scanner.next() != null) {
@@ -124,7 +125,7 @@
       }
 
       assertTrue(fs.exists(tablePath));
-      StorageManager sm = StorageManager.getStorageManager(new TajoConf(conf), tablePath);
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(new TajoConf(conf), tablePath);
 
       Schema schema = new Schema();
       schema.addColumn("id", Type.INT4);
@@ -132,19 +133,19 @@
       schema.addColumn("name",Type.TEXT);
       TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
 
-      List<FileFragment> splits = Lists.newArrayList();
+      List<Fragment> splits = Lists.newArrayList();
       // Get FileFragments in partition batch
       splits.addAll(sm.getSplits("data", meta, schema, partitions.toArray(new Path[partitions.size()])));
       assertEquals(testCount, splits.size());
       // -1 is unknown volumeId
-      assertEquals(-1, splits.get(0).getDiskIds()[0]);
+      assertEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]);
 
       splits.clear();
       splits.addAll(sm.getSplits("data", meta, schema,
           partitions.subList(0, partitions.size() / 2).toArray(new Path[partitions.size() / 2])));
       assertEquals(testCount / 2, splits.size());
       assertEquals(1, splits.get(0).getHosts().length);
-      assertEquals(-1, splits.get(0).getDiskIds()[0]);
+      assertEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]);
       fs.close();
     } finally {
       cluster.shutdown();
@@ -176,7 +177,7 @@
         DFSTestUtil.createFile(fs, tmpFile, 10, (short) 2, 0xDEADDEADl);
       }
       assertTrue(fs.exists(tablePath));
-      StorageManager sm = StorageManager.getStorageManager(new TajoConf(conf), tablePath);
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(new TajoConf(conf), tablePath);
 
       Schema schema = new Schema();
       schema.addColumn("id", Type.INT4);
@@ -184,13 +185,13 @@
       schema.addColumn("name", Type.TEXT);
       TableMeta meta = CatalogUtil.newTableMeta(StoreType.CSV);
 
-      List<FileFragment> splits = Lists.newArrayList();
+      List<Fragment> splits = Lists.newArrayList();
       splits.addAll(sm.getSplits("data", meta, schema, tablePath));
 
       assertEquals(testCount, splits.size());
       assertEquals(2, splits.get(0).getHosts().length);
-      assertEquals(2, splits.get(0).getDiskIds().length);
-      assertNotEquals(-1, splits.get(0).getDiskIds()[0]);
+      assertEquals(2, ((FileFragment)splits.get(0)).getDiskIds().length);
+      assertNotEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]);
       fs.close();
     } finally {
       cluster.shutdown();
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFileSystems.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
similarity index 82%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestFileSystems.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
index bec0daf..ff7fe13 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFileSystems.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
@@ -1,13 +1,13 @@
 /**

  * Licensed to the Apache Software Foundation (ASF) under one

- * or more contributor license agreements.??See the NOTICE file

+ * or more contributor license agreements.  See the NOTICE file

  * distributed with this work for additional information

- * regarding copyright ownership.??The ASF licenses this file

+ * 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

+ * with the License.  You may obtain a copy of the License at

  *

- *?????http://www.apache.org/licenses/LICENSE-2.0

+ *     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,

@@ -18,7 +18,6 @@
 

 package org.apache.tajo.storage;

 

-import org.apache.hadoop.conf.Configuration;

 import org.apache.hadoop.fs.FileStatus;

 import org.apache.hadoop.fs.FileSystem;

 import org.apache.hadoop.fs.LocalFileSystem;

@@ -31,7 +30,7 @@
 import org.apache.tajo.conf.TajoConf;

 import org.apache.tajo.datum.Datum;

 import org.apache.tajo.datum.DatumFactory;

-import org.apache.tajo.storage.fragment.FileFragment;

+import org.apache.tajo.storage.fragment.Fragment;

 import org.junit.After;

 import org.junit.Before;

 import org.junit.Test;

@@ -51,21 +50,21 @@
 public class TestFileSystems {

 

   private static String TEST_PATH = "target/test-data/TestFileSystem";

-  private Configuration conf;

-  private StorageManager sm;

+  private TajoConf conf;

+  private FileStorageManager sm;

   private FileSystem fs;

   private Path testDir;

 

   public TestFileSystems(FileSystem fs) throws IOException {

     this.fs = fs;

-    this.conf = fs.getConf();

-    this.testDir = getTestDir(this.fs, TEST_PATH);

-    this.sm = StorageManager.getStorageManager(new TajoConf(this.conf));

+    this.conf = new TajoConf(fs.getConf());

+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);

+    testDir = getTestDir(this.fs, TEST_PATH);

   }

 

   public Path getTestDir(FileSystem fs, String dir) throws IOException {

     Path path = new Path(dir);

-    if (fs.exists(path))

+    if(fs.exists(path))

       fs.delete(path, true);

 

     fs.mkdirs(path);

@@ -127,12 +126,12 @@
     appender.close();

     FileStatus fileStatus = fs.getFileStatus(path);

 

-    List<FileFragment> splits = sm.getSplits("table", meta, schema, path);

+    List<Fragment> splits = sm.getSplits("table", meta, schema, path);

     int splitSize = (int) Math.ceil(fileStatus.getLen() / (double) fileStatus.getBlockSize());

     assertEquals(splitSize, splits.size());

 

-    for (FileFragment fragment : splits) {

-      assertTrue(fragment.getEndKey() <= fileStatus.getBlockSize());

+    for (Fragment fragment : splits) {

+      assertTrue(fragment.getLength() <= fileStatus.getBlockSize());

     }

   }

 }
\ No newline at end of file
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
similarity index 95%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
index 4512d00..d8e359f 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
@@ -64,8 +64,8 @@
 
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.TEXTFILE);
     Path tablePath = new Path(testDir, "line.data");
-    FileAppender appender = (FileAppender) StorageManager.getStorageManager(conf).getAppender(meta, schema,
-        tablePath);
+    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(
+        null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
     int tupleNum = 10000;
@@ -120,8 +120,8 @@
     meta.putOption("compression.codec", DeflateCodec.class.getCanonicalName());
 
     Path tablePath = new Path(testDir, "line1." + DeflateCodec.class.getSimpleName());
-    FileAppender appender = (FileAppender) StorageManager.getStorageManager(conf).getAppender(meta, schema,
-        tablePath);
+    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(
+        null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
     int tupleNum = 10000;
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
similarity index 93%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
index e6714b5..a0daa7d 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
@@ -22,7 +22,6 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
@@ -32,7 +31,9 @@
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
 import org.junit.Before;
 import org.junit.Test;
@@ -94,7 +95,7 @@
     conf.setStrings("tajo.storage.projectable-scanner", "rcfile", "parquet", "avro");
     testDir = CommonTestingUtil.getTestDir(TEST_PATH);
     fs = testDir.getFileSystem(conf);
-    sm = StorageManager.getStorageManager(conf, testDir);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
   }
 
   @Test
@@ -114,7 +115,7 @@
     }
 
     Path table1Path = new Path(testDir, storeType + "_1.data");
-    Appender appender1 = StorageManager.getStorageManager(conf).getAppender(meta, schema, table1Path);
+    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(null, null, meta, schema, table1Path);
     appender1.enableStats();
     appender1.init();
     int tupleNum = 10000;
@@ -136,7 +137,7 @@
     }
 
     Path table2Path = new Path(testDir, storeType + "_2.data");
-    Appender appender2 = StorageManager.getStorageManager(conf).getAppender(meta, schema, table2Path);
+    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(null, null, meta, schema, table2Path);
     appender2.enableStats();
     appender2.init();
 
@@ -158,7 +159,7 @@
 
     FileStatus status1 = fs.getFileStatus(table1Path);
     FileStatus status2 = fs.getFileStatus(table2Path);
-    FileFragment[] fragment = new FileFragment[2];
+    Fragment[] fragment = new Fragment[2];
     fragment[0] = new FileFragment("tablet1", table1Path, 0, status1.getLen());
     fragment[1] = new FileFragment("tablet1", table2Path, 0, status2.getLen());
 
@@ -166,7 +167,7 @@
     targetSchema.addColumn(schema.getColumn(0));
     targetSchema.addColumn(schema.getColumn(2));
 
-    Scanner scanner = new MergeScanner(conf, schema, meta, TUtil.<FileFragment>newList(fragment), targetSchema);
+    Scanner scanner = new MergeScanner(conf, schema, meta, TUtil.newList(fragment), targetSchema);
     assertEquals(isProjectableStorage(meta.getStoreType()), scanner.isProjectable());
 
     scanner.init();
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
similarity index 100%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
similarity index 90%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
index bd1a1f9..15998f2 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -141,7 +141,8 @@
 
       TableMeta meta = CatalogUtil.newTableMeta(storeType);
       Path tablePath = new Path(testDir, "Splitable.data");
-      Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+      Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.enableStats();
       appender.init();
       int tupleNum = 10000;
@@ -165,7 +166,7 @@
       tablets[0] = new FileFragment("Splitable", tablePath, 0, randomNum);
       tablets[1] = new FileFragment("Splitable", tablePath, randomNum, (fileLen - randomNum));
 
-      Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+      Scanner scanner = sm.getScanner(meta, schema, tablets[0], schema);
       assertTrue(scanner.isSplittable());
       scanner.init();
       int tupleCnt = 0;
@@ -174,7 +175,7 @@
       }
       scanner.close();
 
-      scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
+      scanner = sm.getScanner(meta, schema, tablets[1], schema);
       assertTrue(scanner.isSplittable());
       scanner.init();
       while (scanner.next() != null) {
@@ -195,7 +196,8 @@
 
       TableMeta meta = CatalogUtil.newTableMeta(storeType);
       Path tablePath = new Path(testDir, "Splitable.data");
-      Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+      Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.enableStats();
       appender.init();
       int tupleNum = 10000;
@@ -219,7 +221,7 @@
       tablets[0] = new FileFragment("Splitable", tablePath, 0, randomNum);
       tablets[1] = new FileFragment("Splitable", tablePath, randomNum, (fileLen - randomNum));
 
-      Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+      Scanner scanner = sm.getScanner(meta, schema, tablets[0], schema);
       assertTrue(scanner.isSplittable());
       scanner.init();
       int tupleCnt = 0;
@@ -228,7 +230,7 @@
       }
       scanner.close();
 
-      scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
+      scanner = sm.getScanner(meta, schema, tablets[1], schema);
       assertTrue(scanner.isSplittable());
       scanner.init();
       while (scanner.next() != null) {
@@ -255,7 +257,8 @@
     }
 
     Path tablePath = new Path(testDir, "testProjection.data");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
     int tupleNum = 10000;
     VTuple vTuple;
@@ -275,7 +278,7 @@
     Schema target = new Schema();
     target.addColumn("age", Type.INT8);
     target.addColumn("score", Type.FLOAT4);
-    Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment, target);
+    Scanner scanner = sm.getScanner(meta, schema, fragment, target);
     scanner.init();
     int tupleCnt = 0;
     Tuple tuple;
@@ -324,8 +327,9 @@
       meta.putOption(StorageConstants.AVRO_SCHEMA_URL, path);
     }
 
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
 
     QueryId queryid = new QueryId("12345", 5);
@@ -355,7 +359,7 @@
 
     FileStatus status = fs.getFileStatus(tablePath);
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  sm.getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -401,7 +405,8 @@
     }
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
 
     QueryId queryid = new QueryId("12345", 5);
@@ -444,7 +449,7 @@
 
     FileStatus status = fs.getFileStatus(tablePath);
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -488,7 +493,8 @@
     meta.putOption(StorageConstants.CSVFILE_SERDE, TextSerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
 
@@ -519,7 +525,7 @@
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -557,7 +563,8 @@
     meta.putOption(StorageConstants.RCFILE_SERDE, BinarySerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
 
@@ -588,7 +595,7 @@
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -626,7 +633,8 @@
     meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
 
@@ -657,7 +665,7 @@
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     assertTrue(scanner instanceof SequenceFileScanner);
@@ -699,7 +707,8 @@
     meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, BinarySerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
 
@@ -730,7 +739,7 @@
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     assertTrue(scanner instanceof SequenceFileScanner);
@@ -760,7 +769,8 @@
       TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
 
       Path tablePath = new Path(testDir, "testTime.data");
-      Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+      Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.init();
 
       Tuple tuple = new VTuple(3);
@@ -775,7 +785,7 @@
 
       FileStatus status = fs.getFileStatus(tablePath);
       FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-      Scanner scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+      Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
       scanner.init();
 
       Tuple retrieved;
@@ -801,8 +811,8 @@
 
     TableMeta meta = CatalogUtil.newTableMeta(storeType);
     Path tablePath = new Path(testDir, "Seekable.data");
-    FileAppender appender = (FileAppender) StorageManager.getStorageManager(conf).getAppender(meta, schema,
-        tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    FileAppender appender = (FileAppender) sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
     int tupleNum = 100000;
@@ -843,8 +853,8 @@
     long readBytes = 0;
     long readRows = 0;
     for (long offset : offsets) {
-      scanner = StorageManager.getStorageManager(conf).getScanner(meta, schema,
-          new FileFragment("table", tablePath, prevOffset, offset - prevOffset), schema);
+      scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema,
+	        new FileFragment("table", tablePath, prevOffset, offset - prevOffset), schema);
       scanner.init();
 
       while (scanner.next() != null) {
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
similarity index 96%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
index a79e8ab..7b83894 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
@@ -36,9 +36,7 @@
 import java.net.URISyntaxException;
 import java.net.URL;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 /**
  * Tests for {@link org.apache.tajo.storage.avro.AvroUtil}.
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
similarity index 91%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
index 7900195..383740d 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
@@ -89,7 +89,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindValue_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
@@ -124,7 +124,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -147,7 +147,7 @@
     tuple = new VTuple(keySchema.size());
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + storeType + ".idx"), keySchema, comp);
     reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     for (int i = 0; i < TUPLE_NUM - 1; i++) {
@@ -177,8 +177,8 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testBuildIndexWithAppender_" + storeType);
-    FileAppender appender = (FileAppender) StorageManager.getStorageManager(conf).getAppender(meta, schema,
-        tablePath);
+    FileAppender appender = (FileAppender) ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(meta, schema, tablePath);
     appender.init();
 
     SortSpec[] sortKeys = new SortSpec[2];
@@ -226,7 +226,7 @@
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     for (int i = 0; i < TUPLE_NUM - 1; i++) {
@@ -256,7 +256,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = StorageUtil.concatPath(testDir, "testFindOmittedValue_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i += 2) {
@@ -289,7 +289,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -326,7 +326,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyValue_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
@@ -360,7 +360,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -383,7 +383,7 @@
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyValue_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple result;
@@ -416,7 +416,8 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyOmittedValue_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i += 2) {
@@ -450,7 +451,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -473,7 +474,7 @@
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyOmittedValue_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple result;
@@ -495,7 +496,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindMinValue" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
@@ -529,7 +530,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -554,7 +555,7 @@
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     tuple.put(0, DatumFactory.createInt8(0));
@@ -578,7 +579,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testMinMax_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 5; i < TUPLE_NUM; i += 2) {
@@ -612,7 +613,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -682,7 +683,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testConcurrentAccess_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
@@ -717,7 +718,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -762,7 +763,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindValueDescOrder_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
@@ -798,7 +799,7 @@
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -823,7 +824,7 @@
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     for (int i = (TUPLE_NUM - 1); i > 0; i--) {
@@ -853,7 +854,7 @@
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyValueDescOrder_" + storeType);
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
similarity index 96%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
index c0dda1f..d7c9f49 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
@@ -78,7 +78,7 @@
     Path tablePath = StorageUtil.concatPath(testDir, "testFindValueInSingleCSV", "table.csv");
     fs.mkdirs(tablePath.getParent());
 
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
@@ -167,7 +167,7 @@
     Path tablePath = StorageUtil.concatPath(testDir, "testFindNextKeyValueInSingleCSV",
         "table1.csv");
     fs.mkdirs(tablePath.getParent());
-    Appender appender = StorageManager.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
similarity index 97%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
index 038bc17..70282d9 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
@@ -73,7 +73,7 @@
     FileSystem fs = FileSystem.getLocal(conf);
     FileStatus status = fs.getFileStatus(tablePath);
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple tuple = scanner.next();
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
similarity index 95%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
index 0a01dc4..109fed9 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
@@ -19,12 +19,6 @@
 package org.apache.tajo.storage.parquet;
 
 import com.google.common.base.Charsets;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,13 +29,14 @@
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
-
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotNull;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.*;
 
 public class TestReadWrite {
   private static final String HELLO = "hello";
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
similarity index 99%
rename from tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
rename to tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
index 49a162b..517e00e 100644
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
@@ -18,18 +18,16 @@
 
 package org.apache.tajo.storage.parquet;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.common.TajoDataTypes.Type;
-
 import org.junit.Test;
-
 import parquet.schema.MessageType;
 import parquet.schema.MessageTypeParser;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import static org.junit.Assert.assertEquals;
 
 /**
diff --git a/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
similarity index 100%
rename from tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
rename to tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json
diff --git a/tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
similarity index 100%
rename from tajo-storage/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
rename to tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance2.json
diff --git a/tajo-storage/src/test/resources/dataset/TestJsonSerDe/testVariousType.json b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
similarity index 100%
rename from tajo-storage/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
rename to tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json
diff --git a/tajo-storage/src/test/resources/dataset/testLineText.txt b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testLineText.txt
similarity index 100%
rename from tajo-storage/src/test/resources/dataset/testLineText.txt
rename to tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testLineText.txt
diff --git a/tajo-storage/src/test/resources/dataset/testVariousTypes.avsc b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc
similarity index 100%
rename from tajo-storage/src/test/resources/dataset/testVariousTypes.avsc
rename to tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc
diff --git a/tajo-storage/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
similarity index 94%
rename from tajo-storage/src/test/resources/storage-default.xml
rename to tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
index f4c81c7..737284b 100644
--- a/tajo-storage/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
@@ -25,6 +25,16 @@
     <value>org.apache.tajo.storage.s3.SmallBlockS3FileSystem</value>
   </property>
 
+  <!-- Storage Manager Configuration -->
+  <property>
+    <name>tajo.storage.manager.hdfs.class</name>
+    <value>org.apache.tajo.storage.FileStorageManager</value>
+  </property>
+  <property>
+    <name>tajo.storage.manager.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
+  </property>
+
   <!--- Registered Scanner Handler -->
   <property>
     <name>tajo.storage.scanner-handler</name>
diff --git a/tajo-yarn-pullserver/pom.xml b/tajo-yarn-pullserver/pom.xml
index a7644a1..3daec5c 100644
--- a/tajo-yarn-pullserver/pom.xml
+++ b/tajo-yarn-pullserver/pom.xml
@@ -57,7 +57,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>