You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2015/06/25 10:08:53 UTC

[01/15] tajo git commit: Revert "TAJO-1408 Make IntermediateEntryProto more compact. (Contributed by navis, Committed by hyunsik)"

Repository: tajo
Updated Branches:
  refs/heads/index_support 00a8c658e -> 17dfe86cc


Revert "TAJO-1408 Make IntermediateEntryProto more compact. (Contributed by navis, Committed by hyunsik)"

This reverts commit fab63900cf44b61d571fb9c2982285bb8b669702.

Conflicts:
	CHANGES


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

Branch: refs/heads/index_support
Commit: 7c8477dd3ac15188353044981746ecfef87f2f02
Parents: b24d18f
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Jun 24 03:05:45 2015 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Jun 24 03:05:45 2015 -0700

----------------------------------------------------------------------
 CHANGES                                         |  3 -
 .../java/org/apache/tajo/util/NumberUtil.java   | 54 ------------
 .../apache/tajo/querymaster/Repartitioner.java  | 18 ++--
 .../java/org/apache/tajo/querymaster/Task.java  | 56 +++++++------
 .../tajo/worker/ExecutionBlockContext.java      | 31 +++++--
 .../src/main/proto/TajoWorkerProtocol.proto     | 16 +++-
 .../apache/tajo/master/TestRepartitioner.java   | 77 ++++++++++-------
 .../tajo/querymaster/TestIntermediateEntry.java | 24 +++---
 .../tajo/storage/HashShuffleAppender.java       | 87 ++++++--------------
 .../storage/HashShuffleAppenderManager.java     | 12 +--
 10 files changed, 174 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 170b928..1f69ac0 100644
--- a/CHANGES
+++ b/CHANGES
@@ -72,9 +72,6 @@ Release 0.11.0 - unreleased
     eliminates an important kind of information. 
     (Contributed by Jongyoung Park, Committed by jihoon)
 
-    TAJO-1408: Make IntermediateEntryProto more compact. 
-    (Contributed by navis, Committed by hyunsik)
-
     TAJO-1584: Remove QueryMaster client sharing in TajoMaster and TajoWorker.
     (jinho)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java
index d14e0b4..0d70cc2 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/NumberUtil.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.util;
 
-import com.google.common.primitives.Longs;
 import io.netty.buffer.ByteBuf;
 import io.netty.util.internal.PlatformDependent;
 
@@ -26,8 +25,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.lang.reflect.Constructor;
 import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Iterator;
 
 // this is an implementation copied from LazyPrimitives in hive
 public class NumberUtil {
@@ -1053,55 +1050,4 @@ public class NumberUtil {
 
     return returnNumber;
   }
-
-  public static long mergeToLong(int value1, int value2) {
-    return (long)value1 << 32 | value2 & 0xffffffffl;
-  }
-
-  public static int toHighInt(long value) {
-    return (int)(value >> 32);
-  }
-
-  public static int toLowInt(long value) {
-    return (int)value;
-  }
-
-  public static class PrimitiveLongs implements Iterable<Long> {
-    int index;
-    long[] longArray;
-
-    public PrimitiveLongs(int initLength) {
-      longArray = new long[initLength];
-    }
-    public void add(long value) {
-      reserve(1)[index++] = value;
-    }
-    public void add(long[] value) {
-      System.arraycopy(value, 0, reserve(value.length), index, value.length);
-      index += value.length;
-    }
-    public long[] backingArray() {
-      return longArray;
-    }
-    public long[] toArray() {
-      return Arrays.copyOfRange(longArray, 0, index);
-    }
-    public int size() {
-      return index;
-    }
-    private long[] reserve(int reserve) {
-      if (index + reserve < longArray.length) {
-        return longArray;
-      }
-      int newLength = Math.max(index + reserve, longArray.length << 1);
-      long[] newLongArray = new long[newLength];
-      System.arraycopy(longArray, 0, newLongArray, 0, index);
-      return longArray = newLongArray;
-    }
-
-    @Override
-    public Iterator<Long> iterator() {
-      return Longs.asList(toArray()).iterator();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
index ec09145..2c3e9e2 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
@@ -1044,17 +1044,15 @@ public class Repartitioner {
         firstSplitVolume = splitVolume;
       }
 
-      //Each Pair object in the splits variable is assigned to the next ExecutionBlock's task.
+      //Each Pair object in the splits variable is assigned to the next ExectionBlock's task.
       //The first long value is a offset of the intermediate file and the second long value is length.
-      long[] splits = currentInterm.split(firstSplitVolume, splitVolume);
-      if (splits == null || splits.length == 0) {
+      List<Pair<Long, Long>> splits = currentInterm.split(firstSplitVolume, splitVolume);
+      if (splits == null || splits.isEmpty()) {
         break;
       }
 
-      for (int i = 0; i < splits.length; i += 2) {
-        long offset = splits[i];
-        long length = splits[i + 1];
-        if (fetchListVolume > 0 && fetchListVolume + length >= splitVolume) {
+      for (Pair<Long, Long> eachSplit: splits) {
+        if (fetchListVolume > 0 && fetchListVolume + eachSplit.getSecond() >= splitVolume) {
           if (!fetchListForSingleTask.isEmpty()) {
             fetches.add(fetchListForSingleTask);
           }
@@ -1063,10 +1061,10 @@ public class Repartitioner {
         }
         FetchImpl fetch = new FetchImpl(currentInterm.getPullHost(), SCATTERED_HASH_SHUFFLE,
             ebId, currentInterm.getPartId(), TUtil.newList(currentInterm));
-        fetch.setOffset(offset);
-        fetch.setLength(length);
+        fetch.setOffset(eachSplit.getFirst());
+        fetch.setLength(eachSplit.getSecond());
         fetchListForSingleTask.add(fetch);
-        fetchListVolume += length;
+        fetchListVolume += eachSplit.getSecond();
       }
     }
     if (!fetchListForSingleTask.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
index d2be973..1da623e 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
@@ -23,7 +23,6 @@ import com.google.common.base.Objects;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Longs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -35,6 +34,7 @@ import org.apache.tajo.TajoProtos.TaskAttemptState;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.TaskId;
 import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.ipc.TajoWorkerProtocol.FailureIntermediateProto;
 import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto;
 import org.apache.tajo.master.TaskState;
 import org.apache.tajo.master.event.*;
@@ -44,7 +44,6 @@ 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.NumberUtil;
 import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TajoIdUtils;
 import org.apache.tajo.util.history.TaskHistory;
@@ -786,8 +785,8 @@ public class Task implements EventHandler<TaskEvent> {
     int partId;
     PullHost host;
     long volume;
-    long[] pages;
-    long[] failureRowNums;
+    List<Pair<Long, Integer>> pages;
+    List<Pair<Long, Pair<Integer, Integer>>> failureRowNums;
 
     public IntermediateEntry(IntermediateEntryProto proto) {
       this.ebId = new ExecutionBlockId(proto.getEbId());
@@ -795,12 +794,21 @@ public class Task implements EventHandler<TaskEvent> {
       this.attemptId = proto.getAttemptId();
       this.partId = proto.getPartId();
 
-      String [] pullHost = proto.getAddress().split(":");
+      String[] pullHost = proto.getHost().split(":");
       this.host = new PullHost(pullHost[0], Integer.parseInt(pullHost[1]));
       this.volume = proto.getVolume();
 
-      this.failureRowNums = Longs.toArray(proto.getFailuresList());
-      this.pages = Longs.toArray(proto.getPagesList());
+      failureRowNums = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
+      for (FailureIntermediateProto eachFailure: proto.getFailuresList()) {
+
+        failureRowNums.add(new Pair(eachFailure.getPagePos(),
+            new Pair(eachFailure.getStartRowNum(), eachFailure.getEndRowNum())));
+      }
+
+      pages = new ArrayList<Pair<Long, Integer>>();
+      for (IntermediateEntryProto.PageProto eachPage: proto.getPagesList()) {
+        pages.add(new Pair(eachPage.getPos(), eachPage.getLength()));
+      }
     }
 
     public IntermediateEntry(int taskId, int attemptId, int partId, PullHost host) {
@@ -850,15 +858,15 @@ public class Task implements EventHandler<TaskEvent> {
       return this.volume = volume;
     }
 
-    public long[] getPages() {
+    public List<Pair<Long, Integer>> getPages() {
       return pages;
     }
 
-    public void setPages(long[] pages) {
+    public void setPages(List<Pair<Long, Integer>> pages) {
       this.pages = pages;
     }
 
-    public long[] getFailureRowNums() {
+    public List<Pair<Long, Pair<Integer, Integer>>> getFailureRowNums() {
       return failureRowNums;
     }
 
@@ -867,38 +875,38 @@ public class Task implements EventHandler<TaskEvent> {
       return Objects.hashCode(ebId, taskId, partId, attemptId, host);
     }
 
-    public long[] split(long firstSplitVolume, long splitVolume) {
+    public List<Pair<Long, Long>> split(long firstSplitVolume, long splitVolume) {
+      List<Pair<Long, Long>> splits = new ArrayList<Pair<Long, Long>>();
 
-      if (pages == null || pages.length == 0) {
-        return null;
+      if (pages == null || pages.isEmpty()) {
+        return splits;
       }
+      int pageSize = pages.size();
 
-      NumberUtil.PrimitiveLongs splits = new NumberUtil.PrimitiveLongs(100);
       long currentOffset = -1;
       long currentBytes = 0;
 
       long realSplitVolume = firstSplitVolume > 0 ? firstSplitVolume : splitVolume;
-      for (int i = 0; i < pages.length; i += 2) {
+      for (int i = 0; i < pageSize; i++) {
+        Pair<Long, Integer> eachPage = pages.get(i);
         if (currentOffset == -1) {
-          currentOffset = pages[i];
+          currentOffset = eachPage.getFirst();
         }
-        if (currentBytes > 0 && currentBytes + pages[i + 1] >= realSplitVolume) {
-          splits.add(currentOffset);
-          splits.add(currentBytes);
-          currentOffset = pages[i];
+        if (currentBytes > 0 && currentBytes + eachPage.getSecond() >= realSplitVolume) {
+          splits.add(new Pair(currentOffset, currentBytes));
+          currentOffset = eachPage.getFirst();
           currentBytes = 0;
           realSplitVolume = splitVolume;
         }
 
-        currentBytes += pages[i + 1];
+        currentBytes += eachPage.getSecond();
       }
 
       //add last
       if (currentBytes > 0) {
-        splits.add(currentOffset);
-        splits.add(currentBytes);
+        splits.add(new Pair(currentOffset, currentBytes));
       }
-      return splits.toArray();
+      return splits;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
index 9e4a60f..cbd451d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java
@@ -41,6 +41,7 @@ import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
 import org.apache.tajo.storage.HashShuffleAppenderManager;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.Pair;
 
 import java.io.IOException;
 import java.net.ConnectException;
@@ -339,18 +340,38 @@ public class ExecutionBlockContext {
       }
 
       IntermediateEntryProto.Builder intermediateBuilder = IntermediateEntryProto.newBuilder();
+      IntermediateEntryProto.PageProto.Builder pageBuilder = IntermediateEntryProto.PageProto.newBuilder();
+      FailureIntermediateProto.Builder failureBuilder = FailureIntermediateProto.newBuilder();
 
-      WorkerConnectionInfo connectionInfo = getWorkerContext().getConnectionInfo();
-      String address = connectionInfo.getHost() + ":" + connectionInfo.getPullServerPort();
       for (HashShuffleAppenderManager.HashShuffleIntermediate eachShuffle: shuffles) {
+        List<IntermediateEntryProto.PageProto> pages = Lists.newArrayList();
+        List<FailureIntermediateProto> failureIntermediateItems = Lists.newArrayList();
+
+        for (Pair<Long, Integer> eachPage: eachShuffle.getPages()) {
+          pageBuilder.clear();
+          pageBuilder.setPos(eachPage.getFirst());
+          pageBuilder.setLength(eachPage.getSecond());
+          pages.add(pageBuilder.build());
+        }
+
+        for(Pair<Long, Pair<Integer, Integer>> eachFailure: eachShuffle.getFailureTskTupleIndexes()) {
+          failureBuilder.clear();
+          failureBuilder.setPagePos(eachFailure.getFirst());
+          failureBuilder.setStartRowNum(eachFailure.getSecond().getFirst());
+          failureBuilder.setEndRowNum(eachFailure.getSecond().getSecond());
+          failureIntermediateItems.add(failureBuilder.build());
+        }
+        intermediateBuilder.clear();
+
         intermediateBuilder.setEbId(ebId.getProto())
-            .setAddress(address)
+            .setHost(getWorkerContext().getConnectionInfo().getHost() + ":" +
+                getWorkerContext().getConnectionInfo().getPullServerPort())
             .setTaskId(-1)
             .setAttemptId(-1)
             .setPartId(eachShuffle.getPartId())
             .setVolume(eachShuffle.getVolume())
-            .addAllPages(eachShuffle.getPages())
-            .addAllFailures(eachShuffle.getFailureTskTupleIndexes());
+            .addAllPages(pages)
+            .addAllFailures(failureIntermediateItems);
         intermediateEntries.add(intermediateBuilder.build());
       }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-core/src/main/proto/TajoWorkerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto
index 715b1e6..7cc4171 100644
--- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto
+++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto
@@ -97,15 +97,25 @@ message FetchProto {
     optional int64 length = 12;
 }
 
+message FailureIntermediateProto {
+    required int64 pagePos = 1;
+    required int32 startRowNum = 2;
+    required int32 endRowNum = 3;
+}
+
 message IntermediateEntryProto {
+    message PageProto {
+        required int64 pos = 1;
+        required int32 length = 2;
+    }
     required ExecutionBlockIdProto ebId = 1;
     required int32 taskId = 2;
     required int32 attemptId = 3;
     required int32 partId = 4;
-    required string address = 5;
+    required string host = 5;
     required int64 volume = 6;
-    repeated int64 pages = 7;       // pos : length
-    repeated int64 failures = 8;    // pagePos : startRowNum:endRowNum
+    repeated PageProto pages = 7;
+    repeated FailureIntermediateProto failures = 8;
 }
 
 message ExecutionBlockReport {

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
index 17706f4..9910d79 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
@@ -28,7 +28,6 @@ import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.querymaster.Task;
 import org.apache.tajo.querymaster.Task.IntermediateEntry;
 import org.apache.tajo.querymaster.Repartitioner;
-import org.apache.tajo.util.NumberUtil;
 import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.worker.FetchImpl;
@@ -177,7 +176,20 @@ public class TestRepartitioner {
     List<IntermediateEntry> intermediateEntries = new ArrayList<IntermediateEntry>();
 
     int[] pageLengths = {10 * 1024 * 1024, 10 * 1024 * 1024, 10 * 1024 * 1024, 5 * 1024 * 1024};   //35 MB
-    long expectedTotalLength = makeIntermediates(pageLengths, true, intermediateEntries);
+    long expectedTotalLength = 0;
+    for (int i = 0; i < 20; i++) {
+      List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+      long offset = 0;
+      for (int j = 0; j < pageLengths.length; j++) {
+        pages.add(new Pair(offset, pageLengths[j]));
+        offset += pageLengths[j];
+        expectedTotalLength += pageLengths[j];
+      }
+      IntermediateEntry interm = new IntermediateEntry(i, -1, -1, new Task.PullHost("" + i, i));
+      interm.setPages(pages);
+      interm.setVolume(offset);
+      intermediateEntries.add(interm);
+    }
 
     long splitVolume = 128 * 1024 * 1024;
     List<List<FetchImpl>> fetches = Repartitioner.splitOrMergeIntermediates(null, intermediateEntries,
@@ -209,27 +221,6 @@ public class TestRepartitioner {
     assertEquals(expectedTotalLength, totalLength);
   }
 
-  private long makeIntermediates(int[] pageLengths, boolean uniqueHosts,
-                                 List<IntermediateEntry> intermediateEntries) {
-    long expectedTotalLength = 0;
-    for (int i = 0; i < 20; i++) {
-      NumberUtil.PrimitiveLongs pages = new NumberUtil.PrimitiveLongs(10);
-      long offset = 0;
-      for (int pageLength : pageLengths) {
-        pages.add(offset);
-        pages.add(pageLength);
-        offset += pageLength;
-        expectedTotalLength += pageLength;
-      }
-      IntermediateEntry interm = new IntermediateEntry(i, -1, -1,
-          new Task.PullHost(uniqueHosts ? "" + i : "", uniqueHosts ? i : 0));
-      interm.setPages(pages.toArray());
-      interm.setVolume(offset);
-      intermediateEntries.add(interm);
-    }
-    return expectedTotalLength;
-  }
-
   @Test
   public void testSplitIntermediates() {
     List<IntermediateEntry> intermediateEntries = new ArrayList<IntermediateEntry>();
@@ -243,7 +234,20 @@ public class TestRepartitioner {
       }
     }
 
-    long expectedTotalLength = makeIntermediates(pageLengths, true, intermediateEntries);
+    long expectedTotalLength = 0;
+    for (int i = 0; i < 20; i++) {
+      List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+      long offset = 0;
+      for (int j = 0; j < pageLengths.length; j++) {
+        pages.add(new Pair(offset, pageLengths[j]));
+        offset += pageLengths[j];
+        expectedTotalLength += pageLengths[j];
+      }
+      IntermediateEntry interm = new IntermediateEntry(i, -1, 0, new Task.PullHost("" + i, i));
+      interm.setPages(pages);
+      interm.setVolume(offset);
+      intermediateEntries.add(interm);
+    }
 
     long splitVolume = 128 * 1024 * 1024;
     List<List<FetchImpl>> fetches = Repartitioner.splitOrMergeIntermediates(null, intermediateEntries,
@@ -364,12 +368,12 @@ public class TestRepartitioner {
 
     List<IntermediateEntry> entries = new ArrayList<IntermediateEntry>();
     for (int i = 0; i < 2; i++) {
-      NumberUtil.PrimitiveLongs pages = new NumberUtil.PrimitiveLongs(10);
-      for (long[] pageData : pageDatas) {
-        pages.add(pageData);
+      List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+      for (int j = 0; j < pageDatas.length; j++) {
+        pages.add(new Pair(pageDatas[j][0], (int) (pageDatas[j][1])));
       }
       IntermediateEntry entry = new IntermediateEntry(-1, -1, 1, new Task.PullHost("host" + i , 9000));
-      entry.setPages(pages.toArray());
+      entry.setPages(pages);
 
       entries.add(entry);
     }
@@ -417,7 +421,22 @@ public class TestRepartitioner {
       }
     }
 
-    long expectedTotalLength = makeIntermediates(pageLengths, false, intermediateEntries);
+    long expectedTotalLength = 0;
+    Task.PullHost pullHost = new Task.PullHost("host", 0);
+
+    for (int i = 0; i < 20; i++) {
+      List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+      long offset = 0;
+      for (int j = 0; j < pageLengths.length; j++) {
+        pages.add(new Pair(offset, pageLengths[j]));
+        offset += pageLengths[j];
+        expectedTotalLength += pageLengths[j];
+      }
+      IntermediateEntry interm = new IntermediateEntry(i, -1, 0, pullHost);
+      interm.setPages(pages);
+      interm.setVolume(offset);
+      intermediateEntries.add(interm);
+    }
 
     long splitVolume = 128 * 1024 * 1024;
     List<List<FetchImpl>> fetches = Repartitioner.splitOrMergeIntermediates(null, intermediateEntries,

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java
index b81085b..237fb32 100644
--- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java
@@ -18,9 +18,12 @@
 
 package org.apache.tajo.querymaster;
 
-import org.apache.tajo.util.NumberUtil;
+import org.apache.tajo.util.Pair;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import static org.junit.Assert.assertEquals;
 
 public class TestIntermediateEntry {
@@ -28,22 +31,23 @@ public class TestIntermediateEntry {
   public void testPage() {
     Task.IntermediateEntry interm = new Task.IntermediateEntry(-1, -1, 1, null);
 
-    NumberUtil.PrimitiveLongs pages = new NumberUtil.PrimitiveLongs(10);
-    pages.add(new long[]{0L, 1441275});
-    pages.add(new long[]{1441275L, 1447446});
-    pages.add(new long[]{2888721L, 1442507});
+    List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+    pages.add(new Pair(0L, 1441275));
+    pages.add(new Pair(1441275L, 1447446));
+    pages.add(new Pair(2888721L, 1442507));
 
-    interm.setPages(pages.toArray());
+    interm.setPages(pages);
 
     long splitBytes = 3 * 1024 * 1024;
 
-    long[] splits = interm.split(splitBytes, splitBytes);
-    assertEquals(2 << 1, splits.length);
+    List<Pair<Long, Long>> splits = interm.split(splitBytes, splitBytes);
+    assertEquals(2, splits.size());
 
     long[][] expected = { {0, 1441275 + 1447446}, {1441275 + 1447446, 1442507} };
     for (int i = 0; i < 2; i++) {
-      assertEquals(expected[i][0], splits[i << 1]);
-      assertEquals(expected[i][1], splits[(i << 1) + 1]);
+      Pair<Long, Long> eachSplit = splits.get(i);
+      assertEquals(expected[i][0], eachSplit.getFirst().longValue());
+      assertEquals(expected[i][1], eachSplit.getSecond().longValue());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
index ccf5dae..4c772c9 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
@@ -18,21 +18,16 @@
 
 package org.apache.tajo.storage;
 
-import com.google.common.primitives.Longs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.util.NumberUtil;
-import org.apache.tajo.util.NumberUtil.PrimitiveLongs;
+import org.apache.tajo.util.Pair;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -47,12 +42,12 @@ public class HashShuffleAppender implements Appender {
   private TableStats tableStats;
 
   //<taskId,<page start offset,<task start, task end>>>
-  private Map<TaskAttemptId, PrimitiveLongs> taskTupleIndexes;
+  private Map<TaskAttemptId, List<Pair<Long, Pair<Integer, Integer>>>> taskTupleIndexes;
 
   //page start offset, length
-  private PrimitiveLongs pages = new PrimitiveLongs(100);
+  private List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
 
-  private long[] currentPage;
+  private Pair<Long, Integer> currentPage;
 
   private int pageSize; //MB
 
@@ -73,8 +68,8 @@ public class HashShuffleAppender implements Appender {
 
   @Override
   public void init() throws IOException {
-    currentPage = new long[2];
-    taskTupleIndexes = new HashMap<TaskAttemptId, PrimitiveLongs>();
+    currentPage = new Pair(0L, 0);
+    taskTupleIndexes = new HashMap<TaskAttemptId, List<Pair<Long, Pair<Integer, Integer>>>>();
     rowNumInPage = 0;
   }
 
@@ -101,16 +96,16 @@ public class HashShuffleAppender implements Appender {
       int writtenBytes = (int)(posAfterWritten - currentPos);
 
       int nextRowNum = rowNumInPage + tuples.size();
-      PrimitiveLongs taskIndexes = taskTupleIndexes.get(taskId);
+      List<Pair<Long, Pair<Integer, Integer>>> taskIndexes = taskTupleIndexes.get(taskId);
       if (taskIndexes == null) {
-        taskIndexes = new PrimitiveLongs(100);
+        taskIndexes = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
         taskTupleIndexes.put(taskId, taskIndexes);
       }
-      taskIndexes.add(currentPage[0]);
-      taskIndexes.add(NumberUtil.mergeToLong(rowNumInPage, nextRowNum));
+      taskIndexes.add(
+          new Pair<Long, Pair<Integer, Integer>>(currentPage.getFirst(), new Pair(rowNumInPage, nextRowNum)));
       rowNumInPage = nextRowNum;
 
-      if (posAfterWritten - currentPage[0] > pageSize) {
+      if (posAfterWritten - currentPage.getFirst() > pageSize) {
         nextPage(posAfterWritten);
         rowNumInPage = 0;
       }
@@ -129,9 +124,9 @@ public class HashShuffleAppender implements Appender {
   }
 
   private void nextPage(long pos) {
-    currentPage[1] = pos - currentPage[0];
+    currentPage.setSecond((int) (pos - currentPage.getFirst()));
     pages.add(currentPage);
-    currentPage = new long[] {pos, 0};
+    currentPage = new Pair(pos, 0);
   }
 
   @Override
@@ -162,18 +157,16 @@ public class HashShuffleAppender implements Appender {
       }
       appender.flush();
       offset = appender.getOffset();
-      if (offset > currentPage[0]) {
+      if (offset > currentPage.getFirst()) {
         nextPage(offset);
       }
       appender.close();
       if (LOG.isDebugEnabled()) {
-        int size = pages.size();
-        if (size > 0) {
-          long[] array = pages.backingArray();
-          LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + size
-              + ", lastPage=" + array[size - 2] + ", " + array[size - 1]);
+        if (!pages.isEmpty()) {
+          LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size()
+              + ", lastPage=" + pages.get(pages.size() - 1));
         } else {
-          LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + size);
+          LOG.info(ebId + ",partId=" + partId + " Appender closed: fileLen=" + offset + ", pages=" + pages.size());
         }
       }
       closed.set(true);
@@ -192,48 +185,22 @@ public class HashShuffleAppender implements Appender {
     }
   }
 
-  public PrimitiveLongs getPages() {
+  public List<Pair<Long, Integer>> getPages() {
     return pages;
   }
 
-  public Map<TaskAttemptId, PrimitiveLongs> getTaskTupleIndexes() {
+  public Map<TaskAttemptId, List<Pair<Long, Pair<Integer, Integer>>>> getTaskTupleIndexes() {
     return taskTupleIndexes;
   }
 
-  public Iterable<Long> getMergedTupleIndexes() {
-    return getIterable(taskTupleIndexes.values());
-  }
+  public List<Pair<Long, Pair<Integer, Integer>>> getMergedTupleIndexes() {
+    List<Pair<Long, Pair<Integer, Integer>>> merged = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
 
-  public Iterable<Long> getIterable(final Collection<PrimitiveLongs> values) {
-    return new Iterable<Long>() {
-      @Override
-      public Iterator<Long> iterator() {
-        final Iterator<PrimitiveLongs> iterator1 = values.iterator();
-        return new Iterator<Long>() {
-          Iterator<Long> iterator2 = null;
-          @Override
-          public boolean hasNext() {
-            while (iterator2 == null || !iterator2.hasNext()) {
-              if (!iterator1.hasNext()) {
-                return false;
-              }
-              iterator2 = iterator1.next().iterator();
-            }
-            return true;
-          }
-
-          @Override
-          public Long next() {
-            return iterator2.next();
-          }
-
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException();
-          }
-        };
-      }
-    };
+    for (List<Pair<Long, Pair<Integer, Integer>>> eachFailureIndex: taskTupleIndexes.values()) {
+      merged.addAll(eachFailureIndex);
+    }
+
+    return merged;
   }
 
   public void taskFinished(TaskAttemptId taskId) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/7c8477dd/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
index 66c7f13..1b48fc0 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
@@ -174,14 +174,14 @@ public class HashShuffleAppenderManager {
     private long volume;
 
     //[<page start offset,<task start, task end>>]
-    private Iterable<Long> failureTskTupleIndexes;
+    private Collection<Pair<Long, Pair<Integer, Integer>>> failureTskTupleIndexes;
 
     //[<page start offset, length>]
-    private Iterable<Long> pages;
+    private List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
 
     public HashShuffleIntermediate(int partId, long volume,
-                                   Iterable<Long> pages,
-                                   Iterable<Long> failureTskTupleIndexes) {
+                                   List<Pair<Long, Integer>> pages,
+                                   Collection<Pair<Long, Pair<Integer, Integer>>> failureTskTupleIndexes) {
       this.partId = partId;
       this.volume = volume;
       this.failureTskTupleIndexes = failureTskTupleIndexes;
@@ -196,11 +196,11 @@ public class HashShuffleAppenderManager {
       return volume;
     }
 
-    public Iterable<Long> getFailureTskTupleIndexes() {
+    public Collection<Pair<Long, Pair<Integer, Integer>>> getFailureTskTupleIndexes() {
       return failureTskTupleIndexes;
     }
 
-    public Iterable<Long> getPages() {
+    public List<Pair<Long, Integer>> getPages() {
       return pages;
     }
   }


[02/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
index 80d0fff..ce2a926 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -155,7 +155,7 @@ public class TestStorages {
 
       TableMeta meta = CatalogUtil.newTableMeta(storeType);
       Path tablePath = new Path(testDir, "Splitable.data");
-      FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+      FileTablespace sm = TableSpaceManager.getLocalFs();
       Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.enableStats();
       appender.init();
@@ -210,7 +210,7 @@ public class TestStorages {
 
       TableMeta meta = CatalogUtil.newTableMeta(storeType);
       Path tablePath = new Path(testDir, "Splitable.data");
-      FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+      FileTablespace sm = TableSpaceManager.getLocalFs();
       Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.enableStats();
       appender.init();
@@ -271,7 +271,7 @@ public class TestStorages {
     }
 
     Path tablePath = new Path(testDir, "testProjection.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
     int tupleNum = 10000;
@@ -347,7 +347,7 @@ public class TestStorages {
       meta.putOption(StorageConstants.AVRO_SCHEMA_URL, path);
     }
 
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Path tablePath = new Path(testDir, "testVariousTypes.data");
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
@@ -425,7 +425,7 @@ public class TestStorages {
     }
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
 
@@ -469,7 +469,7 @@ public class TestStorages {
 
     FileStatus status = fs.getFileStatus(tablePath);
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -513,7 +513,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.CSVFILE_SERDE, TextSerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -544,7 +544,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -582,7 +582,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.RCFILE_SERDE, BinarySerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -613,7 +613,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -651,7 +651,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -682,7 +682,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     assertTrue(scanner instanceof SequenceFileScanner);
@@ -724,7 +724,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, BinarySerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -756,7 +756,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     assertTrue(scanner instanceof SequenceFileScanner);
@@ -786,7 +786,7 @@ public class TestStorages {
       TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
 
       Path tablePath = new Path(testDir, "testTime.data");
-      FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+      FileTablespace sm = TableSpaceManager.getLocalFs();
       Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.init();
 
@@ -801,7 +801,7 @@ public class TestStorages {
 
       FileStatus status = fs.getFileStatus(tablePath);
       FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-      Scanner scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+      Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
       scanner.init();
 
       Tuple retrieved;
@@ -827,7 +827,7 @@ public class TestStorages {
 
     TableMeta meta = CatalogUtil.newTableMeta(storeType);
     Path tablePath = new Path(testDir, "Seekable.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     FileAppender appender = (FileAppender) sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -869,7 +869,7 @@ public class TestStorages {
     long readBytes = 0;
     long readRows = 0;
     for (long offset : offsets) {
-      scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema,
+      scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema,
 	        new FileFragment("table", tablePath, prevOffset, offset - prevOffset), schema);
       scanner.init();
 
@@ -909,7 +909,7 @@ public class TestStorages {
     }
 
     if (storeType.equalsIgnoreCase("RAW")) {
-      TableSpaceManager.clearCache();
+      OldStorageManager.clearCache();
       /* TAJO-1250 reproduce BufferOverflow of RAWFile */
       int headerSize = 4 + 2 + 1; //Integer record length + Short null-flag length + 1 byte null flags
       /* max varint32: 5 bytes, max varint64: 10 bytes */
@@ -917,7 +917,7 @@ public class TestStorages {
       conf.setInt(RawFile.WRITE_BUFFER_SIZE, record + headerSize);
     }
 
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Path tablePath = new Path(testDir, "testMaxValue.data");
     Appender appender = sm.getAppender(meta, schema, tablePath);
 
@@ -950,7 +950,7 @@ public class TestStorages {
 
 
     if (storeType.equalsIgnoreCase("RAW")){
-      TableSpaceManager.clearCache();
+      OldStorageManager.clearCache();
     }
   }
 
@@ -972,7 +972,7 @@ public class TestStorages {
     meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
 
     Path tablePath = new Path(testDir, "testLessThanSchemaSize.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, dataSchema, tablePath);
     appender.init();
 
@@ -998,7 +998,7 @@ public class TestStorages {
     inSchema.addColumn("col5", Type.INT8);
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, inSchema, fragment);
+    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, inSchema, fragment);
 
     Schema target = new Schema();
 
@@ -1036,7 +1036,7 @@ public class TestStorages {
     meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
 
     Path tablePath = new Path(testDir, "test_storetype_oversize.data");
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, dataSchema, tablePath);
     appender.init();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/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
index bcfca0f..1a62f52 100644
--- a/tajo-storage/tajo-storage-hdfs/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,8 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindValue_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+        .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
@@ -124,7 +125,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -147,7 +148,7 @@ public class TestBSTIndex {
     tuple = new VTuple(keySchema.size());
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + storeType + ".idx"), keySchema, comp);
     reader.open();
-    scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     for (int i = 0; i < TUPLE_NUM - 1; i++) {
@@ -177,7 +178,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testBuildIndexWithAppender_" + storeType);
-    FileAppender appender = (FileAppender) ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    FileAppender appender = (FileAppender) ((FileTablespace)TableSpaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
 
@@ -226,7 +227,7 @@ public class TestBSTIndex {
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     for (int i = 0; i < TUPLE_NUM - 1; i++) {
@@ -256,7 +257,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = StorageUtil.concatPath(testDir, "testFindOmittedValue_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i += 2) {
@@ -289,7 +290,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -326,7 +327,8 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyValue_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+        .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
@@ -360,7 +362,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -383,7 +385,7 @@ public class TestBSTIndex {
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyValue_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple result;
@@ -416,7 +418,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyOmittedValue_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = (((FileTablespace)TableSpaceManager.getLocalFs()))
         .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
@@ -451,7 +453,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -474,7 +476,7 @@ public class TestBSTIndex {
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyOmittedValue_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple result;
@@ -496,7 +498,8 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindMinValue" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+        .getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
@@ -530,7 +533,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -555,7 +558,7 @@ public class TestBSTIndex {
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     tuple.put(0, DatumFactory.createInt8(0));
@@ -579,7 +582,8 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testMinMax_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+        .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 5; i < TUPLE_NUM; i += 2) {
@@ -613,7 +617,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -683,7 +687,8 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testConcurrentAccess_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+        .getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
@@ -718,7 +723,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -763,7 +768,8 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindValueDescOrder_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+        .getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
@@ -799,7 +805,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -824,7 +830,7 @@ public class TestBSTIndex {
     BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
         keySchema, comp);
     reader.open();
-    scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     for (int i = (TUPLE_NUM - 1); i > 0; i--) {
@@ -854,7 +860,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyValueDescOrder_" + storeType);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;
@@ -889,7 +895,7 @@ public class TestBSTIndex {
     creater.setLoadNum(LOAD_NUM);
     creater.open();
 
-    SeekableScanner scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple keyTuple;
@@ -917,7 +923,7 @@ public class TestBSTIndex {
     assertEquals(keySchema, reader.getKeySchema());
     assertEquals(comp, reader.getComparator());
 
-    scanner = TableSpaceManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
     scanner.init();
 
     Tuple result;

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/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
index 49da04b..5ad7a27 100644
--- a/tajo-storage/tajo-storage-hdfs/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
@@ -77,7 +77,7 @@ public class TestSingleCSVFileBSTIndex {
     Path tablePath = StorageUtil.concatPath(testDir, "testFindValueInSingleCSV", "table.csv");
     fs.mkdirs(tablePath.getParent());
 
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
@@ -166,7 +166,7 @@ public class TestSingleCSVFileBSTIndex {
     Path tablePath = StorageUtil.concatPath(testDir, "testFindNextKeyValueInSingleCSV",
         "table1.csv");
     fs.mkdirs(tablePath.getParent());
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/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
index adc7a05..2fbf5d6 100644
--- a/tajo-storage/tajo-storage-hdfs/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
@@ -69,7 +69,7 @@ public class TestJsonSerDe {
     FileSystem fs = FileSystem.getLocal(conf);
     FileStatus status = fs.getFileStatus(tablePath);
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple tuple = scanner.next();


[06/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
----------------------------------------------------------------------
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 0963658..0e2ce42 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
@@ -109,8 +109,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -148,8 +147,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta("CSV");
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(dep4Meta, dep4Schema, dep4Path);
+    Appender appender4 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     VTuple tuple4 = new VTuple(dep4Schema.size());
     for (int i = 0; i < 11; i++) {
@@ -180,8 +178,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -220,8 +217,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -273,7 +269,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     appender5.flush();
@@ -282,7 +278,7 @@ public class TestFullOuterMergeJoinExec {
     catalog.createTable(phone3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
@@ -316,9 +312,9 @@ public class TestFullOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] dep3Frags =
-        FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testFullOuterMergeJoin0");
@@ -353,9 +349,9 @@ public class TestFullOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testFullOuterMergeJoin1");
@@ -390,9 +386,9 @@ public class TestFullOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testFullOuterMergeJoin2");
@@ -428,9 +424,9 @@ public class TestFullOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] dep4Frags =
-        FileTablespace.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testFullOuterMergeJoin3");
@@ -468,9 +464,9 @@ public class TestFullOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags =
-        FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getUri()),
             Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
@@ -507,9 +503,9 @@ public class TestFullOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags =
-        FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getUri()),
             Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
----------------------------------------------------------------------
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 f1d4718..d54df1c 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
@@ -83,7 +83,7 @@ public class TestHashAntiJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -109,7 +109,7 @@ public class TestHashAntiJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -128,7 +128,7 @@ public class TestHashAntiJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 
@@ -149,9 +149,9 @@ public class TestHashAntiJoinExec {
   @Test
   public final void testHashAntiJoin() throws IOException, PlanningException {
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()), Integer.MAX_VALUE);
+        new Path(people.getUri()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
----------------------------------------------------------------------
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 77d524e..a8826ee 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
@@ -85,7 +85,7 @@ public class TestHashJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -108,7 +108,7 @@ public class TestHashJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -126,7 +126,7 @@ public class TestHashJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
@@ -151,9 +151,9 @@ public class TestHashJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()), Integer.MAX_VALUE);
+        new Path(people.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testHashInnerJoin");
@@ -194,9 +194,9 @@ public class TestHashJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()), Integer.MAX_VALUE);
+        new Path(people.getUri()), Integer.MAX_VALUE);
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testHashInnerJoin");

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
----------------------------------------------------------------------
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 cc9560f..ae90502 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
@@ -84,7 +84,7 @@ public class TestHashSemiJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -110,7 +110,7 @@ public class TestHashSemiJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -133,7 +133,7 @@ public class TestHashSemiJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 
@@ -154,9 +154,9 @@ public class TestHashSemiJoinExec {
   @Test
   public final void testHashSemiJoin() throws IOException, PlanningException {
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()), Integer.MAX_VALUE);
+        new Path(people.getUri()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
@@ -177,8 +177,8 @@ public class TestHashSemiJoinExec {
       MergeJoinExec join = (MergeJoinExec) exec;
       ExternalSortExec sortLeftChild = (ExternalSortExec) join.getLeftChild();
       ExternalSortExec sortRightChild = (ExternalSortExec) join.getRightChild();
-      SeqScanExec scanLeftChild = (SeqScanExec) sortLeftChild.getChild();
-      SeqScanExec scanRightChild = (SeqScanExec) sortRightChild.getChild();
+      SeqScanExec scanLeftChild = sortLeftChild.getChild();
+      SeqScanExec scanRightChild = sortRightChild.getChild();
 
       // 'people' should be outer table. So, the below code guarantees that people becomes the outer table.
       if (scanLeftChild.getTableName().equals("default.people")) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
----------------------------------------------------------------------
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 4b4ed45..bbb441c 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
@@ -105,8 +105,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -135,7 +134,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
@@ -175,7 +174,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
@@ -228,7 +227,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     
@@ -240,7 +239,7 @@ public class TestLeftOuterHashJoinExec {
 
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
@@ -271,9 +270,9 @@ public class TestLeftOuterHashJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
     FileFragment[] dep3Frags = FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(),
-        new Path(dep3.getPath()), Integer.MAX_VALUE);
+        new Path(dep3.getUri()), Integer.MAX_VALUE);
     FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(),
-        new Path(emp3.getPath()), Integer.MAX_VALUE);
+        new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestLeftOuterHashJoinExec0");
@@ -301,9 +300,9 @@ public class TestLeftOuterHashJoinExec {
   @Test
   public final void testLeftOuter_HashJoinExec1() throws IOException, PlanningException {
     FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(),
-        new Path(job3.getPath()), Integer.MAX_VALUE);
+        new Path(job3.getUri()), Integer.MAX_VALUE);
     FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(),
-        new Path(emp3.getPath()), Integer.MAX_VALUE);
+        new Path(emp3.getUri()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
@@ -335,9 +334,9 @@ public class TestLeftOuterHashJoinExec {
   public final void testLeftOuter_HashJoinExec2() throws IOException, PlanningException {
     
     FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(),
-        new Path(emp3.getPath()), Integer.MAX_VALUE);
+        new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(),
-        new Path(job3.getPath()), Integer.MAX_VALUE);
+        new Path(job3.getUri()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
 
@@ -371,9 +370,9 @@ public class TestLeftOuterHashJoinExec {
   public final void testLeftOuter_HashJoinExec3() throws IOException, PlanningException {
     
     FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(),
-        new Path(emp3.getPath()), Integer.MAX_VALUE);
+        new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags = FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(),
-        new Path(phone3.getPath()), Integer.MAX_VALUE);
+        new Path(phone3.getUri()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
@@ -406,9 +405,9 @@ public class TestLeftOuterHashJoinExec {
   public final void testLeftOuter_HashJoinExec4() throws IOException, PlanningException {
     
     FileFragment[] emp3Frags = FileTablespace.splitNG(conf, "default.emp3", emp3.getMeta(),
-        new Path(emp3.getPath()), Integer.MAX_VALUE);
+        new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags = FileTablespace.splitNG(conf, "default.phone3", phone3.getMeta(),
-        new Path(phone3.getPath()), Integer.MAX_VALUE);
+        new Path(phone3.getUri()), Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(phone3Frags, emp3Frags);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
----------------------------------------------------------------------
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 370d03e..d0d0983 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
@@ -85,7 +85,7 @@ public class TestMergeJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -114,7 +114,7 @@ public class TestMergeJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -139,7 +139,7 @@ public class TestMergeJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
   }
 
   @After
@@ -162,9 +162,9 @@ public class TestMergeJoinExec {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
-    FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(), new Path(employee.getPath()),
+    FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(), new Path(employee.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(), new Path(people.getPath()),
+    FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(), new Path(people.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
----------------------------------------------------------------------
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 c73b89f..4866323 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
@@ -83,7 +83,7 @@ public class TestNLJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(schema.size());
@@ -107,7 +107,7 @@ public class TestNLJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -125,7 +125,7 @@ public class TestNLJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
 
     masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
   }
@@ -144,9 +144,9 @@ public class TestNLJoinExec {
   @Test
   public final void testNLCrossJoin() throws IOException, PlanningException {
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()), Integer.MAX_VALUE);
+        new Path(people.getUri()), Integer.MAX_VALUE);
     
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
@@ -173,9 +173,9 @@ public class TestNLJoinExec {
   @Test
   public final void testNLInnerJoin() throws IOException, PlanningException {
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()), Integer.MAX_VALUE);
+        new Path(people.getUri()), Integer.MAX_VALUE);
     
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
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 be519bf..b2a228a 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
@@ -98,7 +98,7 @@ public class TestPhysicalPlanner {
     util.startCatalogCluster();
     conf = util.getConfiguration();
     testDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestPhysicalPlanner");
-    sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    sm = TableSpaceManager.getLocalFs();
     catalog = util.getMiniCatalogCluster().getCatalog();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
@@ -164,7 +164,7 @@ public class TestPhysicalPlanner {
     appender.close();
     catalog.createTable(score);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
@@ -180,7 +180,7 @@ public class TestPhysicalPlanner {
 
     Schema scoreSchmea = score.getSchema();
     TableMeta scoreLargeMeta = CatalogUtil.newTableMeta("RAW", new KeyValueSet());
-    Appender appender =  ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender =  ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(scoreLargeMeta, scoreSchmea, scoreLargePath);
     appender.enableStats();
     appender.init();
@@ -238,7 +238,7 @@ public class TestPhysicalPlanner {
   @Test
   public final void testCreateScanPlan() throws IOException, PlanningException {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testCreateScanPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newTaskAttemptId(masterPlan),
@@ -269,7 +269,7 @@ public class TestPhysicalPlanner {
   @Test
   public final void testCreateScanWithFilterPlan() throws IOException, PlanningException {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testCreateScanWithFilterPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newTaskAttemptId(masterPlan),
@@ -297,7 +297,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testGroupByPlan() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testGroupByPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -328,7 +328,7 @@ public class TestPhysicalPlanner {
   @Test
   public final void testHashGroupByPlanWithALLField() throws IOException, PlanningException {
     // TODO - currently, this query does not use hash-based group operator.
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + 
         "/testHashGroupByPlanWithALLField");
@@ -358,7 +358,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testSortGroupByPlan() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testSortGroupByPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -421,7 +421,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testStorePlan() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -442,7 +442,7 @@ public class TestPhysicalPlanner {
     exec.next();
     exec.close();
 
-    Scanner scanner =  ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Scanner scanner =  ((FileTablespace) TableSpaceManager.getLocalFs())
         .getFileScanner(outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;
@@ -468,7 +468,7 @@ public class TestPhysicalPlanner {
     assertTrue("Checking meaningfulness of test", stats.getNumBytes() > StorageUnit.MB);
 
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.score_large", largeScore.getMeta(),
-        new Path(largeScore.getPath()), Integer.MAX_VALUE);
+        new Path(largeScore.getUri()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testStorePlanWithMaxOutputFileSize");
 
     QueryContext queryContext = new QueryContext(conf, session);
@@ -502,7 +502,7 @@ public class TestPhysicalPlanner {
     // checking the file contents
     long totalNum = 0;
     for (FileStatus status : fs.listStatus(ctx.getOutputPath().getParent())) {
-      Scanner scanner =  ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getFileScanner(
+      Scanner scanner =  ((FileTablespace) TableSpaceManager.getLocalFs()).getFileScanner(
           CatalogUtil.newTableMeta("CSV"),
           rootNode.getOutSchema(),
           status.getPath());
@@ -518,7 +518,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testStorePlanWithRCFile() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testStorePlanWithRCFile");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -539,7 +539,7 @@ public class TestPhysicalPlanner {
     exec.next();
     exec.close();
 
-    Scanner scanner = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getFileScanner(
+    Scanner scanner = ((FileTablespace) TableSpaceManager.getLocalFs()).getFileScanner(
         outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;
@@ -559,7 +559,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testEnforceForDefaultColumnPartitionStorePlan() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -586,7 +586,7 @@ public class TestPhysicalPlanner {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.HASH_PARTITION);
 
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -610,7 +610,7 @@ public class TestPhysicalPlanner {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.SORT_PARTITION);
 
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testStorePlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -626,7 +626,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testPartitionedStorePlan() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     TaskAttemptId id = LocalTajoTestingUtility.newTaskAttemptId(masterPlan);
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), id, new FileFragment[] { frags[0] },
@@ -696,7 +696,7 @@ public class TestPhysicalPlanner {
 
     // Preparing working dir and input fragments
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.score_large", largeScore.getMeta(),
-        new Path(largeScore.getPath()), Integer.MAX_VALUE);
+        new Path(largeScore.getUri()), Integer.MAX_VALUE);
     TaskAttemptId id = LocalTajoTestingUtility.newTaskAttemptId(masterPlan);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testPartitionedStorePlanWithMaxFileSize");
 
@@ -759,7 +759,7 @@ public class TestPhysicalPlanner {
   @Test
   public final void testPartitionedStorePlanWithEmptyGroupingSet()
       throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     TaskAttemptId id = LocalTajoTestingUtility.newTaskAttemptId(masterPlan);
 
@@ -826,7 +826,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testAggregationFunction() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testAggregationFunction");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -857,7 +857,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testCountFunction() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testCountFunction");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -885,7 +885,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testGroupByWithNullValue() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testGroupByWithNullValue");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -911,7 +911,7 @@ public class TestPhysicalPlanner {
   @Test
   public final void testUnionPlan() throws IOException, PlanningException, CloneNotSupportedException {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testUnionPlan");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newTaskAttemptId(masterPlan),
@@ -976,7 +976,7 @@ public class TestPhysicalPlanner {
   //@Test
   public final void testCreateIndex() throws IOException, PlanningException {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testCreateIndex");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newTaskAttemptId(masterPlan),
@@ -1003,7 +1003,7 @@ public class TestPhysicalPlanner {
   @Test
   public final void testDuplicateEliminate() throws IOException, PlanningException {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(),
-        new Path(score.getPath()), Integer.MAX_VALUE);
+        new Path(score.getUri()), Integer.MAX_VALUE);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testDuplicateEliminate");
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
@@ -1037,7 +1037,7 @@ public class TestPhysicalPlanner {
   @Test
   public final void testSortEnforcer() throws IOException, PlanningException {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testSortEnforcer");
     Expr context = analyzer.parse(SORT_QUERY[0]);
@@ -1087,7 +1087,7 @@ public class TestPhysicalPlanner {
 
   @Test
   public final void testGroupByEnforcer() throws IOException, PlanningException {
-    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()),
+    FileFragment[] frags = FileTablespace.splitNG(conf, "default.score", score.getMeta(), new Path(score.getUri()),
         Integer.MAX_VALUE);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testGroupByEnforcer");
     Expr context = analyzer.parse(QUERIES[7]);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
----------------------------------------------------------------------
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 d7bc36b..1b54948 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
@@ -86,7 +86,7 @@ public class TestProgressExternalSortExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("RAW");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
@@ -110,7 +110,7 @@ public class TestProgressExternalSortExec {
         employeePath.toUri());
     catalog.createTable(employee);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
   }
 
   @After
@@ -135,7 +135,7 @@ public class TestProgressExternalSortExec {
 
   private void testProgress(int sortBufferBytesNum) throws Exception {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newTaskAttemptId(), new FileFragment[] { frags[0] }, workDir);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
----------------------------------------------------------------------
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 088de50..c956f29 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
@@ -100,7 +100,7 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
@@ -130,7 +130,7 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
@@ -170,7 +170,7 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
@@ -212,7 +212,7 @@ public class TestRightOuterHashJoinExec {
     catalog.createTable(emp3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 
@@ -229,9 +229,9 @@ public class TestRightOuterHashJoinExec {
 
   @Test
   public final void testRightOuter_HashJoinExec0() throws IOException, PlanningException {
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] dep3Frags = FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
+    FileFragment[] dep3Frags = FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getUri()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
@@ -270,9 +270,9 @@ public class TestRightOuterHashJoinExec {
 
   @Test
   public final void testRightOuter_HashJoinExec1() throws IOException, PlanningException {
-    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
@@ -311,9 +311,9 @@ public class TestRightOuterHashJoinExec {
     @Test
   public final void testRightOuter_HashJoinExec2() throws IOException, PlanningException {
     
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()),
         Integer.MAX_VALUE);
 
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
----------------------------------------------------------------------
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 9d01c8f..25f0ca4 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
@@ -107,7 +107,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
@@ -146,7 +146,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta("CSV");
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender4 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     VTuple tuple4 = new VTuple(dep4Schema.size());
@@ -178,7 +178,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
@@ -218,7 +218,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
@@ -271,7 +271,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
@@ -281,7 +281,7 @@ public class TestRightOuterMergeJoinExec {
     catalog.createTable(phone3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
@@ -314,9 +314,9 @@ public class TestRightOuterMergeJoinExec {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] dep3Frags = FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
+    FileFragment[] dep3Frags = FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags);
 
@@ -352,9 +352,9 @@ public class TestRightOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testRightOuterMergeJoin1");
@@ -388,9 +388,9 @@ public class TestRightOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] job3Frags =
-        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testRightOuterMergeJoin2");
@@ -424,9 +424,9 @@ public class TestRightOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] dep4Frags =
-        FileTablespace.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags);
 
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testRightOuter_MergeJoin3");
@@ -461,9 +461,9 @@ public class TestRightOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
     FileFragment[] phone3Frags =
-        FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getUri()),
             Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 
@@ -498,8 +498,8 @@ public class TestRightOuterMergeJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN);
 
     FileFragment[] emp3Frags =
-        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+        FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()), Integer.MAX_VALUE);
+    FileFragment[] phone3Frags = FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags);
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
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 0f92bec..ce12faf 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
@@ -69,7 +69,7 @@ public class TestSortExec {
     util = TpchTestBase.getInstance().getTestingCluster();
     catalog = util.getMaster().getCatalog();
     workDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    sm = TableSpaceManager.getLocalFs();
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -81,7 +81,7 @@ public class TestSortExec {
     tablePath = StorageUtil.concatPath(workDir, "employee", "table1");
     sm.getFileSystem().mkdirs(tablePath.getParent());
 
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, tablePath);
     appender.init();
     VTuple tuple = new VTuple(schema.size());
@@ -101,7 +101,7 @@ public class TestSortExec {
     catalog.createTable(desc);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
index de2a13e..2758d7d 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java
@@ -71,7 +71,7 @@ public class TestCTASQuery extends QueryTestCaseBase {
     assertEquals("key", partitionDesc.getExpressionSchema().getRootColumns().get(0).getSimpleName());
 
     FileSystem fs = FileSystem.get(testBase.getTestingCluster().getConfiguration());
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
     assertTrue(fs.isDirectory(path));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0")));
@@ -114,7 +114,7 @@ public class TestCTASQuery extends QueryTestCaseBase {
     assertEquals("key", partitionDesc.getExpressionSchema().getRootColumns().get(0).getSimpleName());
 
     FileSystem fs = FileSystem.get(cluster.getConfiguration());
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
     assertTrue(fs.isDirectory(path));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0")));
@@ -267,7 +267,7 @@ public class TestCTASQuery extends QueryTestCaseBase {
       TableDesc desc =  client.getTableDesc("managed_table1");
 
       assertNotNull(desc);
-      assertEquals("managed_table1", new Path(desc.getPath()).getName());
+      assertEquals("managed_table1", new Path(desc.getUri()).getName());
     } else {
       assertFalse(client.existTable("managed_Table1"));
       assertTrue(client.existTable("MANAGED_TABLE1"));
@@ -275,7 +275,7 @@ public class TestCTASQuery extends QueryTestCaseBase {
       TableDesc desc =  client.getTableDesc("MANAGED_TABLE1");
 
       assertNotNull(desc);
-      assertEquals("MANAGED_TABLE1", new Path(desc.getPath()).getName());
+      assertEquals("MANAGED_TABLE1", new Path(desc.getUri()).getName());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
index f1a3ddc..b19a488 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java
@@ -107,9 +107,9 @@ public class TestCreateTable extends QueryTestCaseBase {
     // checking the existence of the table directory and validating the path
     Path warehouseDir = TajoConf.getWarehouseDir(testingCluster.getConfiguration());
     FileSystem fs = warehouseDir.getFileSystem(testingCluster.getConfiguration());
-    assertTrue(fs.exists(new Path(oldTableDesc.getPath())));
+    assertTrue(fs.exists(new Path(oldTableDesc.getUri())));
     assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, originalTableName),
-        new Path(oldTableDesc.getPath()));
+        new Path(oldTableDesc.getUri()));
 
     // Rename
     client.executeQuery("ALTER TABLE " + CatalogUtil.denormalizeIdentifier(oldFQTableName)
@@ -118,8 +118,8 @@ public class TestCreateTable extends QueryTestCaseBase {
     // checking the existence of the new table directory and validating the path
     final String newFQTableName = CatalogUtil.buildFQName(databaseName, newTableName);
     TableDesc newTableDesc = client.getTableDesc(newFQTableName);
-    assertTrue(fs.exists(new Path(newTableDesc.getPath())));
-    assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, newTableName), new Path(newTableDesc.getPath()));
+    assertTrue(fs.exists(new Path(newTableDesc.getUri())));
+    assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, newTableName), new Path(newTableDesc.getUri()));
   }
 
   @Test


[15/15] tajo git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into index_support

Posted by ji...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into index_support

Conflicts:
	tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
	tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
	tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
	tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
	tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
	tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
	tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
	tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
	tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
	tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
	tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
	tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
	tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
	tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
	tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
	tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
	tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
	tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java


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

Branch: refs/heads/index_support
Commit: 17dfe86ccacaf83a81d1f9b45d7c6b82409f0f82
Parents: 00a8c65 2ec307d
Author: Jihoon Son <ji...@apache.org>
Authored: Thu Jun 25 17:08:23 2015 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Thu Jun 25 17:08:23 2015 +0900

----------------------------------------------------------------------
 .travis.yml                                     |   4 +-
 CHANGES                                         |   7 +-
 NOTICE                                          |   2 +-
 .../org/apache/tajo/algebra/CreateTable.java    |  14 +
 .../org/apache/tajo/catalog/CatalogUtil.java    |   7 +
 .../org/apache/tajo/catalog/DDLBuilder.java     |   5 +-
 .../java/org/apache/tajo/catalog/TableDesc.java |   8 +-
 .../org/apache/tajo/catalog/TestTableDesc.java  |   2 +-
 .../tajo/catalog/store/HiveCatalogStore.java    |   2 +-
 .../catalog/store/TestHiveCatalogStore.java     |  25 +-
 .../tajo/catalog/store/AbstractDBStore.java     |  44 +--
 .../src/main/resources/schemas/derby/derby.xml  |   2 +-
 .../main/resources/schemas/mariadb/mariadb.xml  |   3 +-
 .../src/main/resources/schemas/mysql/mysql.xml  |   3 +-
 .../main/resources/schemas/oracle/oracle.xml    |   3 +-
 .../resources/schemas/postgresql/postgresql.xml |   4 +-
 .../cli/tsql/commands/DescTableCommand.java     |   2 +-
 .../main/java/org/apache/tajo/QueryVars.java    |   4 +-
 .../apache/tajo/storage/StorageConstants.java   |   9 +
 .../org/apache/tajo/storage/StorageService.java |  37 ++
 .../java/org/apache/tajo/util/FileUtil.java     |  14 +-
 .../java/org/apache/tajo/util/KeyValueSet.java  |   8 +-
 .../java/org/apache/tajo/util/NumberUtil.java   |  54 ---
 .../org/apache/tajo/util/ReflectionUtil.java    |   6 +-
 .../org/apache/tajo/engine/parser/SQLParser.g4  |   8 +-
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  11 +-
 .../engine/planner/PhysicalPlannerImpl.java     |  11 +-
 .../engine/planner/global/GlobalPlanner.java    |   4 +-
 .../planner/physical/BSTIndexScanExec.java      |   2 +-
 .../planner/physical/ColPartitionStoreExec.java |   2 +-
 .../planner/physical/FilterScanIterator.java    |  56 +++
 .../planner/physical/FullScanIterator.java      |  47 +++
 .../engine/planner/physical/InsertRowsExec.java | 107 +++++
 .../planner/physical/PhysicalPlanUtil.java      |   2 +-
 .../physical/RangeShuffleFileWriteExec.java     |   2 +-
 .../engine/planner/physical/ScanIterator.java   |  33 ++
 .../engine/planner/physical/SeqScanExec.java    |  40 +-
 .../engine/planner/physical/StoreTableExec.java |  23 +-
 .../apache/tajo/engine/query/QueryContext.java  |  71 +++-
 .../org/apache/tajo/master/GlobalEngine.java    |  17 +-
 .../java/org/apache/tajo/master/TajoMaster.java |  22 +-
 .../tajo/master/TajoMasterClientService.java    |   3 +-
 .../apache/tajo/master/exec/DDLExecutor.java    |  58 ++-
 .../exec/NonForwardQueryResultFileScanner.java  |   2 +-
 .../apache/tajo/master/exec/QueryExecutor.java  | 253 ++++++------
 .../master/exec/prehook/CreateTableHook.java    |   8 +-
 .../prehook/DistributedQueryHookManager.java    |   2 +-
 .../master/exec/prehook/InsertIntoHook.java     |   8 +-
 .../java/org/apache/tajo/querymaster/Query.java |  53 ++-
 .../tajo/querymaster/QueryMasterTask.java       | 150 ++-----
 .../apache/tajo/querymaster/Repartitioner.java  |  95 +++--
 .../java/org/apache/tajo/querymaster/Stage.java |  12 +-
 .../java/org/apache/tajo/querymaster/Task.java  |  56 +--
 .../tajo/webapp/QueryExecutorServlet.java       |   2 +-
 .../tajo/worker/ExecutionBlockContext.java      |  31 +-
 .../org/apache/tajo/worker/LegacyTaskImpl.java  |   2 +-
 .../java/org/apache/tajo/worker/TajoWorker.java |   5 +-
 .../java/org/apache/tajo/worker/TaskImpl.java   |   2 +-
 .../src/main/proto/TajoWorkerProtocol.proto     |  16 +-
 .../resources/webapps/admin/catalogview.jsp     |   3 +-
 .../src/main/resources/webapps/admin/index.jsp  |  23 +-
 .../org/apache/tajo/BackendTestingUtil.java     |   2 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |  39 +-
 .../org/apache/tajo/TajoTestingCluster.java     |  35 +-
 .../org/apache/tajo/cli/tsql/TestTajoCli.java   |   4 +-
 .../org/apache/tajo/client/TestTajoClient.java  |  18 +-
 .../apache/tajo/engine/eval/ExprTestBase.java   |   9 +-
 .../tajo/engine/eval/TestEvalTreeUtil.java      |   4 +-
 .../engine/planner/TestLogicalOptimizer.java    |   5 +-
 .../tajo/engine/planner/TestLogicalPlan.java    |   3 +-
 .../tajo/engine/planner/TestLogicalPlanner.java |  92 +++--
 .../tajo/engine/planner/TestPlannerUtil.java    |   5 +-
 .../planner/physical/TestBNLJoinExec.java       |  15 +-
 .../planner/physical/TestExternalSortExec.java  |   6 +-
 .../physical/TestFullOuterHashJoinExec.java     |  29 +-
 .../physical/TestFullOuterMergeJoinExec.java    |  40 +-
 .../planner/physical/TestHashAntiJoinExec.java  |  15 +-
 .../planner/physical/TestHashJoinExec.java      |  14 +-
 .../planner/physical/TestHashSemiJoinExec.java  |  14 +-
 .../physical/TestLeftOuterHashJoinExec.java     |  31 +-
 .../planner/physical/TestMergeJoinExec.java     |  10 +-
 .../engine/planner/physical/TestNLJoinExec.java |  14 +-
 .../planner/physical/TestPhysicalPlanner.java   |  57 ++-
 .../physical/TestProgressExternalSortExec.java  |   6 +-
 .../physical/TestRightOuterHashJoinExec.java    |  20 +-
 .../physical/TestRightOuterMergeJoinExec.java   |  36 +-
 .../engine/planner/physical/TestSortExec.java   |  18 +-
 .../apache/tajo/engine/query/TestCTASQuery.java |   8 +-
 .../tajo/engine/query/TestCreateTable.java      |   8 +-
 .../tajo/engine/query/TestHBaseTable.java       | 274 +++++++------
 .../tajo/engine/query/TestInsertQuery.java      |   6 +-
 .../apache/tajo/engine/query/TestJoinQuery.java |   6 +-
 .../tajo/engine/query/TestTablePartitions.java  |  28 +-
 .../apache/tajo/ha/TestHAServiceHDFSImpl.java   |  88 +++--
 .../org/apache/tajo/jdbc/TestResultSet.java     |   4 +-
 .../tajo/master/TestExecutionBlockCursor.java   |  12 +-
 .../TestNonForwardQueryResultSystemScanner.java | 258 +-----------
 .../apache/tajo/master/TestRepartitioner.java   |  77 ++--
 .../tajo/querymaster/TestIntermediateEntry.java |  24 +-
 .../apache/tajo/querymaster/TestKillQuery.java  |  10 +-
 .../org/apache/tajo/storage/TestRowFile.java    |   5 +-
 .../results/TestHBaseTable/testCATS.result      | 100 -----
 .../results/TestHBaseTable/testCTAS.result      | 100 +++++
 .../testInsertIntoUsingPut.result               |   4 +-
 .../TestHBaseTable/testInsertValues1.result     |   4 +
 .../testGetClusterDetails.result                |   4 +
 .../testGetNextRowsForAggregateFunction.result  |   3 +
 .../testGetNextRowsForTable.result              |   5 +
 .../results/TestTajoCli/testDescTable.result    |   8 +-
 .../testDescTableForNestedSchema.result         |   4 +-
 .../src/main/conf/storage-site.json.template    |  35 ++
 tajo-docs/src/main/sphinx/index.rst             |   1 +
 tajo-docs/src/main/sphinx/storage_plugin.rst    |  47 +++
 tajo-docs/src/main/sphinx/table_management.rst  |   1 +
 .../sphinx/table_management/table_overview.rst  |   7 +
 .../sphinx/table_management/tablespaces.rst     |  45 +++
 .../java/org/apache/tajo/plan/LogicalPlan.java  |   8 +
 .../org/apache/tajo/plan/LogicalPlanner.java    |  62 ++-
 .../tajo/plan/logical/CreateTableNode.java      |  44 +--
 .../apache/tajo/plan/logical/InsertNode.java    |  51 +--
 .../plan/logical/PartitionedTableScanNode.java  |   2 +-
 .../org/apache/tajo/plan/logical/ScanNode.java  |   2 +-
 .../tajo/plan/logical/StoreTableNode.java       |  27 ++
 .../rewrite/rules/PartitionedTableRewriter.java |   4 +-
 .../plan/serder/LogicalNodeDeserializer.java    |  16 +-
 .../tajo/plan/serder/LogicalNodeSerializer.java |  25 +-
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  30 +-
 tajo-plan/src/main/proto/Plan.proto             |  11 +-
 tajo-storage/tajo-storage-common/pom.xml        |  12 +-
 .../org/apache/tajo/storage/FormatProperty.java |  63 +++
 .../org/apache/tajo/storage/MergeScanner.java   |   5 +-
 .../apache/tajo/storage/OldStorageManager.java  | 234 +++++++++++
 .../apache/tajo/storage/StorageProperty.java    |  59 ++-
 .../org/apache/tajo/storage/StorageUtil.java    |   4 +-
 .../apache/tajo/storage/TableSpaceManager.java  | 238 -----------
 .../org/apache/tajo/storage/Tablespace.java     | 211 ++++++----
 .../apache/tajo/storage/TablespaceManager.java  | 390 +++++++++++++++++++
 .../src/main/resources/storage-default.json     |  20 +
 tajo-storage/tajo-storage-hbase/pom.xml         |  11 +
 .../storage/hbase/AbstractHBaseAppender.java    |   2 +-
 .../storage/hbase/AddSortForInsertRewriter.java |  92 -----
 .../tajo/storage/hbase/ColumnMapping.java       |  17 +-
 .../tajo/storage/hbase/HBaseFragment.java       |  28 +-
 .../tajo/storage/hbase/HBasePutAppender.java    |  13 +-
 .../apache/tajo/storage/hbase/HBaseScanner.java |   9 +-
 .../tajo/storage/hbase/HBaseTablespace.java     | 248 +++++++-----
 .../storage/hbase/SortedInsertRewriter.java     | 117 ++++++
 .../src/main/proto/StorageFragmentProtos.proto  |  15 +-
 .../tajo/storage/hbase/TestColumnMapping.java   |   2 +-
 .../storage/hbase/TestHBaseStorageManager.java  | 108 -----
 .../tajo/storage/hbase/TestHBaseTableSpace.java | 134 +++++++
 tajo-storage/tajo-storage-hdfs/pom.xml          |   6 +-
 .../org/apache/tajo/storage/FileAppender.java   |  28 +-
 .../org/apache/tajo/storage/FileTablespace.java | 237 +++++++----
 .../tajo/storage/HashShuffleAppender.java       |  87 ++---
 .../storage/HashShuffleAppenderManager.java     |  19 +-
 .../tajo/storage/TestCompressionStorages.java   |   4 +-
 .../tajo/storage/TestDelimitedTextFile.java     |   8 +-
 .../tajo/storage/TestFileStorageManager.java    | 233 -----------
 .../apache/tajo/storage/TestFileSystems.java    |   2 +-
 .../apache/tajo/storage/TestFileTablespace.java | 250 ++++++++++++
 .../org/apache/tajo/storage/TestLineReader.java |   8 +-
 .../apache/tajo/storage/TestMergeScanner.java   |   6 +-
 .../org/apache/tajo/storage/TestStorages.java   |  48 +--
 .../apache/tajo/storage/index/TestBSTIndex.java |  58 +--
 .../index/TestSingleCSVFileBSTIndex.java        |   4 +-
 .../apache/tajo/storage/json/TestJsonSerDe.java |   2 +-
 167 files changed, 3860 insertions(+), 2682 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/CHANGES
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
index 3d09eb1,c6b9b41..d76ec92
--- 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
@@@ -48,10 -49,7 +48,9 @@@ import org.apache.tajo.plan.LogicalPlan
  import org.apache.tajo.plan.logical.*;
  import org.apache.tajo.plan.serder.LogicalNodeDeserializer;
  import org.apache.tajo.plan.util.PlannerUtil;
 -import org.apache.tajo.storage.*;
 +import org.apache.tajo.storage.FileTablespace;
 +import org.apache.tajo.storage.StorageConstants;
- import org.apache.tajo.storage.TableSpaceManager;
- import org.apache.tajo.storage.Tablespace;
++import org.apache.tajo.storage.TablespaceManager;
  import org.apache.tajo.storage.fragment.FileFragment;
  import org.apache.tajo.storage.fragment.Fragment;
  import org.apache.tajo.storage.fragment.FragmentConvertor;
@@@ -1192,11 -1185,21 +1192,10 @@@ public class PhysicalPlannerImpl implem
      Preconditions.checkNotNull(ctx.getTable(annotation.getCanonicalName()),
          "Error: There is no table matched to %s", annotation.getCanonicalName());
  
 -    FragmentProto [] fragmentProtos = ctx.getTables(annotation.getTableName());
 -    List<FileFragment> fragments =
 -        FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
 -
 -    String indexName = IndexUtil.getIndexNameOfFrag(fragments.get(0), annotation.getSortKeys());
 -    FileTablespace sm = (FileTablespace) TablespaceManager.get(fragments.get(0).getPath().toUri()).get();
 -    String dbName = CatalogUtil.extractQualifier(annotation.getTableName());
 -    String simpleName = CatalogUtil.extractSimpleName(annotation.getTableName());
 -    Path indexPath = new Path(new Path(sm.getTableUri(dbName, simpleName)), "index");
 -
 -    TupleComparator comp = new BaseTupleComparator(annotation.getKeySchema(),
 -        annotation.getSortKeys());
 -    return new BSTIndexScanExec(ctx, annotation, fragments.get(0), new Path(indexPath, indexName),
 -        annotation.getKeySchema(), comp, annotation.getDatum());
 -
 +    FragmentProto [] fragments = ctx.getTables(annotation.getTableName());
- 
 +    Preconditions.checkState(fragments.length == 1);
 +    return new BSTIndexScanExec(ctx, annotation, fragments[0], annotation.getIndexPath(),
 +        annotation.getKeySchema(), annotation.getPredicates());
    }
  
    public static EnforceProperty getAlgorithmEnforceProperty(Enforcer enforcer, LogicalNode node) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
index 34e6f5c,54abca8..521b6b9
--- 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
@@@ -96,98 -67,12 +96,98 @@@ public class BSTIndexScanExec extends P
      this.reader.open();
    }
  
 +  private static Schema mergeSubSchemas(Schema originalSchema, Schema subSchema, Target[] targets, EvalNode qual) {
 +    Schema mergedSchema = new Schema();
 +    Set<Column> qualAndTargets = TUtil.newHashSet();
 +    qualAndTargets.addAll(EvalTreeUtil.findUniqueColumns(qual));
 +    for (Target target : targets) {
 +      qualAndTargets.addAll(EvalTreeUtil.findUniqueColumns(target.getEvalTree()));
 +    }
 +    for (Column column : originalSchema.getRootColumns()) {
 +      if (subSchema.contains(column)
 +          || qualAndTargets.contains(column)
 +          || qualAndTargets.contains(column)) {
 +        mergedSchema.addColumn(column);
 +      }
 +    }
 +    return mergedSchema;
 +  }
 +
    @Override
    public void init() throws IOException {
 +    Schema projected;
 +
 +    // in the case where projected column or expression are given
 +    // the target can be an empty list.
 +    if (plan.hasTargets()) {
 +      projected = new Schema();
 +      Set<Column> columnSet = new HashSet<Column>();
 +
 +      if (plan.hasQual()) {
 +        columnSet.addAll(EvalTreeUtil.findUniqueColumns(qual));
 +      }
 +
 +      for (Target t : plan.getTargets()) {
 +        columnSet.addAll(EvalTreeUtil.findUniqueColumns(t.getEvalTree()));
 +      }
 +
 +      for (Column column : inSchema.getAllColumns()) {
 +        if (columnSet.contains(column)) {
 +          projected.addColumn(column);
 +        }
 +      }
 +
 +    } else {
 +      // no any projected columns, meaning that all columns should be projected.
 +      // TODO - this implicit rule makes code readability bad. So, we should remove it later
 +      projected = outSchema;
 +    }
 +
 +    initScanner(projected);
      super.init();
      progress = 0.0f;
 -    if (qual != null) {
 -      qual.bind(context.getEvalContext(), inSchema);
 +
 +    if (plan.hasQual()) {
 +      if (fileScanner.isProjectable()) {
 +        qual.bind(context.getEvalContext(), projected);
 +      } else {
 +        qual.bind(context.getEvalContext(), inSchema);
 +      }
 +    }
 +  }
 +
 +  private void initScanner(Schema projected) throws IOException {
 +
 +    TableMeta meta;
 +    try {
 +      meta = (TableMeta) plan.getTableDesc().getMeta().clone();
 +    } catch (CloneNotSupportedException e) {
 +      throw new RuntimeException(e);
 +    }
 +
 +    // set system default properties
 +    PlannerUtil.applySystemDefaultToTableProperties(context.getQueryContext(), meta);
 +
 +    // Why we should check nullity? See https://issues.apache.org/jira/browse/TAJO-1422
 +    if (fragment != null) {
 +
 +      Schema fileScanOutSchema = mergeSubSchemas(projected, keySchema, plan.getTargets(), qual);
 +
-       this.fileScanner = TableSpaceManager.getStorageManager(context.getConf(),
++      this.fileScanner = OldStorageManager.getStorageManager(context.getConf(),
 +          plan.getTableDesc().getMeta().getStoreType())
 +          .getSeekableScanner(plan.getTableDesc().getMeta(), plan.getPhysicalSchema(), fragment, fileScanOutSchema);
 +      this.fileScanner.init();
 +
 +      // See Scanner.isProjectable() method Depending on the result of isProjectable(),
 +      // the width of retrieved tuple is changed.
 +      //
 +      // If TRUE, the retrieved tuple will contain only projected fields.
 +      // If FALSE, the retrieved tuple will contain projected fields and NullDatum for non-projected fields.
 +      if (fileScanner.isProjectable()) {
 +        this.projector = new Projector(context, projected, outSchema, plan.getTargets());
 +      } else {
 +        this.projector = new Projector(context, inSchema, outSchema, plan.getTargets());
 +      }
      }
    }
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 422d034,37b497c..91fce57
--- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@@ -43,7 -43,7 +43,6 @@@ import org.apache.tajo.ipc.ClientProtos
  import org.apache.tajo.master.TajoMaster.MasterContext;
  import org.apache.tajo.master.exec.DDLExecutor;
  import org.apache.tajo.master.exec.QueryExecutor;
--import org.apache.tajo.session.Session;
  import org.apache.tajo.plan.*;
  import org.apache.tajo.plan.logical.InsertNode;
  import org.apache.tajo.plan.logical.LogicalRootNode;
@@@ -53,10 -53,8 +52,10 @@@ import org.apache.tajo.plan.verifier.Lo
  import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
  import org.apache.tajo.plan.verifier.VerificationState;
  import org.apache.tajo.plan.verifier.VerifyException;
- import org.apache.tajo.storage.Tablespace;
- import org.apache.tajo.storage.TableSpaceManager;
++import org.apache.tajo.session.Session;
+ import org.apache.tajo.storage.TablespaceManager;
  import org.apache.tajo.util.CommonTestingUtil;
 +import org.apache.tajo.util.IPCUtil;
  
  import java.io.IOException;
  import java.sql.SQLException;
@@@ -95,9 -91,8 +92,9 @@@ public class GlobalEngine extends Abstr
      try  {
        analyzer = new SQLAnalyzer();
        preVerifier = new PreLogicalPlanVerifier(context.getCatalog());
-       planner = new LogicalPlanner(context.getCatalog());
+       planner = new LogicalPlanner(context.getCatalog(), TablespaceManager.getInstance());
 -      optimizer = new LogicalOptimizer(context.getConf());
 +      // Access path rewriter is enabled only in QueryMasterTask
 +      optimizer = new LogicalOptimizer(context.getConf(), context.getCatalog());
        annotatedPlanVerifier = new LogicalPlanVerifier(context.getConf(), context.getCatalog());
      } catch (Throwable t) {
        LOG.error(t.getMessage(), t);

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index c6732b3,7dbe815..76bfb95
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@@ -835,12 -844,11 +835,11 @@@ public class TajoMasterClientService ex
          TableDesc desc;
          try {
            desc = context.getGlobalEngine().getDDLExecutor().createTable(queryContext, request.getName(),
-               meta.getStoreType(), schema,
-               meta, path, true, partitionDesc, false);
+               null, meta.getStoreType(), schema, meta, path.toUri(), true, partitionDesc, false);
          } catch (Exception e) {
            return TableResponse.newBuilder()
 -              .setResultCode(ResultCode.ERROR)
 -              .setErrorMessage(e.getMessage()).build();
 +              .setResult(IPCUtil.buildRequestResult(ResultCode.ERROR, e.getMessage(), null))
 +              .build();
          }
  
          return TableResponse.newBuilder()

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
index 862e6f7,5d42157..2e5584c
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@@ -28,11 -28,10 +28,8 @@@ import org.apache.tajo.QueryId
  import org.apache.tajo.QueryIdFactory;
  import org.apache.tajo.SessionVars;
  import org.apache.tajo.TajoConstants;
--import org.apache.tajo.catalog.CatalogService;
- import org.apache.tajo.catalog.CatalogUtil;
--import org.apache.tajo.catalog.Schema;
--import org.apache.tajo.catalog.TableDesc;
 -import org.apache.tajo.catalog.TableMeta;
++import org.apache.tajo.catalog.*;
 +import org.apache.tajo.catalog.exception.AlreadyExistsIndexException;
  import org.apache.tajo.catalog.proto.CatalogProtos;
  import org.apache.tajo.catalog.statistics.TableStats;
  import org.apache.tajo.common.TajoDataTypes;
@@@ -41,12 -40,13 +38,14 @@@ import org.apache.tajo.datum.DatumFacto
  import org.apache.tajo.engine.planner.global.GlobalPlanner;
  import org.apache.tajo.engine.planner.global.MasterPlan;
  import org.apache.tajo.engine.planner.physical.EvalExprExec;
- import org.apache.tajo.engine.planner.physical.StoreTableExec;
+ import org.apache.tajo.engine.planner.physical.InsertRowsExec;
  import org.apache.tajo.engine.query.QueryContext;
  import org.apache.tajo.ipc.ClientProtos;
 +import org.apache.tajo.ipc.ClientProtos.ResultCode;
  import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
- import org.apache.tajo.master.*;
+ import org.apache.tajo.master.QueryInfo;
+ import org.apache.tajo.master.QueryManager;
+ import org.apache.tajo.master.TajoMaster;
  import org.apache.tajo.master.exec.prehook.CreateTableHook;
  import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
  import org.apache.tajo.master.exec.prehook.InsertIntoHook;
@@@ -65,8 -61,8 +60,9 @@@ import org.apache.tajo.plan.function.py
  import org.apache.tajo.plan.logical.*;
  import org.apache.tajo.plan.util.PlannerUtil;
  import org.apache.tajo.plan.verifier.VerifyException;
+ import org.apache.tajo.session.Session;
  import org.apache.tajo.storage.*;
 +import org.apache.tajo.util.IPCUtil;
  import org.apache.tajo.util.ProtoUtil;
  import org.apache.tajo.worker.TaskAttemptContext;
  
@@@ -422,42 -384,89 +391,89 @@@ public class QueryExecutor 
          fs.rename(eachFile.getPath(), targetFilePath);
        }
      }
+   }
  
-     if (insertNode.hasTargetTable()) {
-       TableStats stats = tableDesc.getStats();
-       long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-       stats.setNumBytes(volume);
-       stats.setNumRows(1);
- 
-       CatalogProtos.UpdateTableStatsProto.Builder builder = CatalogProtos.UpdateTableStatsProto.newBuilder();
-       builder.setTableName(tableDesc.getName());
-       builder.setStats(stats.getProto());
+   /**
+    * Insert row values
+    */
+   private void insertRowValues(QueryContext queryContext,
+                                InsertNode insertNode, SubmitQueryResponse.Builder responseBuilder) {
+     try {
+       String nodeUniqName = insertNode.getTableName() == null ? new Path(insertNode.getUri()).getName() :
+           insertNode.getTableName();
+       String queryId = nodeUniqName + "_" + System.currentTimeMillis();
+ 
+       URI finalOutputUri = insertNode.getUri();
+       Tablespace space = TablespaceManager.get(finalOutputUri).get();
+       TableMeta tableMeta = new TableMeta(insertNode.getStorageType(), insertNode.getOptions());
+       tableMeta.putOption(StorageConstants.INSERT_DIRECTLY, Boolean.TRUE.toString());
+ 
+       FormatProperty formatProperty = space.getFormatProperty(tableMeta);
+ 
+       TaskAttemptContext taskAttemptContext;
+       if (formatProperty.directInsertSupported()) { // if this format and storage supports direct insertion
+         taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, null);
+         taskAttemptContext.setOutputPath(new Path(finalOutputUri));
+ 
+         EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
+         InsertRowsExec exec = new InsertRowsExec(taskAttemptContext, insertNode, evalExprExec);
+ 
+         try {
+           exec.init();
+           exec.next();
+         } finally {
+           exec.close();
+         }
+       } else {
+         URI stagingSpaceUri = space.prepareStagingSpace(context.getConf(), queryId, queryContext, tableMeta);
+         Path stagingDir = new Path(stagingSpaceUri);
+         Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
  
-       catalog.updateTableStats(builder.build());
+         taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
+         taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
+         insertRowsThroughStaging(taskAttemptContext, insertNode, new Path(finalOutputUri), stagingDir, stagingResultDir);
+       }
  
-       responseBuilder.setTableDesc(tableDesc.getProto());
-     } else {
+       // set insert stats (how many rows and bytes)
        TableStats stats = new TableStats();
-       long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-       stats.setNumBytes(volume);
-       stats.setNumRows(1);
- 
-       // Empty TableDesc
-       List<CatalogProtos.ColumnProto> columns = new ArrayList<CatalogProtos.ColumnProto>();
-       CatalogProtos.TableDescProto tableDescProto = CatalogProtos.TableDescProto.newBuilder()
-           .setTableName(nodeUniqName)
-           .setMeta(CatalogProtos.TableProto.newBuilder().setStoreType("CSV").build())
-           .setSchema(CatalogProtos.SchemaProto.newBuilder().addAllFields(columns).build())
-           .setStats(stats.getProto())
-           .build();
- 
-       responseBuilder.setTableDesc(tableDescProto);
-     }
+       stats.setNumBytes(taskAttemptContext.getResultStats().getNumBytes());
+       stats.setNumRows(taskAttemptContext.getResultStats().getNumRows());
+ 
+       if (insertNode.hasTargetTable()) {
+         CatalogProtos.UpdateTableStatsProto.Builder builder = CatalogProtos.UpdateTableStatsProto.newBuilder();
+         builder.setTableName(insertNode.getTableName());
+         builder.setStats(stats.getProto());
+ 
+         catalog.updateTableStats(builder.build());
+ 
+         TableDesc desc = new TableDesc(
+             insertNode.getTableName(),
+             insertNode.getTargetSchema(),
+             tableMeta,
+             finalOutputUri);
+         responseBuilder.setTableDesc(desc.getProto());
+ 
+       } else { // If INSERT INTO LOCATION
+ 
+         // Empty TableDesc
+         List<CatalogProtos.ColumnProto> columns = new ArrayList<CatalogProtos.ColumnProto>();
+         CatalogProtos.TableDescProto tableDescProto = CatalogProtos.TableDescProto.newBuilder()
+             .setTableName(nodeUniqName)
+             .setMeta(CatalogProtos.TableProto.newBuilder().setStoreType("CSV").build())
+             .setSchema(CatalogProtos.SchemaProto.newBuilder().addAllFields(columns).build())
+             .setStats(stats.getProto())
+             .build();
+ 
+         responseBuilder.setTableDesc(tableDescProto);
+       }
  
-     // If queryId == NULL_QUERY_ID and MaxRowNum == -1, TajoCli prints only number of inserted rows.
-     responseBuilder.setMaxRowNum(-1);
-     responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
-     responseBuilder.setResult(IPCUtil.buildOkRequestResult());
+       // If queryId == NULL_QUERY_ID and MaxRowNum == -1, TajoCli prints only number of inserted rows.
+       responseBuilder.setMaxRowNum(-1);
+       responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
 -      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
++      responseBuilder.setResult(IPCUtil.buildOkRequestResult());
+     } catch (Throwable t) {
+       throw new RuntimeException(t);
+     }
    }
  
    public void executeDistributedQuery(QueryContext queryContext, Session session,
@@@ -502,47 -515,15 +522,34 @@@
      }
    }
  
 +  private void checkIndexExistence(final QueryContext queryContext, final CreateIndexNode createIndexNode)
 +      throws IOException {
 +    String databaseName, simpleIndexName, qualifiedIndexName;
 +    if (CatalogUtil.isFQTableName(createIndexNode.getIndexName())) {
 +      String[] splits = CatalogUtil.splitFQTableName(createIndexNode.getIndexName());
 +      databaseName = splits[0];
 +      simpleIndexName = splits[1];
 +      qualifiedIndexName = createIndexNode.getIndexName();
 +    } else {
 +      databaseName = queryContext.getCurrentDatabase();
 +      simpleIndexName = createIndexNode.getIndexName();
 +      qualifiedIndexName = CatalogUtil.buildFQName(databaseName, simpleIndexName);
 +    }
 +
 +    if (catalog.existIndexByName(databaseName, simpleIndexName)) {
 +      throw new AlreadyExistsIndexException(qualifiedIndexName);
 +    }
 +  }
 +
-   public static MasterPlan compileMasterPlan(LogicalPlan plan, QueryContext context, GlobalPlanner planner)
+   public MasterPlan compileMasterPlan(LogicalPlan plan, QueryContext context, GlobalPlanner planner)
        throws Exception {
  
-     String storeType = PlannerUtil.getStoreType(plan);
-     if (storeType != null) {
-       Tablespace sm = TableSpaceManager.getStorageManager(planner.getConf(), storeType);
-       StorageProperty storageProperty = sm.getStorageProperty();
-       if (storageProperty.isSortedInsert()) {
-         String tableName = PlannerUtil.getStoreTableName(plan);
-         LogicalRootNode rootNode = plan.getRootBlock().getRoot();
-         TableDesc tableDesc = PlannerUtil.getTableDesc(planner.getCatalog(), rootNode.getChild());
-         if (tableDesc == null) {
-           throw new VerifyException("Can't get table meta data from catalog: " + tableName);
-         }
-         List<LogicalPlanRewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
-             context, tableDesc);
-         if (storageSpecifiedRewriteRules != null) {
-           for (LogicalPlanRewriteRule eachRule: storageSpecifiedRewriteRules) {
-             eachRule.rewrite(new LogicalPlanRewriteRuleContext(context, plan));
-           }
-         }
-       }
+     LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+     TableDesc tableDesc = PlannerUtil.getTableDesc(planner.getCatalog(), rootNode.getChild());
+ 
+     if (tableDesc != null) {
+       Tablespace space = TablespaceManager.get(tableDesc.getUri()).get();
+       space.rewritePlan(context, plan);
      }
  
      MasterPlan masterPlan = new MasterPlan(QueryIdFactory.NULL_QUERY_ID, context, plan);

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
index bb83b8c,9d5838d..23fa497
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
@@@ -30,10 -30,9 +30,11 @@@ import org.apache.hadoop.yarn.state.*
  import org.apache.hadoop.yarn.util.Clock;
  import org.apache.tajo.ExecutionBlockId;
  import org.apache.tajo.QueryId;
+ import org.apache.tajo.QueryVars;
  import org.apache.tajo.SessionVars;
  import org.apache.tajo.TajoProtos.QueryState;
 +import org.apache.tajo.catalog.*;
 +import org.apache.tajo.catalog.exception.CatalogException;
  import org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
  import org.apache.tajo.catalog.CatalogService;
  import org.apache.tajo.catalog.TableDesc;

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
index 0737422,1f5e7a3..550b1ee
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
@@@ -50,15 -40,16 +40,18 @@@ import org.apache.tajo.ipc.TajoWorkerPr
  import org.apache.tajo.master.TajoContainerProxy;
  import org.apache.tajo.master.event.*;
  import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+ import org.apache.tajo.plan.LogicalOptimizer;
+ import org.apache.tajo.plan.LogicalPlan;
+ import org.apache.tajo.plan.LogicalPlanner;
  import org.apache.tajo.plan.logical.LogicalNode;
+ import org.apache.tajo.plan.logical.LogicalRootNode;
  import org.apache.tajo.plan.logical.NodeType;
  import org.apache.tajo.plan.logical.ScanNode;
- import org.apache.tajo.plan.verifier.VerifyException;
+ import org.apache.tajo.plan.util.PlannerUtil;
  import org.apache.tajo.session.Session;
 -import org.apache.tajo.storage.*;
++import org.apache.tajo.storage.FormatProperty;
 +import org.apache.tajo.storage.Tablespace;
- import org.apache.tajo.storage.StorageProperty;
- import org.apache.tajo.storage.StorageUtil;
- import org.apache.tajo.storage.TableSpaceManager;
++import org.apache.tajo.storage.TablespaceManager;
  import org.apache.tajo.util.metrics.TajoMetrics;
  import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
  import org.apache.tajo.worker.AbstractResourceAllocator;
@@@ -315,36 -300,22 +302,21 @@@ public class QueryMasterTask extends Co
          LOG.warn("Query already started");
          return;
        }
 -
 -
 +      LOG.info(SessionVars.INDEX_ENABLED.keyname() + " : " + queryContext.getBool(SessionVars.INDEX_ENABLED));
        CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
-       LogicalPlanner planner = new LogicalPlanner(catalog);
+       LogicalPlanner planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -      LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
 +      LogicalOptimizer optimizer = new LogicalOptimizer(systemConf, catalog);
        Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
        jsonExpr = null; // remove the possible OOM
+ 
        plan = planner.createPlan(queryContext, expr);
+       optimizer.optimize(queryContext, plan);
  
-       String storeType = PlannerUtil.getStoreType(plan);
-       if (storeType != null) {
-         sm = TableSpaceManager.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<LogicalPlanRewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
-               getQueryTaskContext().getQueryContext(), tableDesc);
-           if (storageSpecifiedRewriteRules != null) {
-             for (LogicalPlanRewriteRule eachRule: storageSpecifiedRewriteRules) {
-               optimizer.addRuleAfterToJoinOpt(eachRule);
-             }
-           }
-         }
-       }
+       // when a given uri is null, TablespaceManager.get will return the default tablespace.
+       space = TablespaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
+       space.rewritePlan(queryContext, plan);
  
-       optimizer.optimize(queryContext, plan);
+       initStagingDir();
  
        for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
          LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index 9118e73,07a09ad..2c336a4
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@@ -44,7 -42,8 +43,6 @@@ import org.apache.tajo.plan.expr.EvalCo
  import org.apache.tajo.plan.expr.EvalNode;
  import org.apache.tajo.plan.serder.EvalNodeDeserializer;
  import org.apache.tajo.plan.serder.EvalNodeSerializer;
 -import org.apache.tajo.engine.query.QueryContext;
--import org.apache.tajo.catalog.SchemaUtil;
  import org.apache.tajo.plan.serder.PlanProto;
  import org.apache.tajo.plan.verifier.LogicalPlanVerifier;
  import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
@@@ -66,9 -66,9 +65,7 @@@ import java.util.TimeZone
  
  import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
  import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME;
--import static org.junit.Assert.assertEquals;
--import static org.junit.Assert.assertFalse;
--import static org.junit.Assert.fail;
++import static org.junit.Assert.*;
  
  public class ExprTestBase {
    private static TajoTestingCluster util;
@@@ -103,8 -103,8 +100,8 @@@
  
      analyzer = new SQLAnalyzer();
      preLogicalPlanVerifier = new PreLogicalPlanVerifier(cat);
-     planner = new LogicalPlanner(cat);
+     planner = new LogicalPlanner(cat, TablespaceManager.getInstance());
 -    optimizer = new LogicalOptimizer(util.getConfiguration());
 +    optimizer = new LogicalOptimizer(util.getConfiguration(), cat);
      annotatedPlanVerifier = new LogicalPlanVerifier(util.getConfiguration(), cat);
    }
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
index 133dec1,afa3472..8042aef
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
@@@ -103,10 -103,10 +103,11 @@@ public class TestLogicalOptimizer 
  
      catalog.createFunction(funcDesc);
      sqlAnalyzer = new SQLAnalyzer();
-     planner = new LogicalPlanner(catalog);
+     planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    optimizer = new LogicalOptimizer(util.getConfiguration());
++    optimizer = new LogicalOptimizer(util.getConfiguration(), catalog);
  
      defaultContext = LocalTajoTestingUtility.createDummyContext(util.getConfiguration());
 +    optimizer = new LogicalOptimizer(util.getConfiguration(), catalog);
    }
  
    @AfterClass

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index dbd6e6d,0f37763..0cd5ced
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@@ -881,31 -889,8 +889,31 @@@ public class TestLogicalPlanner 
    }
  
    @Test
 +  public final void testCreateIndexNode() throws PlanningException {
 +    QueryContext qc = new QueryContext(util.getConfiguration(), session);
 +    Expr expr = sqlAnalyzer.parse(QUERIES[11]);
 +    LogicalPlan rootNode = planner.createPlan(qc, expr);
 +    LogicalNode plan = rootNode.getRootBlock().getRoot();
 +    testJsonSerDerObject(plan);
 +
 +    LogicalRootNode root = (LogicalRootNode) plan;
 +    assertEquals(NodeType.CREATE_INDEX, root.getChild().getType());
 +    CreateIndexNode createIndexNode = root.getChild();
 +
 +    assertEquals(NodeType.PROJECTION, createIndexNode.getChild().getType());
 +    ProjectionNode projNode = createIndexNode.getChild();
 +
 +    assertEquals(NodeType.SELECTION, projNode.getChild().getType());
 +    SelectionNode selNode = projNode.getChild();
 +
 +    assertEquals(NodeType.SCAN, selNode.getChild().getType());
 +    ScanNode scanNode = selNode.getChild();
 +    assertEquals(CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), scanNode.getTableName());
 +  }
 +
 +  @Test
    public final void testAsterisk() throws CloneNotSupportedException, PlanningException {
-     QueryContext qc = new QueryContext(util.getConfiguration(), session);
+     QueryContext qc = createQueryContext();
  
      Expr expr = sqlAnalyzer.parse(QUERIES[13]);
      LogicalPlan planNode = planner.createPlan(qc, expr);

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
index 2178d5c,1b64a8f..0684ff6
--- 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
@@@ -28,14 -28,14 +28,15 @@@ import org.apache.tajo.conf.TajoConf
  import org.apache.tajo.datum.Datum;
  import org.apache.tajo.datum.DatumFactory;
  import org.apache.tajo.engine.parser.SQLAnalyzer;
--import org.apache.tajo.engine.planner.*;
++import org.apache.tajo.engine.planner.PhysicalPlanner;
++import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
  import org.apache.tajo.engine.planner.enforce.Enforcer;
++import org.apache.tajo.engine.query.QueryContext;
  import org.apache.tajo.plan.LogicalOptimizer;
  import org.apache.tajo.plan.LogicalPlan;
  import org.apache.tajo.plan.LogicalPlanner;
  import org.apache.tajo.plan.PlanningException;
  import org.apache.tajo.plan.logical.LogicalNode;
--import org.apache.tajo.engine.query.QueryContext;
  import org.apache.tajo.storage.*;
  import org.apache.tajo.storage.fragment.FileFragment;
  import org.apache.tajo.util.CommonTestingUtil;
@@@ -130,8 -128,8 +131,8 @@@ public class TestHashAntiJoinExec 
      people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
      catalog.createTable(people);
      analyzer = new SQLAnalyzer();
-     planner = new LogicalPlanner(catalog);
+     planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    optimizer = new LogicalOptimizer(conf);
 +    optimizer = new LogicalOptimizer(conf, catalog);
    }
  
    @After

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
index 697624a,afa273b..cbf0739
--- 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
@@@ -135,8 -133,8 +135,8 @@@ public class TestHashSemiJoinExec 
      people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
      catalog.createTable(people);
      analyzer = new SQLAnalyzer();
-     planner = new LogicalPlanner(catalog);
+     planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    optimizer = new LogicalOptimizer(conf);
 +    optimizer = new LogicalOptimizer(conf, catalog);
    }
  
    @After

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index 8fe03be,dff0cbe..cdfb8cc
--- 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
@@@ -162,13 -162,12 +162,12 @@@ public class TestPhysicalPlanner 
      }
      appender.flush();
      appender.close();
 +
 +    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
      catalog.createTable(score);
      analyzer = new SQLAnalyzer();
-     planner = new LogicalPlanner(catalog);
+     planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    optimizer = new LogicalOptimizer(conf);
 -
 -    defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
 +    optimizer = new LogicalOptimizer(conf, catalog);
- 
      masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
  
      createLargeScoreTable();
@@@ -971,19 -970,14 +970,19 @@@
    }
  
    public final String [] createIndexStmt = {
 -      "create index idx_employee on employee using bst (name null first, empId desc)"
 +      "create index idx_employee on employee using TWO_LEVEL_BIN_TREE (name null first, empId desc)"
    };
  
 -  //@Test
 +  @Test
    public final void testCreateIndex() throws IOException, PlanningException {
      FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-         new Path(employee.getPath()), Integer.MAX_VALUE);
+         new Path(employee.getUri()), Integer.MAX_VALUE);
      Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testCreateIndex");
 +    Path indexPath = StorageUtil.concatPath(TajoConf.getWarehouseDir(conf), "default/idx_employee");
 +    if (sm.getFileSystem().exists(indexPath)) {
 +      sm.getFileSystem().delete(indexPath, true);
 +    }
 +
      TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
          LocalTajoTestingUtility.newTaskAttemptId(masterPlan),
          new FileFragment[] {frags[0]}, workDir);

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
index b54b4d9,4690e71..a19ac34
--- 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
@@@ -30,11 -30,11 +30,17 @@@ import org.apache.tajo.conf.TajoConf
  import org.apache.tajo.datum.Datum;
  import org.apache.tajo.datum.DatumFactory;
  import org.apache.tajo.engine.parser.SQLAnalyzer;
--import org.apache.tajo.engine.planner.*;
++import org.apache.tajo.engine.planner.PhysicalPlanner;
++import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
++import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
++import org.apache.tajo.engine.planner.UniformRangePartition;
  import org.apache.tajo.engine.planner.enforce.Enforcer;
--import org.apache.tajo.plan.*;
--import org.apache.tajo.plan.logical.LogicalNode;
  import org.apache.tajo.engine.query.QueryContext;
++import org.apache.tajo.plan.LogicalOptimizer;
++import org.apache.tajo.plan.LogicalPlan;
++import org.apache.tajo.plan.LogicalPlanner;
++import org.apache.tajo.plan.PlanningException;
++import org.apache.tajo.plan.logical.LogicalNode;
  import org.apache.tajo.plan.util.PlannerUtil;
  import org.apache.tajo.storage.*;
  import org.apache.tajo.storage.fragment.FileFragment;
@@@ -101,10 -100,9 +107,10 @@@ public class TestSortExec 
          tablePath.toUri());
      catalog.createTable(desc);
  
 +    queryContext = new QueryContext(conf);
      analyzer = new SQLAnalyzer();
-     planner = new LogicalPlanner(catalog);
+     planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    optimizer = new LogicalOptimizer(conf);
 +    optimizer = new LogicalOptimizer(conf, catalog);
    }
  
    public static String[] QUERIES = {

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
index af093a8,7c61cc7..e5b8470
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
@@@ -80,10 -79,9 +79,9 @@@ public class TestExecutionBlockCursor 
      }
  
      analyzer = new SQLAnalyzer();
-     logicalPlanner = new LogicalPlanner(catalog);
+     logicalPlanner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    optimizer = new LogicalOptimizer(conf);
 +    optimizer = new LogicalOptimizer(conf, catalog);
  
-     Tablespace sm  = TableSpaceManager.getFileStorageManager(conf);
      dispatcher = new AsyncDispatcher();
      dispatcher.init(conf);
      dispatcher.start();

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
index 10c6836,1351716..e7f51a8
--- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
@@@ -37,7 -37,7 +37,10 @@@ import org.apache.tajo.engine.query.Tas
  import org.apache.tajo.ipc.QueryCoordinatorProtocol;
  import org.apache.tajo.ipc.TajoWorkerProtocol;
  import org.apache.tajo.master.cluster.WorkerConnectionInfo;
--import org.apache.tajo.master.event.*;
++import org.apache.tajo.master.event.QueryEvent;
++import org.apache.tajo.master.event.QueryEventType;
++import org.apache.tajo.master.event.StageEvent;
++import org.apache.tajo.master.event.StageEventType;
  import org.apache.tajo.plan.LogicalOptimizer;
  import org.apache.tajo.plan.LogicalPlan;
  import org.apache.tajo.plan.LogicalPlanner;
@@@ -104,8 -105,8 +108,8 @@@ public class TestKillQuery 
      Session session = LocalTajoTestingUtility.createDummySession();
      CatalogService catalog = cluster.getMaster().getCatalog();
  
-     LogicalPlanner planner = new LogicalPlanner(catalog);
+     LogicalPlanner planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    LogicalOptimizer optimizer = new LogicalOptimizer(conf);
 +    LogicalOptimizer optimizer = new LogicalOptimizer(conf, catalog);
      Expr expr =  analyzer.parse(queryStr);
      LogicalPlan plan = planner.createPlan(defaultContext, expr);
  
@@@ -168,8 -169,8 +172,8 @@@
      Session session = LocalTajoTestingUtility.createDummySession();
      CatalogService catalog = cluster.getMaster().getCatalog();
  
-     LogicalPlanner planner = new LogicalPlanner(catalog);
+     LogicalPlanner planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 -    LogicalOptimizer optimizer = new LogicalOptimizer(conf);
 +    LogicalOptimizer optimizer = new LogicalOptimizer(conf, catalog);
      Expr expr =  analyzer.parse(queryStr);
      LogicalPlan plan = planner.createPlan(defaultContext, expr);
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------
diff --cc tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
index 48b74c0,a2480c9..d32045f
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
@@@ -49,7 -48,9 +50,8 @@@ import org.apache.tajo.plan.nameresolve
  import org.apache.tajo.plan.rewrite.rules.ProjectionPushDownRule;
  import org.apache.tajo.plan.util.ExprFinder;
  import org.apache.tajo.plan.util.PlannerUtil;
 -import org.apache.tajo.catalog.SchemaUtil;
  import org.apache.tajo.plan.verifier.VerifyException;
+ import org.apache.tajo.storage.StorageService;
  import org.apache.tajo.util.KeyValueSet;
  import org.apache.tajo.util.Pair;
  import org.apache.tajo.util.StringUtils;

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-plan/src/main/proto/Plan.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
index 0000000,ef33a8e..d12c6bd
mode 000000,100644..100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
@@@ -1,0 -1,250 +1,234 @@@
+ /**
+  * 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.annotations.VisibleForTesting;
+ import com.google.common.base.Preconditions;
+ 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.FileSystem;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.tajo.TaskAttemptId;
+ import org.apache.tajo.catalog.Schema;
+ import org.apache.tajo.catalog.TableMeta;
+ import org.apache.tajo.conf.TajoConf;
+ import org.apache.tajo.storage.fragment.Fragment;
+ 
+ import java.io.IOException;
+ import java.lang.reflect.Constructor;
+ import java.net.URI;
+ import java.util.Map;
+ import java.util.concurrent.ConcurrentHashMap;
+ 
+ /**
+  * It handles available table spaces and cache TableSpace instances.
+  */
+ public class OldStorageManager {
+   private static final Log LOG = LogFactory.getLog(OldStorageManager.class);
+ 
+   /**
+    * 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>>();
+   private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
+       Configuration.class,
+       Schema.class,
+       TableMeta.class,
+       Fragment.class
+   };
+   private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
+       Configuration.class,
+       TaskAttemptId.class,
+       Schema.class,
+       TableMeta.class,
+       Path.class
+   };
+   /**
+    * Cache of Tablespace.
+    * Key is manager key(warehouse path) + store type
+    */
+   private static final Map<String, Tablespace> storageManagers = Maps.newHashMap();
+   /**
+    * Cache of constructors for each class. Pins the classes so they
+    * can't be garbage collected until ReflectionUtils can be collected.
+    */
+   protected static Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
+ 
+   /**
+    * Clear all class cache
+    */
+   @VisibleForTesting
+   protected synchronized static void clearCache() {
+     CONSTRUCTOR_CACHE.clear();
+     SCANNER_HANDLER_CACHE.clear();
+     APPENDER_HANDLER_CACHE.clear();
+     storageManagers.clear();
+   }
+ 
+   /**
+    * Close Tablespace
+    * @throws java.io.IOException
+    */
+   public static void shutdown() throws IOException {
+     synchronized(storageManagers) {
+       for (Tablespace eachTablespace : storageManagers.values()) {
+         eachTablespace.close();
+       }
+     }
+     clearCache();
+   }
+ 
+   /**
+    * Returns the proper Tablespace instance according to the storeType.
+    *
+    * @param tajoConf Tajo system property.
+    * @param storeType Storage type
+    * @return
+    * @throws IOException
+    */
+   public static Tablespace getStorageManager(TajoConf tajoConf, String storeType) throws IOException {
+     FileSystem fileSystem = TajoConf.getWarehouseDir(tajoConf).getFileSystem(tajoConf);
+     if (fileSystem != null) {
+       return getStorageManager(tajoConf, fileSystem.getUri(), storeType);
+     } else {
+       return getStorageManager(tajoConf, null, storeType);
+     }
+   }
+ 
+   /**
+    * Returns the proper Tablespace instance according to the storeType
+    *
+    * @param tajoConf Tajo system property.
+    * @param uri Key that can identify each storage manager(may be a path)
+    * @param storeType Storage type
+    * @return
+    * @throws IOException
+    */
+   public static synchronized Tablespace getStorageManager(
+       TajoConf tajoConf, URI uri, String storeType) throws IOException {
+     Preconditions.checkNotNull(tajoConf);
+     Preconditions.checkNotNull(uri);
+     Preconditions.checkNotNull(storeType);
+ 
+     String typeName;
+     if (storeType.equalsIgnoreCase("HBASE")) {
+       typeName = "hbase";
+     } else {
+       typeName = "hdfs";
+     }
+ 
+     synchronized (storageManagers) {
+       String storeKey = typeName + "_" + uri.toString();
+       Tablespace manager = storageManagers.get(storeKey);
+ 
+       if (manager == null) {
+         Class<? extends Tablespace> storageManagerClass =
+             tajoConf.getClass(String.format("tajo.storage.manager.%s.class", typeName), null, Tablespace.class);
+ 
+         if (storageManagerClass == null) {
+           throw new IOException("Unknown Storage Type: " + typeName);
+         }
+ 
+         try {
+           Constructor<? extends Tablespace> constructor =
+               (Constructor<? extends Tablespace>) CONSTRUCTOR_CACHE.get(storageManagerClass);
+           if (constructor == null) {
+             constructor = storageManagerClass.getDeclaredConstructor(TablespaceManager.TABLESPACE_PARAM);
+             constructor.setAccessible(true);
+             CONSTRUCTOR_CACHE.put(storageManagerClass, constructor);
+           }
+           manager = constructor.newInstance(new Object[]{"noname", uri});
+         } catch (Exception e) {
+           throw new RuntimeException(e);
+         }
+         manager.init(tajoConf);
+         storageManagers.put(storeKey, manager);
+       }
+ 
+       return manager;
+     }
+   }
+ 
+   /**
 -   * 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 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);
 -  }
 -
 -  /**
+    * 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, TaskAttemptId 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;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
index c89f043,52e223d..0995e91
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
@@@ -245,22 -261,15 +261,30 @@@ public abstract class Tablespace 
      return scanner;
    }
  
+   public Appender getAppenderForInsertRow(OverridableConf queryContext,
+                                           TaskAttemptId taskAttemptId,
+                                           TableMeta meta,
+                                           Schema schema,
+                                           Path workDir) throws IOException {
+     return getAppender(queryContext, taskAttemptId, meta, schema, workDir);
+   }
+ 
    /**
 +   * 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 IOException
 +   */
 +  public synchronized SeekableScanner getSeekableScanner(TableMeta meta, Schema schema, FragmentProto fragment,
 +                                                         Schema target) throws IOException {
 +    return (SeekableScanner)this.getScanner(meta, schema, fragment, target);
 +  }
 +
 +  /**
     * Returns Appender instance.
     * @param queryContext Query property.
     * @param taskAttemptId Task id.

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
index 845c2d7,18bb7ed..bcb02c6
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
@@@ -49,13 -49,9 +49,10 @@@ import org.apache.tajo.plan.logical.Cre
  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.LogicalPlanRewriteRule;
++import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRuleContext;
  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 org.apache.tajo.util.*;
  
  import java.io.BufferedReader;
  import java.io.IOException;
@@@ -1062,13 -1089,10 +1090,10 @@@ public class HBaseTablespace extends Ta
      }
    }
  
-   public List<LogicalPlanRewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
-     if ("false".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
-       List<LogicalPlanRewriteRule> rules = new ArrayList<LogicalPlanRewriteRule>();
-       rules.add(new AddSortForInsertRewriter(tableDesc, getIndexColumns(tableDesc)));
-       return rules;
-     } else {
-       return null;
+   @Override
+   public void rewritePlan(OverridableConf context, LogicalPlan plan) throws PlanningException {
 -    if (REWRITE_RULE.isEligible(context, plan)) {
 -      REWRITE_RULE.rewrite(context, plan);
++    if (REWRITE_RULE.isEligible(new LogicalPlanRewriteRuleContext(context, plan))) {
++      REWRITE_RULE.rewrite(new LogicalPlanRewriteRuleContext(context, plan));
      }
    }
  

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
index 0000000,ebf557e..b6143f3
mode 000000,100644..100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
@@@ -1,0 -1,116 +1,117 @@@
+ /**
+  * 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.OverridableConf;
+ import org.apache.tajo.catalog.Column;
+ import org.apache.tajo.catalog.Schema;
+ import org.apache.tajo.catalog.SortSpec;
+ import org.apache.tajo.plan.LogicalPlan;
+ import org.apache.tajo.plan.PlanningException;
+ import org.apache.tajo.plan.logical.*;
+ import org.apache.tajo.plan.logical.SortNode.SortPurpose;
+ import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
++import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRuleContext;
+ import org.apache.tajo.util.KeyValueSet;
+ 
+ import java.io.IOException;
+ import java.util.ArrayList;
+ import java.util.List;
+ 
+ /**
+  * This rewrite rule injects a sort operation to preserve the writing rows in
+  * an ascending order of HBase row keys, required by HFile.
+  */
+ public class SortedInsertRewriter implements LogicalPlanRewriteRule {
+ 
+   @Override
+   public String getName() {
+     return "SortedInsertRewriter";
+   }
+ 
+   @Override
 -  public boolean isEligible(OverridableConf queryContext, LogicalPlan plan) {
 -    boolean hbaseMode = "false".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"));
 -    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
++  public boolean isEligible(LogicalPlanRewriteRuleContext context) {
++    boolean hbaseMode = "false".equalsIgnoreCase(context.getQueryContext().get(HBaseStorageConstants.INSERT_PUT_MODE, "false"));
++    LogicalRootNode rootNode = context.getPlan().getRootBlock().getRoot();
+     LogicalNode node = rootNode.getChild();
+     return hbaseMode && node.getType() == NodeType.CREATE_TABLE || node.getType() == NodeType.INSERT;
+   }
+ 
+   public static Column[] getIndexColumns(Schema tableSchema, KeyValueSet tableProperty) throws IOException {
+     List<Column> indexColumns = new ArrayList<Column>();
+ 
+     ColumnMapping columnMapping = new ColumnMapping(tableSchema, tableProperty);
+ 
+     boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
+     for (int i = 0; i < isRowKeys.length; i++) {
+       if (isRowKeys[i]) {
+         indexColumns.add(tableSchema.getColumn(i));
+       }
+     }
+ 
+     return indexColumns.toArray(new Column[]{});
+   }
+ 
+   @Override
 -  public LogicalPlan rewrite(OverridableConf queryContext, LogicalPlan plan) throws PlanningException {
++  public LogicalPlan rewrite(LogicalPlanRewriteRuleContext context) throws PlanningException {
++    LogicalPlan plan = context.getPlan();
+     LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+ 
+     StoreTableNode storeTable = rootNode.getChild();
+     Schema tableSchema = storeTable.getTableSchema();
+ 
+     Column[] sortColumns;
+     try {
+       sortColumns = getIndexColumns(tableSchema, storeTable.getOptions());
+     } catch (IOException e) {
+       throw new PlanningException(e);
+     }
+ 
+     int[] sortColumnIndexes = new int[sortColumns.length];
+     for (int i = 0; i < sortColumns.length; i++) {
+       sortColumnIndexes[i] = tableSchema.getColumnId(sortColumns[i].getQualifiedName());
+     }
+ 
+     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;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/tajo/blob/17dfe86c/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
----------------------------------------------------------------------
diff --cc tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
index 2fab8db,22fb607..f0a302a
--- a/tajo-storage/tajo-storage-hdfs/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
@@@ -124,8 -125,7 +125,8 @@@ public class TestBSTIndex 
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -148,8 -148,7 +149,8 @@@
      tuple = new VTuple(keySchema.size());
      BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + storeType + ".idx"), keySchema, comp);
      reader.open();
-     scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      for (int i = 0; i < TUPLE_NUM - 1; i++) {
@@@ -228,8 -227,7 +229,8 @@@
      BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
          keySchema, comp);
      reader.open();
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      for (int i = 0; i < TUPLE_NUM - 1; i++) {
@@@ -292,8 -290,7 +293,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -364,8 -362,7 +366,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -388,8 -385,7 +390,8 @@@
      BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyValue_" + storeType + ".idx"),
          keySchema, comp);
      reader.open();
-     scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple result;
@@@ -457,8 -453,7 +459,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -481,8 -476,7 +483,8 @@@
      BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyOmittedValue_" + storeType + ".idx"),
          keySchema, comp);
      reader.open();
-     scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple result;
@@@ -538,8 -533,7 +541,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -564,8 -558,7 +567,8 @@@
      BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
          keySchema, comp);
      reader.open();
-     scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      tuple.put(0, DatumFactory.createInt8(0));
@@@ -623,8 -617,7 +627,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -729,8 -723,7 +734,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -811,8 -805,7 +817,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -837,8 -830,7 +843,8 @@@
      BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
          keySchema, comp);
      reader.open();
-     scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      for (int i = (TUPLE_NUM - 1); i > 0; i--) {
@@@ -903,8 -895,7 +909,8 @@@
      creater.setLoadNum(LOAD_NUM);
      creater.open();
  
-     SeekableScanner scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    SeekableScanner scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple keyTuple;
@@@ -932,8 -923,7 +938,8 @@@
      assertEquals(keySchema, reader.getKeySchema());
      assertEquals(comp, reader.getComparator());
  
-     scanner = TableSpaceManager.getStorageManager(conf, meta.getStoreType()).
 -    scanner = OldStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
++    scanner = OldStorageManager.getStorageManager(conf, meta.getStoreType()).
 +        getSeekableScanner(meta, schema, tablet.getProto(), schema);
      scanner.init();
  
      Tuple result;


[03/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
index 845c2d7..5fac0cf 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.storage.hbase;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -44,12 +43,12 @@ 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.PlanningException;
 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.LogicalPlanRewriteRule;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.Bytes;
@@ -60,6 +59,7 @@ import org.apache.tajo.util.TUtil;
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.net.URI;
 import java.util.*;
 
 /**
@@ -68,14 +68,44 @@ import java.util.*;
 public class HBaseTablespace extends Tablespace {
   private static final Log LOG = LogFactory.getLog(HBaseTablespace.class);
 
+  public static final StorageProperty HBASE_STORAGE_PROPERTIES = new StorageProperty(false, true, true, false);
+
+  public static final FormatProperty HFILE_FORMAT_PROPERTIES = new FormatProperty(true);
+
+  private Configuration hbaseConf;
+
+  private final static SortedInsertRewriter REWRITE_RULE = new SortedInsertRewriter();
+
   private Map<HConnectionKey, HConnection> connMap = new HashMap<HConnectionKey, HConnection>();
 
-  public HBaseTablespace(String storeType) {
-    super(storeType);
+  public HBaseTablespace(String spaceName, URI uri) {
+    super(spaceName, uri);
   }
 
   @Override
   public void storageInit() throws IOException {
+    this.hbaseConf = HBaseConfiguration.create(conf);
+    String zkQuorum = extractQuorum(uri);
+    String [] splits = zkQuorum.split(":");
+    hbaseConf.set(HConstants.ZOOKEEPER_QUORUM, splits[0]);
+    hbaseConf.set(HConstants.ZOOKEEPER_CLIENT_PORT, splits[1]);
+  }
+
+  @Override
+  public void setConfig(String name, String value) {
+  }
+
+  @Override
+  public void setConfigs(Map<String, String> configs) {
+  }
+
+  public Configuration getHbaseConf() {
+    return hbaseConf;
+  }
+
+  @Override
+  public long getTableVolume(URI uri) throws IOException {
+    return 0;
   }
 
   @Override
@@ -93,13 +123,13 @@ public class HBaseTablespace extends Tablespace {
 
   @Override
   public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
-    createTable(tableDesc.getMeta(), tableDesc.getSchema(), tableDesc.isExternal(), ifNotExists);
+    createTable(tableDesc.getUri(), 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,
+  private void createTable(URI uri, TableMeta tableMeta, Schema schema,
                            boolean isExternal, boolean ifNotExists) throws IOException {
     String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
     if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
@@ -113,7 +143,7 @@ public class HBaseTablespace extends Tablespace {
       throw new IOException("Columns property has more entry than Tajo table columns");
     }
 
-    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta.getOptions());
     int numRowKeys = 0;
     boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
     for (int i = 0; i < isRowKeyMappings.length; i++) {
@@ -138,8 +168,7 @@ public class HBaseTablespace extends Tablespace {
       }
     }
 
-    Configuration hConf = getHBaseConfiguration(conf, tableMeta);
-    HBaseAdmin hAdmin =  new HBaseAdmin(hConf);
+    HBaseAdmin hAdmin =  new HBaseAdmin(getHbaseConf());
 
     try {
       if (isExternal) {
@@ -210,7 +239,7 @@ public class HBaseTablespace extends Tablespace {
       return null;
     }
 
-    ColumnMapping columnMapping = new ColumnMapping(schema, meta);
+    ColumnMapping columnMapping = new ColumnMapping(schema, meta.getOptions());
     boolean[] isBinaryColumns = columnMapping.getIsBinaryColumns();
     boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
 
@@ -291,45 +320,23 @@ public class HBaseTablespace extends Tablespace {
   }
 
   /**
-   * Creates Configuration instance and sets with hbase connection options.
+   * It extracts quorum addresses from a Hbase Tablespace URI.
+   * For example, consider an example URI 'hbase:zk://host1:2171,host2:2172,host3:2173/table1'.
+   * <code>extractQuorum</code> will extract only 'host1:2171,host2:2172,host3:2173'.
    *
-   * @param conf
-   * @param tableMeta
-   * @return
-   * @throws java.io.IOException
+   * @param uri Hbase Tablespace URI
+   * @return Quorum addresses
    */
-  public static Configuration getHBaseConfiguration(Configuration conf, TableMeta tableMeta) throws IOException {
-    Configuration hbaseConf = (conf == null) ? HBaseConfiguration.create() : HBaseConfiguration.create(conf);
-
-    String zkQuorum = hbaseConf.get(HConstants.ZOOKEEPER_QUORUM);
-    if (tableMeta.containsOption(HBaseStorageConstants.META_ZK_QUORUM_KEY)) {
-      zkQuorum = tableMeta.getOption(HBaseStorageConstants.META_ZK_QUORUM_KEY, "");
-      hbaseConf.set(HConstants.ZOOKEEPER_QUORUM, zkQuorum);
-    }
-
-    if (zkQuorum == null || zkQuorum.trim().isEmpty()) {
-      throw new IOException("HBase mapped table is required a '" +
-          HBaseStorageConstants.META_ZK_QUORUM_KEY + "' attribute.");
-    }
+  static String extractQuorum(URI uri) {
+    String uriStr = uri.toString();
+    int start = uriStr.indexOf("/") + 2;
+    int pathIndex = uriStr.lastIndexOf("/");
 
-    String zkPort = hbaseConf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
-    if (tableMeta.containsOption(HBaseStorageConstants.META_ZK_CLIENT_PORT)) {
-      zkPort = tableMeta.getOption(HBaseStorageConstants.META_ZK_CLIENT_PORT, "");
-      hbaseConf.set(HConstants.ZOOKEEPER_CLIENT_PORT, zkPort);
-    }
-
-    if (zkPort == null || zkPort.trim().isEmpty()) {
-      throw new IOException("HBase mapped table is required a '" +
-        HBaseStorageConstants.META_ZK_CLIENT_PORT + "' attribute.");
-    }
-
-    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());
-      }
+    if (pathIndex < start) {
+      return uriStr.substring(start);
+    } else {
+      return uriStr.substring(start, pathIndex);
     }
-    return hbaseConf;
   }
 
   /**
@@ -348,7 +355,7 @@ public class HBaseTablespace extends Tablespace {
     }
     TableName hTableName = TableName.valueOf(hbaseTableName);
 
-    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta.getOptions());
 
     HTableDescriptor hTableDescriptor = new HTableDescriptor(hTableName);
 
@@ -369,7 +376,7 @@ public class HBaseTablespace extends Tablespace {
 
   @Override
   public void purgeTable(TableDesc tableDesc) throws IOException {
-    HBaseAdmin hAdmin =  new HBaseAdmin(getHBaseConfiguration(conf, tableDesc.getMeta()));
+    HBaseAdmin hAdmin =  new HBaseAdmin(hbaseConf);
 
     try {
       HTableDescriptor hTableDesc = parseHTableDescriptor(tableDesc.getMeta(), tableDesc.getSchema());
@@ -381,6 +388,11 @@ public class HBaseTablespace extends Tablespace {
     }
   }
 
+  @Override
+  public URI getTableUri(String databaseName, String tableName) {
+    return URI.create(uri.toString() + "/" + tableName);
+  }
+
   /**
    * Returns columns which are mapped to the rowkey of the hbase table.
    *
@@ -389,7 +401,7 @@ public class HBaseTablespace extends Tablespace {
    * @throws java.io.IOException
    */
   private Column[] getIndexableColumns(TableDesc tableDesc) throws IOException {
-    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta().getOptions());
     boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
     int[] rowKeyIndexes = columnMapping.getRowKeyFieldIndexes();
 
@@ -407,15 +419,14 @@ public class HBaseTablespace extends Tablespace {
 
   @Override
   public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc, ScanNode scanNode) throws IOException {
-    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta().getOptions());
 
     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));
+      htable = new HTable(hbaseConf, 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) {
@@ -424,8 +435,12 @@ public class HBaseTablespace extends Tablespace {
           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());
+        Fragment fragment = new HBaseFragment(
+            tableDesc.getUri(),
+            fragmentId, htable.getName().getNameAsString(),
+            HConstants.EMPTY_BYTE_ARRAY,
+            HConstants.EMPTY_BYTE_ARRAY,
+            regLoc.getHostname());
         fragments.add(fragment);
         return fragments;
       }
@@ -458,7 +473,7 @@ public class HBaseTablespace extends Tablespace {
         stopRows = TUtil.newList(HConstants.EMPTY_END_ROW);
       }
 
-      hAdmin =  new HBaseAdmin(hconf);
+      hAdmin =  new HBaseAdmin(hbaseConf);
       Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
 
       // region startkey -> HBaseFragment
@@ -499,8 +514,12 @@ public class HBaseTablespace extends Tablespace {
                 prevFragment.setStopRow(fragmentStop);
               }
             } else {
-              HBaseFragment fragment = new HBaseFragment(fragmentId, htable.getName().getNameAsString(),
-                  fragmentStart, fragmentStop, location.getHostname());
+              HBaseFragment fragment = new HBaseFragment(tableDesc.getUri(),
+                  fragmentId,
+                  htable.getName().getNameAsString(),
+                  fragmentStart,
+                  fragmentStop,
+                  location.getHostname());
 
               // get region size
               boolean foundLength = false;
@@ -557,7 +576,7 @@ public class HBaseTablespace extends Tablespace {
                               TaskAttemptId 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);
+      return new HBasePutAppender(conf, uri, taskAttemptId, schema, meta, workDir);
     } else {
       return super.getAppender(queryContext, taskAttemptId, meta, schema, workDir);
     }
@@ -566,17 +585,16 @@ public class HBaseTablespace extends Tablespace {
   @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));
+      htable = new HTable(hbaseConf, 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);
+      hAdmin =  new HBaseAdmin(hbaseConf);
       Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
 
       List<Fragment> fragments = new ArrayList<Fragment>(keys.getFirst().length);
@@ -599,8 +617,13 @@ public class HBaseTablespace extends Tablespace {
           serverLoadMap.put(location.getServerName(), serverLoad);
         }
 
-        HBaseFragment fragment = new HBaseFragment(tableDesc.getName(), htable.getName().getNameAsString(),
-            location.getRegionInfo().getStartKey(), location.getRegionInfo().getEndKey(), location.getHostname());
+        HBaseFragment fragment = new HBaseFragment(
+            tableDesc.getUri(),
+            tableDesc.getName(),
+            htable.getName().getNameAsString(),
+            location.getRegionInfo().getStartKey(),
+            location.getRegionInfo().getEndKey(),
+            location.getHostname());
 
         // get region size
         boolean foundLength = false;
@@ -642,7 +665,7 @@ public class HBaseTablespace extends Tablespace {
     }
   }
 
-  public HConnection getConnection(Configuration hbaseConf) throws IOException {
+  public HConnection getConnection() throws IOException {
     synchronized(connMap) {
       HConnectionKey key = new HConnectionKey(hbaseConf);
       HConnection conn = connMap.get(key);
@@ -937,18 +960,17 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
-                               LogicalPlan plan, Schema schema,
-                               TableDesc tableDesc) throws IOException {
+  public Path commitTable(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);
     }
-    Preconditions.checkArgument(tableDesc.getName() != null && tableDesc.getPath() == null);
 
     Path stagingDir = new Path(queryContext.get(QueryVars.STAGING_DIR));
     Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
 
-    Configuration hbaseConf = HBaseTablespace.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
+    Configuration hbaseConf = HBaseConfiguration.create(this.hbaseConf);
     hbaseConf.set("hbase.loadincremental.threads.max", "2");
 
     JobContextImpl jobContext = new JobContextImpl(hbaseConf,
@@ -993,8 +1015,7 @@ public class HBaseTablespace extends Tablespace {
         sortKeyIndexes[i] = inputSchema.getColumnId(sortSpecs[i].getSortKey().getQualifiedName());
       }
 
-      ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
-      Configuration hbaseConf = HBaseTablespace.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
+      ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta().getOptions());
 
       HTable htable = new HTable(hbaseConf, columnMapping.getHbaseTableName());
       try {
@@ -1062,59 +1083,45 @@ public class HBaseTablespace extends Tablespace {
     }
   }
 
-  public List<LogicalPlanRewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
-    if ("false".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
-      List<LogicalPlanRewriteRule> rules = new ArrayList<LogicalPlanRewriteRule>();
-      rules.add(new AddSortForInsertRewriter(tableDesc, getIndexColumns(tableDesc)));
-      return rules;
-    } else {
-      return null;
+  @Override
+  public void rewritePlan(OverridableConf context, LogicalPlan plan) throws PlanningException {
+    if (REWRITE_RULE.isEligible(context, plan)) {
+      REWRITE_RULE.rewrite(context, plan);
     }
   }
 
-  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 getProperty() {
+    return HBASE_STORAGE_PROPERTIES;
   }
 
   @Override
-  public StorageProperty getStorageProperty() {
-    StorageProperty storageProperty = new StorageProperty();
-    storageProperty.setSortedInsert(true);
-    storageProperty.setSupportsInsertInto(true);
-    return storageProperty;
+  public FormatProperty getFormatProperty(String format) {
+    return HFILE_FORMAT_PROPERTIES;
   }
 
-  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
+  public void prepareTable(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());
+        createTable(
+            ((CreateTableNode) node).getUri(), tableMeta, cNode.getTableSchema(),
+            cNode.isExternal(), cNode.isIfNotExists());
       }
     }
   }
 
   @Override
-  public void rollbackOutputCommit(LogicalNode node) throws IOException {
+  public void rollbackTable(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));
 
+      HBaseAdmin hAdmin =  new HBaseAdmin(this.hbaseConf);
+      TableMeta tableMeta = new TableMeta(cNode.getStorageType(), cNode.getOptions());
       try {
         HTableDescriptor hTableDesc = parseHTableDescriptor(tableMeta, cNode.getTableSchema());
         LOG.info("Delete table cause query failed:" + new String(hTableDesc.getName()));
@@ -1127,7 +1134,7 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException  {
+  public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException  {
     if (tableDesc != null) {
       Schema tableSchema = tableDesc.getSchema();
       if (tableSchema.size() != outSchema.size()) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
new file mode 100644
index 0000000..ebf557e
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/SortedInsertRewriter.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.plan.logical.SortNode.SortPurpose;
+import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
+import org.apache.tajo.util.KeyValueSet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This rewrite rule injects a sort operation to preserve the writing rows in
+ * an ascending order of HBase row keys, required by HFile.
+ */
+public class SortedInsertRewriter implements LogicalPlanRewriteRule {
+
+  @Override
+  public String getName() {
+    return "SortedInsertRewriter";
+  }
+
+  @Override
+  public boolean isEligible(OverridableConf queryContext, LogicalPlan plan) {
+    boolean hbaseMode = "false".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"));
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+    LogicalNode node = rootNode.getChild();
+    return hbaseMode && node.getType() == NodeType.CREATE_TABLE || node.getType() == NodeType.INSERT;
+  }
+
+  public static Column[] getIndexColumns(Schema tableSchema, KeyValueSet tableProperty) throws IOException {
+    List<Column> indexColumns = new ArrayList<Column>();
+
+    ColumnMapping columnMapping = new ColumnMapping(tableSchema, tableProperty);
+
+    boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
+    for (int i = 0; i < isRowKeys.length; i++) {
+      if (isRowKeys[i]) {
+        indexColumns.add(tableSchema.getColumn(i));
+      }
+    }
+
+    return indexColumns.toArray(new Column[]{});
+  }
+
+  @Override
+  public LogicalPlan rewrite(OverridableConf queryContext, LogicalPlan plan) throws PlanningException {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+
+    StoreTableNode storeTable = rootNode.getChild();
+    Schema tableSchema = storeTable.getTableSchema();
+
+    Column[] sortColumns;
+    try {
+      sortColumns = getIndexColumns(tableSchema, storeTable.getOptions());
+    } catch (IOException e) {
+      throw new PlanningException(e);
+    }
+
+    int[] sortColumnIndexes = new int[sortColumns.length];
+    for (int i = 0; i < sortColumns.length; i++) {
+      sortColumnIndexes[i] = tableSchema.getColumnId(sortColumns[i].getQualifiedName());
+    }
+
+    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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto b/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
index 668b116..33d45b3 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
+++ b/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
@@ -25,11 +25,12 @@ option java_generate_equals_and_hash = true;
 import "CatalogProtos.proto";
 
 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;
+  required string uri = 1;
+  required string tableName = 2;
+  required string hbaseTableName = 3;
+  required bytes startRow = 4;
+  required bytes stopRow = 5;
+  required bool last = 6;
+  required int64 length = 7;
+  optional string regionLocation = 8;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java
----------------------------------------------------------------------
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
index dd52324..5b1e2bd 100644
--- 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
@@ -43,7 +43,7 @@ public class TestColumnMapping {
 
     TableMeta tableMeta = new TableMeta("HBASE", keyValueSet);
 
-    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta.getOptions());
 
     List<String> cfNames = columnMapping.getColumnFamilyNames();
     assertEquals(2, cfNames.size());

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
----------------------------------------------------------------------
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
deleted file mode 100644
index b59fe7b..0000000
--- a/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
+++ /dev/null
@@ -1,108 +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.hbase;
-
-import org.apache.tajo.catalog.Column;
-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.TableSpaceManager;
-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);
-
-    HBaseTablespace storageManager =
-        (HBaseTablespace) TableSpaceManager.getStorageManager(new TajoConf(), "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());
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java
new file mode 100644
index 0000000..f7cbb5a
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java
@@ -0,0 +1,134 @@
+/**
+ * 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.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.TableSpaceManager;
+import org.apache.tajo.util.Pair;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestHBaseTableSpace {
+  @BeforeClass
+  public static void setUp() throws IOException {
+    String tableSpaceUri = "hbase:zk://host1:2171";
+    HBaseTablespace hBaseTablespace = new HBaseTablespace("cluster1", URI.create(tableSpaceUri));
+    hBaseTablespace.init(new TajoConf());
+    TableSpaceManager.addTableSpaceForTest(hBaseTablespace);
+  }
+
+  @Test
+  public void testExtractQuorum() {
+    assertEquals("host1:2171", HBaseTablespace.extractQuorum(URI.create("hbase:zk://host1:2171")));
+    assertEquals("host1:2171", HBaseTablespace.extractQuorum(URI.create("hbase:zk://host1:2171/table1")));
+    assertEquals("host1:2171,host2:2172",
+        HBaseTablespace.extractQuorum(URI.create("hbase:zk://host1:2171,host2:2172/table1")));
+  }
+
+  @Test
+  public void testTablespaceHandler() throws Exception {
+    assertTrue((TableSpaceManager.getByName("cluster1").get()) instanceof HBaseTablespace);
+    assertTrue((TableSpaceManager.get(URI.create("hbase:zk://host1:2171")).get())
+        instanceof HBaseTablespace);
+  }
+
+  @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);
+
+    HBaseTablespace storageManager = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    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());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/pom.xml b/tajo-storage/tajo-storage-hdfs/pom.xml
index 3a59ec9..9b98b0d 100644
--- a/tajo-storage/tajo-storage-hdfs/pom.xml
+++ b/tajo-storage/tajo-storage-hdfs/pom.xml
@@ -74,6 +74,8 @@
         <configuration>
           <excludes>
             <exclude>src/test/resources/dataset/**</exclude>
+            <exclude>src/main/resources/*.json</exclude>
+            <exclude>src/test/resources/*.json</exclude>
           </excludes>
         </configuration>
       </plugin>
@@ -350,10 +352,6 @@
       <groupId>io.netty</groupId>
       <artifactId>netty-buffer</artifactId>
     </dependency>
-    <dependency>
-      <groupId>net.minidev</groupId>
-      <artifactId>json-smart</artifactId>
-    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
index 5c8242f..081fa3f 100644
--- a/tajo-storage/tajo-storage-hdfs/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,6 +18,7 @@
 
 package org.apache.tajo.storage;
 
+import com.google.common.base.Optional;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -51,20 +52,23 @@ public abstract class FileAppender implements Appender {
     this.workDir = workDir;
     this.taskAttemptId = taskAttemptId;
 
-    try {
-      if (taskAttemptId != null) {
-        if (!(conf instanceof TajoConf)) {
-          throw new IllegalArgumentException("Configuration must be an instance of TajoConf");
-        }
 
-        this.path = ((FileTablespace) TableSpaceManager.getFileStorageManager((TajoConf) conf))
-            .getAppenderFilePath(taskAttemptId, workDir);
-      } else {
-        this.path = workDir;
+    if (taskAttemptId != null) {
+      if (!(conf instanceof TajoConf)) {
+        throw new IllegalArgumentException("Configuration must be an instance of TajoConf");
       }
-    } catch (IOException e) {
-      LOG.error(e.getMessage(), e);
-      throw new IllegalStateException("Error while opeining FileAppender: " + e.getMessage(), e);
+
+      Optional<FileTablespace> spaceResult = TableSpaceManager.get(workDir.toUri());
+
+      if (!spaceResult.isPresent()) {
+        throw new IllegalStateException("No TableSpace for " + workDir.toUri());
+      }
+
+      FileTablespace space = spaceResult.get();
+      this.path = space.getAppenderFilePath(taskAttemptId, workDir);
+
+    } else {
+      this.path = workDir;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
index 6ab8574..2ce1f09 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
@@ -35,18 +35,28 @@ 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.LogicalPlanRewriteRule;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.Bytes;
 import org.apache.tajo.util.TUtil;
 
 import java.io.IOException;
+import java.net.URI;
 import java.text.NumberFormat;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT;
+
 public class FileTablespace extends Tablespace {
+
+  public static final PathFilter hiddenFileFilter = new PathFilter() {
+    public boolean accept(Path p) {
+      String name = p.getName();
+      return !name.startsWith("_") && !name.startsWith(".");
+    }
+  };
   private final Log LOG = LogFactory.getLog(FileTablespace.class);
 
   static final String OUTPUT_FILE_PREFIX="part-";
@@ -83,27 +93,54 @@ public class FileTablespace extends Tablespace {
       };
 
   protected FileSystem fs;
-  protected Path tableBaseDir;
+  protected Path basePath;
   protected boolean blocksMetadataEnabled;
   private static final HdfsVolumeId zeroVolumeId = new HdfsVolumeId(Bytes.toBytes(0));
 
-  public FileTablespace(String storeType) {
-    super(storeType);
+  public FileTablespace(String spaceName, URI uri) {
+    super(spaceName, uri);
   }
 
   @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,
-        DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
-    if (!this.blocksMetadataEnabled)
+    this.basePath = new Path(uri);
+    this.fs = basePath.getFileSystem(conf);
+    this.conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, fs.getUri().toString());
+
+    this.blocksMetadataEnabled =
+        conf.getBoolean(DFS_HDFS_BLOCKS_METADATA_ENABLED, DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
+
+    if (!this.blocksMetadataEnabled) {
       LOG.warn("does not support block metadata. ('dfs.datanode.hdfs-blocks-metadata.enabled')");
+    }
+  }
+
+  @Override
+  public void setConfig(String name, String value) {
+    conf.set(name, value);
+  }
+
+  @Override
+  public void setConfigs(Map<String, String> configs) {
+    for (Map.Entry<String, String> c : configs.entrySet()) {
+      conf.set(c.getKey(), c.getValue());
+    }
+  }
+
+  @Override
+  public long getTableVolume(URI uri) throws IOException {
+    Path path = new Path(uri);
+    ContentSummary summary = fs.getContentSummary(path);
+    return summary.getLength();
+  }
+
+  @Override
+  public URI getRootUri() {
+    return fs.getUri();
   }
 
   public Scanner getFileScanner(TableMeta meta, Schema schema, Path path)
       throws IOException {
-    FileSystem fs = path.getFileSystem(conf);
     FileStatus status = fs.getFileStatus(path);
     return getFileScanner(meta, schema, path, status);
   }
@@ -128,8 +165,9 @@ public class FileTablespace extends Tablespace {
     return fileSystem.exists(path);
   }
 
-  public Path getTablePath(String tableName) {
-    return new Path(tableBaseDir, tableName);
+  @Override
+  public URI getTableUri(String databaseName, String tableName) {
+    return StorageUtil.concatPath(basePath, databaseName, tableName).toUri();
   }
 
   private String partitionPath = "";
@@ -154,12 +192,12 @@ public class FileTablespace extends Tablespace {
   }
 
   public FileFragment[] split(String tableName) throws IOException {
-    Path tablePath = new Path(tableBaseDir, tableName);
+    Path tablePath = new Path(basePath, tableName);
     return split(tableName, tablePath, fs.getDefaultBlockSize());
   }
 
   public FileFragment[] split(String tableName, long fragmentSize) throws IOException {
-    Path tablePath = new Path(tableBaseDir, tableName);
+    Path tablePath = new Path(basePath, tableName);
     return split(tableName, tablePath, fragmentSize);
   }
 
@@ -314,7 +352,6 @@ public class FileTablespace extends Tablespace {
     for (int i = 0; i < dirs.length; ++i) {
       Path p = dirs[i];
 
-      FileSystem fs = p.getFileSystem(conf);
       FileStatus[] matches = fs.globStatus(p, inputFilter);
       if (matches == null) {
         errors.add(new IOException("Input path does not exist: " + p));
@@ -323,8 +360,7 @@ public class FileTablespace extends Tablespace {
       } else {
         for (FileStatus globStat : matches) {
           if (globStat.isDirectory()) {
-            for (FileStatus stat : fs.listStatus(globStat.getPath(),
-                inputFilter)) {
+            for (FileStatus stat : fs.listStatus(globStat.getPath(), inputFilter)) {
               result.add(stat);
             }
           } else {
@@ -492,8 +528,6 @@ public class FileTablespace extends Tablespace {
     List<BlockLocation> blockLocations = Lists.newArrayList();
 
     for (Path p : inputs) {
-      FileSystem fs = p.getFileSystem(conf);
-
       ArrayList<FileStatus> files = Lists.newArrayList();
       if (fs.isFile(p)) {
         files.addAll(Lists.newArrayList(fs.getFileStatus(p)));
@@ -586,7 +620,7 @@ public class FileTablespace extends Tablespace {
       return;
     }
 
-    DistributedFileSystem fs = (DistributedFileSystem)DistributedFileSystem.get(conf);
+    DistributedFileSystem fs = (DistributedFileSystem) this.fs;
     int lsLimit = conf.getInt(DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     int blockLocationIdx = 0;
 
@@ -629,7 +663,7 @@ public class FileTablespace extends Tablespace {
 
   @Override
   public List<Fragment> getSplits(String tableName, TableDesc table, ScanNode scanNode) throws IOException {
-    return getSplits(tableName, table.getMeta(), table.getSchema(), new Path(table.getPath()));
+    return getSplits(tableName, table.getMeta(), table.getSchema(), new Path(table.getUri()));
   }
 
   @Override
@@ -640,13 +674,13 @@ public class FileTablespace extends Tablespace {
       String simpleTableName = splitted[1];
 
       // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} )
-      Path tablePath = StorageUtil.concatPath(tableBaseDir, databaseName, simpleTableName);
-      tableDesc.setPath(tablePath.toUri());
+      Path tablePath = StorageUtil.concatPath(basePath, databaseName, simpleTableName);
+      tableDesc.setUri(tablePath.toUri());
     } else {
-      Preconditions.checkState(tableDesc.getPath() != null, "ERROR: LOCATION must be given.");
+      Preconditions.checkState(tableDesc.getUri() != null, "ERROR: LOCATION must be given.");
     }
 
-    Path path = new Path(tableDesc.getPath());
+    Path path = new Path(tableDesc.getUri());
 
     FileSystem fs = path.getFileSystem(conf);
     TableStats stats = new TableStats();
@@ -679,7 +713,7 @@ public class FileTablespace extends Tablespace {
   @Override
   public void purgeTable(TableDesc tableDesc) throws IOException {
     try {
-      Path path = new Path(tableDesc.getPath());
+      Path path = new Path(tableDesc.getUri());
       FileSystem fs = path.getFileSystem(conf);
       LOG.info("Delete table data dir: " + path);
       fs.delete(path, true);
@@ -692,7 +726,7 @@ public class FileTablespace extends Tablespace {
   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());
+    Path tablePath = new Path(tableDesc.getUri());
     FileSystem fs = tablePath.getFileSystem(conf);
 
     //In the case of partitioned table, we should return same partition key data files.
@@ -704,7 +738,7 @@ public class FileTablespace extends Tablespace {
     List<FileStatus> nonZeroLengthFiles = new ArrayList<FileStatus>();
     if (fs.exists(tablePath)) {
       if (!partitionPath.isEmpty()) {
-        Path partPath = new Path(tableDesc.getPath() + partitionPath);
+        Path partPath = new Path(tableDesc.getUri() + partitionPath);
         if (fs.exists(partPath)) {
           getNonZeroLengthDataFiles(fs, partPath, nonZeroLengthFiles, currentPage, numResultFragments,
                   new AtomicInteger(0), tableDesc.hasPartition(), this.currentDepth, partitionDepth);
@@ -768,7 +802,7 @@ public class FileTablespace extends Tablespace {
     // Intermediate directory
     if (fs.isDirectory(path)) {
 
-      FileStatus[] files = fs.listStatus(path, Tablespace.hiddenFileFilter);
+      FileStatus[] files = fs.listStatus(path, hiddenFileFilter);
 
       if (files != null && files.length > 0) {
 
@@ -817,44 +851,43 @@ public class FileTablespace extends Tablespace {
     }
   }
 
-  @Override
-  public StorageProperty getStorageProperty() {
-    StorageProperty storageProperty = new StorageProperty();
-    storageProperty.setSortedInsert(false);
-    if (storeType.equalsIgnoreCase("RAW")) {
-      storageProperty.setSupportsInsertInto(false);
-    } else {
-      storageProperty.setSupportsInsertInto(true);
-    }
+  private static final StorageProperty FileStorageProperties = new StorageProperty(true, true, true, true);
+  private static final FormatProperty GeneralFileProperties = new FormatProperty(false);
+  private static final FormatProperty HFileProperties = new FormatProperty(true);
 
-    return storageProperty;
+  @Override
+  public StorageProperty getProperty() {
+    return FileStorageProperties;
   }
 
   @Override
-  public void close() {
+  public FormatProperty getFormatProperty(String format) {
+    if (format.equalsIgnoreCase("hbase")) {
+      return HFileProperties;
+    } else {
+      return GeneralFileProperties;
+    }
   }
 
   @Override
-  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
+  public void close() {
   }
 
   @Override
-  public void rollbackOutputCommit(LogicalNode node) throws IOException {
+  public void prepareTable(LogicalNode node) throws IOException {
   }
 
   @Override
-  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException {
+  public void rollbackTable(LogicalNode node) throws IOException {
   }
 
   @Override
-  public List<LogicalPlanRewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc)
-      throws IOException {
-    return null;
+  public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException {
   }
 
   @Override
-  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId, LogicalPlan plan,
-                               Schema schema, TableDesc tableDesc) throws IOException {
+  public Path commitTable(OverridableConf queryContext, ExecutionBlockId finalEbId, LogicalPlan plan,
+                          Schema schema, TableDesc tableDesc) throws IOException {
     return commitOutputData(queryContext, true);
   }
 
@@ -879,8 +912,8 @@ public class FileTablespace extends Tablespace {
     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));
+    if (!queryContext.get(QueryVars.OUTPUT_TABLE_URI, "").isEmpty()) {
+      finalOutputDir = new Path(queryContext.get(QueryVars.OUTPUT_TABLE_URI));
       try {
         FileSystem fs = stagingResultDir.getFileSystem(conf);
 
@@ -949,7 +982,7 @@ public class FileTablespace extends Tablespace {
               if (fs.exists(finalOutputDir)) {
                 fs.mkdirs(oldTableDir);
 
-                for (FileStatus status : fs.listStatus(finalOutputDir, Tablespace.hiddenFileFilter)) {
+                for (FileStatus status : fs.listStatus(finalOutputDir, hiddenFileFilter)) {
                   fs.rename(status.getPath(), oldTableDir);
                 }
 
@@ -971,7 +1004,7 @@ public class FileTablespace extends Tablespace {
               if (movedToOldTable && !committed) {
 
                 // if commit is failed, recover the old data
-                for (FileStatus status : fs.listStatus(finalOutputDir, Tablespace.hiddenFileFilter)) {
+                for (FileStatus status : fs.listStatus(finalOutputDir, hiddenFileFilter)) {
                   fs.delete(status.getPath(), true);
                 }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
index 1b48fc0..7937d38 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
@@ -82,8 +82,9 @@ public class HashShuffleAppenderManager {
         if (!fs.exists(dataFile.getParent())) {
           fs.mkdirs(dataFile.getParent());
         }
-        FileAppender appender = (FileAppender)((FileTablespace) TableSpaceManager.getFileStorageManager(tajoConf))
-            .getAppender(meta, outSchema, dataFile);
+
+        FileTablespace space = (FileTablespace) TableSpaceManager.get(dataFile.toUri()).get();
+        FileAppender appender = (FileAppender) space.getAppender(meta, outSchema, dataFile);
         appender.enableStats();
         appender.init();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
index 75ad0d5..ab63d55 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
@@ -122,7 +122,7 @@ public class TestCompressionStorages {
 
     String fileName = "Compression_" + codec.getSimpleName();
     Path tablePath = new Path(testDir, fileName);
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.enableStats();
 
     appender.init();
@@ -154,7 +154,7 @@ public class TestCompressionStorages {
     FileFragment[] tablets = new FileFragment[1];
     tablets[0] = new FileFragment(fileName, tablePath, 0, fileLen);
 
-    Scanner scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, tablets[0], schema);
 
     if (storeType.equalsIgnoreCase("CSV")) {
       if (SplittableCompressionCodec.class.isAssignableFrom(codec)) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
index 9726ecc..2d919cd 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
@@ -103,7 +103,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "-1");
     FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
-    Scanner scanner =  TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple tuple;
@@ -125,7 +125,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "1");
     FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
-    Scanner scanner =  TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     assertNotNull(scanner.next());
@@ -147,7 +147,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "0");
     FileFragment fragment =  getFileFragment("testErrorTolerance2.json");
-    Scanner scanner =  TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     try {
@@ -166,7 +166,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "1");
     FileFragment fragment = getFileFragment("testErrorTolerance3.json");
-    Scanner scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     try {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java
deleted file mode 100644
index a6d6077..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java
+++ /dev/null
@@ -1,233 +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 com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.*;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.common.TajoDataTypes.Type;
-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.apache.tajo.util.CommonTestingUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.UUID;
-
-import static org.junit.Assert.*;
-
-public class TestFileStorageManager {
-	private TajoConf conf;
-	private static String TEST_PATH = "target/test-data/TestFileStorageManager";
-  private Path testDir;
-  private FileSystem fs;
-
-	@Before
-	public void setUp() throws Exception {
-		conf = new TajoConf();
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-	}
-
-	@After
-	public void tearDown() throws Exception {
-	}
-
-  @Test
-	public final void testGetScannerAndAppender() throws IOException {
-		Schema schema = new Schema();
-		schema.addColumn("id", Type.INT4);
-		schema.addColumn("age",Type.INT4);
-		schema.addColumn("name",Type.TEXT);
-
-		TableMeta meta = CatalogUtil.newTableMeta("CSV");
-		
-		VTuple[] tuples = new VTuple[4];
-		for(int i=0; i < tuples.length; i++) {
-		  tuples[i] = new VTuple(new Datum[] {
-          DatumFactory.createInt4(i),
-		      DatumFactory.createInt4(i + 32),
-		      DatumFactory.createText("name" + i)});
-		}
-
-    Path path = StorageUtil.concatPath(testDir, "testGetScannerAndAppender", "table.csv");
-    fs.mkdirs(path.getParent());
-    FileTablespace fileStorageManager = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
-    assertEquals(fs.getUri(), fileStorageManager.getFileSystem().getUri());
-
-		Appender appender = fileStorageManager.getAppender(meta, schema, path);
-    appender.init();
-		for(Tuple t : tuples) {
-		  appender.addTuple(t);
-		}
-		appender.close();
-
-		Scanner scanner = fileStorageManager.getFileScanner(meta, schema, path);
-    scanner.init();
-		int i=0;
-		while(scanner.next() != null) {
-			i++;
-		}
-		assertEquals(4,i);
-	}
-
-  @Test
-  public void testGetSplit() throws Exception {
-    final Configuration conf = new HdfsConfiguration();
-    String testDataPath = TEST_PATH + "/" + UUID.randomUUID().toString();
-    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, testDataPath);
-    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
-    conf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, false);
-
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(1).build();
-    cluster.waitClusterUp();
-    TajoConf tajoConf = new TajoConf(conf);
-    tajoConf.setVar(TajoConf.ConfVars.ROOT_DIR, cluster.getFileSystem().getUri() + "/tajo");
-
-    int testCount = 10;
-    Path tablePath = new Path("/testGetSplit");
-    try {
-      DistributedFileSystem fs = cluster.getFileSystem();
-
-      // Create test partitions
-      List<Path> partitions = Lists.newArrayList();
-      for (int i =0; i < testCount; i++){
-        Path tmpFile = new Path(tablePath, String.valueOf(i));
-        DFSTestUtil.createFile(fs, new Path(tmpFile, "tmpfile.dat"), 10, (short) 2, 0xDEADDEADl);
-        partitions.add(tmpFile);
-      }
-
-      assertTrue(fs.exists(tablePath));
-      FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(tajoConf);
-      assertEquals(fs.getUri(), sm.getFileSystem().getUri());
-
-      Schema schema = new Schema();
-      schema.addColumn("id", Type.INT4);
-      schema.addColumn("age",Type.INT4);
-      schema.addColumn("name",Type.TEXT);
-      TableMeta meta = CatalogUtil.newTableMeta("CSV");
-
-      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, ((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, ((FileFragment)splits.get(0)).getDiskIds()[0]);
-      fs.close();
-    } finally {
-      cluster.shutdown(true);
-    }
-  }
-
-  @Test
-  public void testGetSplitWithBlockStorageLocationsBatching() throws Exception {
-    final Configuration conf = new HdfsConfiguration();
-    String testDataPath = TEST_PATH + "/" + UUID.randomUUID().toString();
-    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, testDataPath);
-    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
-    conf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, true);
-
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(2).build();
-    cluster.waitClusterUp();
-
-    TajoConf tajoConf = new TajoConf(conf);
-    tajoConf.setVar(TajoConf.ConfVars.ROOT_DIR, cluster.getFileSystem().getUri() + "/tajo");
-
-    int testCount = 10;
-    Path tablePath = new Path("/testGetSplitWithBlockStorageLocationsBatching");
-    try {
-      DistributedFileSystem fs = cluster.getFileSystem();
-
-      // Create test files
-      for (int i = 0; i < testCount; i++) {
-        Path tmpFile = new Path(tablePath, "tmpfile" + i + ".dat");
-        DFSTestUtil.createFile(fs, tmpFile, 10, (short) 2, 0xDEADDEADl);
-      }
-      assertTrue(fs.exists(tablePath));
-      FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(tajoConf);
-      assertEquals(fs.getUri(), sm.getFileSystem().getUri());
-
-      Schema schema = new Schema();
-      schema.addColumn("id", Type.INT4);
-      schema.addColumn("age", Type.INT4);
-      schema.addColumn("name", Type.TEXT);
-      TableMeta meta = CatalogUtil.newTableMeta("CSV");
-
-      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, ((FileFragment)splits.get(0)).getDiskIds().length);
-      assertNotEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]);
-      fs.close();
-    } finally {
-      cluster.shutdown(true);
-    }
-  }
-
-  @Test
-  public void testStoreType() throws Exception {
-    final Configuration hdfsConf = new HdfsConfiguration();
-    String testDataPath = TEST_PATH + "/" + UUID.randomUUID().toString();
-    hdfsConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, testDataPath);
-    hdfsConf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
-    hdfsConf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, true);
-
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(hdfsConf)
-        .numDataNodes(2).build();
-    cluster.waitClusterUp();
-
-    TajoConf tajoConf = new TajoConf(hdfsConf);
-    tajoConf.setVar(TajoConf.ConfVars.ROOT_DIR, cluster.getFileSystem().getUri() + "/tajo");
-
-    try {
-      /* Local FileSystem */
-      FileTablespace sm = (FileTablespace) TableSpaceManager.getStorageManager(conf, "CSV");
-      assertEquals(fs.getUri(), sm.getFileSystem().getUri());
-
-      /* Distributed FileSystem */
-      sm = (FileTablespace) TableSpaceManager.getStorageManager(tajoConf, "CSV");
-      assertNotEquals(fs.getUri(), sm.getFileSystem().getUri());
-      assertEquals(cluster.getFileSystem().getUri(), sm.getFileSystem().getUri());
-    } finally {
-      cluster.shutdown(true);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
index a6c238b..9237e07 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
@@ -57,7 +57,7 @@ public class TestFileSystems {
   public TestFileSystems(FileSystem fs) throws IOException {
     this.fs = fs;
     this.conf = new TajoConf(fs.getConf());
-    sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    sm = TableSpaceManager.getLocalFs();
     testDir = getTestDir(this.fs, TEST_PATH);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java
new file mode 100644
index 0000000..ec3e143
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java
@@ -0,0 +1,250 @@
+/**
+ * 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.base.Optional;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.*;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.common.TajoDataTypes.Type;
+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.apache.tajo.util.CommonTestingUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+
+public class TestFileTablespace {
+	private TajoConf conf;
+	private static String TEST_PATH = "target/test-data/TestFileTablespace";
+  private Path testDir;
+  private FileSystem localFs;
+
+	@Before
+	public void setUp() throws Exception {
+		conf = new TajoConf();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    localFs = testDir.getFileSystem(conf);
+	}
+
+	@After
+	public void tearDown() throws Exception {
+	}
+
+  @Test
+	public final void testGetScannerAndAppender() throws IOException {
+		Schema schema = new Schema();
+		schema.addColumn("id", Type.INT4);
+		schema.addColumn("age",Type.INT4);
+		schema.addColumn("name",Type.TEXT);
+
+		TableMeta meta = CatalogUtil.newTableMeta("CSV");
+		
+		VTuple[] tuples = new VTuple[4];
+		for(int i=0; i < tuples.length; i++) {
+		  tuples[i] = new VTuple(new Datum[] {
+          DatumFactory.createInt4(i),
+		      DatumFactory.createInt4(i + 32),
+		      DatumFactory.createText("name" + i)});
+		}
+
+    Path path = StorageUtil.concatPath(testDir, "testGetScannerAndAppender", "table.csv");
+    localFs.mkdirs(path.getParent());
+    FileTablespace fileStorageManager = (FileTablespace) TableSpaceManager.getLocalFs();
+    assertEquals(localFs.getUri(), fileStorageManager.getFileSystem().getUri());
+
+		Appender appender = fileStorageManager.getAppender(meta, schema, path);
+    appender.init();
+		for(Tuple t : tuples) {
+		  appender.addTuple(t);
+		}
+		appender.close();
+
+		Scanner scanner = fileStorageManager.getFileScanner(meta, schema, path);
+    scanner.init();
+		int i=0;
+		while(scanner.next() != null) {
+			i++;
+		}
+		assertEquals(4,i);
+	}
+
+  @Test
+  public void testGetSplit() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    String testDataPath = TEST_PATH + "/" + UUID.randomUUID().toString();
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, testDataPath);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
+    conf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, false);
+
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(1).build();
+    cluster.waitClusterUp();
+    TajoConf tajoConf = new TajoConf(conf);
+    tajoConf.setVar(TajoConf.ConfVars.ROOT_DIR, cluster.getFileSystem().getUri() + "/tajo");
+
+    int testCount = 10;
+    Path tablePath = new Path("/testGetSplit");
+    try {
+      DistributedFileSystem fs = cluster.getFileSystem();
+
+      // Create test partitions
+      List<Path> partitions = Lists.newArrayList();
+      for (int i =0; i < testCount; i++){
+        Path tmpFile = new Path(tablePath, String.valueOf(i));
+        DFSTestUtil.createFile(fs, new Path(tmpFile, "tmpfile.dat"), 10, (short) 2, 0xDEADDEADl);
+        partitions.add(tmpFile);
+      }
+
+      assertTrue(fs.exists(tablePath));
+      FileTablespace space = new FileTablespace("testGetSplit", fs.getUri());
+      space.init(new TajoConf(conf));
+      assertEquals(fs.getUri(), space.getUri());
+
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age",Type.INT4);
+      schema.addColumn("name",Type.TEXT);
+      TableMeta meta = CatalogUtil.newTableMeta("CSV");
+
+      List<Fragment> splits = Lists.newArrayList();
+      // Get FileFragments in partition batch
+      splits.addAll(space.getSplits("data", meta, schema, partitions.toArray(new Path[partitions.size()])));
+      assertEquals(testCount, splits.size());
+      // -1 is unknown volumeId
+      assertEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]);
+
+      splits.clear();
+      splits.addAll(space.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, ((FileFragment)splits.get(0)).getDiskIds()[0]);
+      fs.close();
+    } finally {
+      cluster.shutdown(true);
+    }
+  }
+
+  @Test
+  public void testGetSplitWithBlockStorageLocationsBatching() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    String testDataPath = TEST_PATH + "/" + UUID.randomUUID().toString();
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, testDataPath);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
+    conf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, true);
+
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(2).build();
+    cluster.waitClusterUp();
+
+    TajoConf tajoConf = new TajoConf(conf);
+    tajoConf.setVar(TajoConf.ConfVars.ROOT_DIR, cluster.getFileSystem().getUri() + "/tajo");
+
+    int testCount = 10;
+    Path tablePath = new Path("/testGetSplitWithBlockStorageLocationsBatching");
+    try {
+      DistributedFileSystem fs = cluster.getFileSystem();
+
+      // Create test files
+      for (int i = 0; i < testCount; i++) {
+        Path tmpFile = new Path(tablePath, "tmpfile" + i + ".dat");
+        DFSTestUtil.createFile(fs, tmpFile, 10, (short) 2, 0xDEADDEADl);
+      }
+      assertTrue(fs.exists(tablePath));
+
+      FileTablespace sm = new FileTablespace("testGetSplitWithBlockStorageLocationsBatching", fs.getUri());
+      sm.init(new TajoConf(conf));
+
+      assertEquals(fs.getUri(), sm.getUri());
+
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age", Type.INT4);
+      schema.addColumn("name", Type.TEXT);
+      TableMeta meta = CatalogUtil.newTableMeta("CSV");
+
+      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, ((FileFragment)splits.get(0)).getDiskIds().length);
+      assertNotEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]);
+      fs.close();
+    } finally {
+      cluster.shutdown(true);
+    }
+  }
+
+  @Test
+  public void testGetFileTablespace() throws Exception {
+    final Configuration hdfsConf = new HdfsConfiguration();
+    String testDataPath = TEST_PATH + "/" + UUID.randomUUID().toString();
+    hdfsConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, testDataPath);
+    hdfsConf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
+    hdfsConf.setBoolean(DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, true);
+
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(hdfsConf).numDataNodes(2).build();
+    cluster.waitClusterUp();
+    URI uri = URI.create(cluster.getFileSystem().getUri() + "/tajo");
+
+    Optional<Tablespace> existingTs = Optional.absent();
+    try {
+      /* Local FileSystem */
+      FileTablespace space = TableSpaceManager.getLocalFs();
+      assertEquals(localFs.getUri(), space.getFileSystem().getUri());
+
+      FileTablespace distTablespace = new FileTablespace("testGetFileTablespace", uri);
+      distTablespace.init(conf);
+      existingTs = TableSpaceManager.addTableSpaceForTest(distTablespace);
+
+      /* Distributed FileSystem */
+      space = (FileTablespace) TableSpaceManager.get(uri).get();
+      assertEquals(cluster.getFileSystem().getUri(), space.getFileSystem().getUri());
+
+      space = (FileTablespace) TableSpaceManager.getByName("testGetFileTablespace").get();
+      assertEquals(cluster.getFileSystem().getUri(), space.getFileSystem().getUri());
+
+    } finally {
+
+      if (existingTs.isPresent()) {
+        TableSpaceManager.addTableSpaceForTest(existingTs.get());
+      }
+
+      cluster.shutdown(true);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
index 266f906..c13ce16 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
@@ -65,7 +65,7 @@ public class TestLineReader {
 
     TableMeta meta = CatalogUtil.newTableMeta("TEXT");
     Path tablePath = new Path(testDir, "line.data");
-    FileAppender appender = (FileAppender) TableSpaceManager.getFileStorageManager(conf).getAppender(
+    FileAppender appender = (FileAppender) TableSpaceManager.getLocalFs().getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -118,7 +118,7 @@ public class TestLineReader {
     meta.putOption("compression.codec", DeflateCodec.class.getCanonicalName());
 
     Path tablePath = new Path(testDir, "testLineDelimitedReaderWithCompression." + DeflateCodec.class.getSimpleName());
-    FileAppender appender = (FileAppender) TableSpaceManager.getFileStorageManager(conf).getAppender(
+    FileAppender appender = (FileAppender) (TableSpaceManager.getLocalFs()).getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -176,7 +176,7 @@ public class TestLineReader {
     TableMeta meta = CatalogUtil.newTableMeta("TEXT");
 
     Path tablePath = new Path(testDir, "testLineDelimitedReader");
-    FileAppender appender = (FileAppender) TableSpaceManager.getFileStorageManager(conf).getAppender(
+    FileAppender appender = (FileAppender) TableSpaceManager.getLocalFs().getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -279,7 +279,7 @@ public class TestLineReader {
 
     TableMeta meta = CatalogUtil.newTableMeta("TEXT");
     Path tablePath = new Path(testDir, "testSeekableByteBufLineReader.data");
-    FileAppender appender = (FileAppender) TableSpaceManager.getFileStorageManager(conf).getAppender(
+    FileAppender appender = (FileAppender) TableSpaceManager.getLocalFs().getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
index 03a601d..79928ff 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
@@ -94,7 +94,7 @@ public class TestMergeScanner {
     conf.setStrings("tajo.storage.projectable-scanner", "rcfile", "parquet", "avro");
     testDir = CommonTestingUtil.getTestDir(TEST_PATH);
     fs = testDir.getFileSystem(conf);
-    sm = TableSpaceManager.getFileStorageManager(conf);
+    sm = TableSpaceManager.getLocalFs();
   }
 
   @Test
@@ -114,7 +114,7 @@ public class TestMergeScanner {
     }
 
     Path table1Path = new Path(testDir, storeType + "_1.data");
-    Appender appender1 = TableSpaceManager.getFileStorageManager(conf).getAppender(null, null, meta, schema, table1Path);
+    Appender appender1 = TableSpaceManager.getLocalFs().getAppender(null, null, meta, schema, table1Path);
     appender1.enableStats();
     appender1.init();
     int tupleNum = 10000;
@@ -136,7 +136,7 @@ public class TestMergeScanner {
     }
 
     Path table2Path = new Path(testDir, storeType + "_2.data");
-    Appender appender2 = TableSpaceManager.getFileStorageManager(conf).getAppender(null, null, meta, schema, table2Path);
+    Appender appender2 = TableSpaceManager.getLocalFs().getAppender(null, null, meta, schema, table2Path);
     appender2.enableStats();
     appender2.init();
 


[11/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces. (missed commits)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
----------------------------------------------------------------------
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 ae90502..afa273b 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
@@ -84,7 +84,7 @@ public class TestHashSemiJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -110,7 +110,7 @@ public class TestHashSemiJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -133,7 +133,7 @@ public class TestHashSemiJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
----------------------------------------------------------------------
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 bbb441c..c93a1b4 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
@@ -105,7 +105,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -134,7 +134,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender2 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
@@ -174,7 +174,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender3 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
@@ -227,7 +227,7 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender5 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     
@@ -239,7 +239,7 @@ public class TestLeftOuterHashJoinExec {
 
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
----------------------------------------------------------------------
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 d0d0983..c4e7752 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
@@ -85,7 +85,7 @@ public class TestMergeJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -114,7 +114,7 @@ public class TestMergeJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -139,7 +139,7 @@ public class TestMergeJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
----------------------------------------------------------------------
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 4866323..1b30ef8 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
@@ -83,7 +83,7 @@ public class TestNLJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(schema.size());
@@ -107,7 +107,7 @@ public class TestNLJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -125,7 +125,7 @@ public class TestNLJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 
     masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
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 b2a228a..dff0cbe 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
@@ -98,7 +98,7 @@ public class TestPhysicalPlanner {
     util.startCatalogCluster();
     conf = util.getConfiguration();
     testDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/TestPhysicalPlanner");
-    sm = TableSpaceManager.getLocalFs();
+    sm = TablespaceManager.getLocalFs();
     catalog = util.getMiniCatalogCluster().getCatalog();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
@@ -164,7 +164,7 @@ public class TestPhysicalPlanner {
     appender.close();
     catalog.createTable(score);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
@@ -180,7 +180,7 @@ public class TestPhysicalPlanner {
 
     Schema scoreSchmea = score.getSchema();
     TableMeta scoreLargeMeta = CatalogUtil.newTableMeta("RAW", new KeyValueSet());
-    Appender appender =  ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender =  ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(scoreLargeMeta, scoreSchmea, scoreLargePath);
     appender.enableStats();
     appender.init();
@@ -442,7 +442,7 @@ public class TestPhysicalPlanner {
     exec.next();
     exec.close();
 
-    Scanner scanner =  ((FileTablespace) TableSpaceManager.getLocalFs())
+    Scanner scanner =  ((FileTablespace) TablespaceManager.getLocalFs())
         .getFileScanner(outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;
@@ -502,7 +502,7 @@ public class TestPhysicalPlanner {
     // checking the file contents
     long totalNum = 0;
     for (FileStatus status : fs.listStatus(ctx.getOutputPath().getParent())) {
-      Scanner scanner =  ((FileTablespace) TableSpaceManager.getLocalFs()).getFileScanner(
+      Scanner scanner =  ((FileTablespace) TablespaceManager.getLocalFs()).getFileScanner(
           CatalogUtil.newTableMeta("CSV"),
           rootNode.getOutSchema(),
           status.getPath());
@@ -539,7 +539,7 @@ public class TestPhysicalPlanner {
     exec.next();
     exec.close();
 
-    Scanner scanner = ((FileTablespace) TableSpaceManager.getLocalFs()).getFileScanner(
+    Scanner scanner = ((FileTablespace) TablespaceManager.getLocalFs()).getFileScanner(
         outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
----------------------------------------------------------------------
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 1b54948..d1da787 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
@@ -86,7 +86,7 @@ public class TestProgressExternalSortExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("RAW");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
@@ -110,7 +110,7 @@ public class TestProgressExternalSortExec {
         employeePath.toUri());
     catalog.createTable(employee);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
----------------------------------------------------------------------
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 c956f29..f581db8 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
@@ -100,7 +100,7 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender1 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
@@ -130,7 +130,7 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender2 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
@@ -170,7 +170,7 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender3 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
@@ -212,7 +212,7 @@ public class TestRightOuterHashJoinExec {
     catalog.createTable(emp3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
----------------------------------------------------------------------
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 25f0ca4..d86b229 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
@@ -107,7 +107,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender1 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
@@ -146,7 +146,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta("CSV");
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender4 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     VTuple tuple4 = new VTuple(dep4Schema.size());
@@ -178,7 +178,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender2 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
@@ -218,7 +218,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender3 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
@@ -271,7 +271,7 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender5 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
@@ -281,7 +281,7 @@ public class TestRightOuterMergeJoinExec {
     catalog.createTable(phone3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
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 ce12faf..4690e71 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
@@ -69,7 +69,7 @@ public class TestSortExec {
     util = TpchTestBase.getInstance().getTestingCluster();
     catalog = util.getMaster().getCatalog();
     workDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    sm = TableSpaceManager.getLocalFs();
+    sm = TablespaceManager.getLocalFs();
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -81,7 +81,7 @@ public class TestSortExec {
     tablePath = StorageUtil.concatPath(workDir, "employee", "table1");
     sm.getFileSystem().mkdirs(tablePath.getParent());
 
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, tablePath);
     appender.init();
     VTuple tuple = new VTuple(schema.size());
@@ -101,7 +101,7 @@ public class TestSortExec {
     catalog.createTable(desc);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
----------------------------------------------------------------------
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
index 3d2d857..569111c 100644
--- 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
@@ -38,7 +38,7 @@ 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.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.hbase.*;
@@ -82,7 +82,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
     tableSpaceUri = "hbase:zk://" + hostName + ":" + zkPort;
     HBaseTablespace hBaseTablespace = new HBaseTablespace("cluster1", URI.create(tableSpaceUri));
     hBaseTablespace.init(new TajoConf(testingCluster.getHBaseUtil().getConf()));
-    TableSpaceManager.addTableSpaceForTest(hBaseTablespace);
+    TablespaceManager.addTableSpaceForTest(hBaseTablespace);
   }
 
   @AfterClass
@@ -213,7 +213,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
     assertTableExists("external_hbase_mapped_table");
 
-    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get();
     HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
@@ -253,7 +253,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
     assertTableExists("external_hbase_mapped_table");
 
-    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get();
     HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
@@ -306,7 +306,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
     assertTableExists("external_hbase_mapped_table");
 
-    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get();
     HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
@@ -343,7 +343,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
     assertTableExists("external_hbase_mapped_table");
 
-    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HBaseTablespace space = (HBaseTablespace) TablespaceManager.getByName("cluster1").get();
     HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
@@ -477,7 +477,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
     EvalNode evalNodeEq = new BinaryEval(EvalType.EQUAL, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
         new ConstEval(new TextDatum("021")));
     scanNode.setQual(evalNodeEq);
-    Tablespace tablespace = TableSpaceManager.getByName("cluster1").get();
+    Tablespace tablespace = TablespaceManager.getByName("cluster1").get();
     List<Fragment> fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode);
     assertEquals(1, fragments.size());
     assertEquals("021", new String(((HBaseFragment)fragments.get(0)).getStartRow()));
@@ -683,6 +683,48 @@ public class TestHBaseTable extends QueryTestCaseBase {
   }
 
   @Test
+  public void testInsertValues1() throws Exception {
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) " +
+        "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close();
+
+    assertTableExists("hbase_mapped_table");
+    TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
+
+    executeString("insert into hbase_mapped_table select 'aaa', 'a12', 'a34', 1").close();
+    executeString("insert into hbase_mapped_table select 'bbb', 'b12', 'b34', 2").close();
+    executeString("insert into hbase_mapped_table select 'ccc', 'c12', 'c34', 3").close();
+    executeString("insert into hbase_mapped_table select 'ddd', 'd12', 'd34', 4").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 {
     executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " +
         "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a', " +
@@ -1301,10 +1343,8 @@ public class TestHBaseTable extends QueryTestCaseBase {
     }
   }
 
-  private String resultSetToString(ResultScanner scanner,
-                                   byte[][] cfNames, byte[][] qualifiers,
-                                   boolean[] binaries,
-                                   Schema schema) throws Exception {
+  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 ) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
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 1478690..dd67e06 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
@@ -264,7 +264,7 @@ public class TestJoinQuery extends QueryTestCaseBase {
         }
         Path dataPath = new Path(table.getUri().toString(), fileIndex + ".csv");
         fileIndex++;
-        appender = (((FileTablespace)TableSpaceManager.getLocalFs()))
+        appender = (((FileTablespace) TablespaceManager.getLocalFs()))
             .getAppender(tableMeta, schema, dataPath);
         appender.init();
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
index c714749..265f075 100644
--- a/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
+++ b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
@@ -20,10 +20,13 @@ package org.apache.tajo.ha;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.TpchTestBase;
 import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.service.ServiceTracker;
@@ -32,58 +35,51 @@ import org.junit.Test;
 
 import static junit.framework.Assert.assertTrue;
 import static junit.framework.TestCase.assertEquals;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertNotEquals;
 
 public class TestHAServiceHDFSImpl  {
   private TajoTestingCluster cluster;
-  private TajoMaster backupMaster;
 
-  private TajoConf conf;
-  private TajoClient client;
+  private TajoMaster primaryMaster;
+  private TajoMaster backupMaster;
 
   private Path haPath, activePath, backupPath;
 
-  private String masterAddress;
-
   @Test
   public final void testAutoFailOver() throws Exception {
-    cluster = new TajoTestingCluster(true);
-
-    cluster.startMiniCluster(1);
-    conf = cluster.getConfiguration();
-    client = cluster.newTajoClient();
+    cluster = TpchTestBase.getInstance().getTestingCluster();
 
     try {
       FileSystem fs = cluster.getDefaultFileSystem();
 
-      ServiceTracker serviceTracker = ServiceTrackerFactory.get(conf);
-      masterAddress = serviceTracker.getUmbilicalAddress().getHostName();
-
-      setConfiguration();
+      TajoConf primaryConf = setConfigForHAMaster();
+      primaryMaster = new TajoMaster();
+      primaryMaster.init(primaryConf);
+      primaryMaster.start();
 
+      TajoConf backupConf = setConfigForHAMaster();
       backupMaster = new TajoMaster();
-      backupMaster.init(conf);
+      backupMaster.init(backupConf);
       backupMaster.start();
 
-      assertNotEquals(cluster.getMaster().getMasterName(), backupMaster.getMasterName());
+      ServiceTracker tracker = ServiceTrackerFactory.get(primaryConf);
 
+      assertNotEquals(primaryMaster.getMasterName(), backupMaster.getMasterName());
       verifySystemDirectories(fs);
 
       assertEquals(2, fs.listStatus(activePath).length);
       assertEquals(1, fs.listStatus(backupPath).length);
 
       assertTrue(fs.exists(new Path(activePath, HAConstants.ACTIVE_LOCK_FILE)));
-      assertTrue(fs.exists(new Path(activePath, cluster.getMaster().getMasterName().replaceAll(":", "_"))));
+      assertTrue(fs.exists(new Path(activePath, primaryMaster.getMasterName().replaceAll(":", "_"))));
       assertTrue(fs.exists(new Path(backupPath, backupMaster.getMasterName().replaceAll(":", "_"))));
 
-      createDatabaseAndTable();
-      verifyDataBaseAndTable();
-      client.close();
+      createDatabaseAndTable(tracker);
+      verifyDataBaseAndTable(tracker);
 
-      cluster.getMaster().stop();
+      primaryMaster.stop();
 
-      client = cluster.newTajoClient();
-      verifyDataBaseAndTable();
+      verifyDataBaseAndTable(tracker);
 
       assertEquals(2, fs.listStatus(activePath).length);
       assertEquals(0, fs.listStatus(backupPath).length);
@@ -91,25 +87,23 @@ public class TestHAServiceHDFSImpl  {
       assertTrue(fs.exists(new Path(activePath, HAConstants.ACTIVE_LOCK_FILE)));
       assertTrue(fs.exists(new Path(activePath, backupMaster.getMasterName().replaceAll(":", "_"))));
     } finally {
-      client.close();
       backupMaster.stop();
-      cluster.shutdownMiniCluster();
     }
   }
 
-  private void setConfiguration() {
-    conf = cluster.getConfiguration();
+  private TajoConf setConfigForHAMaster() {
+    TajoConf conf = new TajoConf(cluster.getConfiguration());
 
     conf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS,
-      masterAddress + ":" + NetUtils.getFreeSocketPort());
+        "localhost:" + NetUtils.getFreeSocketPort());
     conf.setVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS,
-      masterAddress + ":" + NetUtils.getFreeSocketPort());
+        "localhost:" + NetUtils.getFreeSocketPort());
     conf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS,
-      masterAddress + ":" + NetUtils.getFreeSocketPort());
+        "localhost:" + NetUtils.getFreeSocketPort());
     conf.setVar(TajoConf.ConfVars.CATALOG_ADDRESS,
-      masterAddress + ":" + NetUtils.getFreeSocketPort());
+        "localhost:" + NetUtils.getFreeSocketPort());
     conf.setVar(TajoConf.ConfVars.TAJO_MASTER_INFO_ADDRESS,
-        masterAddress + ":" + NetUtils.getFreeSocketPort());
+        "localhost:" + NetUtils.getFreeSocketPort());
     conf.setIntVar(TajoConf.ConfVars.REST_SERVICE_PORT,
         NetUtils.getFreeSocketPort());
 
@@ -126,6 +120,8 @@ public class TestHAServiceHDFSImpl  {
     conf.setIntVar(TajoConf.ConfVars.WORKER_RPC_SERVER_WORKER_THREAD_NUM, 2);
     conf.setIntVar(TajoConf.ConfVars.CATALOG_RPC_SERVER_WORKER_THREAD_NUM, 2);
     conf.setIntVar(TajoConf.ConfVars.SHUFFLE_RPC_SERVER_WORKER_THREAD_NUM, 2);
+
+    return conf;
   }
 
   private void verifySystemDirectories(FileSystem fs) throws Exception {
@@ -139,14 +135,26 @@ public class TestHAServiceHDFSImpl  {
     assertTrue(fs.exists(backupPath));
   }
 
-  private void createDatabaseAndTable() throws Exception {
-    client.executeQuery("CREATE TABLE default.table1 (age int);");
-    client.executeQuery("CREATE TABLE default.table2 (age int);");
+  private void createDatabaseAndTable(ServiceTracker tracker) throws Exception {
+    TajoClient client = null;
+    try {
+      client = new TajoClientImpl(tracker);
+      client.executeQuery("CREATE TABLE default.ha_test1 (age int);");
+      client.executeQuery("CREATE TABLE default.ha_test2 (age int);");
+    } finally {
+      IOUtils.cleanup(null, client);
+    }
   }
 
-  private void verifyDataBaseAndTable() throws Exception {
-    client.existDatabase("default");
-    client.existTable("default.table1");
-    client.existTable("default.table2");
+  private void verifyDataBaseAndTable(ServiceTracker tracker) throws Exception {
+    TajoClient client = null;
+    try {
+      client = new TajoClientImpl(tracker);
+      client.existDatabase("default");
+      client.existTable("default.ha_test1");
+      client.existTable("default.ha_test2");
+    } finally {
+      IOUtils.cleanup(null, client);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
----------------------------------------------------------------------
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 fc25c27..3d32c08 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
@@ -66,7 +66,7 @@ public class TestResultSet {
   public static void setup() throws Exception {
     util = TpchTestBase.getInstance().getTestingCluster();
     conf = util.getConfiguration();
-    sm = TableSpaceManager.getDefault();
+    sm = TablespaceManager.getDefault();
 
     scoreSchema = new Schema();
     scoreSchema.addColumn("deptname", Type.TEXT);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
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 48966bc..7c61cc7 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
@@ -34,7 +34,7 @@ import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.plan.LogicalOptimizer;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -79,7 +79,7 @@ public class TestExecutionBlockCursor {
     }
 
     analyzer = new SQLAnalyzer();
-    logicalPlanner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    logicalPlanner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
 
     dispatcher = new AsyncDispatcher();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
index 6322732..e8d59d0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
@@ -22,18 +22,17 @@ import static org.junit.Assert.*;
 import static org.hamcrest.CoreMatchers.*;
 
 import java.io.File;
+import java.sql.ResultSet;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.tajo.LocalTajoTestingUtility;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.*;
 import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.benchmark.TPCH;
 import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.client.ResultSetUtil;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
@@ -62,256 +61,21 @@ import org.junit.Test;
 
 import com.google.protobuf.ByteString;
 
-public class TestNonForwardQueryResultSystemScanner {
-  
-  private class CollectionMatcher<T> extends TypeSafeDiagnosingMatcher<Iterable<? extends T>> {
-    
-    private final Matcher<? extends T> matcher;
-    
-    public CollectionMatcher(Matcher<? extends T> matcher) {
-      this.matcher = matcher;
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("a collection containing ").appendDescriptionOf(this.matcher);
-    }
-
-    @Override
-    protected boolean matchesSafely(Iterable<? extends T> item, Description mismatchDescription) {
-      boolean isFirst = true;
-      Iterator<? extends T> iterator = item.iterator();
-      
-      while (iterator.hasNext()) {
-        T obj = iterator.next();
-        if (this.matcher.matches(obj)) {
-          return true;
-        }
-        
-        if (!isFirst) {
-          mismatchDescription.appendText(", ");
-        }
-        
-        this.matcher.describeMismatch(obj, mismatchDescription);
-        isFirst = false;
-      }
-      return false;
-    }
-    
-  }
-  
-  private <T> Matcher<Iterable<? extends T>> hasItem(Matcher<? extends T> matcher) {
-    return new CollectionMatcher<T>(matcher);
-  }
-
-  private static LocalTajoTestingUtility testUtil;
-  private static TajoTestingCluster testingCluster;
-  private static TajoConf conf;
-  private static MasterContext masterContext;
-  
-  private static SQLAnalyzer analyzer;
-  private static LogicalPlanner logicalPlanner;
-  private static LogicalOptimizer logicalOptimizer;
-  
-  private static void setupTestingCluster() throws Exception {
-    testUtil = new LocalTajoTestingUtility();
-    String[] names, paths;
-    Schema[] schemas;
-    
-    TPCH tpch = new TPCH();
-    tpch.loadSchemas();
-    tpch.loadQueries();
-    
-    names = new String[] {"customer", "lineitem", "nation", "orders", "part", "partsupp", 
-        "region", "supplier", "empty_orders"};
-    schemas = new Schema[names.length];
-    for (int i = 0; i < names.length; i++) {
-      schemas[i] = tpch.getSchema(names[i]);
-    }
-
-    File file;
-    paths = new String[names.length];
-    for (int i = 0; i < names.length; i++) {
-      file = new File("src/test/tpch/" + names[i] + ".tbl");
-      if(!file.exists()) {
-        file = new File(System.getProperty("user.dir") + "/tajo-core/src/test/tpch/" + names[i]
-            + ".tbl");
-      }
-      paths[i] = file.getAbsolutePath();
-    }
-    
-    KeyValueSet opt = new KeyValueSet();
-    opt.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    testUtil.setup(names, paths, schemas, opt);
-    
-    testingCluster = testUtil.getTestingCluster();
-  }
-  
-  @BeforeClass
-  public static void setUp() throws Exception {
-    setupTestingCluster();
-    
-    conf = testingCluster.getConfiguration();
-    masterContext = testingCluster.getMaster().getContext();
-    
-    GlobalEngine globalEngine = masterContext.getGlobalEngine();
-    analyzer = globalEngine.getAnalyzer();
-    logicalPlanner = globalEngine.getLogicalPlanner();
-    logicalOptimizer = globalEngine.getLogicalOptimizer();
-  }
-  
-  @AfterClass
-  public static void tearDown() throws Exception {
-    try {
-      Thread.sleep(2000);
-    } catch (Exception ignored) {
-    }
-    
-    testUtil.shutdown();
-  }
-  
-  private NonForwardQueryResultScanner getScanner(String sql) throws Exception {
-    QueryId queryId = QueryIdFactory.newQueryId(masterContext.getResourceManager().getSeedQueryId());
-    String sessionId = UUID.randomUUID().toString();
-    
-    return getScanner(sql, queryId, sessionId);
-  }
-  
-  private NonForwardQueryResultScanner getScanner(String sql, QueryId queryId, String sessionId) throws Exception {
-    QueryContext queryContext = LocalTajoTestingUtility.createDummyContext(conf);
-    
-    Expr expr = analyzer.parse(sql);
-    LogicalPlan logicalPlan = logicalPlanner.createPlan(queryContext, expr);
-    logicalOptimizer.optimize(logicalPlan);
-    
-    int maxRow = Integer.MAX_VALUE;
-    if (logicalPlan.getRootBlock().hasNode(NodeType.LIMIT)) {
-      LimitNode limitNode = logicalPlan.getRootBlock().getNode(NodeType.LIMIT);
-      maxRow = (int) limitNode.getFetchFirstNum();
-    }
-    
-    NonForwardQueryResultScanner queryResultScanner = 
-        new NonForwardQueryResultSystemScanner(masterContext, logicalPlan, queryId,
-            sessionId, maxRow);
-    
-    return queryResultScanner;
-  }
-  
-  @Test
-  public void testInit() throws Exception {
-    QueryId queryId = QueryIdFactory.newQueryId(masterContext.getResourceManager().getSeedQueryId());
-    String sessionId = UUID.randomUUID().toString();
-    NonForwardQueryResultScanner queryResultScanner = 
-        getScanner("SELECT SPACE_ID, SPACE_URI FROM INFORMATION_SCHEMA.TABLESPACE",
-            queryId, sessionId);
-    
-    queryResultScanner.init();
-    
-    assertThat(queryResultScanner.getQueryId(), is(notNullValue()));
-    assertThat(queryResultScanner.getLogicalSchema(), is(notNullValue()));
-    assertThat(queryResultScanner.getSessionId(), is(notNullValue()));
-    assertThat(queryResultScanner.getTableDesc(), is(notNullValue()));
-    
-    assertThat(queryResultScanner.getQueryId(), is(queryId));
-    assertThat(queryResultScanner.getSessionId(), is(sessionId));
-    
-    assertThat(queryResultScanner.getLogicalSchema().size(), is(2));
-    assertThat(queryResultScanner.getLogicalSchema().getColumn("space_id"), is(notNullValue()));
-  }
-  
-  private List<Tuple> getTupleList(RowStoreDecoder decoder, List<ByteString> bytes) {
-    List<Tuple> tuples = new ArrayList<Tuple>(bytes.size());
-    
-    for (ByteString byteString: bytes) {
-      Tuple aTuple = decoder.toTuple(byteString.toByteArray());
-      tuples.add(aTuple);
-    }
-    
-    return tuples;
-  }
-  
-  private <T> Matcher<Tuple> getTupleMatcher(final int fieldId, final Matcher<T> matcher) {
-    return new TypeSafeDiagnosingMatcher<Tuple>() {
-
-      @Override
-      public void describeTo(Description description) {
-        description.appendDescriptionOf(matcher);
-      }
-
-      @Override
-      protected boolean matchesSafely(Tuple item, Description mismatchDescription) {
-        Object itemValue = null;
-
-        Type type = item.type(fieldId);
-        if (type == Type.TEXT) {
-          itemValue = item.getText(fieldId);
-        } else if (type == Type.INT4) {
-          itemValue = item.getInt4(fieldId);
-        } else if (type == Type.INT8) {
-          itemValue = item.getInt8(fieldId);
-        }
-        
-        if (itemValue != null && matcher.matches(itemValue)) {
-          return true;
-        }
-        
-        matcher.describeMismatch(itemValue, mismatchDescription);
-        return false;
-      }
-    };
-  }
-  
+public class TestNonForwardQueryResultSystemScanner extends QueryTestCaseBase {
   @Test
   public void testGetNextRowsForAggregateFunction() throws Exception {
-    NonForwardQueryResultScanner queryResultScanner = 
-        getScanner("SELECT COUNT(*) FROM INFORMATION_SCHEMA.TABLES");
-    
-    queryResultScanner.init();
-    
-    List<ByteString> rowBytes = queryResultScanner.getNextRows(100);
-    
-    assertThat(rowBytes.size(), is(1));
-    
-    RowStoreDecoder decoder = RowStoreUtil.createDecoder(queryResultScanner.getLogicalSchema());
-    List<Tuple> tuples = getTupleList(decoder, rowBytes);
-    
-    assertThat(tuples.size(), is(1));
-    assertThat(tuples, hasItem(getTupleMatcher(0, is(9L))));
+    assertQueryStr("SELECT COUNT(*) FROM INFORMATION_SCHEMA.TABLES " +
+        "WHERE TABLE_NAME = 'lineitem' OR TABLE_NAME = 'nation' OR TABLE_NAME = 'customer'");
   }
-  
+
   @Test
   public void testGetNextRowsForTable() throws Exception {
-    NonForwardQueryResultScanner queryResultScanner =
-        getScanner("SELECT TABLE_NAME, TABLE_TYPE FROM INFORMATION_SCHEMA.TABLES");
-    
-    queryResultScanner.init();
-    
-    List<ByteString> rowBytes = queryResultScanner.getNextRows(100);
-    
-    assertThat(rowBytes.size(), is(9));
-    
-    RowStoreDecoder decoder = RowStoreUtil.createDecoder(queryResultScanner.getLogicalSchema());
-    List<Tuple> tuples = getTupleList(decoder, rowBytes);;
-    
-    assertThat(tuples.size(), is(9));
-    assertThat(tuples, hasItem(getTupleMatcher(0, is("lineitem"))));
+    assertQueryStr("SELECT TABLE_NAME, TABLE_TYPE FROM INFORMATION_SCHEMA.TABLES " +
+        "WHERE TABLE_NAME = 'lineitem' OR TABLE_NAME = 'nation' OR TABLE_NAME = 'customer'");
   }
-  
+
   @Test
   public void testGetClusterDetails() throws Exception {
-    NonForwardQueryResultScanner queryResultScanner =
-        getScanner("SELECT TYPE FROM INFORMATION_SCHEMA.CLUSTER");
-    
-    queryResultScanner.init();
-    
-    List<ByteString> rowBytes = queryResultScanner.getNextRows(100);
-    
-    assertThat(rowBytes.size(), is(2));
-    
-    RowStoreDecoder decoder = RowStoreUtil.createDecoder(queryResultScanner.getLogicalSchema());
-    List<Tuple> tuples = getTupleList(decoder, rowBytes);
-    
-    assertThat(tuples.size(), is(2));
-    assertThat(tuples, hasItem(getTupleMatcher(0, is("QueryMaster"))));
+    assertQueryStr("SELECT TYPE FROM INFORMATION_SCHEMA.CLUSTER");
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
index edddc5a..1351716 100644
--- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
@@ -45,7 +45,7 @@ import org.apache.tajo.plan.serder.PlanProto;
 import org.apache.tajo.service.ServiceTracker;
 import org.apache.tajo.session.Session;
 import org.apache.tajo.storage.HashShuffleAppenderManager;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.history.HistoryReader;
 import org.apache.tajo.util.history.HistoryWriter;
@@ -105,7 +105,7 @@ public class TestKillQuery {
     Session session = LocalTajoTestingUtility.createDummySession();
     CatalogService catalog = cluster.getMaster().getCatalog();
 
-    LogicalPlanner planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    LogicalPlanner planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(queryStr);
     LogicalPlan plan = planner.createPlan(defaultContext, expr);
@@ -169,7 +169,7 @@ public class TestKillQuery {
     Session session = LocalTajoTestingUtility.createDummySession();
     CatalogService catalog = cluster.getMaster().getCatalog();
 
-    LogicalPlanner planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    LogicalPlanner planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(queryStr);
     LogicalPlan plan = planner.createPlan(defaultContext, expr);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java
----------------------------------------------------------------------
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 863c7b5..f48a71e 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
@@ -67,7 +67,7 @@ public class TestRowFile {
 
     TableMeta meta = CatalogUtil.newTableMeta("ROWFILE");
 
-    FileTablespace sm = (FileTablespace) TableSpaceManager.get(cluster.getDefaultFileSystem().getUri()).get();
+    FileTablespace sm = (FileTablespace) TablespaceManager.get(cluster.getDefaultFileSystem().getUri()).get();
 
     Path tablePath = new Path("/test");
     Path metaPath = new Path(tablePath, ".meta");

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/resources/results/TestHBaseTable/testInsertValues1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertValues1.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertValues1.result
new file mode 100644
index 0000000..45d730a
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertValues1.result
@@ -0,0 +1,4 @@
+aaa, a12, {"": "a34"}, 1
+bbb, b12, {"": "b34"}, 2
+ccc, c12, {"": "c34"}, 3
+ddd, d12, {"": "d34"}, 4

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetClusterDetails.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetClusterDetails.result b/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetClusterDetails.result
new file mode 100644
index 0000000..9f12294
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetClusterDetails.result
@@ -0,0 +1,4 @@
+type
+-------------------------------
+QueryMaster
+Worker
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForAggregateFunction.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForAggregateFunction.result b/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForAggregateFunction.result
new file mode 100644
index 0000000..07dd98b
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForAggregateFunction.result
@@ -0,0 +1,3 @@
+?count_2
+-------------------------------
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForTable.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForTable.result b/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForTable.result
new file mode 100644
index 0000000..fd37504
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestNonForwardQueryResultSystemScanner/testGetNextRowsForTable.result
@@ -0,0 +1,5 @@
+table_name,table_type
+-------------------------------
+customer,EXTERNAL
+lineitem,EXTERNAL
+nation,EXTERNAL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java
index 17f79da..22f4781 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlan.java
@@ -239,6 +239,14 @@ public class LogicalPlan {
     return queryBlocks.get(ROOT_BLOCK);
   }
 
+  public LogicalRootNode getRootNode() {
+    return queryBlocks.get(ROOT_BLOCK).getRoot();
+  }
+
+  public Schema getOutputSchema() {
+    return getRootNode().getOutSchema();
+  }
+
   public QueryBlock getBlock(String blockName) {
     return queryBlocks.get(blockName);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------
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 16ca368..441e047 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
@@ -910,6 +910,22 @@ public class PlannerUtil {
     }
   }
 
+  public static TableDesc getOutputTableDesc(LogicalPlan plan) {
+    LogicalNode [] found = findAllNodes(plan.getRootNode().getChild(), NodeType.CREATE_TABLE, NodeType.INSERT);
+
+    if (found.length == 0) {
+      return new TableDesc(null, plan.getRootNode().getOutSchema(), "TEXT", new KeyValueSet(), null);
+    } else {
+      StoreTableNode storeNode = (StoreTableNode) found[0];
+      return new TableDesc(
+          storeNode.getTableName(),
+          storeNode.getOutSchema(),
+          storeNode.getStorageType(),
+          storeNode.getOptions(),
+          storeNode.getUri());
+    }
+  }
+
   public static TableDesc getTableDesc(CatalogService catalog, LogicalNode node) throws IOException {
     if (node.getType() == NodeType.ROOT) {
       node = ((LogicalRootNode)node).getChild();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java
index 0f0cd10..547a6f2 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java
@@ -18,14 +18,46 @@
 
 package org.apache.tajo.storage;
 
+/**
+ * Format properties
+ */
 public class FormatProperty {
-  private boolean sortedInsertRequired;
 
-  public FormatProperty(boolean sortedInsertRequired) {
-    this.sortedInsertRequired = sortedInsertRequired;
+  /** if this format supports insert operation */
+  private boolean insertable;
+  /** if this format supports direct insertion (e.g., HBASE or JDBC-based storages) */
+  private boolean directInsert;
+  /** if this format supports staging phase */
+  private boolean stagingSupport;
+
+  public FormatProperty(boolean insertable, boolean directInsert, boolean stagingSupport) {
+    this.insertable = insertable;
+    this.stagingSupport = stagingSupport;
+    this.directInsert = directInsert;
+  }
+
+  /**
+   * Return if this format supports staging phase
+   * @return True if this format supports staging phase
+   */
+  public boolean isInsertable() {
+    return insertable;
+  }
+
+  /**
+   * Return if this format supports direct insertion (e.g., HBASE or JDBC-based storages)
+   * @return True if this format supports direct insertion
+   */
+  public boolean directInsertSupported() {
+    return directInsert;
   }
 
-  public boolean sortedInsertRequired() {
-    return sortedInsertRequired;
+  /**
+   * Return if this format supports staging phase
+   *
+   * @return True if this format supports staging phase
+   */
+  public boolean isStagingSupport() {
+    return stagingSupport;
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
index ce573be..67a2f86 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
@@ -64,7 +64,7 @@ public class MergeScanner implements Scanner {
 
     long numBytes = 0;
     for (Fragment eachFileFragment: rawFragmentList) {
-      long fragmentLength = TableSpaceManager.guessFragmentVolume((TajoConf) conf, eachFileFragment);
+      long fragmentLength = TablespaceManager.guessFragmentVolume((TajoConf) conf, eachFileFragment);
       if (fragmentLength > 0) {
         numBytes += fragmentLength;
         fragments.add(eachFileFragment);
@@ -131,7 +131,7 @@ public class MergeScanner implements Scanner {
   private Scanner getNextScanner() throws IOException {
     if (iterator.hasNext()) {
       currentFragment = iterator.next();
-      currentScanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, currentFragment, target);
+      currentScanner = TablespaceManager.getLocalFs().getScanner(meta, schema, currentFragment, target);
       currentScanner.init();
       return currentScanner;
     } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
index 12b236f..ef33a8e 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
@@ -31,7 +31,6 @@ import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.FileUtil;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -158,7 +157,7 @@ public class OldStorageManager {
           Constructor<? extends Tablespace> constructor =
               (Constructor<? extends Tablespace>) CONSTRUCTOR_CACHE.get(storageManagerClass);
           if (constructor == null) {
-            constructor = storageManagerClass.getDeclaredConstructor(TableSpaceManager.TABLESPACE_PARAM);
+            constructor = storageManagerClass.getDeclaredConstructor(TablespaceManager.TABLESPACE_PARAM);
             constructor.setAccessible(true);
             CONSTRUCTOR_CACHE.put(storageManagerClass, constructor);
           }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
index 38d0734..c1db34e 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
@@ -18,20 +18,39 @@
 
 package org.apache.tajo.storage;
 
+/**
+ * Storage Properties
+ */
 public class StorageProperty {
-  private boolean movable;
-  private boolean writable;
-  private boolean insertable;
-  private boolean absolutePathAllowed;
+  /** default file format */
+  private final String defaultFormat;
+  /** if this storage supports move operator */
+  private final boolean movable;
+  /** if this storage supports is writable */
+  private final boolean writable;
+  /** if this storage allows use of artibrary paths */
+  private final boolean absolutePathAllowed;
+
+  public StorageProperty(String defaultFormat,
+                         boolean movable,
+                         boolean writable,
+                         boolean absolutePathAllowed) {
 
-  public StorageProperty(boolean movable, boolean writable, boolean isInsertable, boolean absolutePathAllowed) {
+    this.defaultFormat = defaultFormat;
     this.movable = movable;
     this.writable = writable;
-    this.insertable = isInsertable;
     this.absolutePathAllowed = absolutePathAllowed;
   }
 
   /**
+   * Return default file format
+   * @return Default file format
+   */
+  public String defaultFormat() {
+    return defaultFormat;
+  }
+
+  /**
    * Move-like operation is allowed
    *
    * @return true if move operation is available
@@ -50,18 +69,9 @@ public class StorageProperty {
   }
 
   /**
-   * this storage supports insert operation?
-   *
-   * @return true if insert operation is allowed.
-   */
-  public boolean isInsertable() {
-    return insertable;
-  }
-
-  /**
    * Does this storage allows the use of arbitrary absolute paths outside tablespace?
    *
-   * @return
+   * @return True if this storage allows accesses to artibrary paths.
    */
   public boolean isArbitraryPathAllowed() {
     return this.absolutePathAllowed;

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java
deleted file mode 100644
index ef04509..0000000
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java
+++ /dev/null
@@ -1,390 +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 com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
-import net.minidev.json.JSONObject;
-import net.minidev.json.parser.JSONParser;
-import net.minidev.json.parser.ParseException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.Pair;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.net.URI;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.UUID;
-
-import static org.apache.tajo.storage.StorageConstants.LOCAL_FS_URI;
-
-/**
- * It handles available table spaces and cache TableSpace instances.
- *
- * Default tablespace must be a filesystem-based one.
- * HDFS and S3 can be a default tablespace if a Tajo cluster is in fully distributed mode.
- * Local file system can be a default tablespace if a Tajo cluster runs on a single machine.
- */
-public class TableSpaceManager implements StorageService {
-  private static final Log LOG = LogFactory.getLog(TableSpaceManager.class);
-
-  public static final String DEFAULT_CONFIG_FILE = "storage-default.json";
-  public static final String SITE_CONFIG_FILE = "storage-site.json";
-
-  /** default tablespace name */
-  public static final String DEFAULT_TABLESPACE_NAME = "default";
-
-  private final static TajoConf systemConf = new TajoConf();
-  private final static JSONParser parser = new JSONParser(JSONParser.MODE_JSON_SIMPLE | JSONParser.IGNORE_CONTROL_CHAR);
-
-  // The relation ship among name, URI, Tablespaces must be kept 1:1:1.
-  protected static final Map<String, URI> SPACES_URIS_MAP = Maps.newHashMap();
-  protected static final TreeMap<URI, Tablespace> TABLE_SPACES = Maps.newTreeMap();
-
-  protected static final Map<Class<?>, Constructor<?>> CONSTRUCTORS = Maps.newHashMap();
-  protected static final Map<String, Class<? extends Tablespace>> TABLE_SPACE_HANDLERS = Maps.newHashMap();
-
-  public static final Class [] TABLESPACE_PARAM = new Class [] {String.class, URI.class};
-
-  static {
-    instance = new TableSpaceManager();
-  }
-  /**
-   * Singleton instance
-   */
-  private static final TableSpaceManager instance;
-
-  private TableSpaceManager() {
-    initForDefaultConfig(); // loading storage-default.json
-    initSiteConfig();       // storage-site.json will override the configs of storage-default.json
-    addWarehouseAsSpace();  // adding a warehouse directory for a default tablespace
-    addLocalFsTablespace(); // adding a tablespace using local file system by default
-  }
-
-  private void addWarehouseAsSpace() {
-    Path warehouseDir = TajoConf.getWarehouseDir(systemConf);
-    registerTableSpace(DEFAULT_TABLESPACE_NAME, warehouseDir.toUri(), null, true, false);
-  }
-
-  private void addLocalFsTablespace() {
-    if (TABLE_SPACES.headMap(LOCAL_FS_URI, true).firstEntry() == null) {
-      String tmpName = UUID.randomUUID().toString();
-      registerTableSpace(tmpName, LOCAL_FS_URI, null, false, false);
-    }
-  }
-
-  public static TableSpaceManager getInstance() {
-    return instance;
-  }
-
-  private void initForDefaultConfig() {
-    JSONObject json = loadFromConfig(DEFAULT_CONFIG_FILE);
-    if (json == null) {
-      throw new IllegalStateException("There is no " + SITE_CONFIG_FILE);
-    }
-    applyConfig(json, false);
-  }
-
-  private void initSiteConfig() {
-    JSONObject json = loadFromConfig(SITE_CONFIG_FILE);
-
-    // if there is no storage-site.json file, nothing happen.
-    if (json != null) {
-      applyConfig(json, true);
-    }
-  }
-
-  private JSONObject loadFromConfig(String fileName) {
-    String json;
-    try {
-      json = FileUtil.readTextFileFromResource(fileName);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    if (json != null) {
-      return parseJson(json);
-    } else {
-      return null;
-    }
-  }
-
-  private static JSONObject parseJson(String json) {
-    try {
-      return (JSONObject) parser.parse(json);
-    } catch (ParseException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private void applyConfig(JSONObject json, boolean override) {
-    loadStorages(json);
-    loadTableSpaces(json, override);
-  }
-
-  private void loadStorages(JSONObject json) {
-    JSONObject spaces = (JSONObject) json.get(KEY_STORAGES);
-
-    if (spaces != null) {
-      Pair<String, Class<? extends Tablespace>> pair = null;
-      for (Map.Entry<String, Object> entry : spaces.entrySet()) {
-
-        try {
-          pair = extractStorage(entry);
-        } catch (ClassNotFoundException e) {
-          LOG.warn(e);
-          continue;
-        }
-
-        TABLE_SPACE_HANDLERS.put(pair.getFirst(), pair.getSecond());
-      }
-    }
-  }
-
-  private Pair<String, Class<? extends Tablespace>> extractStorage(Map.Entry<String, Object> entry)
-      throws ClassNotFoundException {
-
-    String storageType = entry.getKey();
-    JSONObject storageDesc = (JSONObject) entry.getValue();
-    String handlerClass = (String) storageDesc.get(KEY_STORAGE_HANDLER);
-
-    return new Pair<String, Class<? extends Tablespace>>(
-        storageType,(Class<? extends Tablespace>) Class.forName(handlerClass));
-  }
-
-  private void loadTableSpaces(JSONObject json, boolean override) {
-    JSONObject spaces = (JSONObject) json.get(KEY_SPACES);
-
-    if (spaces != null) {
-      for (Map.Entry<String, Object> entry : spaces.entrySet()) {
-        AddTableSpace(entry.getKey(), (JSONObject) entry.getValue(), override);
-      }
-    }
-  }
-
-  public static void AddTableSpace(String spaceName, JSONObject spaceDesc, boolean override) {
-    boolean defaultSpace = Boolean.parseBoolean(spaceDesc.getAsString("default"));
-    URI spaceUri = URI.create(spaceDesc.getAsString("uri"));
-
-    if (defaultSpace) {
-      registerTableSpace(DEFAULT_TABLESPACE_NAME, spaceUri, spaceDesc, true, override);
-    }
-    registerTableSpace(spaceName, spaceUri, spaceDesc, true, override);
-  }
-
-  private static void registerTableSpace(String spaceName, URI uri, JSONObject spaceDesc,
-                                         boolean visible, boolean override) {
-    Tablespace tableSpace = initializeTableSpace(spaceName, uri, visible);
-    tableSpace.setVisible(visible);
-
-    try {
-      tableSpace.init(systemConf);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    putTablespace(tableSpace, override);
-
-    // If the arbitrary path is allowed, root uri is also added as a tablespace
-    if (tableSpace.getProperty().isArbitraryPathAllowed()) {
-      URI rootUri = tableSpace.getRootUri();
-      // if there already exists or the rootUri is 'file:/', it won't overwrite the tablespace.
-      if (!TABLE_SPACES.containsKey(rootUri) && !rootUri.toString().startsWith(LOCAL_FS_URI.toString())) {
-        String tmpName = UUID.randomUUID().toString();
-        registerTableSpace(tmpName, rootUri, spaceDesc, false, override);
-      }
-    }
-  }
-
-  private static void putTablespace(Tablespace space, boolean override) {
-    // It is a device to keep the relationship among name, URI, and tablespace 1:1:1.
-
-    boolean nameExist = SPACES_URIS_MAP.containsKey(space.getName());
-    boolean uriExist = TABLE_SPACES.containsKey(space.uri);
-
-    boolean mismatch = nameExist && !SPACES_URIS_MAP.get(space.getName()).equals(space.getUri());
-    mismatch = mismatch || uriExist && TABLE_SPACES.get(space.uri).equals(space);
-
-    if (!override && mismatch) {
-      throw new RuntimeException("Name or URI of Tablespace must be unique.");
-    }
-
-    SPACES_URIS_MAP.put(space.getName(), space.getUri());
-    // We must guarantee that the same uri results in the same tablespace instance.
-    TABLE_SPACES.put(space.getUri(), space);
-  }
-
-  /**
-   * 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 guessFragmentVolume(TajoConf conf, Fragment fragment) {
-    if (fragment.getLength() == TajoConstants.UNKNOWN_LENGTH) {
-      return conf.getLongVar(TajoConf.ConfVars.FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH);
-    } else {
-      return fragment.getLength();
-    }
-  }
-
-  public static final String KEY_STORAGES = "storages"; // storages
-  public static final String KEY_STORAGE_HANDLER = "handler"; // storages/?/handler
-  public static final String KEY_STORAGE_DEFAULT_FORMAT = "default-format"; // storages/?/default-format
-
-  public static final String KEY_SPACES = "spaces";
-
-  private static Tablespace initializeTableSpace(String spaceName, URI uri, boolean visible) {
-    Preconditions.checkNotNull(uri.getScheme(), "URI must include scheme, but it was " + uri);
-    Class<? extends Tablespace> clazz = TABLE_SPACE_HANDLERS.get(uri.getScheme());
-
-    if (clazz == null) {
-      throw new RuntimeException("There is no tablespace for " + uri.toString());
-    }
-
-    try {
-      Constructor<? extends Tablespace> constructor =
-          (Constructor<? extends Tablespace>) CONSTRUCTORS.get(clazz);
-
-      if (constructor == null) {
-        constructor = clazz.getDeclaredConstructor(TABLESPACE_PARAM);
-        constructor.setAccessible(true);
-        CONSTRUCTORS.put(clazz, constructor);
-      }
-
-      return constructor.newInstance(new Object[]{spaceName, uri});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @VisibleForTesting
-  public static Optional<Tablespace> addTableSpaceForTest(Tablespace space) {
-    Tablespace existing;
-    synchronized (SPACES_URIS_MAP) {
-      // Remove existing one
-      SPACES_URIS_MAP.remove(space.getName());
-      existing = TABLE_SPACES.remove(space.getUri());
-
-      // Add anotherone for test
-      registerTableSpace(space.name, space.uri, null, true, true);
-    }
-    // if there is an existing one, return it.
-    return Optional.fromNullable(existing);
-  }
-
-  public Iterable<String> getSupportSchemes() {
-    return TABLE_SPACE_HANDLERS.keySet();
-  }
-
-  /**
-   * Get tablespace for the given URI. If uri is null, the default tablespace will be returned
-   *
-   * @param uri Table or Table Fragment URI.
-   * @param <T> Tablespace class type
-   * @return Tablespace. If uri is null, the default tablespace will be returned.
-   */
-  public static <T extends Tablespace> Optional<T> get(@Nullable String uri) {
-
-    if (uri == null || uri.isEmpty()) {
-      return (Optional<T>) Optional.of(getDefault());
-    }
-
-    Tablespace lastOne = null;
-
-    // Find the longest matched one. For example, assume that the caller tries to find /x/y/z, and
-    // there are /x and /x/y. In this case, /x/y will be chosen because it is more specific.
-    for (Map.Entry<URI, Tablespace> entry: TABLE_SPACES.headMap(URI.create(uri), true).entrySet()) {
-      if (uri.startsWith(entry.getKey().toString())) {
-        lastOne = entry.getValue();
-      }
-    }
-    return (Optional<T>) Optional.fromNullable(lastOne);
-  }
-
-  /**
-   * Get tablespace for the given URI. If uri is null, the default tablespace will be returned
-   *
-   * @param uri Table or Table Fragment URI.
-   * @param <T> Tablespace class type
-   * @return Tablespace. If uri is null, the default tablespace will be returned.
-   */
-  public static <T extends Tablespace> Optional<T> get(@Nullable URI uri) {
-    if (uri == null) {
-      return (Optional<T>) Optional.of(getDefault());
-    } else {
-      return (Optional<T>) get(uri.toString());
-    }
-  }
-
-  /**
-   * It returns the default tablespace. This method ensures that it always return the tablespace.
-   *
-   * @return
-   */
-  public static <T extends Tablespace> T getDefault() {
-    return (T) getByName(DEFAULT_TABLESPACE_NAME).get();
-  }
-
-  public static <T extends Tablespace> T getLocalFs() {
-    return (T) get(LOCAL_FS_URI).get();
-  }
-
-  public static Optional<? extends Tablespace> getByName(String name) {
-    URI uri = SPACES_URIS_MAP.get(name);
-    if (uri != null) {
-      return Optional.of(TABLE_SPACES.get(uri));
-    } else {
-      return Optional.absent();
-    }
-  }
-
-  public static Optional<? extends Tablespace> getAnyByScheme(String scheme) {
-    for (Map.Entry<URI, Tablespace> entry : TABLE_SPACES.entrySet()) {
-      String uriScheme = entry.getKey().getScheme();
-      if (uriScheme != null && uriScheme.equalsIgnoreCase(scheme)) {
-        return Optional.of(entry.getValue());
-      }
-    }
-
-    return Optional.absent();
-  }
-
-  @Override
-  public URI getTableURI(@Nullable String spaceName, String databaseName, String tableName) {
-    Tablespace space = spaceName == null ? getDefault() : getByName(spaceName).get();
-    return space.getTableUri(databaseName, tableName);
-  }
-
-  public static Iterable<Tablespace> getAllTablespaces() {
-    return TABLE_SPACES.values();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
index 77c5d05..52e223d 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
@@ -18,9 +18,11 @@
 
 package org.apache.tajo.storage;
 
+import com.google.common.base.Optional;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.OverridableConf;
+import org.apache.tajo.QueryVars;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.SortSpec;
@@ -149,7 +151,7 @@ public abstract class Tablespace {
    */
   public abstract StorageProperty getProperty();
 
-  public abstract FormatProperty getFormatProperty(String dataFormat);
+  public abstract FormatProperty getFormatProperty(TableMeta meta);
 
   /**
    * Release storage manager resource
@@ -259,6 +261,14 @@ public abstract class Tablespace {
     return scanner;
   }
 
+  public Appender getAppenderForInsertRow(OverridableConf queryContext,
+                                          TaskAttemptId taskAttemptId,
+                                          TableMeta meta,
+                                          Schema schema,
+                                          Path workDir) throws IOException {
+    return getAppender(queryContext, taskAttemptId, meta, schema, workDir);
+  }
+
   /**
    * Returns Appender instance.
    * @param queryContext Query property.
@@ -395,4 +405,11 @@ public abstract class Tablespace {
       return false;
     }
   }
+
+  public abstract URI getStagingUri(OverridableConf context, String queryId, TableMeta meta) throws IOException;
+
+  public URI prepareStagingSpace(TajoConf conf, String queryId, OverridableConf context,
+                                 TableMeta meta) throws IOException {
+    throw new IOException("Staging the output result is not supported in this storage");
+  }
 }


[09/15] tajo git commit: Fixed compilation error after revert.

Posted by ji...@apache.org.
Fixed compilation error after revert.


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

Branch: refs/heads/index_support
Commit: d44072762cb0b8e6dbb989e598074d160657ad0e
Parents: d0f3701
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Jun 24 14:35:24 2015 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Jun 24 14:35:24 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/tajo/storage/HashShuffleAppenderManager.java   | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/d4407276/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
index 7937d38..bd5502d 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
@@ -30,9 +30,11 @@ import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.util.Pair;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;


[08/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces.

Posted by ji...@apache.org.
TAJO-1616: Implement TablespaceManager to load Tablespaces.

Closes #602


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

Branch: refs/heads/index_support
Commit: d0f37012faf1a8a3259210287e9bc7762ce62001
Parents: 7c8477d
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Jun 24 14:33:34 2015 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Jun 24 14:33:34 2015 -0700

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../org/apache/tajo/algebra/CreateTable.java    |  14 +
 .../org/apache/tajo/catalog/CatalogUtil.java    |   7 +
 .../org/apache/tajo/catalog/DDLBuilder.java     |   5 +-
 .../java/org/apache/tajo/catalog/TableDesc.java |   8 +-
 .../org/apache/tajo/catalog/TestTableDesc.java  |   2 +-
 .../tajo/catalog/store/HiveCatalogStore.java    |   2 +-
 .../catalog/store/TestHiveCatalogStore.java     |  25 +-
 .../tajo/catalog/store/AbstractDBStore.java     |  44 +-
 .../src/main/resources/schemas/derby/derby.xml  |   2 +-
 .../main/resources/schemas/mariadb/mariadb.xml  |   3 +-
 .../src/main/resources/schemas/mysql/mysql.xml  |   3 +-
 .../main/resources/schemas/oracle/oracle.xml    |   3 +-
 .../resources/schemas/postgresql/postgresql.xml |   4 +-
 .../cli/tsql/commands/DescTableCommand.java     |   2 +-
 .../main/java/org/apache/tajo/QueryVars.java    |   4 +-
 .../apache/tajo/storage/StorageConstants.java   |   6 +
 .../org/apache/tajo/storage/StorageService.java |  37 ++
 .../java/org/apache/tajo/util/FileUtil.java     |  14 +-
 .../org/apache/tajo/util/ReflectionUtil.java    |   6 +-
 .../org/apache/tajo/engine/parser/SQLParser.g4  |   8 +-
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  11 +-
 .../engine/planner/PhysicalPlannerImpl.java     |  14 +-
 .../engine/planner/global/GlobalPlanner.java    |   4 +-
 .../planner/physical/BSTIndexScanExec.java      |   2 +-
 .../planner/physical/ColPartitionStoreExec.java |   2 +-
 .../planner/physical/PhysicalPlanUtil.java      |   2 +-
 .../physical/RangeShuffleFileWriteExec.java     |   2 +-
 .../engine/planner/physical/SeqScanExec.java    |   6 +-
 .../engine/planner/physical/StoreTableExec.java |  19 +-
 .../apache/tajo/engine/query/QueryContext.java  |  67 ++-
 .../org/apache/tajo/master/GlobalEngine.java    |  13 +-
 .../java/org/apache/tajo/master/TajoMaster.java |  22 +-
 .../tajo/master/TajoMasterClientService.java    |   3 +-
 .../apache/tajo/master/exec/DDLExecutor.java    |  58 ++-
 .../exec/NonForwardQueryResultFileScanner.java  |   2 +-
 .../apache/tajo/master/exec/QueryExecutor.java  | 238 +++++----
 .../master/exec/prehook/CreateTableHook.java    |  10 +-
 .../prehook/DistributedQueryHookManager.java    |   2 +-
 .../master/exec/prehook/InsertIntoHook.java     |   8 +-
 .../java/org/apache/tajo/querymaster/Query.java |  51 +-
 .../tajo/querymaster/QueryMasterTask.java       |  72 ++-
 .../apache/tajo/querymaster/Repartitioner.java  |  77 +--
 .../java/org/apache/tajo/querymaster/Stage.java |  12 +-
 .../tajo/webapp/QueryExecutorServlet.java       |   2 +-
 .../org/apache/tajo/worker/LegacyTaskImpl.java  |   2 +-
 .../java/org/apache/tajo/worker/TajoWorker.java |   5 +-
 .../java/org/apache/tajo/worker/TaskImpl.java   |   2 +-
 .../resources/webapps/admin/catalogview.jsp     |   3 +-
 .../src/main/resources/webapps/admin/index.jsp  |  23 +-
 .../org/apache/tajo/BackendTestingUtil.java     |   2 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |   4 +-
 .../org/apache/tajo/TajoTestingCluster.java     |  35 +-
 .../org/apache/tajo/cli/tsql/TestTajoCli.java   |   4 +-
 .../org/apache/tajo/client/TestTajoClient.java  |  18 +-
 .../apache/tajo/engine/eval/ExprTestBase.java   |   3 +-
 .../tajo/engine/eval/TestEvalTreeUtil.java      |   3 +-
 .../engine/planner/TestLogicalOptimizer.java    |   4 +-
 .../tajo/engine/planner/TestLogicalPlan.java    |   3 +-
 .../tajo/engine/planner/TestLogicalPlanner.java |  92 ++--
 .../tajo/engine/planner/TestPlannerUtil.java    |   5 +-
 .../planner/physical/TestBNLJoinExec.java       |  15 +-
 .../planner/physical/TestBSTIndexExec.java      |   6 +-
 .../planner/physical/TestExternalSortExec.java  |   6 +-
 .../physical/TestFullOuterHashJoinExec.java     |  29 +-
 .../physical/TestFullOuterMergeJoinExec.java    |  40 +-
 .../planner/physical/TestHashAntiJoinExec.java  |  10 +-
 .../planner/physical/TestHashJoinExec.java      |  14 +-
 .../planner/physical/TestHashSemiJoinExec.java  |  14 +-
 .../physical/TestLeftOuterHashJoinExec.java     |  31 +-
 .../planner/physical/TestMergeJoinExec.java     |  10 +-
 .../engine/planner/physical/TestNLJoinExec.java |  14 +-
 .../planner/physical/TestPhysicalPlanner.java   |  56 +--
 .../physical/TestProgressExternalSortExec.java  |   6 +-
 .../physical/TestRightOuterHashJoinExec.java    |  20 +-
 .../physical/TestRightOuterMergeJoinExec.java   |  36 +-
 .../engine/planner/physical/TestSortExec.java   |   6 +-
 .../apache/tajo/engine/query/TestCTASQuery.java |   8 +-
 .../tajo/engine/query/TestCreateTable.java      |   8 +-
 .../tajo/engine/query/TestHBaseTable.java       | 226 +++++----
 .../tajo/engine/query/TestInsertQuery.java      |   6 +-
 .../apache/tajo/engine/query/TestJoinQuery.java |   6 +-
 .../tajo/engine/query/TestTablePartitions.java  |  28 +-
 .../org/apache/tajo/jdbc/TestResultSet.java     |   4 +-
 .../tajo/master/TestExecutionBlockCursor.java   |  10 +-
 .../apache/tajo/querymaster/TestKillQuery.java  |   5 +-
 .../org/apache/tajo/storage/TestRowFile.java    |   5 +-
 .../results/TestHBaseTable/testCATS.result      | 100 ----
 .../results/TestHBaseTable/testCTAS.result      | 100 ++++
 .../testInsertIntoUsingPut.result               |   4 +-
 .../results/TestTajoCli/testDescTable.result    |   8 +-
 .../testDescTableForNestedSchema.result         |   4 +-
 .../src/main/conf/storage-site.json.template    |  35 ++
 tajo-docs/src/main/sphinx/index.rst             |   1 +
 tajo-docs/src/main/sphinx/storage_plugin.rst    |  47 ++
 tajo-docs/src/main/sphinx/table_management.rst  |   1 +
 .../sphinx/table_management/table_overview.rst  |   7 +
 .../sphinx/table_management/tablespaces.rst     |  45 ++
 .../org/apache/tajo/plan/LogicalPlanner.java    |  64 ++-
 .../tajo/plan/logical/CreateTableNode.java      |  44 +-
 .../apache/tajo/plan/logical/InsertNode.java    |  51 +-
 .../plan/logical/PartitionedTableScanNode.java  |   2 +-
 .../org/apache/tajo/plan/logical/ScanNode.java  |   2 +-
 .../tajo/plan/logical/StoreTableNode.java       |  27 ++
 .../rewrite/rules/PartitionedTableRewriter.java |   4 +-
 .../plan/serder/LogicalNodeDeserializer.java    |  17 +-
 .../tajo/plan/serder/LogicalNodeSerializer.java |  25 +-
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  14 +-
 tajo-plan/src/main/proto/Plan.proto             |  11 +-
 tajo-storage/tajo-storage-common/pom.xml        |  12 +-
 .../org/apache/tajo/storage/FormatProperty.java |  31 ++
 .../org/apache/tajo/storage/MergeScanner.java   |   5 +-
 .../apache/tajo/storage/OldStorageManager.java  | 251 ++++++++++
 .../apache/tajo/storage/StorageProperty.java    |  49 +-
 .../org/apache/tajo/storage/StorageUtil.java    |   4 +-
 .../apache/tajo/storage/TableSpaceManager.java  | 480 ++++++++++++-------
 .../org/apache/tajo/storage/Tablespace.java     | 194 +++++---
 .../src/main/resources/storage-default.json     |  20 +
 tajo-storage/tajo-storage-hbase/pom.xml         |  11 +
 .../storage/hbase/AbstractHBaseAppender.java    |   2 +-
 .../storage/hbase/AddSortForInsertRewriter.java |  91 ----
 .../tajo/storage/hbase/ColumnMapping.java       |  17 +-
 .../tajo/storage/hbase/HBaseFragment.java       |  28 +-
 .../tajo/storage/hbase/HBasePutAppender.java    |  11 +-
 .../apache/tajo/storage/hbase/HBaseScanner.java |   9 +-
 .../tajo/storage/hbase/HBaseTablespace.java     | 209 ++++----
 .../storage/hbase/SortedInsertRewriter.java     | 116 +++++
 .../src/main/proto/StorageFragmentProtos.proto  |  15 +-
 .../tajo/storage/hbase/TestColumnMapping.java   |   2 +-
 .../storage/hbase/TestHBaseStorageManager.java  | 108 -----
 .../tajo/storage/hbase/TestHBaseTableSpace.java | 134 ++++++
 tajo-storage/tajo-storage-hdfs/pom.xml          |   6 +-
 .../org/apache/tajo/storage/FileAppender.java   |  28 +-
 .../org/apache/tajo/storage/FileTablespace.java | 137 ++++--
 .../storage/HashShuffleAppenderManager.java     |   5 +-
 .../tajo/storage/TestCompressionStorages.java   |   4 +-
 .../tajo/storage/TestDelimitedTextFile.java     |   8 +-
 .../tajo/storage/TestFileStorageManager.java    | 233 ---------
 .../apache/tajo/storage/TestFileSystems.java    |   2 +-
 .../apache/tajo/storage/TestFileTablespace.java | 250 ++++++++++
 .../org/apache/tajo/storage/TestLineReader.java |   8 +-
 .../apache/tajo/storage/TestMergeScanner.java   |   6 +-
 .../org/apache/tajo/storage/TestStorages.java   |  48 +-
 .../apache/tajo/storage/index/TestBSTIndex.java |  58 ++-
 .../index/TestSingleCSVFileBSTIndex.java        |   4 +-
 .../apache/tajo/storage/json/TestJsonSerDe.java |   2 +-
 146 files changed, 2966 insertions(+), 1972 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 1f69ac0..75ffd39 100644
--- a/CHANGES
+++ b/CHANGES
@@ -350,6 +350,8 @@ Release 0.11.0 - unreleased
 
   SUB TASKS
 
+    TAJO-1616: Implement TablespaceManager to load Tablespaces. (hyunsik)
+
     TAJO-1615: Implement TaskManager. (jinho)
 
     TAJO-1599: Implement NodeResourceManager and Status updater. (jinho)

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-algebra/src/main/java/org/apache/tajo/algebra/CreateTable.java
----------------------------------------------------------------------
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 2d4a241..5d1599d 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
@@ -38,6 +38,8 @@ public class CreateTable extends Expr {
   private String tableName;
   @Expose @SerializedName("Attributes")
   private ColumnDefinition [] tableElements;
+  @Expose @SerializedName("SpaceName")
+  private String spaceName;
   @Expose @SerializedName("StorageType")
   private String storageType;
   @Expose @SerializedName("Location")
@@ -100,6 +102,18 @@ public class CreateTable extends Expr {
     this.tableElements = tableElements;
   }
 
+  public boolean hasTableSpaceName() {
+    return spaceName != null;
+  }
+
+  public void setTableSpaceName(String spaceName) {
+    this.spaceName = spaceName;
+  }
+
+  public String getTableSpaceName() {
+    return spaceName;
+  }
+
   public boolean hasStorageType() {
     return storageType != null;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
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 a2e4a9d..6c6915b 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
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.catalog;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.DataTypeUtil;
@@ -31,6 +32,7 @@ import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.exception.InvalidOperationException;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.StringUtils;
 import org.apache.tajo.util.TUtil;
 
@@ -227,6 +229,11 @@ public class CatalogUtil {
     return sb.toString();
   }
 
+  public static Pair<String, String> separateQualifierAndName(String name) {
+    Preconditions.checkArgument(isFQTableName(name), "Must be a qualified name.");
+    return new Pair<String, String>(extractQualifier(name), extractSimpleName(name));
+  }
+
   /**
    * Extract a qualification name from an identifier.
    *

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
index 65640b9..62dd894 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.catalog;
 
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
-import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.util.KeyValueSet;
 
 import java.util.Map;
@@ -32,7 +31,7 @@ public class DDLBuilder {
     sb.append("--\n")
       .append("-- Name: ").append(CatalogUtil.denormalizeIdentifier(desc.getName())).append("; Type: TABLE;")
       .append(" Storage: ").append(desc.getMeta().getStoreType());
-    sb.append("\n-- Path: ").append(desc.getPath());
+    sb.append("\n-- Path: ").append(desc.getUri());
     sb.append("\n--\n");
     sb.append("CREATE EXTERNAL TABLE ").append(CatalogUtil.denormalizeIdentifier(desc.getName()));
     buildSchema(sb, desc.getSchema());
@@ -109,7 +108,7 @@ public class DDLBuilder {
   }
 
   private static void buildLocationClause(StringBuilder sb, TableDesc desc) {
-    sb.append(" LOCATION '").append(desc.getPath()).append("'");
+    sb.append(" LOCATION '").append(desc.getUri()).append("'");
   }
 
   private static void buildPartitionClause(StringBuilder sb, TableDesc desc) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TableDesc.java
----------------------------------------------------------------------
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 17f9146..4700322 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
@@ -89,12 +89,16 @@ public class TableDesc implements ProtoObject<TableDescProto>, GsonObject, Clone
   public String getName() {
     return this.tableName;
   }
+
+  public boolean hasUri() {
+    return this.uri != null;
+  }
 	
-	public void setPath(URI uri) {
+	public void setUri(URI uri) {
 		this.uri = uri;
 	}
 	
-  public URI getPath() {
+  public URI getUri() {
     return this.uri;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
index 9d84de6..41a0832 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
@@ -78,7 +78,7 @@ public class TestTableDesc {
     TableDesc desc = new TableDesc("table1", schema, info, path.toUri());
     assertEquals("table1", desc.getName());
     
-    assertEquals(path.toUri(), desc.getPath());
+    assertEquals(path.toUri(), desc.getUri());
     assertEquals(info, desc.getMeta());
     testClone(desc);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
index 3ea263a..8f23db4 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
@@ -443,7 +443,7 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
         table.setTableType(TableType.EXTERNAL_TABLE.name());
         table.putToParameters("EXTERNAL", "TRUE");
 
-        Path tablePath = new Path(tableDesc.getPath());
+        Path tablePath = new Path(tableDesc.getUri());
         FileSystem fs = tablePath.getFileSystem(conf);
         if (fs.isFile(tablePath)) {
           LOG.warn("A table path is a file, but HiveCatalogStore does not allow a file path.");

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
index 6e91b89..946d271 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
@@ -43,7 +43,6 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
 import static org.junit.Assert.*;
 
 /**
@@ -105,7 +104,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, CUSTOMER));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -134,7 +133,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, REGION));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -163,7 +162,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, REGION));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -197,7 +196,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, SUPPLIER));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -247,7 +246,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, NATION));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -261,8 +260,8 @@ public class TestHiveCatalogStore {
       assertEquals(partitionSchema.getColumn(i).getSimpleName(), partitionSchema1.getColumn(i).getSimpleName());
     }
 
-    testAddPartition(table1.getPath(), NATION, "n_nationkey=10/n_date=20150101");
-    testAddPartition(table1.getPath(), NATION, "n_nationkey=20/n_date=20150102");
+    testAddPartition(table1.getUri(), NATION, "n_nationkey=10/n_date=20150101");
+    testAddPartition(table1.getUri(), NATION, "n_nationkey=20/n_date=20150102");
 
     testDropPartition(NATION, "n_nationkey=10/n_date=20150101");
     testDropPartition(NATION, "n_nationkey=20/n_date=20150102");
@@ -370,7 +369,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, tableName));
     FileSystem fs = FileSystem.getLocal(new Configuration());
-    assertTrue(fs.exists(new Path(table1.getPath())));
+    assertTrue(fs.exists(new Path(table1.getUri())));
 
     store.dropTable(DB_NAME, tableName);
     assertFalse(store.existTable(DB_NAME, tableName));
@@ -395,7 +394,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, REGION));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -424,7 +423,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, REGION));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -456,7 +455,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, CUSTOMER));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());
@@ -491,7 +490,7 @@ public class TestHiveCatalogStore {
 
     TableDesc table1 = new TableDesc(store.getTable(DB_NAME, tableName));
     assertEquals(table.getName(), table1.getName());
-    assertEquals(table.getPath(), table1.getPath());
+    assertEquals(table.getUri(), table1.getUri());
     assertEquals(table.getSchema().size(), table1.getSchema().size());
     for (int i = 0; i < table.getSchema().size(); i++) {
       assertEquals(table.getSchema().getColumn(i).getSimpleName(), table1.getSchema().getColumn(i).getSimpleName());

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
index 34740c0..043c8bc 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
@@ -761,36 +761,25 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
 
       int dbid = getDatabaseId(databaseName);
 
-      if (table.getIsExternal()) {
-        String sql = "INSERT INTO TABLES (DB_ID, " + COL_TABLES_NAME + ", TABLE_TYPE, PATH, STORE_TYPE) VALUES(?, ?, ?, ?, ?) ";
+      String sql = "INSERT INTO TABLES (DB_ID, " + COL_TABLES_NAME +
+          ", TABLE_TYPE, PATH, STORE_TYPE) VALUES(?, ?, ?, ?, ?) ";
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(sql);
-        }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(sql);
+      }
 
-        pstmt = conn.prepareStatement(sql);
-        pstmt.setInt(1, dbid);
-        pstmt.setString(2, tableName);
+      pstmt = conn.prepareStatement(sql);
+      pstmt.setInt(1, dbid);
+      pstmt.setString(2, tableName);
+      if (table.getIsExternal()) {
         pstmt.setString(3, TableType.EXTERNAL_TABLE.name());
-        pstmt.setString(4, table.getPath());
-        pstmt.setString(5, table.getMeta().getStoreType());
-        pstmt.executeUpdate();
-        pstmt.close();
       } else {
-        String sql = "INSERT INTO TABLES (DB_ID, " + COL_TABLES_NAME + ", TABLE_TYPE, STORE_TYPE) VALUES(?, ?, ?, ?) ";
-
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(sql);
-        }
-
-        pstmt = conn.prepareStatement(sql);
-        pstmt.setInt(1, dbid);
-        pstmt.setString(2, tableName);
         pstmt.setString(3, TableType.BASE_TABLE.name());
-        pstmt.setString(4, table.getMeta().getStoreType());
-        pstmt.executeUpdate();
-        pstmt.close();
       }
+      pstmt.setString(4, table.getPath());
+      pstmt.setString(5, table.getMeta().getStoreType());
+      pstmt.executeUpdate();
+      pstmt.close();
 
       String tidSql =
           "SELECT TID from " + TB_TABLES + " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_NAME + "=?";
@@ -1612,12 +1601,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo
         tableBuilder.setIsExternal(true);
       }
 
-      if (tableType == TableType.BASE_TABLE) {
-        tableBuilder.setPath(databaseIdAndUri.getSecond() + "/" + tableName);
-      } else {
-        tableBuilder.setPath(res.getString(4).trim());
-      }
-
+      tableBuilder.setPath(res.getString(4).trim());
       storeType = res.getString(5).trim();
 
       res.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
index a4ff00f..e0bd469 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml
@@ -24,7 +24,7 @@
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-	<tns:base version="4">
+	<tns:base version="5">
 		<tns:objects>
 			<tns:Object order="0" type="table" name="META">
 				<tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
index 79ccd0a..7485da1 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mariadb/mariadb.xml
@@ -19,12 +19,13 @@
 <tns:store xmlns:tns="http://tajo.apache.org/catalogstore" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
       Catalog base version history
+      * 5 - 2015-06-15: Implement TablespaceManager to load Tablespaces (TAJO-1616)
       * 4 - 2015-03-27: Partition Schema (TAJO-1284)
       * 3 - 2015-03-12: Nested Schema (TAJO-1329)
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-  <tns:base version="4">
+  <tns:base version="5">
     <tns:objects>
       <tns:Object order="0" type="table" name="META">
         <tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
index 34337fb..2bde04f 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/mysql/mysql.xml
@@ -19,12 +19,13 @@
 <tns:store xmlns:tns="http://tajo.apache.org/catalogstore" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
     Catalog base version history
+    * 5 - 2015-06-15: Implement TablespaceManager to load Tablespaces (TAJO-1616)
     * 4 - 2015-03-27: Partition Schema (TAJO-1284)
     * 3 - 2015-03-12: Nested Schema (TAJO-1329)
     * 2 - 2014-06-09: First versioning
     * 1-  Before 2013-03-20
   -->
-  <tns:base version="4">
+  <tns:base version="5">
     <tns:objects>
       <tns:Object order="0" type="table" name="META">
         <tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
index 323e22c..2778e0c 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/oracle/oracle.xml
@@ -19,12 +19,13 @@
 <tns:store xmlns:tns="http://tajo.apache.org/catalogstore" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
       Catalog base version history
+      * 5 - 2015-06-15: Implement TablespaceManager to load Tablespaces (TAJO-1616)
       * 4 - 2015-03-27: Partition Schema (TAJO-1284)
       * 3 - 2015-03-12: Nested Schema (TAJO-1329)
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-  <tns:base version="4">
+  <tns:base version="5">
     <tns:objects>
   		<tns:Object order="0" type="table" name="meta">
   			<tns:sql><![CDATA[

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
index 554acd5..0051242 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
+++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/postgresql/postgresql.xml
@@ -21,12 +21,14 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 xsi:schemaLocation="http://tajo.apache.org/catalogstore ../DBMSSchemaDefinition.xsd ">
   <!--
       Catalog base version history
+      * 5 - 2015-06-15: Implement TablespaceManager to load Tablespaces (TAJO-1616)
+      * 5 - 2015-06-15: Implement TablespaceManager to load Tablespaces (TAJO-1616)
       * 4 - 2015-03-27: Partition Schema (TAJO-1284)
       * 3 - 2015-03-12: Nested Schema (TAJO-1329)
       * 2 - 2014-06-09: First versioning
       * 1-  Before 2013-03-20
     -->
-	<tns:base version="4">
+	<tns:base version="5">
 		<tns:objects>
 			<tns:Object name="META" type="table" order="0">
 				<tns:sql><![CDATA[CREATE TABLE META (VERSION INT NOT NULL)]]></tns:sql>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
index 6df26b7..6b2905a 100644
--- a/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tsql/commands/DescTableCommand.java
@@ -78,7 +78,7 @@ public class DescTableCommand extends TajoShellCommand {
   protected String toFormattedString(TableDesc desc) {
     StringBuilder sb = new StringBuilder();
     sb.append("\ntable name: ").append(desc.getName()).append("\n");
-    sb.append("table path: ").append(desc.getPath()).append("\n");
+    sb.append("table uri: ").append(desc.getUri()).append("\n");
     sb.append("store type: ").append(desc.getMeta().getStoreType()).append("\n");
     if (desc.getStats() != null) {
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-common/src/main/java/org/apache/tajo/QueryVars.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java
index 55ca700..a6d5d1d 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java
@@ -22,9 +22,11 @@ import org.apache.tajo.validation.Validator;
 
 public enum QueryVars implements ConfigKey {
   COMMAND_TYPE,
+  DEFAULT_SPACE_URI,
+  DEFAULT_SPACE_ROOT_URI,
   STAGING_DIR,
   OUTPUT_TABLE_NAME,
-  OUTPUT_TABLE_PATH,
+  OUTPUT_TABLE_URI,
   OUTPUT_PARTITIONS,
   OUTPUT_OVERWRITE,
   OUTPUT_AS_DIRECTORY,

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index d2c6c1c..a9923a5 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -20,8 +20,14 @@ package org.apache.tajo.storage;
 
 import org.apache.tajo.TajoConstants;
 
+import java.net.URI;
+
 public class StorageConstants {
 
+  // Tablespace  -------------------------------------------------
+
+  public static final URI LOCAL_FS_URI = URI.create("file:/");
+
   // Common table properties -------------------------------------------------
 
   // time zone

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java
new file mode 100644
index 0000000..1057097
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageService.java
@@ -0,0 +1,37 @@
+/*
+ * 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 javax.annotation.Nullable;
+import java.net.URI;
+
+/**
+ * TablespaceManager interface for loosely coupled usages
+ */
+public interface StorageService {
+  /**
+   * Get Table URI
+   *
+   * @param spaceName Tablespace name. If it is null, the default space will be used
+   * @param databaseName Database name
+   * @param tableName Table name
+   * @return Table URI
+   */
+  URI getTableURI(@Nullable String spaceName, String databaseName, String tableName);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
index 3e3d3a2..39f4c29 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java
@@ -87,8 +87,20 @@ public class FileUtil {
     return ClassLoader.getSystemResource(resource);
   }
 
+  /**
+   * It returns a string from a text file found in classpath.
+   *
+   * @param resource Resource file name
+   * @return String contents if exists. Otherwise, it will return null.
+   * @throws IOException
+   */
   public static String readTextFileFromResource(String resource) throws IOException {
-    return readTextFromStream(ClassLoader.getSystemResourceAsStream(resource));
+    InputStream stream = ClassLoader.getSystemResourceAsStream(resource);
+    if (stream != null) {
+      return readTextFromStream(stream);
+    } else {
+      return null;
+    }
   }
 
   public static String readTextFromStream(InputStream inputStream)

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
index e2def69..5a712c0 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/ReflectionUtil.java
@@ -25,9 +25,9 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 public class ReflectionUtil {
-  private static final Class<?>[] EMPTY_PARAM = new Class[]{};
-  private static final Object [] EMPTY_OBJECT = new Object[] {};
-  private static final Class<?>[] CONF_PARAM = new Class[]{TajoConf.class};
+  public static final Class<?>[] EMPTY_PARAM = new Class[]{};
+  public static final Object [] EMPTY_OBJECT = new Object[] {};
+  public static final Class<?>[] CONF_PARAM = new Class[]{TajoConf.class};
 
   /**
    * Caches of constructors for each class. Pins the classes so they

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------
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 3ab11bd..469b2a2 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
@@ -98,11 +98,11 @@ if_exists
   ;
 
 create_table_statement
-  : 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)?
+  : CREATE EXTERNAL TABLE (if_not_exists)? table_name table_elements (TABLESPACE spacename=identifier)? USING storage_type=identifier
+    (param_clause)? (table_partitioning_clauses)? (LOCATION uri=Character_String_Literal)?
+  | CREATE TABLE (if_not_exists)? table_name table_elements (TABLESPACE spacename=identifier)? (USING storage_type=identifier)?
     (param_clause)? (table_partitioning_clauses)? (AS query_expression)?
-  | CREATE TABLE (if_not_exists)? table_name (USING storage_type=identifier)?
+  | CREATE TABLE (if_not_exists)? table_name (TABLESPACE spacename=identifier)? (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
   ;

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
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 7c99868..62bb0f9 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
@@ -1252,9 +1252,9 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
       createTable.setTableElements(elements);
       createTable.setStorageType(storageType);
 
-      if (PlannerUtil.isFileStorageType(storageType)) {
-        String path = stripQuote(ctx.path.getText());
-        createTable.setLocation(path);
+      if (checkIfExist(ctx.LOCATION())) {
+        String uri = stripQuote(ctx.uri.getText());
+        createTable.setLocation(uri);
       }
     } else {
       if (checkIfExist(ctx.table_elements())) {
@@ -1262,6 +1262,11 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
         createTable.setTableElements(elements);
       }
 
+      if (checkIfExist(ctx.TABLESPACE())) {
+        String spaceName = ctx.spacename.getText();
+        createTable.setTableSpaceName(spaceName);
+      }
+
       if (checkIfExist(ctx.USING())) {
         String fileType = ctx.storage_type.getText();
         createTable.setStorageType(fileType);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
index 30cb24f..f0b2f5e 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,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.SessionVars;
+import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.SortSpec;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -251,7 +252,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
       FragmentProto[] fragmentProtos = ctx.getTables(tableId);
       List<Fragment> fragments = FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
       for (Fragment frag : fragments) {
-        size += Tablespace.getFragmentLength(ctx.getConf(), frag);
+        size += TableSpaceManager.guessFragmentVolume(ctx.getConf(), frag);
       }
     }
     return size;
@@ -924,9 +925,10 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
         if (broadcastFlag) {
           PartitionedTableScanNode partitionedTableScanNode = (PartitionedTableScanNode) scanNode;
           List<Fragment> fileFragments = TUtil.newList();
-          FileTablespace fileStorageManager = (FileTablespace) TableSpaceManager.getFileStorageManager(ctx.getConf());
+
+          FileTablespace space = (FileTablespace) TableSpaceManager.get(scanNode.getTableDesc().getUri()).get();
           for (Path path : partitionedTableScanNode.getInputPaths()) {
-            fileFragments.addAll(TUtil.newList(fileStorageManager.split(scanNode.getCanonicalName(), path)));
+            fileFragments.addAll(TUtil.newList(space.split(scanNode.getCanonicalName(), path)));
           }
 
           FragmentProto[] fragments =
@@ -1188,8 +1190,10 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
         FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
 
     String indexName = IndexUtil.getIndexNameOfFrag(fragments.get(0), annotation.getSortKeys());
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(ctx.getConf());
-    Path indexPath = new Path(sm.getTablePath(annotation.getTableName()), "index");
+    FileTablespace sm = (FileTablespace) TableSpaceManager.get(fragments.get(0).getPath().toUri()).get();
+    String dbName = CatalogUtil.extractQualifier(annotation.getTableName());
+    String simpleName = CatalogUtil.extractSimpleName(annotation.getTableName());
+    Path indexPath = new Path(new Path(sm.getTableUri(dbName, simpleName)), "index");
 
     TupleComparator comp = new BaseTupleComparator(annotation.getKeySchema(),
         annotation.getSortKeys());

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
index 89e887a..ba4833b 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
@@ -48,6 +48,7 @@ import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.rewrite.rules.ProjectionPushDownRule;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.visitor.BasicLogicalPlanVisitor;
+import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.ReflectionUtil;
 import org.apache.tajo.util.TUtil;
@@ -170,7 +171,8 @@ public class GlobalPlanner {
         "Channel schema (" + channel.getSrcId().getId() + " -> " + channel.getTargetId().getId() +
             ") is not initialized");
     TableMeta meta = new TableMeta(channel.getStoreType(), new KeyValueSet());
-    TableDesc desc = new TableDesc(channel.getSrcId().toString(), channel.getSchema(), meta, new Path("/").toUri());
+    TableDesc desc = new TableDesc(
+        channel.getSrcId().toString(), channel.getSchema(), meta, StorageConstants.LOCAL_FS_URI);
     ScanNode scanNode = plan.createNode(ScanNode.class);
     scanNode.init(desc);
     return scanNode;

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java
----------------------------------------------------------------------
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 bc6975a..54abca8 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
@@ -56,7 +56,7 @@ public class BSTIndexScanExec extends PhysicalExec {
     this.qual = scanNode.getQual();
     this.datum = datum;
 
-    this.fileScanner = TableSpaceManager.getSeekableScanner(context.getConf(),
+    this.fileScanner = OldStorageManager.getSeekableScanner(context.getConf(),
         scanNode.getTableDesc().getMeta(), scanNode.getInSchema(), fragment, outSchema);
     this.fileScanner.init();
     this.projector = new Projector(context, inSchema, outSchema, scanNode.getTargets());

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
----------------------------------------------------------------------
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 3121671..969998c 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
@@ -165,7 +165,7 @@ public abstract class ColPartitionStoreExec extends UnaryPhysicalExec {
       actualFilePath = new Path(lastFileName + "_" + suffixId);
     }
 
-    appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(context.getConf()))
+    appender = ((FileTablespace) TableSpaceManager.get(lastFileName.toUri()).get())
         .getAppender(meta, outSchema, actualFilePath);
 
     appender.enableStats();

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
----------------------------------------------------------------------
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 d240edb..deda498 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
@@ -74,7 +74,7 @@ public class PhysicalPlanUtil {
    */
   public static CatalogProtos.FragmentProto[] getNonZeroLengthDataFiles(TajoConf tajoConf,TableDesc tableDesc,
                                                           int fileIndex, int numResultFiles) throws IOException {
-    Path path = new Path(tableDesc.getPath());
+    Path path = new Path(tableDesc.getUri());
     FileSystem fs = path.getFileSystem(tajoConf);
 
     //In the case of partitioned table, we should return same partition key data files.

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
----------------------------------------------------------------------
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 3dd1cd9..fb29e4f 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
@@ -77,7 +77,7 @@ public class RangeShuffleFileWriteExec extends UnaryPhysicalExec {
         context.getDataChannel().getStoreType() : "RAW");
     FileSystem fs = new RawLocalFileSystem();
     fs.mkdirs(storeTablePath);
-    this.appender = (FileAppender) ((FileTablespace) TableSpaceManager.getFileStorageManager(context.getConf()))
+    this.appender = (FileAppender) ((FileTablespace) TableSpaceManager.getDefault())
         .getAppender(meta, outSchema, new Path(storeTablePath, "output"));
     this.appender.enableStats();
     this.appender.init();

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
index b01af6c..d2ae3bd 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
@@ -202,10 +202,8 @@ public class SeqScanExec extends ScanExec {
             FragmentConvertor.convert(context.getConf(), fragments), projected
         );
       } else {
-        Tablespace tablespace = TableSpaceManager.getStorageManager(
-            context.getConf(), plan.getTableDesc().getMeta().getStoreType());
-        this.scanner = tablespace.getScanner(meta,
-            plan.getPhysicalSchema(), fragments[0], projected);
+        Tablespace tablespace = TableSpaceManager.get(plan.getTableDesc().getUri()).get();
+        this.scanner = tablespace.getScanner(meta, plan.getPhysicalSchema(), fragments[0], projected);
       }
       scanner.init();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
----------------------------------------------------------------------
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 5b17eee..dd8768e 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
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.engine.planner.physical;
 
+import com.google.common.base.Optional;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -35,6 +36,7 @@ import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
+import java.net.URI;
 
 /**
  * This is a physical executor to store a table part into a specified storage.
@@ -90,17 +92,26 @@ public class StoreTableExec extends UnaryPhysicalExec {
         lastFileName = new Path(lastFileName + "_" + suffixId);
       }
 
-      appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(context.getConf()))
-          .getAppender(meta, appenderSchema, lastFileName);
+      Optional<FileTablespace> spaceRes = TableSpaceManager.get(lastFileName.toUri());
+      if (!spaceRes.isPresent())  {
+        throw new IllegalStateException("No Tablespace for " + lastFileName.toUri());
+      }
+
+      FileTablespace space = spaceRes.get();
+      appender = space.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 = TableSpaceManager.getStorageManager(context.getConf(), meta.getStoreType()).getAppender(
+      Path stagingDir = context.getQueryContext().getStagingDir();
+      appender = TableSpaceManager.get(stagingDir.toUri()).get().getAppender(
           context.getQueryContext(),
-          context.getTaskId(), meta, appenderSchema, context.getQueryContext().getStagingDir());
+          context.getTaskId(),
+          meta,
+          appenderSchema,
+          stagingDir);
     }
 
     appender.enableStats();

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
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 ee50221..7696c6c 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
@@ -28,6 +28,8 @@ import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.session.Session;
 
+import java.net.URI;
+
 import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
 
 /**
@@ -63,6 +65,40 @@ public class QueryContext extends OverridableConf {
     return get(SessionVars.USERNAME);
   }
 
+  /**
+   * Set the default tablespace uri
+   *
+   * @param uri The default tablespace uri
+   */
+  public void setDefaultSpaceUri(URI uri) {
+    put(QueryVars.DEFAULT_SPACE_URI, uri.toString());
+  }
+
+  /**
+   * Return the default tablespace uri
+   */
+  public URI getDefaultSpaceUri() {
+    String strVal = get(QueryVars.DEFAULT_SPACE_URI, "");
+    return strVal != null && !strVal.isEmpty() ? URI.create(strVal) : null;
+  }
+
+  /**
+   * Set the root uri of the default tablespace
+   *
+   * @param uri The root uri of the default tablespace
+   */
+  public void setDefaultSpaceRootUri(URI uri) {
+    put(QueryVars.DEFAULT_SPACE_ROOT_URI, uri.toString());
+  }
+
+  /**
+   * Return the root of the default tablespace
+   */
+  public URI getDefaultSpaceRootUri() {
+    String strVal = get(QueryVars.DEFAULT_SPACE_ROOT_URI, "");
+    return strVal != null && !strVal.isEmpty() ? URI.create(strVal) : null;
+  }
+
   public void setStagingDir(Path path) {
     put(QueryVars.STAGING_DIR, path.toUri().toString());
   }
@@ -82,24 +118,33 @@ public class QueryContext extends OverridableConf {
   }
 
   /**
-   * The fact that QueryContext has an output path means this query will write the output to a specific directory.
-   * In other words, this query is 'CREATE TABLE' or 'INSERT (OVERWRITE) INTO (<table name>|LOCATION)' statement.
+   * The final output table's uri. It will be set if a query is CTAS or INSERT (OVERWRITE) INTO statement
    *
-   * @return
+   * @return True if a output table uri is set. Otherwise, it will return false
    */
-  public boolean hasOutputPath() {
-    return containsKey(QueryVars.OUTPUT_TABLE_PATH);
+  public boolean hasOutputTableUri() {
+    return containsKey(QueryVars.OUTPUT_TABLE_URI);
   }
 
-  public void setOutputPath(Path path) {
-    if (path != null) {
-      put(QueryVars.OUTPUT_TABLE_PATH, path.toUri().toString());
+  /**
+   * Set the final output table uri
+   *
+   * @param uri
+   */
+  public void setOutputPath(URI uri) {
+    if (uri != null) {
+      put(QueryVars.OUTPUT_TABLE_URI, uri.toString());
     }
   }
 
-  public Path getOutputPath() {
-    String strVal = get(QueryVars.OUTPUT_TABLE_PATH);
-    return strVal != null ? new Path(strVal) : null;
+  /**
+   * Get the final output table uri
+   *
+   * @return The final output table uri
+   */
+  public URI getOutputTableUri() {
+    String strVal = get(QueryVars.OUTPUT_TABLE_URI);
+    return strVal != null ? URI.create(strVal) : null;
   }
 
   public boolean hasPartition() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
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 2cd585f..e833884 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
@@ -53,7 +53,6 @@ import org.apache.tajo.plan.verifier.LogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.VerificationState;
 import org.apache.tajo.plan.verifier.VerifyException;
-import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 
@@ -68,7 +67,6 @@ public class GlobalEngine extends AbstractService {
   private final static Log LOG = LogFactory.getLog(GlobalEngine.class);
 
   private final MasterContext context;
-  private final Tablespace sm;
 
   private SQLAnalyzer analyzer;
   private CatalogService catalog;
@@ -84,7 +82,6 @@ public class GlobalEngine extends AbstractService {
     super(GlobalEngine.class.getName());
     this.context = context;
     this.catalog = context.getCatalog();
-    this.sm = context.getStorageManager();
 
     this.ddlExecutor = new DDLExecutor(context);
     this.queryExecutor = new QueryExecutor(context, ddlExecutor);
@@ -94,7 +91,7 @@ public class GlobalEngine extends AbstractService {
     try  {
       analyzer = new SQLAnalyzer();
       preVerifier = new PreLogicalPlanVerifier(context.getCatalog());
-      planner = new LogicalPlanner(context.getCatalog());
+      planner = new LogicalPlanner(context.getCatalog(), TableSpaceManager.getInstance());
       optimizer = new LogicalOptimizer(context.getConf());
       annotatedPlanVerifier = new LogicalPlanVerifier(context.getConf(), context.getCatalog());
     } catch (Throwable t) {
@@ -143,6 +140,10 @@ public class GlobalEngine extends AbstractService {
   private QueryContext createQueryContext(Session session) {
     QueryContext newQueryContext =  new QueryContext(context.getConf(), session);
 
+    // Set default space uri and its root uri
+    newQueryContext.setDefaultSpaceUri(TableSpaceManager.getDefault().getUri());
+    newQueryContext.setDefaultSpaceRootUri(TableSpaceManager.getDefault().getRootUri());
+
     String tajoTest = System.getProperty(CommonTestingUtil.TAJO_TEST_KEY);
     if (tajoTest != null && tajoTest.equalsIgnoreCase(CommonTestingUtil.TAJO_TEST_TRUE)) {
       newQueryContext.putAll(CommonTestingUtil.getSessionVarsForTest());
@@ -302,8 +303,8 @@ public class GlobalEngine extends AbstractService {
           InsertNode iNode = rootNode.getChild();
           Schema outSchema = iNode.getChild().getOutSchema();
 
-          TableSpaceManager.getStorageManager(queryContext.getConf(), storeType)
-              .verifyInsertTableSchema(tableDesc, outSchema);
+          TableSpaceManager.get(tableDesc.getUri()).get().verifySchemaToWrite(tableDesc, outSchema);
+
         } catch (Throwable t) {
           state.addVerification(t.getMessage());
         }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
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 c41fdde..e1e85dd 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
@@ -21,7 +21,10 @@ package org.apache.tajo.master;
 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.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.service.CompositeService;
@@ -53,8 +56,6 @@ import org.apache.tajo.rule.SelfDiagnosisRuleSession;
 import org.apache.tajo.service.ServiceTracker;
 import org.apache.tajo.service.ServiceTrackerFactory;
 import org.apache.tajo.session.SessionManager;
-import org.apache.tajo.storage.Tablespace;
-import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.*;
 import org.apache.tajo.util.history.HistoryReader;
 import org.apache.tajo.util.history.HistoryWriter;
@@ -114,7 +115,6 @@ public class TajoMaster extends CompositeService {
 
   private CatalogServer catalogServer;
   private CatalogService catalog;
-  private Tablespace storeManager;
   private GlobalEngine globalEngine;
   private AsyncDispatcher dispatcher;
   private TajoMasterClientService tajoMasterClientService;
@@ -183,7 +183,6 @@ public class TajoMaster extends CompositeService {
       // check the system directory and create if they are not created.
       checkAndInitializeSystemDirectories();
       diagnoseTajoMaster();
-      this.storeManager = TableSpaceManager.getFileStorageManager(systemConf);
 
       catalogServer = new CatalogServer(loadFunctions());
       addIfService(catalogServer);
@@ -211,7 +210,14 @@ public class TajoMaster extends CompositeService {
       throw e;
     }
 
-    super.serviceInit(systemConf);
+    // Try to start up all services in TajoMaster.
+    // If anyone is failed, the master prints out the errors and immediately should shutdowns
+    try {
+      super.serviceInit(systemConf);
+    } catch (Throwable t) {
+      t.printStackTrace();
+      System.exit(1);
+    }
     LOG.info("Tajo Master is initialized.");
   }
 
@@ -477,10 +483,6 @@ public class TajoMaster extends CompositeService {
       return globalEngine;
     }
 
-    public Tablespace getStorageManager() {
-      return storeManager;
-    }
-
     public QueryCoordinatorService getTajoMasterService() {
       return tajoMasterService;
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
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 31eecdc..7dbe815 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
@@ -844,8 +844,7 @@ public class TajoMasterClientService extends AbstractService {
         TableDesc desc;
         try {
           desc = context.getGlobalEngine().getDDLExecutor().createTable(queryContext, request.getName(),
-              meta.getStoreType(), schema,
-              meta, path, true, partitionDesc, false);
+              null, meta.getStoreType(), schema, meta, path.toUri(), true, partitionDesc, false);
         } catch (Exception e) {
           return TableResponse.newBuilder()
               .setResultCode(ResultCode.ERROR)

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
index 93c950e..5e0e639 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
@@ -36,11 +36,12 @@ import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.storage.TableSpaceManager;
 
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -54,12 +55,10 @@ public class DDLExecutor {
 
   private final TajoMaster.MasterContext context;
   private final CatalogService catalog;
-  private final Tablespace tablespace;
 
   public DDLExecutor(TajoMaster.MasterContext context) {
     this.context = context;
     this.catalog = context.getCatalog();
-    this.tablespace = context.getStorageManager();
   }
 
   public boolean execute(QueryContext queryContext, LogicalPlan plan) throws IOException {
@@ -202,17 +201,31 @@ public class DDLExecutor {
     }
 
     if(PlannerUtil.isFileStorageType(createTable.getStorageType()) && createTable.isExternal()){
-      Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given.");
+      Preconditions.checkState(createTable.hasUri(), "ERROR: LOCATION must be given.");
     }
 
-    return createTable(queryContext, createTable.getTableName(), createTable.getStorageType(),
-        createTable.getTableSchema(), meta, createTable.getPath(), createTable.isExternal(),
-        createTable.getPartitionMethod(), ifNotExists);
+    return createTable(
+        queryContext,
+        createTable.getTableName(),
+        createTable.getTableSpaceName(),
+        createTable.getStorageType(),createTable.getTableSchema(),
+        meta,
+        createTable.getUri(),
+        createTable.isExternal(),
+        createTable.getPartitionMethod(),
+        ifNotExists);
   }
 
-  public TableDesc createTable(QueryContext queryContext, String tableName, String storeType,
-                               Schema schema, TableMeta meta, Path path, boolean isExternal,
-                               PartitionMethodDesc partitionDesc, boolean ifNotExists) throws IOException {
+  public TableDesc createTable(QueryContext queryContext,
+                               String tableName,
+                               @Nullable String tableSpaceName,
+                               @Nullable String storeType,
+                               Schema schema,
+                               TableMeta meta,
+                               @Nullable URI uri,
+                               boolean isExternal,
+                               @Nullable PartitionMethodDesc partitionDesc,
+                               boolean ifNotExists) throws IOException {
     String databaseName;
     String simpleTableName;
     if (CatalogUtil.isFQTableName(tableName)) {
@@ -232,18 +245,28 @@ public class DDLExecutor {
         LOG.info("relation \"" + qualifiedName + "\" is already exists." );
         return catalog.getTableDesc(databaseName, simpleTableName);
       } else {
-        throw new AlreadyExistsTableException(CatalogUtil.buildFQName(databaseName, tableName));
+        throw new AlreadyExistsTableException(qualifiedName);
       }
     }
 
-    TableDesc desc = new TableDesc(CatalogUtil.buildFQName(databaseName, simpleTableName),
-        schema, meta, (path != null ? path.toUri(): null), isExternal);
+    Tablespace tableSpace;
+    if (tableSpaceName != null) {
+      tableSpace = TableSpaceManager.getByName(tableSpaceName).get();
+    } else if (uri != null) {
+      tableSpace = TableSpaceManager.get(uri).get();
+    } else {
+      tableSpace = TableSpaceManager.getDefault();
+    }
+
+    TableDesc desc;
+    URI tableUri = isExternal ? uri : tableSpace.getTableUri(databaseName, simpleTableName);
+    desc = new TableDesc(qualifiedName, schema, meta, tableUri, isExternal);
 
     if (partitionDesc != null) {
       desc.setPartitionMethod(partitionDesc);
     }
 
-    TableSpaceManager.getStorageManager(queryContext.getConf(), storeType).createTable(desc, ifNotExists);
+    tableSpace.createTable(desc, ifNotExists);
 
     if (catalog.createTable(desc)) {
       LOG.info("Table " + desc.getName() + " is created (" + desc.getStats().getNumBytes() + ")");
@@ -290,8 +313,7 @@ public class DDLExecutor {
 
     if (purge) {
       try {
-        TableSpaceManager.getStorageManager(queryContext.getConf(),
-            tableDesc.getMeta().getStoreType()).purgeTable(tableDesc);
+        TableSpaceManager.get(tableDesc.getUri()).get().purgeTable(tableDesc);
       } catch (IOException e) {
         throw new InternalError(e.getMessage());
       }
@@ -330,7 +352,7 @@ public class DDLExecutor {
 
       Path warehousePath = new Path(TajoConf.getWarehouseDir(context.getConf()), databaseName);
       TableDesc tableDesc = catalog.getTableDesc(databaseName, simpleTableName);
-      Path tablePath = new Path(tableDesc.getPath());
+      Path tablePath = new Path(tableDesc.getUri());
       if (tablePath.getParent() == null ||
           !tablePath.getParent().toUri().getPath().equals(warehousePath.toUri().getPath())) {
         throw new IOException("Can't truncate external table:" + eachTableName + ", data dir=" + tablePath +
@@ -340,7 +362,7 @@ public class DDLExecutor {
     }
 
     for (TableDesc eachTable: tableDescList) {
-      Path path = new Path(eachTable.getPath());
+      Path path = new Path(eachTable.getUri());
       LOG.info("Truncate table: " + eachTable.getName() + ", delete all data files in " + path);
       FileSystem fs = path.getFileSystem(context.getConf());
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
index 8f6c6f9..ae57453 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
@@ -101,7 +101,7 @@ public class NonForwardQueryResultFileScanner implements NonForwardQueryResultSc
   }
 
   private void initSeqScanExec() throws IOException {
-    Tablespace tablespace = TableSpaceManager.getStorageManager(tajoConf, tableDesc.getMeta().getStoreType());
+    Tablespace tablespace = TableSpaceManager.get(tableDesc.getUri()).get();
     List<Fragment> fragments = null;
     setPartition(tablespace);
     fragments = tablespace.getNonForwardSplit(tableDesc, currentFragmentIndex, MAX_FRAGMENT_NUM_PER_SCAN);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
index 281edad..480f45c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@ -43,24 +43,26 @@ import org.apache.tajo.engine.planner.physical.StoreTableExec;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
-import org.apache.tajo.master.*;
+import org.apache.tajo.master.QueryInfo;
+import org.apache.tajo.master.QueryManager;
+import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.exec.prehook.CreateTableHook;
 import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
 import org.apache.tajo.master.exec.prehook.InsertIntoHook;
-import org.apache.tajo.plan.expr.EvalContext;
-import org.apache.tajo.plan.expr.GeneralFunctionEval;
-import org.apache.tajo.plan.function.python.PythonScriptEngine;
-import org.apache.tajo.plan.function.python.TajoScriptEngine;
-import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
-import org.apache.tajo.querymaster.*;
-import org.apache.tajo.session.Session;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.Target;
+import org.apache.tajo.plan.expr.EvalContext;
 import org.apache.tajo.plan.expr.EvalNode;
+import org.apache.tajo.plan.expr.GeneralFunctionEval;
+import org.apache.tajo.plan.function.python.PythonScriptEngine;
+import org.apache.tajo.plan.function.python.TajoScriptEngine;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.verifier.VerifyException;
+import org.apache.tajo.querymaster.Query;
+import org.apache.tajo.querymaster.QueryMasterTask;
+import org.apache.tajo.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.util.ProtoUtil;
 import org.apache.tajo.worker.TaskAttemptContext;
@@ -329,104 +331,108 @@ public class QueryExecutor {
   }
 
   private void insertNonFromQuery(QueryContext queryContext,
-                                  InsertNode insertNode, SubmitQueryResponse.Builder responseBuilder)
-      throws Exception {
-    String nodeUniqName = insertNode.getTableName() == null ? insertNode.getPath().getName() : insertNode.getTableName();
-    String queryId = nodeUniqName + "_" + System.currentTimeMillis();
-
-    FileSystem fs = TajoConf.getWarehouseDir(context.getConf()).getFileSystem(context.getConf());
-    Path stagingDir = QueryMasterTask.initStagingDir(context.getConf(), queryId.toString(), queryContext);
-    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-
-    TableDesc tableDesc = null;
-    Path finalOutputDir = null;
-    if (insertNode.getTableName() != null) {
-      tableDesc = this.catalog.getTableDesc(insertNode.getTableName());
-      finalOutputDir = new Path(tableDesc.getPath());
-    } else {
-      finalOutputDir = insertNode.getPath();
-    }
-
-    TaskAttemptContext taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
-    taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
-
-    EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
-    StoreTableExec exec = new StoreTableExec(taskAttemptContext, insertNode, evalExprExec);
+                                  InsertNode insertNode, SubmitQueryResponse.Builder responseBuilder) {
     try {
-      exec.init();
-      exec.next();
-    } finally {
-      exec.close();
-    }
+      String nodeUniqName = insertNode.getTableName() == null ? new Path(insertNode.getUri()).getName() :
+          insertNode.getTableName();
+      String queryId = nodeUniqName + "_" + System.currentTimeMillis();
+
+      FileSystem fs = TajoConf.getWarehouseDir(context.getConf()).getFileSystem(context.getConf());
+      Path stagingDir = QueryMasterTask.initStagingDir(context.getConf(), queryId.toString(), queryContext);
+      Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+
+      TableDesc tableDesc = null;
+      Path finalOutputDir;
+      if (insertNode.getTableName() != null) {
+        tableDesc = this.catalog.getTableDesc(insertNode.getTableName());
+        finalOutputDir = new Path(tableDesc.getUri());
+      } else {
+        finalOutputDir = new Path(insertNode.getUri());
+      }
+
+      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
+      taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
 
-    if (insertNode.isOverwrite()) { // 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);
+      EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
+      StoreTableExec exec = new StoreTableExec(taskAttemptContext, insertNode, evalExprExec);
       try {
-        if (fs.exists(finalOutputDir)) {
-          fs.rename(finalOutputDir, oldTableDir);
-          movedToOldTable = fs.exists(oldTableDir);
-        } else { // if the parent does not exist, make its parent directory.
-          fs.mkdirs(finalOutputDir.getParent());
-        }
-        fs.rename(stagingResultDir, finalOutputDir);
-        committed = fs.exists(finalOutputDir);
-      } catch (IOException ioe) {
-        // recover the old table
-        if (movedToOldTable && !committed) {
-          fs.rename(oldTableDir, finalOutputDir);
-        }
+        exec.init();
+        exec.next();
+      } finally {
+        exec.close();
       }
-    } else {
-      FileStatus[] files = fs.listStatus(stagingResultDir);
-      for (FileStatus eachFile: files) {
-        Path targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName());
-        if (fs.exists(targetFilePath)) {
-          targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName() + "_" + System.currentTimeMillis());
+
+      if (insertNode.isOverwrite()) { // 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);
+        try {
+          if (fs.exists(finalOutputDir)) {
+            fs.rename(finalOutputDir, oldTableDir);
+            movedToOldTable = fs.exists(oldTableDir);
+          } else { // if the parent does not exist, make its parent directory.
+            fs.mkdirs(finalOutputDir.getParent());
+          }
+          fs.rename(stagingResultDir, finalOutputDir);
+          committed = fs.exists(finalOutputDir);
+        } catch (IOException ioe) {
+          // recover the old table
+          if (movedToOldTable && !committed) {
+            fs.rename(oldTableDir, finalOutputDir);
+          }
+        }
+      } else {
+        FileStatus[] files = fs.listStatus(stagingResultDir);
+        for (FileStatus eachFile : files) {
+          Path targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName());
+          if (fs.exists(targetFilePath)) {
+            targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName() + "_" + System.currentTimeMillis());
+          }
+          fs.rename(eachFile.getPath(), targetFilePath);
         }
-        fs.rename(eachFile.getPath(), targetFilePath);
       }
-    }
 
-    if (insertNode.hasTargetTable()) {
-      TableStats stats = tableDesc.getStats();
-      long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-      stats.setNumBytes(volume);
-      stats.setNumRows(1);
+      if (insertNode.hasTargetTable()) {
+        TableStats stats = tableDesc.getStats();
+        long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
+        stats.setNumBytes(volume);
+        stats.setNumRows(1);
 
-      CatalogProtos.UpdateTableStatsProto.Builder builder = CatalogProtos.UpdateTableStatsProto.newBuilder();
-      builder.setTableName(tableDesc.getName());
-      builder.setStats(stats.getProto());
+        CatalogProtos.UpdateTableStatsProto.Builder builder = CatalogProtos.UpdateTableStatsProto.newBuilder();
+        builder.setTableName(tableDesc.getName());
+        builder.setStats(stats.getProto());
 
-      catalog.updateTableStats(builder.build());
+        catalog.updateTableStats(builder.build());
 
-      responseBuilder.setTableDesc(tableDesc.getProto());
-    } else {
-      TableStats stats = new TableStats();
-      long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-      stats.setNumBytes(volume);
-      stats.setNumRows(1);
-
-      // Empty TableDesc
-      List<CatalogProtos.ColumnProto> columns = new ArrayList<CatalogProtos.ColumnProto>();
-      CatalogProtos.TableDescProto tableDescProto = CatalogProtos.TableDescProto.newBuilder()
-          .setTableName(nodeUniqName)
-          .setMeta(CatalogProtos.TableProto.newBuilder().setStoreType("CSV").build())
-          .setSchema(CatalogProtos.SchemaProto.newBuilder().addAllFields(columns).build())
-          .setStats(stats.getProto())
-          .build();
-
-      responseBuilder.setTableDesc(tableDescProto);
-    }
+        responseBuilder.setTableDesc(tableDesc.getProto());
+      } else {
+        TableStats stats = new TableStats();
+        long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
+        stats.setNumBytes(volume);
+        stats.setNumRows(1);
+
+        // Empty TableDesc
+        List<CatalogProtos.ColumnProto> columns = new ArrayList<CatalogProtos.ColumnProto>();
+        CatalogProtos.TableDescProto tableDescProto = CatalogProtos.TableDescProto.newBuilder()
+            .setTableName(nodeUniqName)
+            .setMeta(CatalogProtos.TableProto.newBuilder().setStoreType("CSV").build())
+            .setSchema(CatalogProtos.SchemaProto.newBuilder().addAllFields(columns).build())
+            .setStats(stats.getProto())
+            .build();
+
+        responseBuilder.setTableDesc(tableDescProto);
+      }
 
-    // If queryId == NULL_QUERY_ID and MaxRowNum == -1, TajoCli prints only number of inserted rows.
-    responseBuilder.setMaxRowNum(-1);
-    responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
-    responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+      // If queryId == NULL_QUERY_ID and MaxRowNum == -1, TajoCli prints only number of inserted rows.
+      responseBuilder.setMaxRowNum(-1);
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+    } catch (Throwable t) {
+      throw new RuntimeException(t);
+    }
   }
 
   public void executeDistributedQuery(QueryContext queryContext, Session session,
@@ -436,14 +442,17 @@ public class QueryExecutor {
                                       SubmitQueryResponse.Builder responseBuilder) throws Exception {
     LogicalRootNode rootNode = plan.getRootBlock().getRoot();
 
-    String storeType = PlannerUtil.getStoreType(plan);
-    if (storeType != null) {
-      Tablespace sm = TableSpaceManager.getStorageManager(context.getConf(), storeType);
-      StorageProperty storageProperty = sm.getStorageProperty();
-      if (!storageProperty.isSupportsInsertInto()) {
+    TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, plan.getRootBlock().getRoot());
+    if (tableDesc != null) {
+
+      Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
+      StorageProperty storageProperty = space.getProperty();
+
+      if (!storageProperty.isInsertable()) {
         throw new VerifyException("Inserting into non-file storage is not supported.");
       }
-      sm.beforeInsertOrCATS(rootNode.getChild());
+
+      space.prepareTable(rootNode.getChild());
     }
     context.getSystemMetrics().counter("Query", "numDMLQuery").inc();
     hookManager.doHooks(queryContext, plan);
@@ -471,28 +480,15 @@ public class QueryExecutor {
     }
   }
 
-  public static MasterPlan compileMasterPlan(LogicalPlan plan, QueryContext context, GlobalPlanner planner)
+  public MasterPlan compileMasterPlan(LogicalPlan plan, QueryContext context, GlobalPlanner planner)
       throws Exception {
 
-    String storeType = PlannerUtil.getStoreType(plan);
-    if (storeType != null) {
-      Tablespace sm = TableSpaceManager.getStorageManager(planner.getConf(), storeType);
-      StorageProperty storageProperty = sm.getStorageProperty();
-      if (storageProperty.isSortedInsert()) {
-        String tableName = PlannerUtil.getStoreTableName(plan);
-        LogicalRootNode rootNode = plan.getRootBlock().getRoot();
-        TableDesc tableDesc = PlannerUtil.getTableDesc(planner.getCatalog(), rootNode.getChild());
-        if (tableDesc == null) {
-          throw new VerifyException("Can't get table meta data from catalog: " + tableName);
-        }
-        List<LogicalPlanRewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
-            context, tableDesc);
-        if (storageSpecifiedRewriteRules != null) {
-          for (LogicalPlanRewriteRule eachRule: storageSpecifiedRewriteRules) {
-            eachRule.rewrite(context, plan);
-          }
-        }
-      }
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+    TableDesc tableDesc = PlannerUtil.getTableDesc(planner.getCatalog(), rootNode.getChild());
+
+    if (tableDesc != null) {
+      Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
+      space.rewritePlan(context, plan);
     }
 
     MasterPlan masterPlan = new MasterPlan(QueryIdFactory.NULL_QUERY_ID, context, plan);


[12/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces. (missed commits)

Posted by ji...@apache.org.
TAJO-1616: Implement TablespaceManager to load Tablespaces. (missed commits)


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

Branch: refs/heads/index_support
Commit: 90afaa468080d4f743ed2eee8326a38995900807
Parents: d440727
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Jun 24 17:55:10 2015 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Jun 24 17:55:10 2015 -0700

----------------------------------------------------------------------
 .travis.yml                                     |   4 +-
 .../apache/tajo/storage/StorageConstants.java   |   3 +
 .../java/org/apache/tajo/util/KeyValueSet.java  |   8 +-
 .../engine/planner/PhysicalPlannerImpl.java     |   6 +-
 .../planner/physical/ColPartitionStoreExec.java |   2 +-
 .../engine/planner/physical/InsertRowsExec.java | 107 +++++
 .../physical/RangeShuffleFileWriteExec.java     |   2 +-
 .../engine/planner/physical/SeqScanExec.java    |   2 +-
 .../engine/planner/physical/StoreTableExec.java |  10 +-
 .../apache/tajo/engine/query/QueryContext.java  |   4 +-
 .../org/apache/tajo/master/GlobalEngine.java    |  10 +-
 .../apache/tajo/master/exec/DDLExecutor.java    |  10 +-
 .../exec/NonForwardQueryResultFileScanner.java  |   2 +-
 .../apache/tajo/master/exec/QueryExecutor.java  | 179 +++++----
 .../master/exec/prehook/CreateTableHook.java    |   2 -
 .../java/org/apache/tajo/querymaster/Query.java |   6 +-
 .../tajo/querymaster/QueryMasterTask.java       | 118 +-----
 .../apache/tajo/querymaster/Repartitioner.java  |  10 +-
 .../java/org/apache/tajo/querymaster/Stage.java |   4 +-
 .../org/apache/tajo/worker/LegacyTaskImpl.java  |   2 +-
 .../java/org/apache/tajo/worker/TaskImpl.java   |   2 +-
 .../src/main/resources/webapps/admin/index.jsp  |   4 +-
 .../org/apache/tajo/BackendTestingUtil.java     |   2 +-
 .../java/org/apache/tajo/QueryTestCaseBase.java |  35 ++
 .../org/apache/tajo/TajoTestingCluster.java     |  18 +-
 .../org/apache/tajo/cli/tsql/TestTajoCli.java   |   4 +-
 .../apache/tajo/engine/eval/ExprTestBase.java   |   5 +-
 .../tajo/engine/eval/TestEvalTreeUtil.java      |   5 +-
 .../engine/planner/TestLogicalOptimizer.java    |   4 +-
 .../tajo/engine/planner/TestLogicalPlan.java    |   4 +-
 .../tajo/engine/planner/TestLogicalPlanner.java |   4 +-
 .../tajo/engine/planner/TestPlannerUtil.java    |   4 +-
 .../planner/physical/TestBNLJoinExec.java       |   6 +-
 .../planner/physical/TestBSTIndexExec.java      |   4 +-
 .../planner/physical/TestExternalSortExec.java  |   4 +-
 .../physical/TestFullOuterHashJoinExec.java     |  10 +-
 .../physical/TestFullOuterMergeJoinExec.java    |  12 +-
 .../planner/physical/TestHashAntiJoinExec.java  |   6 +-
 .../planner/physical/TestHashJoinExec.java      |   6 +-
 .../planner/physical/TestHashSemiJoinExec.java  |   6 +-
 .../physical/TestLeftOuterHashJoinExec.java     |  10 +-
 .../planner/physical/TestMergeJoinExec.java     |   6 +-
 .../engine/planner/physical/TestNLJoinExec.java |   6 +-
 .../planner/physical/TestPhysicalPlanner.java   |  12 +-
 .../physical/TestProgressExternalSortExec.java  |   4 +-
 .../physical/TestRightOuterHashJoinExec.java    |   8 +-
 .../physical/TestRightOuterMergeJoinExec.java   |  12 +-
 .../engine/planner/physical/TestSortExec.java   |   6 +-
 .../tajo/engine/query/TestHBaseTable.java       |  62 ++-
 .../apache/tajo/engine/query/TestJoinQuery.java |   2 +-
 .../apache/tajo/ha/TestHAServiceHDFSImpl.java   |  88 +++--
 .../org/apache/tajo/jdbc/TestResultSet.java     |   2 +-
 .../tajo/master/TestExecutionBlockCursor.java   |   4 +-
 .../TestNonForwardQueryResultSystemScanner.java | 258 +-----------
 .../apache/tajo/querymaster/TestKillQuery.java  |   6 +-
 .../org/apache/tajo/storage/TestRowFile.java    |   2 +-
 .../TestHBaseTable/testInsertValues1.result     |   4 +
 .../testGetClusterDetails.result                |   4 +
 .../testGetNextRowsForAggregateFunction.result  |   3 +
 .../testGetNextRowsForTable.result              |   5 +
 .../java/org/apache/tajo/plan/LogicalPlan.java  |   8 +
 .../org/apache/tajo/plan/util/PlannerUtil.java  |  16 +
 .../org/apache/tajo/storage/FormatProperty.java |  42 +-
 .../org/apache/tajo/storage/MergeScanner.java   |   4 +-
 .../apache/tajo/storage/OldStorageManager.java  |   3 +-
 .../apache/tajo/storage/StorageProperty.java    |  42 +-
 .../apache/tajo/storage/TableSpaceManager.java  | 390 -------------------
 .../org/apache/tajo/storage/Tablespace.java     |  19 +-
 .../apache/tajo/storage/TablespaceManager.java  | 390 +++++++++++++++++++
 .../tajo/storage/hbase/HBasePutAppender.java    |   4 +-
 .../apache/tajo/storage/hbase/HBaseScanner.java |   4 +-
 .../tajo/storage/hbase/HBaseTablespace.java     |  48 ++-
 .../tajo/storage/hbase/TestHBaseTableSpace.java |  10 +-
 .../org/apache/tajo/storage/FileAppender.java   |   2 +-
 .../org/apache/tajo/storage/FileTablespace.java | 138 +++++--
 .../storage/HashShuffleAppenderManager.java     |   2 +-
 .../tajo/storage/TestCompressionStorages.java   |   4 +-
 .../tajo/storage/TestDelimitedTextFile.java     |   8 +-
 .../apache/tajo/storage/TestFileSystems.java    |   2 +-
 .../apache/tajo/storage/TestFileTablespace.java |  12 +-
 .../org/apache/tajo/storage/TestLineReader.java |   8 +-
 .../apache/tajo/storage/TestMergeScanner.java   |   6 +-
 .../org/apache/tajo/storage/TestStorages.java   |  44 +--
 .../apache/tajo/storage/index/TestBSTIndex.java |  20 +-
 .../index/TestSingleCSVFileBSTIndex.java        |   4 +-
 .../apache/tajo/storage/json/TestJsonSerDe.java |   2 +-
 86 files changed, 1236 insertions(+), 1136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 8fc9b94..61d56fb 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -19,7 +19,7 @@ git:
   depth: 150
 
 jdk:
-  - openjdk7
+  - oraclejdk7
 
 env: PATH=$PATH:$HOME/local/bin
 
@@ -33,7 +33,7 @@ notifications:
   - issues@tajo.apache.org
   irc: "chat.freenode.net#tajo"
 
-
+before_install: ulimit -t 514029
 install: ./dev-support/travis-install-dependencies.sh
 
 script: 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index a9923a5..16cf51d 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -30,6 +30,9 @@ public class StorageConstants {
 
   // Common table properties -------------------------------------------------
 
+  // Insert
+  public static final String INSERT_DIRECTLY = "insert.direct";
+
   // time zone
   public static final String TIMEZONE = "timezone";
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java
----------------------------------------------------------------------
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 0e27769..404606d 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
@@ -115,6 +115,10 @@ public class KeyValueSet implements ProtoObject<KeyValueSetProto>, Cloneable, Gs
     return get(key, null);
   }
 
+  public boolean isTrue(String key) {
+    return getBool(key, false);
+  }
+
   public void setBool(String key, boolean val) {
     set(key, val ? TRUE_STR : FALSE_STR);
   }
@@ -137,9 +141,9 @@ public class KeyValueSet implements ProtoObject<KeyValueSetProto>, Cloneable, Gs
   public boolean getBool(ConfigKey key) {
     String keyName = key.keyname();
     if (key instanceof SessionVars) {
-      return getBool(keyName, ((SessionVars)key).getConfVars().defaultBoolVal);
+      return getBool(keyName, ((SessionVars) key).getConfVars().defaultBoolVal);
     } else if (key instanceof TajoConf.ConfVars) {
-      return getBool(keyName, ((TajoConf.ConfVars)key).defaultBoolVal);
+      return getBool(keyName, ((TajoConf.ConfVars) key).defaultBoolVal);
     }
     return getBool(keyName);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
index f0b2f5e..c6b9b41 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
@@ -252,7 +252,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
       FragmentProto[] fragmentProtos = ctx.getTables(tableId);
       List<Fragment> fragments = FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
       for (Fragment frag : fragments) {
-        size += TableSpaceManager.guessFragmentVolume(ctx.getConf(), frag);
+        size += TablespaceManager.guessFragmentVolume(ctx.getConf(), frag);
       }
     }
     return size;
@@ -926,7 +926,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
           PartitionedTableScanNode partitionedTableScanNode = (PartitionedTableScanNode) scanNode;
           List<Fragment> fileFragments = TUtil.newList();
 
-          FileTablespace space = (FileTablespace) TableSpaceManager.get(scanNode.getTableDesc().getUri()).get();
+          FileTablespace space = (FileTablespace) TablespaceManager.get(scanNode.getTableDesc().getUri()).get();
           for (Path path : partitionedTableScanNode.getInputPaths()) {
             fileFragments.addAll(TUtil.newList(space.split(scanNode.getCanonicalName(), path)));
           }
@@ -1190,7 +1190,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
         FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
 
     String indexName = IndexUtil.getIndexNameOfFrag(fragments.get(0), annotation.getSortKeys());
-    FileTablespace sm = (FileTablespace) TableSpaceManager.get(fragments.get(0).getPath().toUri()).get();
+    FileTablespace sm = (FileTablespace) TablespaceManager.get(fragments.get(0).getPath().toUri()).get();
     String dbName = CatalogUtil.extractQualifier(annotation.getTableName());
     String simpleName = CatalogUtil.extractSimpleName(annotation.getTableName());
     Path indexPath = new Path(new Path(sm.getTableUri(dbName, simpleName)), "index");

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
----------------------------------------------------------------------
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 969998c..76abc6d 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
@@ -165,7 +165,7 @@ public abstract class ColPartitionStoreExec extends UnaryPhysicalExec {
       actualFilePath = new Path(lastFileName + "_" + suffixId);
     }
 
-    appender = ((FileTablespace) TableSpaceManager.get(lastFileName.toUri()).get())
+    appender = ((FileTablespace) TablespaceManager.get(lastFileName.toUri()).get())
         .getAppender(meta, outSchema, actualFilePath);
 
     appender.enableStats();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/InsertRowsExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/InsertRowsExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/InsertRowsExec.java
new file mode 100644
index 0000000..f3a24a7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/InsertRowsExec.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.CatalogUtil;
+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.PersistentStoreNode;
+import org.apache.tajo.plan.logical.StoreTableNode;
+import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.TablespaceManager;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+
+/**
+ * This is a physical executor to store rows immediately.
+ */
+public class InsertRowsExec extends UnaryPhysicalExec {
+  private static final Log LOG = LogFactory.getLog(InsertRowsExec.class);
+
+  private PersistentStoreNode plan;
+  private TableMeta meta;
+  private Appender appender;
+  private Tuple tuple;
+
+  // for file punctuation
+  private TableStats sumStats; // for aggregating all stats of written files
+
+  public InsertRowsExec(TaskAttemptContext context, PersistentStoreNode plan, PhysicalExec child) throws IOException {
+    super(context, plan.getInSchema(), plan.getOutSchema(), child);
+    this.plan = plan;
+  }
+
+  public void init() throws IOException {
+    super.init();
+
+    if (plan.hasOptions()) {
+      meta = CatalogUtil.newTableMeta(plan.getStorageType(), plan.getOptions());
+    } else {
+      meta = CatalogUtil.newTableMeta(plan.getStorageType());
+    }
+
+    PhysicalPlanUtil.setNullCharIfNecessary(context.getQueryContext(), plan, meta);
+    sumStats = new TableStats();
+
+    StoreTableNode storeTableNode = (StoreTableNode) plan;
+    appender = TablespaceManager.get(storeTableNode.getUri()).get().getAppenderForInsertRow(
+        context.getQueryContext(),
+        context.getTaskId(), meta, storeTableNode.getTableSchema(), context.getOutputPath());
+    appender.enableStats();
+    appender.init();
+  }
+
+  /* (non-Javadoc)
+   * @see PhysicalExec#next()
+   */
+  @Override
+  public Tuple next() throws IOException {
+    while((tuple = child.next()) != null) {
+      appender.addTuple(tuple);
+    }
+        
+    return null;
+  }
+
+  @Override
+  public void rescan() throws IOException {
+    // nothing to do
+  }
+
+  public void close() throws IOException {
+    super.close();
+
+    if(appender != null){
+      appender.flush();
+      appender.close();
+
+      // Collect statistics data
+      StatisticsUtil.aggregateTableStat(sumStats, appender.getStats());
+      context.setResultStats(sumStats);
+    }
+
+    appender = null;
+    plan = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
----------------------------------------------------------------------
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 fb29e4f..bbb21fe 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
@@ -77,7 +77,7 @@ public class RangeShuffleFileWriteExec extends UnaryPhysicalExec {
         context.getDataChannel().getStoreType() : "RAW");
     FileSystem fs = new RawLocalFileSystem();
     fs.mkdirs(storeTablePath);
-    this.appender = (FileAppender) ((FileTablespace) TableSpaceManager.getDefault())
+    this.appender = (FileAppender) ((FileTablespace) TablespaceManager.getDefault())
         .getAppender(meta, outSchema, new Path(storeTablePath, "output"));
     this.appender.enableStats();
     this.appender.init();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
index d2ae3bd..599f160 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
@@ -202,7 +202,7 @@ public class SeqScanExec extends ScanExec {
             FragmentConvertor.convert(context.getConf(), fragments), projected
         );
       } else {
-        Tablespace tablespace = TableSpaceManager.get(plan.getTableDesc().getUri()).get();
+        Tablespace tablespace = TablespaceManager.get(plan.getTableDesc().getUri()).get();
         this.scanner = tablespace.getScanner(meta, plan.getPhysicalSchema(), fragments[0], projected);
       }
       scanner.init();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
----------------------------------------------------------------------
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 dd8768e..6031fdb 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
@@ -31,12 +31,14 @@ 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.*;
+import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.FileTablespace;
+import org.apache.tajo.storage.TablespaceManager;
+import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
-import java.net.URI;
 
 /**
  * This is a physical executor to store a table part into a specified storage.
@@ -92,7 +94,7 @@ public class StoreTableExec extends UnaryPhysicalExec {
         lastFileName = new Path(lastFileName + "_" + suffixId);
       }
 
-      Optional<FileTablespace> spaceRes = TableSpaceManager.get(lastFileName.toUri());
+      Optional<FileTablespace> spaceRes = TablespaceManager.get(lastFileName.toUri());
       if (!spaceRes.isPresent())  {
         throw new IllegalStateException("No Tablespace for " + lastFileName.toUri());
       }
@@ -106,7 +108,7 @@ public class StoreTableExec extends UnaryPhysicalExec {
       }
     } else {
       Path stagingDir = context.getQueryContext().getStagingDir();
-      appender = TableSpaceManager.get(stagingDir.toUri()).get().getAppender(
+      appender = TablespaceManager.get(stagingDir.toUri()).get().getAppender(
           context.getQueryContext(),
           context.getTaskId(),
           meta,

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
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 7696c6c..da2f2ad 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
@@ -99,8 +99,8 @@ public class QueryContext extends OverridableConf {
     return strVal != null && !strVal.isEmpty() ? URI.create(strVal) : null;
   }
 
-  public void setStagingDir(Path path) {
-    put(QueryVars.STAGING_DIR, path.toUri().toString());
+  public void setStagingDir(URI uri) {
+    put(QueryVars.STAGING_DIR, uri.toString());
   }
 
   public Path getStagingDir() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
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 e833884..37b497c 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
@@ -53,7 +53,7 @@ import org.apache.tajo.plan.verifier.LogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.VerificationState;
 import org.apache.tajo.plan.verifier.VerifyException;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 
 import java.io.IOException;
@@ -91,7 +91,7 @@ public class GlobalEngine extends AbstractService {
     try  {
       analyzer = new SQLAnalyzer();
       preVerifier = new PreLogicalPlanVerifier(context.getCatalog());
-      planner = new LogicalPlanner(context.getCatalog(), TableSpaceManager.getInstance());
+      planner = new LogicalPlanner(context.getCatalog(), TablespaceManager.getInstance());
       optimizer = new LogicalOptimizer(context.getConf());
       annotatedPlanVerifier = new LogicalPlanVerifier(context.getConf(), context.getCatalog());
     } catch (Throwable t) {
@@ -141,8 +141,8 @@ public class GlobalEngine extends AbstractService {
     QueryContext newQueryContext =  new QueryContext(context.getConf(), session);
 
     // Set default space uri and its root uri
-    newQueryContext.setDefaultSpaceUri(TableSpaceManager.getDefault().getUri());
-    newQueryContext.setDefaultSpaceRootUri(TableSpaceManager.getDefault().getRootUri());
+    newQueryContext.setDefaultSpaceUri(TablespaceManager.getDefault().getUri());
+    newQueryContext.setDefaultSpaceRootUri(TablespaceManager.getDefault().getRootUri());
 
     String tajoTest = System.getProperty(CommonTestingUtil.TAJO_TEST_KEY);
     if (tajoTest != null && tajoTest.equalsIgnoreCase(CommonTestingUtil.TAJO_TEST_TRUE)) {
@@ -303,7 +303,7 @@ public class GlobalEngine extends AbstractService {
           InsertNode iNode = rootNode.getChild();
           Schema outSchema = iNode.getChild().getOutSchema();
 
-          TableSpaceManager.get(tableDesc.getUri()).get().verifySchemaToWrite(tableDesc, outSchema);
+          TablespaceManager.get(tableDesc.getUri()).get().verifySchemaToWrite(tableDesc, outSchema);
 
         } catch (Throwable t) {
           state.addVerification(t.getMessage());

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
index 5e0e639..7104412 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
@@ -36,7 +36,7 @@ import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.StorageUtil;
 
@@ -251,11 +251,11 @@ public class DDLExecutor {
 
     Tablespace tableSpace;
     if (tableSpaceName != null) {
-      tableSpace = TableSpaceManager.getByName(tableSpaceName).get();
+      tableSpace = TablespaceManager.getByName(tableSpaceName).get();
     } else if (uri != null) {
-      tableSpace = TableSpaceManager.get(uri).get();
+      tableSpace = TablespaceManager.get(uri).get();
     } else {
-      tableSpace = TableSpaceManager.getDefault();
+      tableSpace = TablespaceManager.getDefault();
     }
 
     TableDesc desc;
@@ -313,7 +313,7 @@ public class DDLExecutor {
 
     if (purge) {
       try {
-        TableSpaceManager.get(tableDesc.getUri()).get().purgeTable(tableDesc);
+        TablespaceManager.get(tableDesc.getUri()).get().purgeTable(tableDesc);
       } catch (IOException e) {
         throw new InternalError(e.getMessage());
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
index ae57453..ec8760f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
@@ -101,7 +101,7 @@ public class NonForwardQueryResultFileScanner implements NonForwardQueryResultSc
   }
 
   private void initSeqScanExec() throws IOException {
-    Tablespace tablespace = TableSpaceManager.get(tableDesc.getUri()).get();
+    Tablespace tablespace = TablespaceManager.get(tableDesc.getUri()).get();
     List<Fragment> fragments = null;
     setPartition(tablespace);
     fragments = tablespace.getNonForwardSplit(tableDesc, currentFragmentIndex, MAX_FRAGMENT_NUM_PER_SCAN);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
index 480f45c..5d42157 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@ -31,6 +31,7 @@ import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.CatalogService;
 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.statistics.TableStats;
 import org.apache.tajo.common.TajoDataTypes;
@@ -39,7 +40,7 @@ import org.apache.tajo.datum.DatumFactory;
 import org.apache.tajo.engine.planner.global.GlobalPlanner;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.planner.physical.EvalExprExec;
-import org.apache.tajo.engine.planner.physical.StoreTableExec;
+import org.apache.tajo.engine.planner.physical.InsertRowsExec;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
@@ -60,14 +61,13 @@ import org.apache.tajo.plan.function.python.TajoScriptEngine;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.plan.verifier.VerifyException;
-import org.apache.tajo.querymaster.Query;
-import org.apache.tajo.querymaster.QueryMasterTask;
 import org.apache.tajo.session.Session;
 import org.apache.tajo.storage.*;
 import org.apache.tajo.util.ProtoUtil;
 import org.apache.tajo.worker.TaskAttemptContext;
 
 import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -288,7 +288,7 @@ public class QueryExecutor {
       boolean isInsert = rootNode.getChild() != null && rootNode.getChild().getType() == NodeType.INSERT;
       if (isInsert) {
         InsertNode insertNode = rootNode.getChild();
-        insertNonFromQuery(queryContext, insertNode, responseBuilder);
+        insertRowValues(queryContext, insertNode, responseBuilder);
       } else {
         Schema schema = PlannerUtil.targetToSchema(targets);
         RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(schema);
@@ -330,89 +330,123 @@ public class QueryExecutor {
     }
   }
 
-  private void insertNonFromQuery(QueryContext queryContext,
-                                  InsertNode insertNode, SubmitQueryResponse.Builder responseBuilder) {
+  /**
+   * Insert rows through staging phase
+   */
+  private void insertRowsThroughStaging(TaskAttemptContext taskAttemptContext,
+                                        InsertNode insertNode,
+                                        Path finalOutputPath,
+                                        Path stagingDir,
+                                        Path stagingResultDir)
+      throws IOException {
+
+    EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
+    InsertRowsExec exec = new InsertRowsExec(taskAttemptContext, insertNode, evalExprExec);
+
+    try {
+      exec.init();
+      exec.next();
+    } finally {
+      exec.close();
+    }
+
+    FileSystem fs = TajoConf.getWarehouseDir(context.getConf()).getFileSystem(context.getConf());
+
+    if (insertNode.isOverwrite()) { // 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);
+      try {
+        if (fs.exists(finalOutputPath)) {
+          fs.rename(finalOutputPath, oldTableDir);
+          movedToOldTable = fs.exists(oldTableDir);
+        } else { // if the parent does not exist, make its parent directory.
+          fs.mkdirs(finalOutputPath.getParent());
+        }
+        fs.rename(stagingResultDir, finalOutputPath);
+        committed = fs.exists(finalOutputPath);
+      } catch (IOException ioe) {
+        // recover the old table
+        if (movedToOldTable && !committed) {
+          fs.rename(oldTableDir, finalOutputPath);
+        }
+      }
+    } else {
+      FileStatus[] files = fs.listStatus(stagingResultDir);
+      for (FileStatus eachFile : files) {
+        Path targetFilePath = new Path(finalOutputPath, eachFile.getPath().getName());
+        if (fs.exists(targetFilePath)) {
+          targetFilePath = new Path(finalOutputPath, eachFile.getPath().getName() + "_" + System.currentTimeMillis());
+        }
+        fs.rename(eachFile.getPath(), targetFilePath);
+      }
+    }
+  }
+
+  /**
+   * Insert row values
+   */
+  private void insertRowValues(QueryContext queryContext,
+                               InsertNode insertNode, SubmitQueryResponse.Builder responseBuilder) {
     try {
       String nodeUniqName = insertNode.getTableName() == null ? new Path(insertNode.getUri()).getName() :
           insertNode.getTableName();
       String queryId = nodeUniqName + "_" + System.currentTimeMillis();
 
-      FileSystem fs = TajoConf.getWarehouseDir(context.getConf()).getFileSystem(context.getConf());
-      Path stagingDir = QueryMasterTask.initStagingDir(context.getConf(), queryId.toString(), queryContext);
-      Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+      URI finalOutputUri = insertNode.getUri();
+      Tablespace space = TablespaceManager.get(finalOutputUri).get();
+      TableMeta tableMeta = new TableMeta(insertNode.getStorageType(), insertNode.getOptions());
+      tableMeta.putOption(StorageConstants.INSERT_DIRECTLY, Boolean.TRUE.toString());
 
-      TableDesc tableDesc = null;
-      Path finalOutputDir;
-      if (insertNode.getTableName() != null) {
-        tableDesc = this.catalog.getTableDesc(insertNode.getTableName());
-        finalOutputDir = new Path(tableDesc.getUri());
-      } else {
-        finalOutputDir = new Path(insertNode.getUri());
-      }
+      FormatProperty formatProperty = space.getFormatProperty(tableMeta);
 
-      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
-      taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
+      TaskAttemptContext taskAttemptContext;
+      if (formatProperty.directInsertSupported()) { // if this format and storage supports direct insertion
+        taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, null);
+        taskAttemptContext.setOutputPath(new Path(finalOutputUri));
 
-      EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
-      StoreTableExec exec = new StoreTableExec(taskAttemptContext, insertNode, evalExprExec);
-      try {
-        exec.init();
-        exec.next();
-      } finally {
-        exec.close();
-      }
+        EvalExprExec evalExprExec = new EvalExprExec(taskAttemptContext, (EvalExprNode) insertNode.getChild());
+        InsertRowsExec exec = new InsertRowsExec(taskAttemptContext, insertNode, evalExprExec);
 
-      if (insertNode.isOverwrite()) { // 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);
         try {
-          if (fs.exists(finalOutputDir)) {
-            fs.rename(finalOutputDir, oldTableDir);
-            movedToOldTable = fs.exists(oldTableDir);
-          } else { // if the parent does not exist, make its parent directory.
-            fs.mkdirs(finalOutputDir.getParent());
-          }
-          fs.rename(stagingResultDir, finalOutputDir);
-          committed = fs.exists(finalOutputDir);
-        } catch (IOException ioe) {
-          // recover the old table
-          if (movedToOldTable && !committed) {
-            fs.rename(oldTableDir, finalOutputDir);
-          }
+          exec.init();
+          exec.next();
+        } finally {
+          exec.close();
         }
       } else {
-        FileStatus[] files = fs.listStatus(stagingResultDir);
-        for (FileStatus eachFile : files) {
-          Path targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName());
-          if (fs.exists(targetFilePath)) {
-            targetFilePath = new Path(finalOutputDir, eachFile.getPath().getName() + "_" + System.currentTimeMillis());
-          }
-          fs.rename(eachFile.getPath(), targetFilePath);
-        }
+        URI stagingSpaceUri = space.prepareStagingSpace(context.getConf(), queryId, queryContext, tableMeta);
+        Path stagingDir = new Path(stagingSpaceUri);
+        Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+
+        taskAttemptContext = new TaskAttemptContext(queryContext, null, null, null, stagingDir);
+        taskAttemptContext.setOutputPath(new Path(stagingResultDir, "part-01-000000"));
+        insertRowsThroughStaging(taskAttemptContext, insertNode, new Path(finalOutputUri), stagingDir, stagingResultDir);
       }
 
-      if (insertNode.hasTargetTable()) {
-        TableStats stats = tableDesc.getStats();
-        long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-        stats.setNumBytes(volume);
-        stats.setNumRows(1);
+      // set insert stats (how many rows and bytes)
+      TableStats stats = new TableStats();
+      stats.setNumBytes(taskAttemptContext.getResultStats().getNumBytes());
+      stats.setNumRows(taskAttemptContext.getResultStats().getNumRows());
 
+      if (insertNode.hasTargetTable()) {
         CatalogProtos.UpdateTableStatsProto.Builder builder = CatalogProtos.UpdateTableStatsProto.newBuilder();
-        builder.setTableName(tableDesc.getName());
+        builder.setTableName(insertNode.getTableName());
         builder.setStats(stats.getProto());
 
         catalog.updateTableStats(builder.build());
 
-        responseBuilder.setTableDesc(tableDesc.getProto());
-      } else {
-        TableStats stats = new TableStats();
-        long volume = Query.getTableVolume(context.getConf(), finalOutputDir);
-        stats.setNumBytes(volume);
-        stats.setNumRows(1);
+        TableDesc desc = new TableDesc(
+            insertNode.getTableName(),
+            insertNode.getTargetSchema(),
+            tableMeta,
+            finalOutputUri);
+        responseBuilder.setTableDesc(desc.getProto());
+
+      } else { // If INSERT INTO LOCATION
 
         // Empty TableDesc
         List<CatalogProtos.ColumnProto> columns = new ArrayList<CatalogProtos.ColumnProto>();
@@ -445,11 +479,12 @@ public class QueryExecutor {
     TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, plan.getRootBlock().getRoot());
     if (tableDesc != null) {
 
-      Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
-      StorageProperty storageProperty = space.getProperty();
+      Tablespace space = TablespaceManager.get(tableDesc.getUri()).get();
+      FormatProperty formatProperty = space.getFormatProperty(tableDesc.getMeta());
 
-      if (!storageProperty.isInsertable()) {
-        throw new VerifyException("Inserting into non-file storage is not supported.");
+      if (!formatProperty.isInsertable()) {
+        throw new VerifyException(
+            String.format("%s tablespace does not allow INSERT operation.", tableDesc.getUri().toString()));
       }
 
       space.prepareTable(rootNode.getChild());
@@ -487,7 +522,7 @@ public class QueryExecutor {
     TableDesc tableDesc = PlannerUtil.getTableDesc(planner.getCatalog(), rootNode.getChild());
 
     if (tableDesc != null) {
-      Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
+      Tablespace space = TablespaceManager.get(tableDesc.getUri()).get();
       space.rewritePlan(context, plan);
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
index d490001..0c02b6e 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
@@ -24,8 +24,6 @@ import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.logical.CreateTableNode;
 import org.apache.tajo.plan.logical.LogicalRootNode;
 import org.apache.tajo.plan.logical.NodeType;
-import org.apache.tajo.storage.TableSpaceManager;
-import org.apache.tajo.storage.Tablespace;
 
 public class CreateTableHook implements DistributedQueryHook {
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
index 4fef02c..9d5838d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
@@ -48,7 +48,7 @@ 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.StorageConstants;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.util.history.QueryHistory;
@@ -447,7 +447,7 @@ public class Query implements EventHandler<QueryEvent> {
       QueryContext context = query.context.getQueryContext();
 
       if (lastStage != null && context.hasOutputTableUri()) {
-        Tablespace space = TableSpaceManager.get(context.getOutputTableUri()).get();
+        Tablespace space = TablespaceManager.get(context.getOutputTableUri()).get();
         try {
           LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
           space.rollbackTable(rootNode.getChild());
@@ -470,7 +470,7 @@ public class Query implements EventHandler<QueryEvent> {
 
         // If there is not tabledesc, it is a select query without insert or ctas.
         // In this case, we should use default tablespace.
-        Tablespace space = TableSpaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
+        Tablespace space = TablespaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
 
         Path finalOutputDir = space.commitTable(
             query.context.getQueryContext(),

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
index 84f2eac..1f5e7a3 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
@@ -18,15 +18,10 @@
 
 package org.apache.tajo.querymaster;
 
-import com.google.common.base.Optional;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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 org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -61,6 +56,7 @@ import org.apache.tajo.worker.AbstractResourceAllocator;
 import org.apache.tajo.worker.TajoResourceAllocator;
 
 import java.io.IOException;
+import java.net.URI;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
@@ -70,12 +66,6 @@ import static org.apache.tajo.TajoProtos.QueryState;
 public class QueryMasterTask extends CompositeService {
   private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName());
 
-  // query submission directory is private!
-  final public static FsPermission STAGING_DIR_PERMISSION =
-      FsPermission.createImmutable((short) 0700); // rwx--------
-
-  public static final String TMP_STAGING_DIR_PREFIX = ".staging";
-
   private QueryId queryId;
 
   private Session session;
@@ -157,8 +147,6 @@ public class QueryMasterTask extends CompositeService {
       dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
       dispatcher.register(LocalTaskEventType.class, new LocalTaskEventHandler());
 
-      initStagingDir();
-
       queryMetrics = new TajoMetrics(queryId.toString());
 
       super.init(systemConf);
@@ -303,8 +291,9 @@ public class QueryMasterTask extends CompositeService {
         state == QueryState.QUERY_ERROR;
   }
 
+  private LogicalPlan plan;
+
   public synchronized void startQuery() {
-    LogicalPlan plan = null;
     Tablespace space = null;
     try {
       if (query != null) {
@@ -314,7 +303,7 @@ public class QueryMasterTask extends CompositeService {
 
 
       CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
-      LogicalPlanner planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+      LogicalPlanner planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
       LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
       Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
       jsonExpr = null; // remove the possible OOM
@@ -322,10 +311,12 @@ public class QueryMasterTask extends CompositeService {
       plan = planner.createPlan(queryContext, expr);
       optimizer.optimize(queryContext, plan);
 
-      // when a given uri is null, TableSpaceManager.get will return the default tablespace.
-      space = TableSpaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
+      // when a given uri is null, TablespaceManager.get will return the default tablespace.
+      space = TablespaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
       space.rewritePlan(queryContext, plan);
 
+      initStagingDir();
+
       for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
         LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);
         if (scanNodes != null) {
@@ -367,94 +358,25 @@ public class QueryMasterTask extends CompositeService {
   }
 
   private void initStagingDir() throws IOException {
-    Path stagingDir;
+    URI stagingDir;
 
     try {
+      Tablespace tablespace = TablespaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
+      TableDesc desc = PlannerUtil.getOutputTableDesc(plan);
 
-      stagingDir = initStagingDir(systemConf, queryId.toString(), queryContext);
+      FormatProperty formatProperty = tablespace.getFormatProperty(desc.getMeta());
+      if (formatProperty.isStagingSupport()) {
+        stagingDir = tablespace.prepareStagingSpace(systemConf, queryId.toString(), queryContext, desc.getMeta());
 
-      // Create a subdirectories
-      LOG.info("The staging dir '" + stagingDir + "' is created.");
-      queryContext.setStagingDir(stagingDir);
-    } catch (IOException ioe) {
-      LOG.warn("Creating staging dir has been failed.", ioe);
-
-      throw ioe;
-    }
-  }
-
-  /**
-   * It initializes the final output and staging directory and sets
-   * them to variables.
-   */
-  public static Path initStagingDir(TajoConf conf, String queryId, QueryContext context) throws IOException {
-
-    String realUser;
-    String currentUser;
-    UserGroupInformation ugi;
-    ugi = UserGroupInformation.getLoginUser();
-    realUser = ugi.getShortUserName();
-    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
-
-    FileSystem fs;
-    Path stagingDir;
-
-    ////////////////////////////////////////////
-    // Create Output Directory
-    ////////////////////////////////////////////
-
-    String outputPath = context.get(QueryVars.OUTPUT_TABLE_URI, "");
-
-    // The fact that there is no output means that this query is neither CTAS or INSERT (OVERWRITE) INTO
-    // So, this query results won't be materialized as a part of a table.
-    // The result will be temporarily written in the staging directory.
-    if (outputPath.isEmpty()) {
-      // for temporarily written in the storage directory
-      stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
-    } else {
-      Optional<Tablespace> spaceResult = TableSpaceManager.get(outputPath);
-      if (!spaceResult.isPresent()) {
-        throw new IOException("No registered Tablespace for " + outputPath);
+        // Create a staging space
+        LOG.info("The staging dir '" + stagingDir + "' is created.");
+        queryContext.setStagingDir(stagingDir);
       }
 
-      Tablespace space = spaceResult.get();
-      if (space.getProperty().isMovable()) { // checking if this tablespace allows MOVE operation
-        // If this space allows move operation, the staging directory will be underneath the final output table uri.
-        stagingDir = StorageUtil.concatPath(context.getOutputTableUri().toString(), TMP_STAGING_DIR_PREFIX, queryId);
-      } else {
-        stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
-      }
-    }
-
-    // initializ
-    fs = stagingDir.getFileSystem(conf);
-
-    if (fs.exists(stagingDir)) {
-      throw new IOException("The staging directory '" + stagingDir + "' already exists");
-    }
-    fs.mkdirs(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
-    FileStatus fsStatus = fs.getFileStatus(stagingDir);
-    String owner = fsStatus.getOwner();
-
-    if (!owner.isEmpty() && !(owner.equals(currentUser) || owner.equals(realUser))) {
-      throw new IOException("The ownership on the user's query " +
-          "directory " + stagingDir + " is not as expected. " +
-          "It is owned by " + owner + ". The directory must " +
-          "be owned by the submitter " + currentUser + " or " +
-          "by " + realUser);
-    }
-
-    if (!fsStatus.getPermission().equals(STAGING_DIR_PERMISSION)) {
-      LOG.info("Permissions on staging directory " + stagingDir + " are " +
-          "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
-          "to correct value " + STAGING_DIR_PERMISSION);
-      fs.setPermission(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+    } catch (IOException ioe) {
+      LOG.warn("Creating staging space has been failed.", ioe);
+      throw ioe;
     }
-
-    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-    fs.mkdirs(stagingResultDir);
-
-    return stagingDir;
   }
 
   public Query getQuery() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
index 5b8f24a..f30fb64 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
@@ -100,7 +100,7 @@ public class Repartitioner {
           stats[i] = masterContext.getStage(scanEBId).getResultStats().getNumBytes();
         }
 
-        // TODO - We should remove dummy flagment usages
+        // TODO - We should remove dummy fragment usages
         fragments[i] = new FileFragment(scans[i].getCanonicalName(), new Path("/dummy"), 0, 0,
             new String[]{UNKNOWN_HOST});
 
@@ -115,7 +115,7 @@ public class Repartitioner {
         // if table has no data, tablespace will return empty FileFragment.
         // So, we need to handle FileFragment by its size.
         // If we don't check its size, it can cause IndexOutOfBoundsException.
-        Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
+        Tablespace space = TablespaceManager.get(tableDesc.getUri()).get();
         List<Fragment> fileFragments = space.getSplits(scans[i].getCanonicalName(), tableDesc);
         if (fileFragments.size() > 0) {
           fragments[i] = fileFragments.get(0);
@@ -380,7 +380,7 @@ public class Repartitioner {
 
         Path[] partitionScanPaths = null;
         TableDesc tableDesc = masterContext.getTableDescMap().get(eachScan.getCanonicalName());
-        Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
+        Tablespace space = TablespaceManager.get(tableDesc.getUri()).get();
 
         if (eachScan.getType() == NodeType.PARTITIONS_SCAN) {
 
@@ -507,7 +507,7 @@ public class Repartitioner {
       Collection<Fragment> scanFragments;
       Path[] partitionScanPaths = null;
 
-      FileTablespace space = (FileTablespace) TableSpaceManager.get(desc.getUri()).get();
+      FileTablespace space = (FileTablespace) TablespaceManager.get(desc.getUri()).get();
 
       if (scan.getType() == NodeType.PARTITIONS_SCAN) {
         PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)scan;
@@ -645,7 +645,7 @@ public class Repartitioner {
             PlannerUtil.getStoreTableName(masterPlan.getLogicalPlan()));
       }
 
-      Tablespace space = TableSpaceManager.getAnyByScheme(storeType).get();
+      Tablespace space = TablespaceManager.getAnyByScheme(storeType).get();
       ranges = space.getInsertSortRanges(
           stage.getContext().getQueryContext(),
           tableDesc,

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
index a7d605c..1163a6e 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
@@ -60,7 +60,7 @@ import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.querymaster.Task.IntermediateEntry;
 import org.apache.tajo.storage.FileTablespace;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.unit.StorageUnit;
@@ -1084,7 +1084,7 @@ public class Stage implements EventHandler<StageEvent> {
       Collection<Fragment> fragments;
       TableMeta meta = table.getMeta();
 
-      Tablespace tablespace = TableSpaceManager.get(scan.getTableDesc().getUri()).get();
+      Tablespace tablespace = TablespaceManager.get(scan.getTableDesc().getUri()).get();
 
       // Depending on scanner node's type, it creates fragments. If scan is for
       // a partitioned table, It will creates lots fragments for all partitions.

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
index 0df5d4d..f97ce29 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
@@ -160,7 +160,7 @@ public class LegacyTaskImpl implements Task {
         this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys());
       }
     } else {
-      Path outFilePath = ((FileTablespace) TableSpaceManager.get(queryContext.getStagingDir().toUri()).get())
+      Path outFilePath = ((FileTablespace) TablespaceManager.get(queryContext.getStagingDir().toUri()).get())
           .getAppenderFilePath(getId(), queryContext.getStagingDir());
       LOG.info("Output File Path: " + outFilePath);
       context.setOutputPath(outFilePath);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
index 5974693..7697458 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
@@ -154,7 +154,7 @@ public class TaskImpl implements Task {
         this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys());
       }
     } else {
-      Path outFilePath = ((FileTablespace) TableSpaceManager.get(queryContext.getStagingDir().toUri()).get())
+      Path outFilePath = ((FileTablespace) TablespaceManager.get(queryContext.getStagingDir().toUri()).get())
           .getAppenderFilePath(getId(), queryContext.getStagingDir());
       LOG.info("Output File Path: " + outFilePath);
       context.setOutputPath(outFilePath);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/main/resources/webapps/admin/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp
index 43bb6c1..bd84283 100644
--- a/tajo-core/src/main/resources/webapps/admin/index.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/index.jsp
@@ -27,7 +27,7 @@
 <%@ page import="org.apache.tajo.master.rm.WorkerState" %>
 <%@ page import="org.apache.tajo.service.ServiceTracker" %>
 <%@ page import="org.apache.tajo.service.TajoMasterInfo" %>
-<%@ page import="org.apache.tajo.storage.TableSpaceManager" %>
+<%@ page import="org.apache.tajo.storage.TablespaceManager" %>
 <%@ page import="org.apache.tajo.storage.Tablespace" %>
 <%@ page import="org.apache.tajo.util.NetUtils" %>
 <%@ page import="org.apache.tajo.util.TUtil" %>
@@ -141,7 +141,7 @@
   <h3>Tablespaces</h3>
   <table width="100%" class="border_table" border="1">
     <tr><th>Tablespace Name</th><th>URI</th><th>Handler</th></tr>
-    <% for (Tablespace space : TableSpaceManager.getAllTablespaces()) {
+    <% for (Tablespace space : TablespaceManager.getAllTablespaces()) {
       if (space.isVisible()) { %>
     <tr><td><%=space.getName()%></td><td><%=space.getUri()%></td><td><%=space.getClass().getName()%></td></tr>
     <% }}%>

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
----------------------------------------------------------------------
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 ca2378b..5df1122 100644
--- a/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
@@ -46,7 +46,7 @@ public class BackendTestingUtil {
 
   public static void writeTmpTable(TajoConf conf, Path tablePath)
       throws IOException {
-    FileTablespace sm = TableSpaceManager.getDefault();
+    FileTablespace sm = TablespaceManager.getDefault();
     Appender appender;
 
     Path filePath = new Path(tablePath, "table.csv");

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
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 57b1e18..a323f25 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -348,6 +348,41 @@ public class QueryTestCaseBase {
   }
 
   /**
+   * It executes the query file and compare the result against the the result file.
+   *
+   * @throws Exception
+   */
+  public void assertQuery() throws Exception {
+    ResultSet res = null;
+    try {
+      res = executeQuery();
+      assertResultSet(res);
+    } finally {
+      if (res != null) {
+        res.close();
+      }
+    }
+  }
+
+  /**
+   * It executes a given query statement and verifies the result against the the result file.
+   *
+   * @param query A query statement
+   * @throws Exception
+   */
+  public void assertQueryStr(String query) throws Exception {
+    ResultSet res = null;
+    try {
+      res = executeString(query);
+      assertResultSet(res);
+    } finally {
+      if (res != null) {
+        res.close();
+      }
+    }
+  }
+
+  /**
    * Execute a query contained in the file located in src/test/resources/results/<i>ClassName</i>/<i>MethodName</i>.
    * <i>ClassName</i> and <i>MethodName</i> will be replaced by actual executed class and methods.
    *

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
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 acdae85..973f1e8 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -48,7 +48,7 @@ import org.apache.tajo.querymaster.Stage;
 import org.apache.tajo.querymaster.StageState;
 import org.apache.tajo.service.ServiceTrackerFactory;
 import org.apache.tajo.storage.FileTablespace;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.NetUtils;
@@ -59,6 +59,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.Writer;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.sql.ResultSet;
 import java.util.ArrayList;
 import java.util.List;
@@ -346,18 +347,17 @@ public class TajoTestingCluster {
     LOG.info("derby repository is set to "+conf.get(CatalogConstants.CATALOG_URI));
 
     if (!local) {
-      c.setVar(ConfVars.ROOT_DIR, getMiniDFSCluster().getFileSystem().getUri() + "/tajo");
-    } else {
-      c.setVar(ConfVars.ROOT_DIR, "file://" + testBuildDir.getAbsolutePath() + "/tajo");
-    }
+      String tajoRootDir = getMiniDFSCluster().getFileSystem().getUri().toString() + "/tajo";
+      c.setVar(ConfVars.ROOT_DIR, tajoRootDir);
 
-    // Do not need for local file system
-    if (!local) {
+      URI defaultTsUri = TajoConf.getWarehouseDir(c).toUri();
       FileTablespace defaultTableSpace =
-          new FileTablespace(TableSpaceManager.DEFAULT_TABLESPACE_NAME, TajoConf.getWarehouseDir(c).toUri());
+          new FileTablespace(TablespaceManager.DEFAULT_TABLESPACE_NAME, defaultTsUri);
       defaultTableSpace.init(conf);
+      TablespaceManager.addTableSpaceForTest(defaultTableSpace);
 
-      TableSpaceManager.addTableSpaceForTest(defaultTableSpace);
+    } else {
+      c.setVar(ConfVars.ROOT_DIR, "file://" + testBuildDir.getAbsolutePath() + "/tajo");
     }
 
     setupCatalogForTesting(c, testBuildDir);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java b/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
index ce951c6..26e25a4 100644
--- a/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
@@ -32,7 +32,7 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryStatus;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.FileUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -217,7 +217,7 @@ public class TestTajoCli {
 
     if (!cluster.isHiveCatalogStoreRunning()) {
       assertOutputResult(resultFileName, consoleResult, new String[]{"${table.path}"},
-        new String[]{TableSpaceManager.getDefault().getTableUri("default", tableName).toString()});
+        new String[]{TablespaceManager.getDefault().getTableUri("default", tableName).toString()});
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index 328f883..07a09ad 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -24,7 +24,6 @@ import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.cli.tsql.InvalidStatementException;
 import org.apache.tajo.cli.tsql.ParsedResult;
 import org.apache.tajo.cli.tsql.SimpleParser;
@@ -50,7 +49,7 @@ import org.apache.tajo.plan.verifier.LogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.VerificationState;
 import org.apache.tajo.storage.LazyTuple;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
 import org.apache.tajo.util.BytesUtils;
@@ -104,7 +103,7 @@ public class ExprTestBase {
 
     analyzer = new SQLAnalyzer();
     preLogicalPlanVerifier = new PreLogicalPlanVerifier(cat);
-    planner = new LogicalPlanner(cat, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(cat, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(util.getConfiguration());
     annotatedPlanVerifier = new LogicalPlanVerifier(util.getConfiguration(), cat);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
index 80f3459..5a8238c 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
@@ -27,7 +27,6 @@ import org.apache.tajo.algebra.OpType;
 import org.apache.tajo.algebra.Selection;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
@@ -45,7 +44,7 @@ import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.plan.logical.GroupbyNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.nameresolver.NameResolvingMode;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.AfterClass;
@@ -117,7 +116,7 @@ public class TestEvalTreeUtil {
     catalog.createFunction(funcMeta);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 
     String[] QUERIES = {
         "select name, score, age from people where score > 30", // 0

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
index 9aa7ddf..afa3472 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
@@ -34,7 +34,7 @@ import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.junit.AfterClass;
@@ -103,7 +103,7 @@ public class TestLogicalOptimizer {
 
     catalog.createFunction(funcDesc);
     sqlAnalyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(util.getConfiguration());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(util.getConfiguration());

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
index 3cee816..dc9e2b0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.engine.planner;
 
 import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.graph.SimpleDirectedGraph;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
@@ -40,7 +40,7 @@ public class TestLogicalPlan {
   public static void setup() throws Exception {
     util = new TajoTestingCluster();
     util.startCatalogCluster();
-    planner = new LogicalPlanner(util.getMiniCatalogCluster().getCatalog(), TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(util.getMiniCatalogCluster().getCatalog(), TablespaceManager.getInstance());
   }
 
   public static void tearDown() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 351a6af..0f37763 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -42,7 +42,7 @@ import org.apache.tajo.plan.*;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.KeyValueSet;
@@ -131,7 +131,7 @@ public class TestLogicalPlanner {
 
     catalog.createFunction(funcDesc);
     sqlAnalyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
----------------------------------------------------------------------
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 d62eed2..fb35220 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
@@ -39,7 +39,7 @@ import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleComparator;
 import org.apache.tajo.storage.VTuple;
@@ -109,7 +109,7 @@ public class TestPlannerUtil {
 
     catalog.createFunction(funcDesc);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
----------------------------------------------------------------------
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 2464fb1..ace3d0d 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
@@ -86,7 +86,7 @@ public class TestBNLJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(schema.size());
@@ -108,7 +108,7 @@ public class TestBNLJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < INNER_TUPLE_NUM; i += 2) {
@@ -124,7 +124,7 @@ public class TestBNLJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
----------------------------------------------------------------------
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 96a1f36..b4a6063 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
@@ -90,7 +90,7 @@ public class TestBSTIndexExec {
     Path workDir = CommonTestingUtil.getTestDir();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, workDir.toUri().toString());
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
-    sm = TableSpaceManager.getLocalFs();
+    sm = TablespaceManager.getLocalFs();
 
     idxPath = new Path(workDir, "test.idx");
 
@@ -148,7 +148,7 @@ public class TestBSTIndexExec {
     catalog.createTable(desc);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
----------------------------------------------------------------------
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 d94d3f6..cf5220e 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
@@ -82,7 +82,7 @@ public class TestExternalSortExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
@@ -104,7 +104,7 @@ public class TestExternalSortExec {
     employee = new TableDesc("default.employee", schema, employeeMeta, employeePath.toUri());
     catalog.createTable(employee);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
----------------------------------------------------------------------
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 21a101a..dc4dd04 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
@@ -104,7 +104,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -133,7 +133,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -172,7 +172,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -224,7 +224,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender5 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
@@ -234,7 +234,7 @@ public class TestFullOuterHashJoinExec {
     catalog.createTable(phone3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
----------------------------------------------------------------------
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 0e2ce42..8fd61d0 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
@@ -109,7 +109,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -147,7 +147,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta("CSV");
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep4Meta, dep4Schema, dep4Path);
+    Appender appender4 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     VTuple tuple4 = new VTuple(dep4Schema.size());
     for (int i = 0; i < 11; i++) {
@@ -178,7 +178,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -217,7 +217,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -269,7 +269,7 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender5 = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     appender5.flush();
@@ -278,7 +278,7 @@ public class TestFullOuterMergeJoinExec {
     catalog.createTable(phone3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
----------------------------------------------------------------------
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 d54df1c..1b64a8f 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
@@ -83,7 +83,7 @@ public class TestHashAntiJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -109,7 +109,7 @@ public class TestHashAntiJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -128,7 +128,7 @@ public class TestHashAntiJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
----------------------------------------------------------------------
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 a8826ee..b9ee06a 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
@@ -85,7 +85,7 @@ public class TestHashJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(employeeSchema.size());
@@ -108,7 +108,7 @@ public class TestHashJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getLocalFs())
+    appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
@@ -126,7 +126,7 @@ public class TestHashJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
+    planner = new LogicalPlanner(catalog, TablespaceManager.getInstance());
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
 


[14/15] tajo git commit: TAJO-1659: Simplify scan iteration in SeqScan.

Posted by ji...@apache.org.
TAJO-1659: Simplify scan iteration in SeqScan.


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

Branch: refs/heads/index_support
Commit: 2ec307d621828cc18627ef7bd98e9c3c6774c5af
Parents: 03bf843
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Jun 24 20:51:07 2015 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Jun 24 20:51:07 2015 -0700

----------------------------------------------------------------------
 CHANGES                                         |  2 +
 .../planner/physical/FilterScanIterator.java    | 56 ++++++++++++++++++++
 .../planner/physical/FullScanIterator.java      | 47 ++++++++++++++++
 .../engine/planner/physical/ScanIterator.java   | 33 ++++++++++++
 .../engine/planner/physical/SeqScanExec.java    | 34 ++++++------
 5 files changed, 153 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/2ec307d6/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 75ffd39..425ac5d 100644
--- a/CHANGES
+++ b/CHANGES
@@ -27,6 +27,8 @@ Release 0.11.0 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1659: Simplify scan iteration in SeqScan. (hyunsik)
+
     TAJO-751: JDBC driver should support cancel() method.
     (Contributed by navis, Committed by jihoon)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/2ec307d6/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FilterScanIterator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FilterScanIterator.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FilterScanIterator.java
new file mode 100644
index 0000000..fd440d7
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FilterScanIterator.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.tajo.plan.expr.EvalNode;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+/**
+ * This iterator involves filter operation.
+ */
+public class FilterScanIterator implements ScanIterator {
+  private final Scanner scanner;
+  private final EvalNode filter;
+  private Tuple currentTuple;
+
+  public FilterScanIterator(Scanner scanner, EvalNode filter) {
+    this.scanner = scanner;
+    this.filter = filter;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    while((currentTuple = scanner.next()) != null) {
+      if (filter.eval(currentTuple).isTrue()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  @Override
+  public Tuple next() {
+    return currentTuple;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/2ec307d6/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FullScanIterator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FullScanIterator.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FullScanIterator.java
new file mode 100644
index 0000000..a32f33d
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/FullScanIterator.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+/**
+ * This scan iterator performs full scan.
+ */
+public class FullScanIterator implements ScanIterator {
+  private final Scanner scanner;
+  private Tuple currentTuple;
+
+  public FullScanIterator(Scanner scanner) {
+    this.scanner = scanner;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    return (currentTuple = scanner.next()) != null;
+  }
+
+  @Override
+  public Tuple next() {
+    return currentTuple;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/2ec307d6/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanIterator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanIterator.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanIterator.java
new file mode 100644
index 0000000..813d8d0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ScanIterator.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.engine.planner.physical;
+
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+/**
+ * This is a scan iterator implementation for various scan types.
+ * It has the same semantic to java.util.Iterator except throwing IOException.
+ */
+public interface ScanIterator {
+  boolean hasNext() throws IOException;
+
+  Tuple next() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/2ec307d6/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
index 599f160..79e0a5d 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
@@ -62,6 +62,9 @@ public class SeqScanExec extends ScanExec {
 
   private TableStats inputStats;
 
+  // scanner iterator with filter or without filter
+  private ScanIterator scanIt;
+
   public SeqScanExec(TaskAttemptContext context, ScanNode plan,
                      CatalogProtos.FragmentProto [] fragments) throws IOException {
     super(context, plan.getInSchema(), plan.getOutSchema());
@@ -173,6 +176,10 @@ public class SeqScanExec extends ScanExec {
       } else {
         qual.bind(context.getEvalContext(), inSchema);
       }
+
+      scanIt = new FilterScanIterator(scanner, qual);
+    } else {
+      scanIt = new FullScanIterator(scanner);
     }
   }
 
@@ -226,26 +233,15 @@ public class SeqScanExec extends ScanExec {
       return null;
     }
 
-    Tuple tuple;
-    Tuple outTuple = new VTuple(outColumnNum);
-
-    if (!plan.hasQual()) {
-      if ((tuple = scanner.next()) != null) {
-        projector.eval(tuple, outTuple);
-        outTuple.setOffset(tuple.getOffset());
-        return outTuple;
-      } else {
-        return null;
-      }
-    } else {
-      while ((tuple = scanner.next()) != null) {
-        if (qual.eval(tuple).isTrue()) {
-          projector.eval(tuple, outTuple);
-          return outTuple;
-        }
-      }
-      return null;
+    while(scanIt.hasNext()) {
+      Tuple outTuple = new VTuple(outColumnNum);
+      Tuple t = scanIt.next();
+      projector.eval(t, outTuple);
+      outTuple.setOffset(t.getOffset());
+      return outTuple;
     }
+
+    return null;
   }
 
   @Override


[04/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/pom.xml b/tajo-storage/tajo-storage-common/pom.xml
index f7c9676..f1d3438 100644
--- a/tajo-storage/tajo-storage-common/pom.xml
+++ b/tajo-storage/tajo-storage-common/pom.xml
@@ -58,6 +58,12 @@ limitations under the License.
       <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/main/resources/*.json</exclude>
+            <exclude>src/test/resources/*.json</exclude>
+          </excludes>
+        </configuration>
         <executions>
           <execution>
             <phase>verify</phase>
@@ -293,6 +299,10 @@ limitations under the License.
       <groupId>io.netty</groupId>
       <artifactId>netty-buffer</artifactId>
     </dependency>
+    <dependency>
+      <groupId>net.minidev</groupId>
+      <artifactId>json-smart</artifactId>
+    </dependency>
   </dependencies>
 
   <profiles>
@@ -334,4 +344,4 @@ limitations under the License.
       </plugin>
     </plugins>
   </reporting>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java
new file mode 100644
index 0000000..0f0cd10
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FormatProperty.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+public class FormatProperty {
+  private boolean sortedInsertRequired;
+
+  public FormatProperty(boolean sortedInsertRequired) {
+    this.sortedInsertRequired = sortedInsertRequired;
+  }
+
+  public boolean sortedInsertRequired() {
+    return sortedInsertRequired;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
index a8926a0..ce573be 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
@@ -64,7 +64,7 @@ public class MergeScanner implements Scanner {
 
     long numBytes = 0;
     for (Fragment eachFileFragment: rawFragmentList) {
-      long fragmentLength = Tablespace.getFragmentLength((TajoConf) conf, eachFileFragment);
+      long fragmentLength = TableSpaceManager.guessFragmentVolume((TajoConf) conf, eachFileFragment);
       if (fragmentLength > 0) {
         numBytes += fragmentLength;
         fragments.add(eachFileFragment);
@@ -131,8 +131,7 @@ public class MergeScanner implements Scanner {
   private Scanner getNextScanner() throws IOException {
     if (iterator.hasNext()) {
       currentFragment = iterator.next();
-      currentScanner = TableSpaceManager.getStorageManager((TajoConf) conf, meta.getStoreType()).getScanner(meta, schema,
-          currentFragment, target);
+      currentScanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, currentFragment, target);
       currentScanner.init();
       return currentScanner;
     } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
new file mode 100644
index 0000000..12b236f
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/OldStorageManager.java
@@ -0,0 +1,251 @@
+/**
+ * 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.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.TaskAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.FileUtil;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.URI;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * It handles available table spaces and cache TableSpace instances.
+ */
+public class OldStorageManager {
+  private static final Log LOG = LogFactory.getLog(OldStorageManager.class);
+
+  /**
+   * 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>>();
+  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
+      Configuration.class,
+      Schema.class,
+      TableMeta.class,
+      Fragment.class
+  };
+  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
+      Configuration.class,
+      TaskAttemptId.class,
+      Schema.class,
+      TableMeta.class,
+      Path.class
+  };
+  /**
+   * Cache of Tablespace.
+   * Key is manager key(warehouse path) + store type
+   */
+  private static final Map<String, Tablespace> storageManagers = Maps.newHashMap();
+  /**
+   * Cache of constructors for each class. Pins the classes so they
+   * can't be garbage collected until ReflectionUtils can be collected.
+   */
+  protected static Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
+
+  /**
+   * Clear all class cache
+   */
+  @VisibleForTesting
+  protected synchronized static void clearCache() {
+    CONSTRUCTOR_CACHE.clear();
+    SCANNER_HANDLER_CACHE.clear();
+    APPENDER_HANDLER_CACHE.clear();
+    storageManagers.clear();
+  }
+
+  /**
+   * Close Tablespace
+   * @throws java.io.IOException
+   */
+  public static void shutdown() throws IOException {
+    synchronized(storageManagers) {
+      for (Tablespace eachTablespace : storageManagers.values()) {
+        eachTablespace.close();
+      }
+    }
+    clearCache();
+  }
+
+  /**
+   * Returns the proper Tablespace instance according to the storeType.
+   *
+   * @param tajoConf Tajo system property.
+   * @param storeType Storage type
+   * @return
+   * @throws IOException
+   */
+  public static Tablespace getStorageManager(TajoConf tajoConf, String storeType) throws IOException {
+    FileSystem fileSystem = TajoConf.getWarehouseDir(tajoConf).getFileSystem(tajoConf);
+    if (fileSystem != null) {
+      return getStorageManager(tajoConf, fileSystem.getUri(), storeType);
+    } else {
+      return getStorageManager(tajoConf, null, storeType);
+    }
+  }
+
+  /**
+   * Returns the proper Tablespace instance according to the storeType
+   *
+   * @param tajoConf Tajo system property.
+   * @param uri Key that can identify each storage manager(may be a path)
+   * @param storeType Storage type
+   * @return
+   * @throws IOException
+   */
+  public static synchronized Tablespace getStorageManager(
+      TajoConf tajoConf, URI uri, String storeType) throws IOException {
+    Preconditions.checkNotNull(tajoConf);
+    Preconditions.checkNotNull(uri);
+    Preconditions.checkNotNull(storeType);
+
+    String typeName;
+    if (storeType.equalsIgnoreCase("HBASE")) {
+      typeName = "hbase";
+    } else {
+      typeName = "hdfs";
+    }
+
+    synchronized (storageManagers) {
+      String storeKey = typeName + "_" + uri.toString();
+      Tablespace manager = storageManagers.get(storeKey);
+
+      if (manager == null) {
+        Class<? extends Tablespace> storageManagerClass =
+            tajoConf.getClass(String.format("tajo.storage.manager.%s.class", typeName), null, Tablespace.class);
+
+        if (storageManagerClass == null) {
+          throw new IOException("Unknown Storage Type: " + typeName);
+        }
+
+        try {
+          Constructor<? extends Tablespace> constructor =
+              (Constructor<? extends Tablespace>) CONSTRUCTOR_CACHE.get(storageManagerClass);
+          if (constructor == null) {
+            constructor = storageManagerClass.getDeclaredConstructor(TableSpaceManager.TABLESPACE_PARAM);
+            constructor.setAccessible(true);
+            CONSTRUCTOR_CACHE.put(storageManagerClass, constructor);
+          }
+          manager = constructor.newInstance(new Object[]{"noname", uri});
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+        manager.init(tajoConf);
+        storageManagers.put(storeKey, manager);
+      }
+
+      return manager;
+    }
+  }
+
+  /**
+   * 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 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);
+  }
+
+  /**
+   * 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, TaskAttemptId 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
index 6816d08..38d0734 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
@@ -19,22 +19,51 @@
 package org.apache.tajo.storage;
 
 public class StorageProperty {
-  private boolean supportsInsertInto;
-  private boolean sortedInsert;
+  private boolean movable;
+  private boolean writable;
+  private boolean insertable;
+  private boolean absolutePathAllowed;
 
-  public boolean isSupportsInsertInto() {
-    return supportsInsertInto;
+  public StorageProperty(boolean movable, boolean writable, boolean isInsertable, boolean absolutePathAllowed) {
+    this.movable = movable;
+    this.writable = writable;
+    this.insertable = isInsertable;
+    this.absolutePathAllowed = absolutePathAllowed;
   }
 
-  public void setSupportsInsertInto(boolean supportsInsertInto) {
-    this.supportsInsertInto = supportsInsertInto;
+  /**
+   * Move-like operation is allowed
+   *
+   * @return true if move operation is available
+   */
+  public boolean isMovable() {
+    return movable;
   }
 
-  public boolean isSortedInsert() {
-    return sortedInsert;
+  /**
+   * Is it Writable storage?
+   *
+   * @return true if this storage is writable.
+   */
+  public boolean isWritable() {
+    return writable;
   }
 
-  public void setSortedInsert(boolean sortedInsert) {
-    this.sortedInsert = sortedInsert;
+  /**
+   * this storage supports insert operation?
+   *
+   * @return true if insert operation is allowed.
+   */
+  public boolean isInsertable() {
+    return insertable;
+  }
+
+  /**
+   * Does this storage allows the use of arbitrary absolute paths outside tablespace?
+   *
+   * @return
+   */
+  public boolean isArbitraryPathAllowed() {
+    return this.absolutePathAllowed;
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
index 926b5d3..20a5d5c 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
@@ -68,7 +68,7 @@ public class StorageUtil extends StorageConstants {
         return 0;
     }
   }
-  
+
   public static Path concatPath(String parent, String...childs) {
     return concatPath(new Path(parent), childs);
   }
@@ -82,7 +82,7 @@ public class StorageUtil extends StorageConstants {
         sb.append("/");
     }
     
-    return new Path(parent, sb.toString());
+    return new Path(parent + "/" + sb.toString());
   }
 
   static final String fileNamePatternV08 = "part-[0-9]*-[0-9]*";

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java
index a787cdb..ef04509 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableSpaceManager.java
@@ -19,236 +19,372 @@
 package org.apache.tajo.storage;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
+import net.minidev.json.JSONObject;
+import net.minidev.json.parser.JSONParser;
+import net.minidev.json.parser.ParseException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
-import org.apache.tajo.TaskAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.TajoConstants;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.Pair;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
+import java.net.URI;
 import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static org.apache.tajo.storage.StorageConstants.LOCAL_FS_URI;
 
 /**
  * It handles available table spaces and cache TableSpace instances.
+ *
+ * Default tablespace must be a filesystem-based one.
+ * HDFS and S3 can be a default tablespace if a Tajo cluster is in fully distributed mode.
+ * Local file system can be a default tablespace if a Tajo cluster runs on a single machine.
  */
-public class TableSpaceManager {
+public class TableSpaceManager implements StorageService {
+  private static final Log LOG = LogFactory.getLog(TableSpaceManager.class);
 
-  /**
-   * 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>>();
-  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
-      Configuration.class,
-      Schema.class,
-      TableMeta.class,
-      Fragment.class
-  };
-  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
-      Configuration.class,
-      TaskAttemptId.class,
-      Schema.class,
-      TableMeta.class,
-      Path.class
-  };
-  /**
-   * Cache of Tablespace.
-   * Key is manager key(warehouse path) + store type
-   */
-  private static final Map<String, Tablespace> storageManagers = Maps.newHashMap();
-  /**
-   * 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 static final String DEFAULT_CONFIG_FILE = "storage-default.json";
+  public static final String SITE_CONFIG_FILE = "storage-site.json";
+
+  /** default tablespace name */
+  public static final String DEFAULT_TABLESPACE_NAME = "default";
+
+  private final static TajoConf systemConf = new TajoConf();
+  private final static JSONParser parser = new JSONParser(JSONParser.MODE_JSON_SIMPLE | JSONParser.IGNORE_CONTROL_CHAR);
+
+  // The relation ship among name, URI, Tablespaces must be kept 1:1:1.
+  protected static final Map<String, URI> SPACES_URIS_MAP = Maps.newHashMap();
+  protected static final TreeMap<URI, Tablespace> TABLE_SPACES = Maps.newTreeMap();
 
+  protected static final Map<Class<?>, Constructor<?>> CONSTRUCTORS = Maps.newHashMap();
+  protected static final Map<String, Class<? extends Tablespace>> TABLE_SPACE_HANDLERS = Maps.newHashMap();
+
+  public static final Class [] TABLESPACE_PARAM = new Class [] {String.class, URI.class};
+
+  static {
+    instance = new TableSpaceManager();
+  }
   /**
-   * Clear all class cache
+   * Singleton instance
    */
-  @VisibleForTesting
-  protected synchronized static void clearCache() {
-    CONSTRUCTOR_CACHE.clear();
-    SCANNER_HANDLER_CACHE.clear();
-    APPENDER_HANDLER_CACHE.clear();
-    storageManagers.clear();
+  private static final TableSpaceManager instance;
+
+  private TableSpaceManager() {
+    initForDefaultConfig(); // loading storage-default.json
+    initSiteConfig();       // storage-site.json will override the configs of storage-default.json
+    addWarehouseAsSpace();  // adding a warehouse directory for a default tablespace
+    addLocalFsTablespace(); // adding a tablespace using local file system by default
   }
 
-  /**
-   * Close Tablespace
-   * @throws java.io.IOException
-   */
-  public static void shutdown() throws IOException {
-    synchronized(storageManagers) {
-      for (Tablespace eachTablespace : storageManagers.values()) {
-        eachTablespace.close();
-      }
+  private void addWarehouseAsSpace() {
+    Path warehouseDir = TajoConf.getWarehouseDir(systemConf);
+    registerTableSpace(DEFAULT_TABLESPACE_NAME, warehouseDir.toUri(), null, true, false);
+  }
+
+  private void addLocalFsTablespace() {
+    if (TABLE_SPACES.headMap(LOCAL_FS_URI, true).firstEntry() == null) {
+      String tmpName = UUID.randomUUID().toString();
+      registerTableSpace(tmpName, LOCAL_FS_URI, null, false, false);
     }
-    clearCache();
   }
 
-  /**
-   * Returns FileStorageManager instance.
-   *
-   * @param tajoConf Tajo system property.
-   * @return
-   * @throws IOException
-   */
-  public static Tablespace getFileStorageManager(TajoConf tajoConf) throws IOException {
-    return getStorageManager(tajoConf, "CSV");
+  public static TableSpaceManager getInstance() {
+    return instance;
   }
 
-  /**
-   * Returns the proper Tablespace instance according to the storeType.
-   *
-   * @param tajoConf Tajo system property.
-   * @param storeType Storage type
-   * @return
-   * @throws IOException
-   */
-  public static Tablespace getStorageManager(TajoConf tajoConf, String storeType) throws IOException {
-    FileSystem fileSystem = TajoConf.getWarehouseDir(tajoConf).getFileSystem(tajoConf);
-    if (fileSystem != null) {
-      return getStorageManager(tajoConf, storeType, fileSystem.getUri().toString());
-    } else {
-      return getStorageManager(tajoConf, storeType, null);
+  private void initForDefaultConfig() {
+    JSONObject json = loadFromConfig(DEFAULT_CONFIG_FILE);
+    if (json == null) {
+      throw new IllegalStateException("There is no " + SITE_CONFIG_FILE);
     }
+    applyConfig(json, false);
   }
 
-  /**
-   * Returns the proper Tablespace 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 IOException
-   */
-  private static synchronized Tablespace getStorageManager (
-      TajoConf tajoConf, String storeType, String managerKey) throws IOException {
+  private void initSiteConfig() {
+    JSONObject json = loadFromConfig(SITE_CONFIG_FILE);
+
+    // if there is no storage-site.json file, nothing happen.
+    if (json != null) {
+      applyConfig(json, true);
+    }
+  }
+
+  private JSONObject loadFromConfig(String fileName) {
+    String json;
+    try {
+      json = FileUtil.readTextFileFromResource(fileName);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
 
-    String typeName;
-    if (storeType.equalsIgnoreCase("HBASE")) {
-      typeName = "hbase";
+    if (json != null) {
+      return parseJson(json);
     } else {
-      typeName = "hdfs";
+      return null;
     }
+  }
+
+  private static JSONObject parseJson(String json) {
+    try {
+      return (JSONObject) parser.parse(json);
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
 
-    synchronized (storageManagers) {
-      String storeKey = typeName + "_" + managerKey;
-      Tablespace manager = storageManagers.get(storeKey);
+  private void applyConfig(JSONObject json, boolean override) {
+    loadStorages(json);
+    loadTableSpaces(json, override);
+  }
 
-      if (manager == null) {
-        Class<? extends Tablespace> storageManagerClass =
-            tajoConf.getClass(String.format("tajo.storage.manager.%s.class", typeName), null, Tablespace.class);
+  private void loadStorages(JSONObject json) {
+    JSONObject spaces = (JSONObject) json.get(KEY_STORAGES);
 
-        if (storageManagerClass == null) {
-          throw new IOException("Unknown Storage Type: " + typeName);
-        }
+    if (spaces != null) {
+      Pair<String, Class<? extends Tablespace>> pair = null;
+      for (Map.Entry<String, Object> entry : spaces.entrySet()) {
 
         try {
-          Constructor<? extends Tablespace> constructor =
-              (Constructor<? extends Tablespace>) CONSTRUCTOR_CACHE.get(storageManagerClass);
-          if (constructor == null) {
-            constructor = storageManagerClass.getDeclaredConstructor(new Class<?>[]{String.class});
-            constructor.setAccessible(true);
-            CONSTRUCTOR_CACHE.put(storageManagerClass, constructor);
-          }
-          manager = constructor.newInstance(new Object[]{storeType});
-        } catch (Exception e) {
-          throw new RuntimeException(e);
+          pair = extractStorage(entry);
+        } catch (ClassNotFoundException e) {
+          LOG.warn(e);
+          continue;
         }
-        manager.init(tajoConf);
-        storageManagers.put(storeKey, manager);
+
+        TABLE_SPACE_HANDLERS.put(pair.getFirst(), pair.getSecond());
+      }
+    }
+  }
+
+  private Pair<String, Class<? extends Tablespace>> extractStorage(Map.Entry<String, Object> entry)
+      throws ClassNotFoundException {
+
+    String storageType = entry.getKey();
+    JSONObject storageDesc = (JSONObject) entry.getValue();
+    String handlerClass = (String) storageDesc.get(KEY_STORAGE_HANDLER);
+
+    return new Pair<String, Class<? extends Tablespace>>(
+        storageType,(Class<? extends Tablespace>) Class.forName(handlerClass));
+  }
+
+  private void loadTableSpaces(JSONObject json, boolean override) {
+    JSONObject spaces = (JSONObject) json.get(KEY_SPACES);
+
+    if (spaces != null) {
+      for (Map.Entry<String, Object> entry : spaces.entrySet()) {
+        AddTableSpace(entry.getKey(), (JSONObject) entry.getValue(), override);
       }
+    }
+  }
 
-      return manager;
+  public static void AddTableSpace(String spaceName, JSONObject spaceDesc, boolean override) {
+    boolean defaultSpace = Boolean.parseBoolean(spaceDesc.getAsString("default"));
+    URI spaceUri = URI.create(spaceDesc.getAsString("uri"));
+
+    if (defaultSpace) {
+      registerTableSpace(DEFAULT_TABLESPACE_NAME, spaceUri, spaceDesc, true, override);
     }
+    registerTableSpace(spaceName, spaceUri, spaceDesc, true, override);
   }
 
-  /**
-   * 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 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);
+  private static void registerTableSpace(String spaceName, URI uri, JSONObject spaceDesc,
+                                         boolean visible, boolean override) {
+    Tablespace tableSpace = initializeTableSpace(spaceName, uri, visible);
+    tableSpace.setVisible(visible);
+
+    try {
+      tableSpace.init(systemConf);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    putTablespace(tableSpace, override);
+
+    // If the arbitrary path is allowed, root uri is also added as a tablespace
+    if (tableSpace.getProperty().isArbitraryPathAllowed()) {
+      URI rootUri = tableSpace.getRootUri();
+      // if there already exists or the rootUri is 'file:/', it won't overwrite the tablespace.
+      if (!TABLE_SPACES.containsKey(rootUri) && !rootUri.toString().startsWith(LOCAL_FS_URI.toString())) {
+        String tmpName = UUID.randomUUID().toString();
+        registerTableSpace(tmpName, rootUri, spaceDesc, false, override);
+      }
+    }
+  }
+
+  private static void putTablespace(Tablespace space, boolean override) {
+    // It is a device to keep the relationship among name, URI, and tablespace 1:1:1.
+
+    boolean nameExist = SPACES_URIS_MAP.containsKey(space.getName());
+    boolean uriExist = TABLE_SPACES.containsKey(space.uri);
+
+    boolean mismatch = nameExist && !SPACES_URIS_MAP.get(space.getName()).equals(space.getUri());
+    mismatch = mismatch || uriExist && TABLE_SPACES.get(space.uri).equals(space);
+
+    if (!override && mismatch) {
+      throw new RuntimeException("Name or URI of Tablespace must be unique.");
+    }
+
+    SPACES_URIS_MAP.put(space.getName(), space.getUri());
+    // We must guarantee that the same uri results in the same tablespace instance.
+    TABLE_SPACES.put(space.getUri(), space);
   }
 
   /**
-   * Creates a scanner instance.
+   * Return length of the fragment.
+   * In the UNKNOWN_LENGTH case get FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH from the configuration.
    *
-   * @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
+   * @param conf Tajo system property
+   * @param fragment Fragment
+   * @return
    */
-  public static <T> T newScannerInstance(Class<T> theClass, Configuration conf, Schema schema, TableMeta meta,
-                                         Fragment fragment) {
-    T result;
+  public static long guessFragmentVolume(TajoConf conf, Fragment fragment) {
+    if (fragment.getLength() == TajoConstants.UNKNOWN_LENGTH) {
+      return conf.getLongVar(TajoConf.ConfVars.FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH);
+    } else {
+      return fragment.getLength();
+    }
+  }
+
+  public static final String KEY_STORAGES = "storages"; // storages
+  public static final String KEY_STORAGE_HANDLER = "handler"; // storages/?/handler
+  public static final String KEY_STORAGE_DEFAULT_FORMAT = "default-format"; // storages/?/default-format
+
+  public static final String KEY_SPACES = "spaces";
+
+  private static Tablespace initializeTableSpace(String spaceName, URI uri, boolean visible) {
+    Preconditions.checkNotNull(uri.getScheme(), "URI must include scheme, but it was " + uri);
+    Class<? extends Tablespace> clazz = TABLE_SPACE_HANDLERS.get(uri.getScheme());
+
+    if (clazz == null) {
+      throw new RuntimeException("There is no tablespace for " + uri.toString());
+    }
+
     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);
+      Constructor<? extends Tablespace> constructor =
+          (Constructor<? extends Tablespace>) CONSTRUCTORS.get(clazz);
+
+      if (constructor == null) {
+        constructor = clazz.getDeclaredConstructor(TABLESPACE_PARAM);
+        constructor.setAccessible(true);
+        CONSTRUCTORS.put(clazz, constructor);
       }
-      result = meth.newInstance(new Object[]{conf, schema, meta, fragment});
+
+      return constructor.newInstance(new Object[]{spaceName, uri});
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
+  }
+
+  @VisibleForTesting
+  public static Optional<Tablespace> addTableSpaceForTest(Tablespace space) {
+    Tablespace existing;
+    synchronized (SPACES_URIS_MAP) {
+      // Remove existing one
+      SPACES_URIS_MAP.remove(space.getName());
+      existing = TABLE_SPACES.remove(space.getUri());
+
+      // Add anotherone for test
+      registerTableSpace(space.name, space.uri, null, true, true);
+    }
+    // if there is an existing one, return it.
+    return Optional.fromNullable(existing);
+  }
 
-    return result;
+  public Iterable<String> getSupportSchemes() {
+    return TABLE_SPACE_HANDLERS.keySet();
   }
 
   /**
-   * Creates a scanner instance.
+   * Get tablespace for the given URI. If uri is null, the default tablespace will be returned
    *
-   * @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
+   * @param uri Table or Table Fragment URI.
+   * @param <T> Tablespace class type
+   * @return Tablespace. If uri is null, the default tablespace will be returned.
    */
-  public static <T> T newAppenderInstance(Class<T> theClass, Configuration conf, TaskAttemptId 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);
+  public static <T extends Tablespace> Optional<T> get(@Nullable String uri) {
+
+    if (uri == null || uri.isEmpty()) {
+      return (Optional<T>) Optional.of(getDefault());
+    }
+
+    Tablespace lastOne = null;
+
+    // Find the longest matched one. For example, assume that the caller tries to find /x/y/z, and
+    // there are /x and /x/y. In this case, /x/y will be chosen because it is more specific.
+    for (Map.Entry<URI, Tablespace> entry: TABLE_SPACES.headMap(URI.create(uri), true).entrySet()) {
+      if (uri.startsWith(entry.getKey().toString())) {
+        lastOne = entry.getValue();
       }
-      result = meth.newInstance(new Object[]{conf, taskAttemptId, schema, meta, workDir});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
     }
+    return (Optional<T>) Optional.fromNullable(lastOne);
+  }
+
+  /**
+   * Get tablespace for the given URI. If uri is null, the default tablespace will be returned
+   *
+   * @param uri Table or Table Fragment URI.
+   * @param <T> Tablespace class type
+   * @return Tablespace. If uri is null, the default tablespace will be returned.
+   */
+  public static <T extends Tablespace> Optional<T> get(@Nullable URI uri) {
+    if (uri == null) {
+      return (Optional<T>) Optional.of(getDefault());
+    } else {
+      return (Optional<T>) get(uri.toString());
+    }
+  }
+
+  /**
+   * It returns the default tablespace. This method ensures that it always return the tablespace.
+   *
+   * @return
+   */
+  public static <T extends Tablespace> T getDefault() {
+    return (T) getByName(DEFAULT_TABLESPACE_NAME).get();
+  }
+
+  public static <T extends Tablespace> T getLocalFs() {
+    return (T) get(LOCAL_FS_URI).get();
+  }
+
+  public static Optional<? extends Tablespace> getByName(String name) {
+    URI uri = SPACES_URIS_MAP.get(name);
+    if (uri != null) {
+      return Optional.of(TABLE_SPACES.get(uri));
+    } else {
+      return Optional.absent();
+    }
+  }
+
+  public static Optional<? extends Tablespace> getAnyByScheme(String scheme) {
+    for (Map.Entry<URI, Tablespace> entry : TABLE_SPACES.entrySet()) {
+      String uriScheme = entry.getKey().getScheme();
+      if (uriScheme != null && uriScheme.equalsIgnoreCase(scheme)) {
+        return Optional.of(entry.getValue());
+      }
+    }
+
+    return Optional.absent();
+  }
+
+  @Override
+  public URI getTableURI(@Nullable String spaceName, String databaseName, String tableName) {
+    Tablespace space = spaceName == null ? getDefault() : getByName(spaceName).get();
+    return space.getTableUri(databaseName, tableName);
+  }
 
-    return result;
+  public static Iterable<Tablespace> getAllTablespaces() {
+    return TABLE_SPACES.values();
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
index 0626da8..77c5d05 100644
--- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java
@@ -19,10 +19,8 @@
 package org.apache.tajo.storage;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.OverridableConf;
-import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.SortSpec;
@@ -30,16 +28,20 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.exception.UnsupportedException;
 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.ScanNode;
-import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 
 import java.io.IOException;
+import java.net.URI;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Tablespace manages the functions of storing and reading data.
@@ -49,18 +51,24 @@ import java.util.List;
  */
 public abstract class Tablespace {
 
-  public static final PathFilter hiddenFileFilter = new PathFilter() {
-    public boolean accept(Path p) {
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
+  protected final String name;
+  protected final URI uri;
+  /** this space is visible or not. */
+  protected boolean visible = true;
 
   protected TajoConf conf;
-  protected String storeType;
 
-  public Tablespace(String storeType) {
-    this.storeType = storeType;
+  public Tablespace(String name, URI uri) {
+    this.name = name;
+    this.uri = uri;
+  }
+
+  public void setVisible(boolean visible) {
+    this.visible = visible;
+  }
+
+  public Set<String> getDependencies() {
+    return Collections.emptySet();
   }
 
   /**
@@ -69,24 +77,47 @@ public abstract class Tablespace {
    */
   protected abstract void storageInit() throws IOException;
 
+  public String getName() {
+    return name;
+  }
+
+  public URI getUri() {
+    return uri;
+  }
+
+  public boolean isVisible() {
+    return visible;
+  }
+
+  public abstract void setConfig(String name, String value);
+
+  public abstract void setConfigs(Map<String, String> configs);
+
+  public String toString() {
+    return name + "=" + uri.toString();
+  }
+
+  public abstract long getTableVolume(URI uri) 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.
+   * if {@link StorageProperty#isArbitraryPathAllowed} is true,
+   * the storage allows arbitrary path accesses. In this case, the storage must provide the root URI.
    *
-   * @param tableDesc Table description which is created.
-   * @param ifNotExists Creates the table only when the table does not exist.
-   * @throws java.io.IOException
+   * @see {@link StorageProperty#isArbitraryPathAllowed}
+   * @return Root URI
    */
-  public abstract void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException;
+  public URI getRootUri() {
+    throw new UnsupportedException(
+        String.format("Tablespace '%s' does not allow the use of artibrary paths", uri.toString()));
+  }
 
   /**
-   * This method is called after executing "DROP TABLE" statement with the 'PURGE' option
-   * which is the option to delete all the data.
+   * Get Table URI
    *
-   * @param tableDesc
-   * @throws java.io.IOException
+   * @param tableName
+   * @return
    */
-  public abstract void purgeTable(TableDesc tableDesc) throws IOException;
+  public abstract URI getTableUri(String databaseName, String tableName);
 
   /**
    * Returns the splits that will serve as input for the scan tasks. The
@@ -116,7 +147,9 @@ public abstract class Tablespace {
    * It returns the storage property.
    * @return The storage property
    */
-  public abstract StorageProperty getStorageProperty();
+  public abstract StorageProperty getProperty();
+
+  public abstract FormatProperty getFormatProperty(String dataFormat);
 
   /**
    * Release storage manager resource
@@ -137,21 +170,10 @@ public abstract class Tablespace {
    * @throws java.io.IOException
    */
   public abstract TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
-                                                   Schema inputSchema, SortSpec[] sortSpecs,
+                                                   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.
    *
@@ -160,21 +182,13 @@ public abstract class Tablespace {
    */
 
   /**
-   * Returns the current storage type.
-   * @return
-   */
-  public String getStoreType() {
-    return storeType;
-  }
-
-  /**
    * Initialize Tablespace instance. It should be called before using.
    *
    * @param tajoConf
    * @throws java.io.IOException
    */
   public void init(TajoConf tajoConf) throws IOException {
-    this.conf = tajoConf;
+    this.conf = new TajoConf(tajoConf);
     storageInit();
   }
 
@@ -239,7 +253,7 @@ public abstract class Tablespace {
     Scanner scanner;
 
     Class<? extends Scanner> scannerClass = getScannerClass(meta.getStoreType());
-    scanner = TableSpaceManager.newScannerInstance(scannerClass, conf, schema, meta, fragment);
+    scanner = OldStorageManager.newScannerInstance(scannerClass, conf, schema, meta, fragment);
     scanner.setTarget(target.toArray());
 
     return scanner;
@@ -263,18 +277,18 @@ public abstract class Tablespace {
     Class<? extends Appender> appenderClass;
 
     String handlerName = meta.getStoreType().toLowerCase();
-    appenderClass = TableSpaceManager.APPENDER_HANDLER_CACHE.get(handlerName);
+    appenderClass = OldStorageManager.APPENDER_HANDLER_CACHE.get(handlerName);
     if (appenderClass == null) {
       appenderClass = conf.getClass(
           String.format("tajo.storage.appender-handler.%s.class", handlerName), null, Appender.class);
-      TableSpaceManager.APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
+      OldStorageManager.APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
     }
 
     if (appenderClass == null) {
       throw new IOException("Unknown Storage Type: " + meta.getStoreType());
     }
 
-    appender = TableSpaceManager.newAppenderInstance(appenderClass, conf, taskAttemptId, meta, schema, workDir);
+    appender = OldStorageManager.newAppenderInstance(appenderClass, conf, taskAttemptId, meta, schema, workDir);
 
     return appender;
   }
@@ -288,11 +302,11 @@ public abstract class Tablespace {
    */
   public Class<? extends Scanner> getScannerClass(String storeType) throws IOException {
     String handlerName = storeType.toLowerCase();
-    Class<? extends Scanner> scannerClass = TableSpaceManager.SCANNER_HANDLER_CACHE.get(handlerName);
+    Class<? extends Scanner> scannerClass = OldStorageManager.SCANNER_HANDLER_CACHE.get(handlerName);
     if (scannerClass == null) {
       scannerClass = conf.getClass(
           String.format("tajo.storage.scanner-handler.%s.class", handlerName), null, Scanner.class);
-      TableSpaceManager.SCANNER_HANDLER_CACHE.put(handlerName, scannerClass);
+      OldStorageManager.SCANNER_HANDLER_CACHE.put(handlerName, scannerClass);
     }
 
     if (scannerClass == null) {
@@ -303,43 +317,54 @@ public abstract class Tablespace {
   }
 
   /**
-   * Return length of the fragment.
-   * In the UNKNOWN_LENGTH case get FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH from the configuration.
+   * It is called after making logical plan. Storage manager should verify the schema for inserting.
    *
-   * @param conf Tajo system property
-   * @param fragment Fragment
-   * @return
+   * @param tableDesc The table description of insert target.
+   * @param outSchema  The output schema of select query for inserting.
+   * @throws java.io.IOException
    */
-  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();
-    }
+  public abstract void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException;
+
+  /**
+   * Rewrite the logical plan. It is assumed that the final plan will be given in this method.
+   */
+  public void rewritePlan(OverridableConf context, LogicalPlan plan) throws PlanningException {
+    // nothing to do by default
   }
 
-  public abstract void rollbackOutputCommit(LogicalNode node) throws IOException;
+  ////////////////////////////////////////////////////////////////////////////
+  // Table Lifecycle Section
+  ////////////////////////////////////////////////////////////////////////////
 
   /**
-   * It is called after making logical plan. Storage manager should verify the schema for inserting.
+   * 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 The table description of insert target.
-   * @param outSchema  The output schema of select query for inserting.
+   * @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 verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException;
+  public abstract void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException;
 
   /**
-   * Returns the list of storage specified rewrite rules.
-   * This values are used by LogicalOptimizer.
+   * This method is called after executing "DROP TABLE" statement with the 'PURGE' option
+   * which is the option to delete all the data.
    *
-   * @param queryContext The query property
-   * @param tableDesc The description of the target table.
-   * @return The list of storage specified rewrite rules
+   * @param tableDesc
    * @throws java.io.IOException
    */
-  public abstract List<LogicalPlanRewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc)
-      throws IOException;
+  public abstract void purgeTable(TableDesc tableDesc) 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 prepareTable(LogicalNode node) throws IOException;
 
   /**
    * Finalizes result data. Tajo stores result data in the staging directory.
@@ -354,7 +379,20 @@ public abstract class Tablespace {
    * @return Saved path
    * @throws java.io.IOException
    */
-  public abstract Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
-                               LogicalPlan plan, Schema schema,
-                               TableDesc tableDesc) throws IOException;
+  public abstract Path commitTable(OverridableConf queryContext,
+                                   ExecutionBlockId finalEbId,
+                                   LogicalPlan plan, Schema schema,
+                                   TableDesc tableDesc) throws IOException;
+
+  public abstract void rollbackTable(LogicalNode node) throws IOException;
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof Tablespace) {
+      Tablespace other = (Tablespace) obj;
+      return name.equals(other.name) && uri.equals(other.uri);
+    } else {
+      return false;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json
new file mode 100644
index 0000000..40e17f4
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json
@@ -0,0 +1,20 @@
+{
+  "storages": {
+    "hdfs": {
+      "handler": "org.apache.tajo.storage.FileTablespace",
+      "default-format": "text"
+    },
+    "file": {
+      "handler": "org.apache.tajo.storage.FileTablespace",
+      "default-format": "text"
+    },
+    "s3": {
+      "handler": "org.apache.tajo.storage.FileTablespace",
+      "default-format": "text"
+    },
+    "hbase": {
+      "handler": "org.apache.tajo.storage.hbase.HBaseTablespace",
+      "default-format": "hbase"
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/pom.xml b/tajo-storage/tajo-storage-hbase/pom.xml
index 3456b76..5a1dc9a 100644
--- a/tajo-storage/tajo-storage-hbase/pom.xml
+++ b/tajo-storage/tajo-storage-hbase/pom.xml
@@ -61,6 +61,12 @@
       <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/main/resources/*.json</exclude>
+            <exclude>src/test/resources/*.json</exclude>
+          </excludes>
+        </configuration>
         <executions>
           <execution>
             <phase>verify</phase>
@@ -182,6 +188,11 @@
       <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>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
----------------------------------------------------------------------
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
index 425f392..0fc2922 100644
--- 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
@@ -89,7 +89,7 @@ public abstract class AbstractHBaseAppender implements Appender {
     if (enabledStats) {
       stats = new TableStatistics(this.schema);
     }
-    columnMapping = new ColumnMapping(schema, meta);
+    columnMapping = new ColumnMapping(schema, meta.getOptions());
 
     mappingColumnFamilies = columnMapping.getMappingColumns();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 32f1e43..0000000
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.hbase;
-
-import org.apache.tajo.OverridableConf;
-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.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.LogicalPlanRewriteRule;
-import org.apache.tajo.plan.util.PlannerUtil;
-
-public class AddSortForInsertRewriter implements LogicalPlanRewriteRule {
-  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(OverridableConf queryContext, LogicalPlan plan) {
-    String storeType = PlannerUtil.getStoreType(plan);
-    return storeType != null;
-  }
-
-  @Override
-  public LogicalPlan rewrite(OverridableConf queryContext, 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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
----------------------------------------------------------------------
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
index e66a707..0314e8e 100644
--- 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
@@ -20,18 +20,18 @@ 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 org.apache.tajo.util.KeyValueSet;
 
 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 KeyValueSet tableProperty;
 
+  private char rowKeyDelimiter;
   private String hbaseTableName;
 
   private int[] rowKeyFieldIndexes;
@@ -45,16 +45,15 @@ public class ColumnMapping {
 
   private int numRowKeys;
 
-  public ColumnMapping(Schema schema, TableMeta tableMeta) throws IOException {
+  public ColumnMapping(Schema schema, KeyValueSet tableProperty) throws IOException{
     this.schema = schema;
-    this.tableMeta = tableMeta;
-
+    this.tableProperty = tableProperty;
     init();
   }
 
   public void init() throws IOException {
-    hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY);
-    String delim = tableMeta.getOption(HBaseStorageConstants.META_ROWKEY_DELIMITER, "").trim();
+    hbaseTableName = tableProperty.get(HBaseStorageConstants.META_TABLE_KEY);
+    String delim = tableProperty.get(HBaseStorageConstants.META_ROWKEY_DELIMITER, "").trim();
     if (delim.length() > 0) {
       rowKeyDelimiter = delim.charAt(0);
     }
@@ -70,7 +69,7 @@ public class ColumnMapping {
       rowKeyFieldIndexes[i] = -1;
     }
 
-    String columnMapping = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
+    String columnMapping = tableProperty.get(HBaseStorageConstants.META_COLUMNS_KEY, "");
     if (columnMapping == null || columnMapping.isEmpty()) {
       throw new IOException("'columns' property is required.");
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java
----------------------------------------------------------------------
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
index 9ea0bf6..5961751 100644
--- 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
@@ -29,8 +29,12 @@ import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.hbase.StorageFragmentProtos.*;
 
+import java.net.URI;
+
 public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Cloneable {
   @Expose
+  private URI uri;
+  @Expose
   private String tableName;
   @Expose
   private String hbaseTableName;
@@ -45,7 +49,9 @@ public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Clone
   @Expose
   private long length;
 
-  public HBaseFragment(String tableName, String hbaseTableName, byte[] startRow, byte[] stopRow, String regionLocation) {
+  public HBaseFragment(URI uri, String tableName, String hbaseTableName, byte[] startRow, byte[] stopRow,
+                       String regionLocation) {
+    this.uri = uri;
     this.tableName = tableName;
     this.hbaseTableName = hbaseTableName;
     this.startRow = startRow;
@@ -62,6 +68,7 @@ public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Clone
   }
 
   private void init(HBaseFragmentProto proto) {
+    this.uri = URI.create(proto.getUri());
     this.tableName = proto.getTableName();
     this.hbaseTableName = proto.getHbaseTableName();
     this.startRow = proto.getStartRow().toByteArray();
@@ -76,6 +83,10 @@ public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Clone
     return Bytes.compareTo(startRow, t.startRow);
   }
 
+  public URI getUri() {
+    return uri;
+  }
+
   @Override
   public String getTableName() {
     return tableName;
@@ -107,6 +118,7 @@ public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Clone
 
   public Object clone() throws CloneNotSupportedException {
     HBaseFragment frag = (HBaseFragment) super.clone();
+    frag.uri = uri;
     frag.tableName = tableName;
     frag.hbaseTableName = hbaseTableName;
     frag.startRow = startRow;
@@ -137,16 +149,20 @@ public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Clone
 
   @Override
   public String toString() {
-    return "\"fragment\": {\"tableName\": \""+ tableName + "\", hbaseTableName\": \"" + hbaseTableName + "\"" +
-        ", \"startRow\": \"" + new String(startRow) + "\"" +
-        ", \"stopRow\": \"" + new String(stopRow) + "\"" +
-        ", \"length\": \"" + length + "\"}" ;
+    return
+        "\"fragment\": {\"uri:\"" + uri.toString() +"\", \"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)
+    builder
+        .setUri(uri.toString())
+        .setTableName(tableName)
         .setHbaseTableName(hbaseTableName)
         .setStartRow(ByteString.copyFrom(startRow))
         .setStopRow(ByteString.copyFrom(stopRow))

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
----------------------------------------------------------------------
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
index 19fdf80..916aae7 100644
--- 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
@@ -26,28 +26,29 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.storage.Tuple;
 
 import java.io.IOException;
+import java.net.URI;
 
 public class HBasePutAppender extends AbstractHBaseAppender {
+  private URI uri;
   private HTableInterface htable;
   private long totalNumBytes;
 
-  public HBasePutAppender(Configuration conf, TaskAttemptId taskAttemptId,
+  public HBasePutAppender(Configuration conf, URI uri, TaskAttemptId taskAttemptId,
                           Schema schema, TableMeta meta, Path stagingDir) {
     super(conf, taskAttemptId, schema, meta, stagingDir);
+    this.uri = uri;
   }
 
   @Override
   public void init() throws IOException {
     super.init();
 
-    Configuration hbaseConf = HBaseTablespace.getHBaseConfiguration(conf, meta);
-    HConnection hconn = ((HBaseTablespace) TableSpaceManager.getStorageManager((TajoConf) conf, "HBASE"))
-        .getConnection(hbaseConf);
+    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.get(uri).get();
+    HConnection hconn = space.getConnection();
     htable = hconn.getTable(columnMapping.getHbaseTableName());
     htable.setAutoFlushTo(false);
     htable.setWriteBufferSize(5 * 1024 * 1024);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
----------------------------------------------------------------------
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
index 992c13c..16f4c14 100644
--- 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
@@ -117,7 +117,7 @@ public class HBaseScanner implements Scanner {
       targets = schema.toArray();
     }
 
-    columnMapping = new ColumnMapping(schema, meta);
+    columnMapping = new ColumnMapping(schema, meta.getOptions());
     targetIndexes = new int[targets.length];
     int index = 0;
     for (Column eachTargetColumn: targets) {
@@ -133,8 +133,8 @@ public class HBaseScanner implements Scanner {
     rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
     rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
 
-    hbaseConf = HBaseTablespace.getHBaseConfiguration(conf, meta);
-
+    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.get(fragment.getUri()).get();
+    hbaseConf = space.getHbaseConf();
     initScanner();
   }
 
@@ -181,8 +181,7 @@ public class HBaseScanner implements Scanner {
     }
 
     if (htable == null) {
-      HConnection hconn = ((HBaseTablespace) TableSpaceManager.getStorageManager(conf, "HBASE"))
-          .getConnection(hbaseConf);
+      HConnection hconn = ((HBaseTablespace) TableSpaceManager.get(fragment.getUri()).get()).getConnection();
       htable = hconn.getTable(fragment.getHbaseTableName());
     }
     scanner = htable.getScanner(scan);


[05/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/query/TestHBaseTable.java
----------------------------------------------------------------------
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
index b6a4707..3d2d857 100644
--- 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
@@ -33,12 +33,13 @@ import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
 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.Tablespace;
 import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.hbase.*;
 import org.apache.tajo.util.Bytes;
@@ -49,7 +50,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.io.IOException;
 import java.net.InetAddress;
+import java.net.URI;
 import java.sql.ResultSet;
 import java.text.DecimalFormat;
 import java.util.*;
@@ -61,10 +64,11 @@ import static org.junit.Assert.assertEquals;
 public class TestHBaseTable extends QueryTestCaseBase {
   private static final Log LOG = LogFactory.getLog(TestHBaseTable.class);
 
+  private static String tableSpaceUri;
   private static String hostName,zkPort;
 
   @BeforeClass
-  public static void beforeClass() {
+  public static void beforeClass() throws IOException {
     try {
       testingCluster.getHBaseUtil().startHBaseCluster();
       hostName = InetAddress.getLocalHost().getHostName();
@@ -74,6 +78,11 @@ public class TestHBaseTable extends QueryTestCaseBase {
     } catch (Exception e) {
       e.printStackTrace();
     }
+
+    tableSpaceUri = "hbase:zk://" + hostName + ":" + zkPort;
+    HBaseTablespace hBaseTablespace = new HBaseTablespace("cluster1", URI.create(tableSpaceUri));
+    hBaseTablespace.init(new TajoConf(testingCluster.getHBaseUtil().getConf()));
+    TableSpaceManager.addTableSpaceForTest(hBaseTablespace);
   }
 
   @AfterClass
@@ -88,8 +97,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
   @Test
   public void testVerifyCreateHBaseTableRequiredMeta() throws Exception {
     try {
-      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
-          "USING hbase").close();
+      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) TABLESPACE cluster1 USING hbase").close();
 
       fail("hbase table must have 'table' meta");
     } catch (Exception e) {
@@ -97,7 +105,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
     }
 
     try {
-      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) " +
+      executeString("CREATE TABLE hbase_mapped_table1 (col1 text, col2 text) TABLESPACE cluster1 " +
           "USING hbase " +
           "WITH ('table'='hbase_table')").close();
 
@@ -109,10 +117,9 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testCreateHBaseTable() throws Exception {
-    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();
+    executeString(
+        "CREATE TABLE hbase_mapped_table1 (col1 text, col2 text, col3 text, col4 text) TABLESPACE cluster1 " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col2:a,col3:,col2:b')").close();
 
     assertTableExists("hbase_mapped_table1");
 
@@ -138,11 +145,12 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testCreateNotExistsExternalHBaseTable() throws Exception {
+    String sql = String.format(
+        "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') " +
+            "LOCATION '%s/external_hbase_table'", tableSpaceUri);
     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();
+      executeString(sql).close();
       fail("External table should be a existed table.");
     } catch (Exception e) {
       assertTrue(e.getMessage().indexOf("External table should be a existed table.") >= 0);
@@ -153,10 +161,8 @@ public class TestHBaseTable extends QueryTestCaseBase {
   public void testCreateRowFieldWithNonText() throws Exception {
     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();
+          "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'='0:key#b,1:key,col3:,col2:b', " +
+          "'hbase.rowkey.delimiter'='_')").close();
       fail("Key field type should be TEXT type");
     } catch (Exception e) {
       assertTrue(e.getMessage().indexOf("Key field type should be TEXT type") >= 0);
@@ -171,10 +177,11 @@ public class TestHBaseTable extends QueryTestCaseBase {
     hTableDesc.addFamily(new HColumnDescriptor("col3"));
     testingCluster.getHBaseUtil().createTable(hTableDesc);
 
-    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();
+    String sql = String.format(
+        "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') " +
+        "LOCATION '%s/external_hbase_table'", tableSpaceUri);
+    executeString(sql).close();
 
     assertTableExists("external_hbase_mapped_table");
 
@@ -198,15 +205,16 @@ public class TestHBaseTable extends QueryTestCaseBase {
     hTableDesc.addFamily(new HColumnDescriptor("col3"));
     testingCluster.getHBaseUtil().createTable(hTableDesc);
 
-    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();
+    String sql = String.format(
+        "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') " +
+        "LOCATION '%s/external_hbase_table'", tableSpaceUri);
+    executeString(sql).close();
 
     assertTableExists("external_hbase_mapped_table");
 
-    HConnection hconn = ((HBaseTablespace) TableSpaceManager.getStorageManager(conf, "HBASE"))
-        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
     try {
@@ -237,15 +245,16 @@ public class TestHBaseTable extends QueryTestCaseBase {
     hTableDesc.addFamily(new HColumnDescriptor("col3"));
     testingCluster.getHBaseUtil().createTable(hTableDesc);
 
-    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();
+    String sql = String.format(
+        "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') " +
+        "LOCATION '%s/external_hbase_table'", tableSpaceUri);
+    executeString(sql).close();
 
     assertTableExists("external_hbase_mapped_table");
 
-    HConnection hconn = ((HBaseTablespace) TableSpaceManager.getStorageManager(conf, "HBASE"))
-        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
     try {
@@ -289,16 +298,16 @@ public class TestHBaseTable extends QueryTestCaseBase {
     hTableDesc.addFamily(new HColumnDescriptor("col3"));
     testingCluster.getHBaseUtil().createTable(hTableDesc);
 
-    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, col2_key text, col2_value text, col3 text) " +
+    String sql = String.format(
+        "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();
+        "'hbase.rowkey.delimiter'='_') LOCATION '%s/external_hbase_table'", tableSpaceUri);
+    executeString(sql).close();
 
     assertTableExists("external_hbase_mapped_table");
 
-    HConnection hconn = ((HBaseTablespace) TableSpaceManager.getStorageManager(conf, "HBASE"))
-        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
     try {
@@ -326,16 +335,16 @@ public class TestHBaseTable extends QueryTestCaseBase {
     hTableDesc.addFamily(new HColumnDescriptor("col3"));
     testingCluster.getHBaseUtil().createTable(hTableDesc);
 
-    executeString("CREATE EXTERNAL TABLE external_hbase_mapped_table (rk1 text, rk2 text, col3 text) " +
+    String sql = String.format(
+        "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();
+        "'hbase.rowkey.delimiter'='_') LOCATION '%s/external_hbase_table'", tableSpaceUri);
+    executeString(sql).close();
 
     assertTableExists("external_hbase_mapped_table");
 
-    HConnection hconn = ((HBaseTablespace) TableSpaceManager.getStorageManager(conf, "HBASE"))
-        .getConnection(testingCluster.getHBaseUtil().getConf());
+    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HConnection hconn = space.getConnection();
     HTableInterface htable = hconn.getTable("external_hbase_table");
 
     try {
@@ -356,11 +365,11 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testIndexPredication() throws Exception {
-    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();
+    String sql =
+        "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 text) " +
+        "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b', " +
+        "'hbase.split.rowkeys'='010,040,060,080') ";
+    executeString(sql).close();
 
 
     assertTableExists("hbase_mapped_table");
@@ -404,12 +413,11 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testCompositeRowIndexPredication() throws Exception {
+
     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', " +
+        "TABLESPACE cluster1 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();
+        "'hbase.rowkey.delimiter'='_')").close();
 
     assertTableExists("hbase_mapped_table");
     HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
@@ -469,7 +477,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
     EvalNode evalNodeEq = new BinaryEval(EvalType.EQUAL, new FieldEval(tableDesc.getLogicalSchema().getColumn("rk")),
         new ConstEval(new TextDatum("021")));
     scanNode.setQual(evalNodeEq);
-    Tablespace tablespace = TableSpaceManager.getStorageManager(conf, "HBASE");
+    Tablespace tablespace = TableSpaceManager.getByName("cluster1").get();
     List<Fragment> fragments = tablespace.getSplits("hbase_mapped_table", tableDesc, scanNode);
     assertEquals(1, fragments.size());
     assertEquals("021", new String(((HBaseFragment)fragments.get(0)).getStartRow()));
@@ -559,10 +567,8 @@ public class TestHBaseTable extends QueryTestCaseBase {
   @Test
   public void testNonForwardQuery() throws Exception {
     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();
+        "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:#b', " +
+        "'hbase.split.rowkeys'='010,040,060,080')").close();
 
     assertTableExists("hbase_mapped_table");
     HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
@@ -599,10 +605,8 @@ public class TestHBaseTable extends QueryTestCaseBase {
   @Test
   public void testJoin() throws Exception {
     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();
+        "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b', " +
+        "'hbase.split.rowkeys'='010,040,060,080')").close();
 
     assertTableExists("hbase_mapped_table");
     HBaseAdmin hAdmin =  new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
@@ -641,9 +645,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
   @Test
   public void testInsertInto() throws Exception {
     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();
+        "TABLESPACE cluster1 USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -682,11 +684,9 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoMultiRegion() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.split.rowkeys'='010,040,060,080')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -740,11 +740,9 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoMultiRegion2() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -799,11 +797,10 @@ public class TestHBaseTable extends QueryTestCaseBase {
   public void testInsertIntoMultiRegionWithSplitFile() throws Exception {
     String splitFilePath = currentDatasetPath + "/splits.data";
 
-    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.split.rowkeys.file'='" + splitFilePath + "')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -857,12 +854,11 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoMultiRegionMultiRowFields() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text) " +
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.rowkey.delimiter'='_')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -917,11 +913,9 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoBinaryMultiRegion() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk int4, col1 text) " +
+    executeString("CREATE TABLE hbase_mapped_table (rk int4, col1 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -974,11 +968,10 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoColumnKeyValue() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk text, col2_key text, col2_value text, col3 text) " +
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk text, col2_key text, col2_value text, col3 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.rowkey.delimiter'='_')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -1065,11 +1058,10 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoDifferentType() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.split.rowkeys'='1,2,3,4,5,6,7,8,9')").close();
 
     assertTableExists("hbase_mapped_table");
 
@@ -1102,11 +1094,10 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoRowField() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text, col2 text, col3 text) " +
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk1 text, rk2 text, col1 text, col2 text, col3 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.rowkey.delimiter'='_')").close();
 
 
     assertTableExists("hbase_mapped_table");
@@ -1145,7 +1136,7 @@ public class TestHBaseTable extends QueryTestCaseBase {
   }
 
   @Test
-  public void testCATS() throws Exception {
+  public void testCTAS() throws Exception {
     // create test table
     KeyValueSet tableOptions = new KeyValueSet();
     tableOptions.set(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
@@ -1162,13 +1153,11 @@ public class TestHBaseTable extends QueryTestCaseBase {
     TajoTestingCluster.createTable(getCurrentDatabase() + ".base_table",
         schema, tableOptions, datas.toArray(new String[]{}), 2);
 
-    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text) " +
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk text, col1 text) TABLESPACE cluster1 " +
         "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"
+        "'hbase.split.rowkeys'='010,040,060,080') as" +
+        " select id, name from base_table"
     ).close();
 
     assertTableExists("hbase_mapped_table");
@@ -1199,15 +1188,21 @@ public class TestHBaseTable extends QueryTestCaseBase {
       if (htable != null) {
         htable.close();
       }
+
+      // TODO - rollback should support its corresponding hbase table
+      HBaseAdmin hAdmin = new HBaseAdmin(testingCluster.getHBaseUtil().getConf());
+      if (hAdmin.tableExists("hbase_table")) {
+        hAdmin.disableTable("hbase_table");
+        hAdmin.deleteTable("hbase_table");
+      }
     }
   }
 
   @Test
   public void testInsertIntoUsingPut() throws Exception {
-    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();
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text, col3 int4) TABLESPACE cluster1 " +
+        "USING hbase WITH ('table'='hbase_table', 'columns'=':key,col1:a,col2:,col3:b#b')").close();
 
     assertTableExists("hbase_mapped_table");
     TableDesc tableDesc = catalog.getTableDesc(getCurrentDatabase(), "hbase_mapped_table");
@@ -1219,8 +1214,10 @@ public class TestHBaseTable extends QueryTestCaseBase {
     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();
+      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");
 
@@ -1253,11 +1250,10 @@ public class TestHBaseTable extends QueryTestCaseBase {
 
   @Test
   public void testInsertIntoLocation() throws Exception {
-    executeString("CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text) " +
+    executeString(
+        "CREATE TABLE hbase_mapped_table (rk text, col1 text, col2 text) TABLESPACE cluster1 " +
         "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();
+        "'hbase.split.rowkeys'='010,040,060,080')").close();
 
     assertTableExists("hbase_mapped_table");
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
index b4334f6..7a671d8 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java
@@ -277,7 +277,7 @@ public class TestInsertQuery extends QueryTestCaseBase {
       TableDesc tableDesc = testingCluster.getMaster().getCatalog().getTableDesc(getCurrentDatabase(), tableName);
       assertNotNull(tableDesc);
 
-      Path path = new Path(tableDesc.getPath());
+      Path path = new Path(tableDesc.getUri());
       FileSystem fs = path.getFileSystem(testingCluster.getConfiguration());
 
       FileStatus[] files = fs.listStatus(path);
@@ -484,10 +484,10 @@ public class TestInsertQuery extends QueryTestCaseBase {
     }
 
     FileSystem fs = FileSystem.get(testingCluster.getConfiguration());
-    assertTrue(fs.exists(new Path(desc.getPath())));
+    assertTrue(fs.exists(new Path(desc.getUri())));
     CompressionCodecFactory factory = new CompressionCodecFactory(testingCluster.getConfiguration());
 
-    for (FileStatus file : fs.listStatus(new Path(desc.getPath()))) {
+    for (FileStatus file : fs.listStatus(new Path(desc.getUri()))) {
       CompressionCodec codec = factory.getCodec(file.getPath());
       assertTrue(codec instanceof DeflateCodec);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
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 a65c165..1478690 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
@@ -262,9 +262,9 @@ public class TestJoinQuery extends QueryTestCaseBase {
           appender.flush();
           appender.close();
         }
-        Path dataPath = new Path(table.getPath().toString(), fileIndex + ".csv");
+        Path dataPath = new Path(table.getUri().toString(), fileIndex + ".csv");
         fileIndex++;
-        appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+        appender = (((FileTablespace)TableSpaceManager.getLocalFs()))
             .getAppender(tableMeta, schema, dataPath);
         appender.init();
       }
@@ -279,7 +279,7 @@ public class TestJoinQuery extends QueryTestCaseBase {
   protected static void addEmptyDataFile(String tableName, boolean isPartitioned) throws Exception {
     TableDesc table = client.getTableDesc(tableName);
 
-    Path path = new Path(table.getPath());
+    Path path = new Path(table.getUri());
     FileSystem fs = path.getFileSystem(conf);
     if (isPartitioned) {
       List<Path> partitionPathList = getPartitionPathList(fs, path);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index 0d98b91..397b9ef 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -236,7 +236,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
   private void assertPartitionDirectories(TableDesc desc) throws IOException {
     FileSystem fs = FileSystem.get(conf);
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
     assertTrue(fs.isDirectory(path));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0")));
@@ -361,7 +361,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
 
     FileSystem fs = FileSystem.get(conf);
     assertTrue(fs.isDirectory(path));
@@ -434,7 +434,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     res.close();
 
     TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
 
     FileSystem fs = FileSystem.get(conf);
     assertTrue(fs.isDirectory(path));
@@ -486,7 +486,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
     res.close();
 
     desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
-    path = new Path(desc.getPath());
+    path = new Path(desc.getUri());
 
     assertTrue(fs.isDirectory(path));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1")));
@@ -515,7 +515,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
         "R\n" +
         "R\n";
 
-    String tableData = getTableFileContents(new Path(desc.getPath()));
+    String tableData = getTableFileContents(new Path(desc.getUri()));
     assertEquals(expected, tableData);
 
     res = executeString("select * from " + tableName + " where col2 = 2");
@@ -589,7 +589,7 @@ public class TestTablePartitions extends QueryTestCaseBase {
 
     desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName);
 
-    ContentSummary summary = fs.getContentSummary(new Path(desc.getPath()));
+    ContentSummary summary = fs.getContentSummary(new Path(desc.getUri()));
 
     assertEquals(summary.getDirectoryCount(), 1L);
     assertEquals(summary.getFileCount(), 0L);
@@ -627,10 +627,10 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
 
     FileSystem fs = FileSystem.get(conf);
-    assertTrue(fs.exists(new Path(desc.getPath())));
+    assertTrue(fs.exists(new Path(desc.getUri())));
     CompressionCodecFactory factory = new CompressionCodecFactory(conf);
 
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=2")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=3")));
@@ -676,10 +676,10 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
 
     FileSystem fs = FileSystem.get(conf);
-    assertTrue(fs.exists(new Path(desc.getPath())));
+    assertTrue(fs.exists(new Path(desc.getUri())));
     CompressionCodecFactory factory = new CompressionCodecFactory(conf);
 
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=2")));
@@ -733,10 +733,10 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
 
     FileSystem fs = FileSystem.get(conf);
-    assertTrue(fs.exists(new Path(desc.getPath())));
+    assertTrue(fs.exists(new Path(desc.getUri())));
     CompressionCodecFactory factory = new CompressionCodecFactory(conf);
 
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1/col3=17.0")));
@@ -828,10 +828,10 @@ public class TestTablePartitions extends QueryTestCaseBase {
     }
 
     FileSystem fs = FileSystem.get(conf);
-    assertTrue(fs.exists(new Path(desc.getPath())));
+    assertTrue(fs.exists(new Path(desc.getUri())));
     CompressionCodecFactory factory = new CompressionCodecFactory(conf);
 
-    Path path = new Path(desc.getPath());
+    Path path = new Path(desc.getUri());
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1")));
     assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1/col3=17.0")));

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
----------------------------------------------------------------------
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 f6fd88f..fc25c27 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
@@ -66,7 +66,7 @@ public class TestResultSet {
   public static void setup() throws Exception {
     util = TpchTestBase.getInstance().getTestingCluster();
     conf = util.getConfiguration();
-    sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    sm = TableSpaceManager.getDefault();
 
     scoreSchema = new Schema();
     scoreSchema.addColumn("deptname", Type.TEXT);
@@ -74,7 +74,7 @@ public class TestResultSet {
     scoreMeta = CatalogUtil.newTableMeta("CSV");
     TableStats stats = new TableStats();
 
-    Path p = sm.getTablePath("score");
+    Path p = new Path(sm.getTableUri("default", "score"));
     sm.getFileSystem().mkdirs(p);
     Appender appender = sm.getAppender(scoreMeta, scoreSchema, new Path(p, "score"));
     RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(scoreSchema);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
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 0f90722..48966bc 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
@@ -27,14 +27,13 @@ import org.apache.tajo.catalog.statistics.TableStats;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.global.ExecutionBlock;
-import org.apache.tajo.plan.LogicalOptimizer;
-import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.LogicalPlanner;
 import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
 import org.apache.tajo.engine.planner.global.GlobalPlanner;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.storage.Tablespace;
+import org.apache.tajo.plan.LogicalOptimizer;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.LogicalPlanner;
 import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.AfterClass;
@@ -80,10 +79,9 @@ public class TestExecutionBlockCursor {
     }
 
     analyzer = new SQLAnalyzer();
-    logicalPlanner = new LogicalPlanner(catalog);
+    logicalPlanner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
 
-    Tablespace sm  = TableSpaceManager.getFileStorageManager(conf);
     dispatcher = new AsyncDispatcher();
     dispatcher.init(conf);
     dispatcher.start();

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
index 0cec3da..edddc5a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
@@ -45,6 +45,7 @@ import org.apache.tajo.plan.serder.PlanProto;
 import org.apache.tajo.service.ServiceTracker;
 import org.apache.tajo.session.Session;
 import org.apache.tajo.storage.HashShuffleAppenderManager;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.history.HistoryReader;
 import org.apache.tajo.util.history.HistoryWriter;
@@ -104,7 +105,7 @@ public class TestKillQuery {
     Session session = LocalTajoTestingUtility.createDummySession();
     CatalogService catalog = cluster.getMaster().getCatalog();
 
-    LogicalPlanner planner = new LogicalPlanner(catalog);
+    LogicalPlanner planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(queryStr);
     LogicalPlan plan = planner.createPlan(defaultContext, expr);
@@ -168,7 +169,7 @@ public class TestKillQuery {
     Session session = LocalTajoTestingUtility.createDummySession();
     CatalogService catalog = cluster.getMaster().getCatalog();
 
-    LogicalPlanner planner = new LogicalPlanner(catalog);
+    LogicalPlanner planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     LogicalOptimizer optimizer = new LogicalOptimizer(conf);
     Expr expr =  analyzer.parse(queryStr);
     LogicalPlan plan = planner.createPlan(defaultContext, expr);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java
----------------------------------------------------------------------
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 b19a2e5..863c7b5 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
@@ -67,8 +67,7 @@ public class TestRowFile {
 
     TableMeta meta = CatalogUtil.newTableMeta("ROWFILE");
 
-    FileTablespace sm =
-        (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = (FileTablespace) TableSpaceManager.get(cluster.getDefaultFileSystem().getUri()).get();
 
     Path tablePath = new Path("/test");
     Path metaPath = new Path(tablePath, ".meta");
@@ -109,7 +108,7 @@ public class TestRowFile {
 
     int tupleCnt = 0;
     start = System.currentTimeMillis();
-    Scanner scanner = TableSpaceManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    Scanner scanner = sm.getScanner(meta, schema, fragment);
     scanner.init();
     while ((tuple=scanner.next()) != null) {
       tupleCnt++;

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/resources/results/TestHBaseTable/testCATS.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testCATS.result b/tajo-core/src/test/resources/results/TestHBaseTable/testCATS.result
deleted file mode 100644
index 72013f2..0000000
--- a/tajo-core/src/test/resources/results/TestHBaseTable/testCATS.result
+++ /dev/null
@@ -1,100 +0,0 @@
-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

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/resources/results/TestHBaseTable/testCTAS.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testCTAS.result b/tajo-core/src/test/resources/results/TestHBaseTable/testCTAS.result
new file mode 100644
index 0000000..72013f2
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testCTAS.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

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result
index 1c22960..e0c97ef 100644
--- a/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result
+++ b/tajo-core/src/test/resources/results/TestHBaseTable/testInsertIntoUsingPut.result
@@ -1,3 +1,3 @@
-1, 1996-04-12, {"": "N"}, 7311
+1, 1996-03-13, {"": "N"}, 7706
 2, 1997-01-28, {"": "N"}, 1191
-3, 1993-11-09, {"": "R"}, 6540
+3, 1994-02-02, {"": "R"}, 1798

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/resources/results/TestTajoCli/testDescTable.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCli/testDescTable.result b/tajo-core/src/test/resources/results/TestTajoCli/testDescTable.result
index ae2af45..d3800ab 100644
--- a/tajo-core/src/test/resources/results/TestTajoCli/testDescTable.result
+++ b/tajo-core/src/test/resources/results/TestTajoCli/testDescTable.result
@@ -1,7 +1,7 @@
 OK
 
 table name: default.TEST_DESC_TABLE
-table path: ${table.path}
+table uri: ${table.path}
 store type: CSV
 number of rows: 0
 volume: 0 B
@@ -15,7 +15,7 @@ col2	INT4
 
 
 table name: default.TEST_DESC_TABLE
-table path: ${table.path}
+table uri: ${table.path}
 store type: CSV
 number of rows: 0
 volume: 0 B
@@ -24,6 +24,4 @@ Options:
 
 schema: 
 col1	INT4
-col2	INT4
-
-
+col2	INT4
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/resources/results/TestTajoCli/testDescTableForNestedSchema.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoCli/testDescTableForNestedSchema.result b/tajo-core/src/test/resources/results/TestTajoCli/testDescTableForNestedSchema.result
index 83f360b..7eff4af 100644
--- a/tajo-core/src/test/resources/results/TestTajoCli/testDescTableForNestedSchema.result
+++ b/tajo-core/src/test/resources/results/TestTajoCli/testDescTableForNestedSchema.result
@@ -1,7 +1,7 @@
 OK
 
 table name: default.TEST_DESC_TABLE_NESTED
-table path: ${table.path}
+table uri: ${table.path}
 store type: CSV
 number of rows: 0
 volume: 0 B
@@ -16,7 +16,7 @@ col3	RECORD (col4 RECORD (col5 TEXT))
 
 
 table name: default.TEST_DESC_TABLE_NESTED
-table path: ${table.path}
+table uri: ${table.path}
 store type: CSV
 number of rows: 0
 volume: 0 B

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-dist/src/main/conf/storage-site.json.template
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/conf/storage-site.json.template b/tajo-dist/src/main/conf/storage-site.json.template
new file mode 100644
index 0000000..2d7b19d
--- /dev/null
+++ b/tajo-dist/src/main/conf/storage-site.json.template
@@ -0,0 +1,35 @@
+/**
+ * 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.
+ */
+
+/* HBase Storage Plugin and Tablespace Example */
+/*
+{
+  "spaces": {
+    "hbase-cluster1": {
+      "uri": "hbase://quorum1:port,quorum2:port/"
+    }
+  },
+
+  "storages": {
+    "hbase": {
+      "handler": "org.apache.tajo.storage.hbase.HBaseTablespace",
+      "default-format": "hbase"
+    }
+  }
+}
+*/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-docs/src/main/sphinx/index.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/index.rst b/tajo-docs/src/main/sphinx/index.rst
index 730bed4..ec65fd3 100644
--- a/tajo-docs/src/main/sphinx/index.rst
+++ b/tajo-docs/src/main/sphinx/index.rst
@@ -37,6 +37,7 @@ Table of Contents:
    functions
    table_management
    table_partitioning
+   storage_plugin
    index_overview
    backup_and_restore
    hive_integration

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-docs/src/main/sphinx/storage_plugin.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/storage_plugin.rst b/tajo-docs/src/main/sphinx/storage_plugin.rst
new file mode 100644
index 0000000..d9c6838
--- /dev/null
+++ b/tajo-docs/src/main/sphinx/storage_plugin.rst
@@ -0,0 +1,47 @@
+*************************************
+Storage Plugin
+*************************************
+
+Overview
+========
+
+Tajo supports various storage systems, such as HDFS, Amazon S3, Openstack Swift, and HBase. Also, we have a plan to support RDBMS storages like Oracle, MySQL, PostgreSQL. Tajo already embeds HDFS, S3, Openstack, and HBase, and also Tajo allows users to register custom storages and data formats to Tajo cluster instances. This section describes how you register custom storages and data types.
+
+Register custom storage
+=======================
+
+First of all, your storage implementation should be packed as a jar file. Then, please copy the jar file into ``tajo/extlib`` directory. Next, you should copy ``conf/storage-site.json.template`` into ``conf/storage-site.json`` and modify the file like the below.
+
+Configuration
+=============
+
+Tajo has a default configuration for builtin storages, such as HDFS, local file system, and Amazon S3. it also allows users to add custom storage plugins
+
+``conf/storage-site.json`` file has the following struct:
+
+.. code-block:: json
+
+  {
+    "storages": {
+      "${scheme}": {
+        "handler": "${class name}"
+      }
+    }
+  }
+
+Each storage instance (i.e., :doc:`/table_management/tablespaces`) is identified by an URI. The scheme of URI plays a role to identify storage type. For example, ``hdfs://`` is used for Hdfs storage, ``jdbc://`` is used for JDBC-based storage, and ``hbase://`` is used for HBase storage. 
+
+You should substitute a scheme name without ``://`` for ``${scheme}``.
+
+See an example for HBase storage.
+
+.. code-block:: json
+
+  {
+    "storages": {
+      "hbase": {
+        "handler": "org.apache.tajo.storage.hbase.HBaseTablespace",
+        "default-format": "hbase"
+      }
+    }
+  }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-docs/src/main/sphinx/table_management.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/table_management.rst b/tajo-docs/src/main/sphinx/table_management.rst
index 2b21ddc..5a4693e 100644
--- a/tajo-docs/src/main/sphinx/table_management.rst
+++ b/tajo-docs/src/main/sphinx/table_management.rst
@@ -8,5 +8,6 @@ In Tajo, a table is a logical view of one data sources. Logically, one table con
     :maxdepth: 1
 
     table_management/table_overview
+    table_management/tablespaces
     table_management/file_formats
     table_management/compression
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-docs/src/main/sphinx/table_management/table_overview.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/table_management/table_overview.rst b/tajo-docs/src/main/sphinx/table_management/table_overview.rst
index b63fb69..5818106 100644
--- a/tajo-docs/src/main/sphinx/table_management/table_overview.rst
+++ b/tajo-docs/src/main/sphinx/table_management/table_overview.rst
@@ -5,6 +5,13 @@ Overview of Tajo Tables
 Overview
 ========
 
+Tablespaces
+===========
+
+Tablespaces is a physical location where files or data objects representing data rows can be stored. Once defined, a tablespace can be referred to by a name when creating a database or a table. Especially, it is very useful when a Tajo cluster instance should use heterogeneous storage systems such as HDFS, MySQL, and Oracle because each tablespace can be specified to use a different storage system. 
+
+Please refer to :doc:`/table_management/tablespaces` if you want to know more information about tablespaces.
+
 Managed Table
 ================
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-docs/src/main/sphinx/table_management/tablespaces.rst
----------------------------------------------------------------------
diff --git a/tajo-docs/src/main/sphinx/table_management/tablespaces.rst b/tajo-docs/src/main/sphinx/table_management/tablespaces.rst
new file mode 100644
index 0000000..964491c
--- /dev/null
+++ b/tajo-docs/src/main/sphinx/table_management/tablespaces.rst
@@ -0,0 +1,45 @@
+*************************************
+Tablespaces
+*************************************
+
+Tablespaces in Tajo allow users to define locations in the storage system where the files or data objects representing database objects can be stored. Once defined, a tablespace can be referred to by name when creating a database or a table. Especially, it is very useful when a Tajo cluster instance should use heterogeneous storage systems such as HDFS, MySQL, and Oracle.
+
+Configuration
+=============
+
+By default, Tajo use in ``${tajo.rootdir}/warehouse`` in ``conf/tajo-site.xml`` as a default tablespace. It also allows users to register additional tablespaces. 
+
+``conf/storage-site.json`` file.
+
+The configuration file has the following struct:
+
+.. code-block:: json
+
+  {
+    "spaces": {
+      "${table_space_name}": {
+        "uri": "hbase://quorum1:port,quorum2:port/"
+      }
+    }
+  }
+
+The following is an example for two tablespaces for hbase and hdfs:
+
+.. code-block:: json
+
+  {
+    "spaces": {
+      "hbase-cluster1": {
+        "uri": "hbase://quorum1:port,quorum2:port/"
+      },
+
+      "ssd": {
+        "uri": "hdfs://host:port/data/ssd"
+      }
+    }
+  }
+
+
+.. note::
+
+  Also, each tablespace can use different storage type. Please see :doc:`/storage_plugin` if you want to know more information about it.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
index 5571cdf..a2480c9 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java
@@ -30,13 +30,13 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.tajo.OverridableConf;
+import org.apache.tajo.QueryVars;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.algebra.WindowSpec;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.plan.LogicalPlan.QueryBlock;
@@ -50,11 +50,13 @@ import org.apache.tajo.plan.util.ExprFinder;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.catalog.SchemaUtil;
 import org.apache.tajo.plan.verifier.VerifyException;
+import org.apache.tajo.storage.StorageService;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.Pair;
 import org.apache.tajo.util.StringUtils;
 import org.apache.tajo.util.TUtil;
 
+import java.net.URI;
 import java.util.*;
 
 import static org.apache.tajo.algebra.CreateTable.PartitionType;
@@ -67,13 +69,17 @@ import static org.apache.tajo.plan.LogicalPlan.BlockType;
 public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContext, LogicalNode> {
   private static Log LOG = LogFactory.getLog(LogicalPlanner.class);
   private final CatalogService catalog;
+  private final StorageService storage;
+
   private final LogicalPlanPreprocessor preprocessor;
   private final EvalTreeOptimizer evalOptimizer;
   private final ExprAnnotator exprAnnotator;
   private final ExprNormalizer normalizer;
 
-  public LogicalPlanner(CatalogService catalog) {
+  public LogicalPlanner(CatalogService catalog, StorageService storage) {
     this.catalog = catalog;
+    this.storage = storage;
+
     this.exprAnnotator = new ExprAnnotator(catalog);
     this.preprocessor = new LogicalPlanPreprocessor(catalog, exprAnnotator);
     this.normalizer = new ExprNormalizer();
@@ -1345,9 +1351,10 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
   }
 
   private void updatePhysicalInfo(TableDesc desc) {
-    if (desc.getPath() != null && desc.getMeta().getStoreType() != "SYSTEM") {
+    if (desc.getUri() != null &&
+        desc.getMeta().getStoreType() != "SYSTEM" && PlannerUtil.isFileStorageType(desc.getMeta().getStoreType())) {
       try {
-        Path path = new Path(desc.getPath());
+        Path path = new Path(desc.getUri());
         FileSystem fs = path.getFileSystem(new Configuration());
         FileStatus status = fs.getFileStatus(path);
         if (desc.getStats() != null && (status.isDirectory() || status.isFile())) {
@@ -1689,7 +1696,13 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
     insertNode.setInSchema(childSchema);
     insertNode.setOutSchema(childSchema);
     insertNode.setTableSchema(childSchema);
-    insertNode.setTargetLocation(new Path(expr.getLocation()));
+
+    // Rewrite
+    URI targetUri = URI.create(expr.getLocation());
+    if (targetUri.getScheme() == null) {
+      targetUri = URI.create(context.getQueryContext().get(QueryVars.DEFAULT_SPACE_ROOT_URI) + "/" + targetUri);
+    }
+    insertNode.setUri(targetUri);
 
     if (expr.hasStorageType()) {
       insertNode.setStorageType(expr.getStorageType());
@@ -1742,7 +1755,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
 
     createTableNode.setExternal(parentTableDesc.isExternal());
     if(parentTableDesc.isExternal()) {
-      createTableNode.setPath(new Path(parentTableDesc.getPath()));
+      createTableNode.setUri(parentTableDesc.getUri());
     }
     return createTableNode;
   }
@@ -1762,8 +1775,15 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
           CatalogUtil.buildFQName(context.queryContext.get(SessionVars.CURRENT_DATABASE), expr.getTableName()));
     }
     // This is CREATE TABLE <tablename> LIKE <parentTable>
-    if(expr.getLikeParentTableName() != null)
+    if(expr.getLikeParentTableName() != null) {
       return handleCreateTableLike(context, expr, createTableNode);
+    }
+
+    if (expr.hasTableSpaceName()) {
+      createTableNode.setTableSpaceName(expr.getTableSpaceName());
+    }
+
+    createTableNode.setUri(getCreatedTableURI(context, expr));
 
     if (expr.hasStorageType()) { // If storage type (using clause) is specified
       createTableNode.setStorageType(expr.getStorageType());
@@ -1771,8 +1791,6 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
       createTableNode.setStorageType("CSV");
     }
 
-
-
     // Set default storage properties to table
     KeyValueSet properties = CatalogUtil.newPhysicalProperties(createTableNode.getStorageType());
 
@@ -1789,8 +1807,6 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
 
     createTableNode.setOptions(properties);
 
-
-
     if (expr.hasPartition()) {
       if (expr.getPartitionMethod().getPartitionType().equals(PartitionType.COLUMN)) {
         createTableNode.setPartitionMethod(getPartitionMethod(context, expr.getTableName(), expr.getPartitionMethod()));
@@ -1850,14 +1866,32 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
         createTableNode.setExternal(true);
       }
 
-      if (expr.hasLocation()) {
-        createTableNode.setPath(new Path(expr.getLocation()));
-      }
-
       return createTableNode;
     }
   }
 
+  /**
+   * Return a table uri to be created
+   *
+   * @param context PlanContext
+   * @param createTable An algebral expression for create table
+   * @return a Table uri to be created on a given table space
+   */
+  private URI getCreatedTableURI(PlanContext context, CreateTable createTable) {
+
+    if (createTable.hasLocation()) {
+      return URI.create(createTable.getLocation());
+    } else {
+
+      String tableName = createTable.getTableName();
+      String databaseName = CatalogUtil.isFQTableName(tableName) ?
+          CatalogUtil.extractQualifier(tableName) : context.queryContext.get(SessionVars.CURRENT_DATABASE);
+
+      return storage.getTableURI(
+          createTable.getTableSpaceName(), databaseName, CatalogUtil.extractSimpleName(tableName));
+    }
+  }
+
   private PartitionMethodDesc getPartitionMethod(PlanContext context,
                                                  String tableName,
                                                  CreateTable.PartitionMethodDescExpr expr) throws PlanningException {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java
index 0ab62d5..46ea458 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/CreateTableNode.java
@@ -20,15 +20,14 @@ package org.apache.tajo.plan.logical;
 
 import com.google.common.base.Objects;
 import com.google.gson.annotations.Expose;
-import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.plan.PlanString;
-import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
 
+import java.net.URI;
+
 public class CreateTableNode extends StoreTableNode implements Cloneable {
-  @Expose private Schema schema;
-  @Expose private Path path;
+  @Expose private String tableSpaceName;
   @Expose private boolean external;
   @Expose private boolean ifNotExists;
 
@@ -41,34 +40,26 @@ public class CreateTableNode extends StoreTableNode implements Cloneable {
     return child == null ? 0 : 1;
   }
 
-  public void setTableSchema(Schema schema) {
-    this.schema = schema;
-  }
-    
-  public Schema getTableSchema() {
-    return this.schema;
-  }
-
   public Schema getLogicalSchema() {
     if (hasPartition()) {
-      Schema logicalSchema = new Schema(schema);
+      Schema logicalSchema = new Schema(tableSchema);
       logicalSchema.addColumns(getPartitionMethod().getExpressionSchema());
       return logicalSchema;
     } else {
-      return schema;
+      return tableSchema;
     }
   }
 
-  public boolean hasPath() {
-    return this.path != null;
+  public boolean hasTableSpaceName() {
+    return tableSpaceName != null;
   }
 
-  public void setPath(Path path) {
-    this.path = path;
+  public String getTableSpaceName() {
+    return tableSpaceName;
   }
-  
-  public Path getPath() {
-    return this.path;
+
+  public void setTableSpaceName(String tableSpaceName) {
+    this.tableSpaceName = tableSpaceName;
   }
 
   public boolean isExternal() {
@@ -97,7 +88,7 @@ public class CreateTableNode extends StoreTableNode implements Cloneable {
   }
 
   public int hashCode() {
-    return super.hashCode() ^ Objects.hashCode(schema, path, external, ifNotExists) * 31;
+    return super.hashCode() ^ Objects.hashCode(tableSchema, uri, external, ifNotExists) * 31;
   }
   
   @Override
@@ -105,9 +96,8 @@ public class CreateTableNode extends StoreTableNode implements Cloneable {
     if (obj instanceof CreateTableNode) {
       CreateTableNode other = (CreateTableNode) obj;
       boolean eq = super.equals(other);
-      eq &= this.schema.equals(other.schema);
+      eq &= TUtil.checkEquals(tableSpaceName, other.tableSpaceName);
       eq &= this.external == other.external;
-      eq &= TUtil.checkEquals(path, other.path);
       eq &= ifNotExists == other.ifNotExists;;
       return eq;
     } else {
@@ -118,12 +108,8 @@ public class CreateTableNode extends StoreTableNode implements Cloneable {
   @Override
   public Object clone() throws CloneNotSupportedException {
     CreateTableNode createTableNode = (CreateTableNode) super.clone();
-    createTableNode.tableName = tableName;
-    createTableNode.schema = (Schema) schema.clone();
-    createTableNode.storageType = storageType;
+    createTableNode.tableSpaceName = tableSpaceName;
     createTableNode.external = external;
-    createTableNode.path = path != null ? new Path(path.toString()) : null;
-    createTableNode.options = (KeyValueSet) (options != null ? options.clone() : null);
     createTableNode.ifNotExists = ifNotExists;
     return createTableNode;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/logical/InsertNode.java
----------------------------------------------------------------------
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 ee15951..cb3dcee 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
@@ -20,21 +20,18 @@ package org.apache.tajo.plan.logical;
 
 import com.google.gson.annotations.Expose;
 
-import org.apache.hadoop.fs.Path;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.plan.PlanString;
 import org.apache.tajo.util.TUtil;
 
 public class InsertNode extends StoreTableNode implements Cloneable {
+  /** Overwrite or just insert */
   @Expose private boolean overwrite;
-  @Expose private Schema tableSchema;
-
   /** a target schema of a target table */
   @Expose private Schema targetSchema;
   /** a output schema of select clause */
   @Expose private Schema projectedSchema;
-  @Expose private Path path;
 
   public InsertNode(int pid) {
     super(pid, NodeType.INSERT);
@@ -47,8 +44,8 @@ public class InsertNode extends StoreTableNode implements Cloneable {
     } else {
       tableSchema = desc.getSchema();
     }
-    if (desc.getPath() != null) {
-      setPath(new Path(desc.getPath()));
+    if (desc.getUri() != null) {
+      setUri(desc.getUri());
     }
     setOptions(desc.getMeta().getOptions());
     setStorageType(desc.getMeta().getStoreType());
@@ -58,10 +55,6 @@ public class InsertNode extends StoreTableNode implements Cloneable {
     }
   }
 
-  public void setTargetLocation(Path path) {
-    this.path = path;
-  }
-
   public void setSubQuery(LogicalNode subQuery) {
     this.setChild(subQuery);
     this.setInSchema(subQuery.getOutSchema());
@@ -76,14 +69,6 @@ public class InsertNode extends StoreTableNode implements Cloneable {
     this.overwrite = overwrite;
   }
 
-  public Schema getTableSchema() {
-    return tableSchema;
-  }
-
-  public void setTableSchema(Schema tableSchema) {
-    this.tableSchema = tableSchema;
-  }
-
   public boolean hasTargetSchema() {
     return this.targetSchema != null;
   }
@@ -108,28 +93,12 @@ public class InsertNode extends StoreTableNode implements Cloneable {
     this.projectedSchema = projected;
   }
 
-  public boolean hasPath() {
-    return this.path != null;
-  }
-
-  public void setPath(Path path) {
-    this.path = path;
-  }
-  
-  public Path getPath() {
-    return this.path;
-  }
-
-  public boolean hasStorageType() {
-    return this.storageType != null;
-  }
-  
   @Override
   public int hashCode() {
     final int prime = 31;
     int result = 1;
     result = prime * result + (overwrite ? 1231 : 1237);
-    result = prime * result + ((path == null) ? 0 : path.hashCode());
+    result = prime * result + ((uri == null) ? 0 : uri.hashCode());
     result = prime * result + ((projectedSchema == null) ? 0 : projectedSchema.hashCode());
     result = prime * result + ((tableSchema == null) ? 0 : tableSchema.hashCode());
     result = prime * result + ((targetSchema == null) ? 0 : targetSchema.hashCode());
@@ -142,9 +111,8 @@ public class InsertNode extends StoreTableNode implements Cloneable {
       InsertNode other = (InsertNode) obj;
       boolean eq = super.equals(other);
       eq &= this.overwrite == other.overwrite;
-      eq &= TUtil.checkEquals(this.tableSchema, other.tableSchema);
       eq &= TUtil.checkEquals(this.targetSchema, other.targetSchema);
-      eq &= TUtil.checkEquals(path, other.path);
+      eq &= TUtil.checkEquals(this.projectedSchema, other.projectedSchema);
       return eq;
     } else {
       return false;
@@ -157,7 +125,8 @@ public class InsertNode extends StoreTableNode implements Cloneable {
     insertNode.overwrite = overwrite;
     insertNode.tableSchema = new Schema(tableSchema);
     insertNode.targetSchema = targetSchema != null ? new Schema(targetSchema) : null;
-    insertNode.path = path != null ? new Path(path.toString()) : null;
+    insertNode.projectedSchema = projectedSchema != null ? new Schema(projectedSchema) : null;
+    insertNode.uri = uri != null ? uri : null;
     return insertNode;
   }
   
@@ -166,8 +135,8 @@ public class InsertNode extends StoreTableNode implements Cloneable {
     if (hasTargetTable()) {
       sb.append(",table=").append(tableName);
     }
-    if (hasPath()) {
-      sb.append(", location=").append(path);
+    if (hasUri()) {
+      sb.append(", location=").append(uri);
     }
     sb.append(")");
     return sb.toString();
@@ -195,7 +164,7 @@ public class InsertNode extends StoreTableNode implements Cloneable {
         planString.addExplan(getTargetSchema().toString());
       }
     } else {
-      planString.addExplan("LOCATION " + path);
+      planString.addExplan("LOCATION " + uri);
     }
     return planString;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/logical/PartitionedTableScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/PartitionedTableScanNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/PartitionedTableScanNode.java
index 6fd969a..a4bb94c 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/PartitionedTableScanNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/PartitionedTableScanNode.java
@@ -63,7 +63,7 @@ public class PartitionedTableScanNode extends ScanNode {
     if (hasQual()) {
       sb.append(", filter=").append(qual);
     }
-    sb.append(", path=").append(getTableDesc().getPath()).append(")");
+    sb.append(", uri=").append(getTableDesc().getUri()).append(")");
 	  return sb.toString();
 	}
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/logical/ScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/ScanNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/ScanNode.java
index a22f592..0ba988f 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/ScanNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/ScanNode.java
@@ -162,7 +162,7 @@ public class ScanNode extends RelationNode implements Projectable, SelectableNod
     if (hasQual()) {
       sb.append(", filter=").append(qual);
     }
-    sb.append(", path=").append(getTableDesc().getPath()).append(")");
+    sb.append(", path=").append(getTableDesc().getUri()).append(")");
     return sb.toString();
 	}
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/logical/StoreTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/StoreTableNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/StoreTableNode.java
index 3a40f83..170e13c 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/logical/StoreTableNode.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/logical/StoreTableNode.java
@@ -20,12 +20,17 @@ package org.apache.tajo.plan.logical;
 
 import com.google.gson.annotations.Expose;
 
+import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.plan.PlanString;
 import org.apache.tajo.util.TUtil;
 
+import java.net.URI;
+
 public class StoreTableNode extends PersistentStoreNode implements Cloneable {
   @Expose protected String tableName;
+  @Expose protected URI uri;
+  @Expose protected Schema tableSchema;
   @Expose private PartitionMethodDesc partitionDesc;
 
   public StoreTableNode(int pid) {
@@ -57,6 +62,26 @@ public class StoreTableNode extends PersistentStoreNode implements Cloneable {
     return this.tableName;
   }
 
+  public boolean hasUri() {
+    return this.uri != null;
+  }
+
+  public void setUri(URI uri) {
+    this.uri = uri;
+  }
+
+  public URI getUri() {
+    return this.uri;
+  }
+
+  public void setTableSchema(Schema schema) {
+    this.tableSchema = schema;
+  }
+
+  public Schema getTableSchema() {
+    return this.tableSchema;
+  }
+
   public boolean hasPartition() {
     return this.partitionDesc != null;
   }
@@ -93,6 +118,8 @@ public class StoreTableNode extends PersistentStoreNode implements Cloneable {
       StoreTableNode other = (StoreTableNode) obj;
       boolean eq = super.equals(other);
       eq = eq && TUtil.checkEquals(this.tableName, other.tableName);
+      eq = eq && TUtil.checkEquals(uri, other.uri);
+      eq = tableSchema.equals(other.tableSchema);
       eq = eq && TUtil.checkEquals(partitionDesc, other.partitionDesc);
       return eq;
     } else {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java
----------------------------------------------------------------------
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 4b9fd48..3b1f1a8 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
@@ -259,9 +259,9 @@ public class PartitionedTableRewriter implements LogicalPlanRewriteRule {
 
     if (indexablePredicateSet.size() > 0) { // There are at least one indexable predicates
       return findFilteredPaths(queryContext, paritionValuesSchema,
-          indexablePredicateSet.toArray(new EvalNode[indexablePredicateSet.size()]), new Path(table.getPath()));
+          indexablePredicateSet.toArray(new EvalNode[indexablePredicateSet.size()]), new Path(table.getUri()));
     } else { // otherwise, we will get all partition paths.
-      return findFilteredPaths(queryContext, paritionValuesSchema, null, new Path(table.getPath()));
+      return findFilteredPaths(queryContext, paritionValuesSchema, null, new Path(table.getUri()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
index 694e81c..c1d9f9a 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java
@@ -37,6 +37,7 @@ import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.TUtil;
 
+import java.net.URI;
 import java.util.*;
 
 /**
@@ -477,10 +478,14 @@ public class LogicalNodeDeserializer {
       createTable.setPartitionMethod(new PartitionMethodDesc(storeTableNodeSpec.getPartitionMethod()));
     }
 
-    createTable.setTableSchema(convertSchema(createTableNodeSpec.getSchema()));
+    createTable.setTableSchema(convertSchema(storeTableNodeSpec.getTableSchema()));
+
+    if (createTableNodeSpec.hasTablespaceName()) {
+     createTable.setTableSpaceName(createTableNodeSpec.getTablespaceName());
+    }
     createTable.setExternal(createTableNodeSpec.getExternal());
-    if (createTableNodeSpec.getExternal() && createTableNodeSpec.hasPath()) {
-      createTable.setPath(new Path(createTableNodeSpec.getPath()));
+    if (createTableNodeSpec.getExternal() && storeTableNodeSpec.hasUri()) {
+      createTable.setUri(URI.create(storeTableNodeSpec.getUri()));
     }
     createTable.setIfNotExists(createTableNodeSpec.getIfNotExists());
 
@@ -512,16 +517,14 @@ public class LogicalNodeDeserializer {
     }
 
     insertNode.setOverwrite(insertNodeSpec.getOverwrite());
-    insertNode.setTableSchema(convertSchema(insertNodeSpec.getTableSchema()));
+    insertNode.setTableSchema(convertSchema(storeTableNodeSpec.getTableSchema()));
     if (insertNodeSpec.hasTargetSchema()) {
       insertNode.setTargetSchema(convertSchema(insertNodeSpec.getTargetSchema()));
     }
     if (insertNodeSpec.hasProjectedSchema()) {
       insertNode.setProjectedSchema(convertSchema(insertNodeSpec.getProjectedSchema()));
     }
-    if (insertNodeSpec.hasPath()) {
-      insertNode.setPath(new Path(insertNodeSpec.getPath()));
-    }
+    insertNode.setUri(URI.create(storeTableNodeSpec.getUri()));
 
     return insertNode;
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
index 88d831e..6737756 100644
--- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
+++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeSerializer.java
@@ -491,11 +491,10 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe
     PlanProto.StoreTableNodeSpec.Builder storeTableBuilder = buildStoreTableNodeSpec(node);
 
     PlanProto.CreateTableNodeSpec.Builder createTableBuilder = PlanProto.CreateTableNodeSpec.newBuilder();
-    createTableBuilder.setSchema(node.getTableSchema().getProto());
-    createTableBuilder.setExternal(node.isExternal());
-    if (node.isExternal() && node.hasPath()) {
-      createTableBuilder.setPath(node.getPath().toString());
+    if (node.hasTableSpaceName()) {
+      createTableBuilder.setTablespaceName(node.getTableSpaceName());
     }
+    createTableBuilder.setExternal(node.isExternal());
     createTableBuilder.setIfNotExists(node.isIfNotExists());
 
     PlanProto.LogicalNode.Builder nodeBuilder = createNodeBuilder(context, node);
@@ -605,16 +604,13 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe
 
     PlanProto.InsertNodeSpec.Builder insertNodeSpec = PlanProto.InsertNodeSpec.newBuilder();
     insertNodeSpec.setOverwrite(node.isOverwrite());
-    insertNodeSpec.setTableSchema(node.getTableSchema().getProto());
+
     if (node.hasProjectedSchema()) {
       insertNodeSpec.setProjectedSchema(node.getProjectedSchema().getProto());
     }
     if (node.hasTargetSchema()) {
       insertNodeSpec.setTargetSchema(node.getTargetSchema().getProto());
     }
-    if (node.hasPath()) {
-      insertNodeSpec.setPath(node.getPath().toString());
-    }
 
     PlanProto.LogicalNode.Builder nodeBuilder = createNodeBuilder(context, node);
     nodeBuilder.setPersistentStore(persistentStoreBuilder);
@@ -641,12 +637,19 @@ public class LogicalNodeSerializer extends BasicLogicalPlanVisitor<LogicalNodeSe
 
   private static PlanProto.StoreTableNodeSpec.Builder buildStoreTableNodeSpec(StoreTableNode node) {
     PlanProto.StoreTableNodeSpec.Builder storeTableBuilder = PlanProto.StoreTableNodeSpec.newBuilder();
-    if (node.hasPartition()) {
-      storeTableBuilder.setPartitionMethod(node.getPartitionMethod().getProto());
-    }
+
     if (node.hasTableName()) { // It will be false if node is for INSERT INTO LOCATION '...'
       storeTableBuilder.setTableName(node.getTableName());
     }
+
+    if (node.hasUri()) {
+      storeTableBuilder.setUri(node.getUri().toString());
+    }
+    storeTableBuilder.setTableSchema(node.getTableSchema().getProto());
+
+    if (node.hasPartition()) {
+      storeTableBuilder.setPartitionMethod(node.getPartitionMethod().getProto());
+    }
     return storeTableBuilder;
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/java/org/apache/tajo/plan/util/PlannerUtil.java
----------------------------------------------------------------------
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 f66614f..16ca368 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
@@ -26,7 +26,6 @@ import org.apache.tajo.SessionVars;
 import org.apache.tajo.algebra.*;
 import org.apache.tajo.annotation.Nullable;
 import org.apache.tajo.catalog.*;
-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,9 +40,6 @@ import org.apache.tajo.util.TUtil;
 import java.io.IOException;
 import java.util.*;
 
-import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.CSV;
-import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType.TEXTFILE;
-
 public class PlannerUtil {
 
   public static final Column [] EMPTY_COLUMNS = new Column[] {};
@@ -915,6 +911,10 @@ public class PlannerUtil {
   }
 
   public static TableDesc getTableDesc(CatalogService catalog, LogicalNode node) throws IOException {
+    if (node.getType() == NodeType.ROOT) {
+      node = ((LogicalRootNode)node).getChild();
+    }
+
     if (node.getType() == NodeType.CREATE_TABLE) {
       return createTableDesc((CreateTableNode)node);
     }
@@ -935,7 +935,7 @@ public class PlannerUtil {
         }
       }
     } else {
-      if (insertNode.getPath() != null) {
+      if (insertNode.getUri() != null) {
         //insert ... location
         return createTableDesc(insertNode);
       }
@@ -951,7 +951,7 @@ public class PlannerUtil {
             createTableNode.getTableName(),
             createTableNode.getTableSchema(),
             meta,
-            createTableNode.getPath() != null ? createTableNode.getPath().toUri() : null);
+            createTableNode.getUri() != null ? createTableNode.getUri() : null);
 
     tableDescTobeCreated.setExternal(createTableNode.isExternal());
 
@@ -970,7 +970,7 @@ public class PlannerUtil {
             insertNode.getTableName(),
             insertNode.getTableSchema(),
             meta,
-            insertNode.getPath() != null ? insertNode.getPath().toUri() : null);
+            insertNode.getUri() != null ? insertNode.getUri() : null);
 
     if (insertNode.hasPartition()) {
       tableDescTobeCreated.setPartitionMethod(insertNode.getPartitionMethod());

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-plan/src/main/proto/Plan.proto
----------------------------------------------------------------------
diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto
index 77a21b7..40b7891 100644
--- a/tajo-plan/src/main/proto/Plan.proto
+++ b/tajo-plan/src/main/proto/Plan.proto
@@ -223,22 +223,21 @@ message PersistentStoreNode {
 
 message StoreTableNodeSpec { // required PersistentStoreNode
   optional string tableName = 1; // 'INSERT INTO LOCATION' does not require 'table name'.
-  optional PartitionMethodProto partitionMethod = 2;
+  optional string uri = 2;
+  required SchemaProto table_schema = 3;
+  optional PartitionMethodProto partitionMethod = 4;
 }
 
 message InsertNodeSpec { // required PersistentStoreNode and StoreTableSpec
   required bool overwrite = 1;
-  required SchemaProto tableSchema = 2;
-  optional SchemaProto targetSchema = 4;
+  optional SchemaProto targetSchema = 2;
   optional SchemaProto projectedSchema = 3;
-  optional string path = 5;
 }
 
 message CreateTableNodeSpec { // required PersistentStoreNode and StoreTableNodeSpec
-  required SchemaProto schema = 1;
+  optional string tablespace_name = 1;
   required bool external = 2;
   required bool ifNotExists = 3;
-  optional string path = 4;
 }
 
 message DropTableNode {


[07/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
index 5e3d0b6..d490001 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/CreateTableHook.java
@@ -19,13 +19,13 @@
 package org.apache.tajo.master.exec.prehook;
 
 import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.logical.CreateTableNode;
 import org.apache.tajo.plan.logical.LogicalRootNode;
 import org.apache.tajo.plan.logical.NodeType;
-import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.Tablespace;
 
 public class CreateTableHook implements DistributedQueryHook {
 
@@ -43,8 +43,10 @@ public class CreateTableHook implements DistributedQueryHook {
     String databaseName = splitted[0];
     String tableName = splitted[1];
     queryContext.setOutputTable(tableName);
-    queryContext.setOutputPath(
-        StorageUtil.concatPath(TajoConf.getWarehouseDir(queryContext.getConf()), databaseName, tableName));
+
+    // set the final output table uri
+    queryContext.setOutputPath(createTableNode.getUri());
+
     if(createTableNode.getPartitionMethod() != null) {
       queryContext.setPartitionMethod(createTableNode.getPartitionMethod());
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java
index 3dba176..f403092 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/DistributedQueryHookManager.java
@@ -37,7 +37,7 @@ public class DistributedQueryHookManager {
         try {
           hook.hook(queryContext, plan);
         } catch (Throwable t) {
-          t.printStackTrace();
+          throw new RuntimeException(t);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java
index 14c4d8d..c309f57 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/prehook/InsertIntoHook.java
@@ -39,23 +39,21 @@ public class InsertIntoHook implements DistributedQueryHook {
 
     // Set QueryContext settings, such as output table name and output path.
     // It also remove data files if overwrite is true.
-    Path outputPath;
     if (insertNode.hasTargetTable()) { // INSERT INTO [TB_NAME]
       queryContext.setOutputTable(insertNode.getTableName());
-      queryContext.setOutputPath(insertNode.getPath());
       if (insertNode.hasPartition()) {
         queryContext.setPartitionMethod(insertNode.getPartitionMethod());
       }
     } else { // INSERT INTO LOCATION ...
       // When INSERT INTO LOCATION, must not set output table.
-      outputPath = insertNode.getPath();
       queryContext.setFileOutput();
-      queryContext.setOutputPath(outputPath);
     }
 
+    // Set the final output table uri
+    queryContext.setOutputPath(insertNode.getUri());
+
     if (insertNode.isOverwrite()) {
       queryContext.setOutputOverwrite();
     }
-
   }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
index 23808b5..4fef02c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.state.*;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryVars;
 import org.apache.tajo.SessionVars;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
@@ -48,6 +49,7 @@ import org.apache.tajo.master.event.*;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.util.TUtil;
 import org.apache.tajo.util.history.QueryHistory;
 import org.apache.tajo.util.history.StageHistory;
@@ -427,40 +429,59 @@ public class Query implements EventHandler<QueryEvent> {
       } else {
         finalState = QueryState.QUERY_ERROR;
       }
+
+      // When a query is failed
       if (finalState != QueryState.QUERY_SUCCEEDED) {
         Stage lastStage = query.getStage(stageEvent.getExecutionBlockId());
-        if (lastStage != null && lastStage.getTableMeta() != null) {
-          String storeType = lastStage.getTableMeta().getStoreType();
-          if (storeType != null) {
-            LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
-            try {
-              TableSpaceManager.getStorageManager(query.systemConf, storeType).rollbackOutputCommit(rootNode.getChild());
-            } catch (IOException e) {
-              LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
-            }
-          }
-        }
+        handleQueryFailure(query, lastStage);
       }
+
       query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
       query.setFinishTime();
 
       return finalState;
     }
 
+    // handle query failures
+    private void handleQueryFailure(Query query, Stage lastStage) {
+      QueryContext context = query.context.getQueryContext();
+
+      if (lastStage != null && context.hasOutputTableUri()) {
+        Tablespace space = TableSpaceManager.get(context.getOutputTableUri()).get();
+        try {
+          LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
+          space.rollbackTable(rootNode.getChild());
+        } catch (IOException e) {
+          LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
+        }
+      }
+    }
+
     private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
       Stage lastStage = query.getStage(event.getExecutionBlockId());
-      String storeType = lastStage.getTableMeta().getStoreType();
+
       try {
+
         LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
         CatalogService catalog = lastStage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
         TableDesc tableDesc =  PlannerUtil.getTableDesc(catalog, rootNode.getChild());
 
-        Path finalOutputDir = TableSpaceManager.getStorageManager(query.systemConf, storeType)
-            .commitOutputData(query.context.getQueryContext(),
-                lastStage.getId(), lastStage.getMasterPlan().getLogicalPlan(), lastStage.getSchema(), tableDesc);
+        QueryContext queryContext = query.context.getQueryContext();
+
+        // If there is not tabledesc, it is a select query without insert or ctas.
+        // In this case, we should use default tablespace.
+        Tablespace space = TableSpaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
+
+        Path finalOutputDir = space.commitTable(
+            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 (Exception e) {
         query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
         return QueryState.QUERY_ERROR;

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
index 2809a70..84f2eac 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
@@ -18,6 +18,7 @@
 
 package org.apache.tajo.querymaster;
 
+import com.google.common.base.Optional;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -51,14 +52,9 @@ import org.apache.tajo.plan.logical.LogicalNode;
 import org.apache.tajo.plan.logical.LogicalRootNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.ScanNode;
-import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.plan.verifier.VerifyException;
 import org.apache.tajo.session.Session;
-import org.apache.tajo.storage.Tablespace;
-import org.apache.tajo.storage.StorageProperty;
-import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.util.metrics.TajoMetrics;
 import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
 import org.apache.tajo.worker.AbstractResourceAllocator;
@@ -308,43 +304,28 @@ public class QueryMasterTask extends CompositeService {
   }
 
   public synchronized void startQuery() {
-    Tablespace sm = null;
     LogicalPlan plan = null;
+    Tablespace space = null;
     try {
       if (query != null) {
         LOG.warn("Query already started");
         return;
       }
+
+
       CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
-      LogicalPlanner planner = new LogicalPlanner(catalog);
+      LogicalPlanner planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
       LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
       Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
       jsonExpr = null; // remove the possible OOM
-      plan = planner.createPlan(queryContext, expr);
-
-      String storeType = PlannerUtil.getStoreType(plan);
-      if (storeType != null) {
-        sm = TableSpaceManager.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<LogicalPlanRewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
-              getQueryTaskContext().getQueryContext(), tableDesc);
-          if (storageSpecifiedRewriteRules != null) {
-            for (LogicalPlanRewriteRule eachRule: storageSpecifiedRewriteRules) {
-              optimizer.addRuleAfterToJoinOpt(eachRule);
-            }
-          }
-        }
-      }
 
+      plan = planner.createPlan(queryContext, expr);
       optimizer.optimize(queryContext, plan);
 
+      // when a given uri is null, TableSpaceManager.get will return the default tablespace.
+      space = TableSpaceManager.get(queryContext.get(QueryVars.OUTPUT_TABLE_URI, "")).get();
+      space.rewritePlan(queryContext, plan);
+
       for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
         LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);
         if (scanNodes != null) {
@@ -374,10 +355,10 @@ public class QueryMasterTask extends CompositeService {
       LOG.error(t.getMessage(), t);
       initError = t;
 
-      if (plan != null && sm != null) {
+      if (plan != null && space != null) {
         LogicalRootNode rootNode = plan.getRootBlock().getRoot();
         try {
-          sm.rollbackOutputCommit(rootNode.getChild());
+          space.rollbackTable(rootNode.getChild());
         } catch (IOException e) {
           LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
         }
@@ -422,16 +403,27 @@ public class QueryMasterTask extends CompositeService {
     // Create Output Directory
     ////////////////////////////////////////////
 
-    String outputPath = context.get(QueryVars.OUTPUT_TABLE_PATH, "");
-    if (context.isCreateTable() || context.isInsert()) {
-      if (outputPath == null || outputPath.isEmpty()) {
-        // hbase
-        stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+    String outputPath = context.get(QueryVars.OUTPUT_TABLE_URI, "");
+
+    // The fact that there is no output means that this query is neither CTAS or INSERT (OVERWRITE) INTO
+    // So, this query results won't be materialized as a part of a table.
+    // The result will be temporarily written in the staging directory.
+    if (outputPath.isEmpty()) {
+      // for temporarily written in the storage directory
+      stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+    } else {
+      Optional<Tablespace> spaceResult = TableSpaceManager.get(outputPath);
+      if (!spaceResult.isPresent()) {
+        throw new IOException("No registered Tablespace for " + outputPath);
+      }
+
+      Tablespace space = spaceResult.get();
+      if (space.getProperty().isMovable()) { // checking if this tablespace allows MOVE operation
+        // If this space allows move operation, the staging directory will be underneath the final output table uri.
+        stagingDir = StorageUtil.concatPath(context.getOutputTableUri().toString(), TMP_STAGING_DIR_PREFIX, queryId);
       } else {
-        stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
+        stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
       }
-    } else {
-      stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
     }
 
     // initializ

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
index 2c3e9e2..5b8f24a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
@@ -36,7 +36,6 @@ import org.apache.tajo.engine.planner.UniformRangePartition;
 import org.apache.tajo.engine.planner.enforce.Enforcer;
 import org.apache.tajo.engine.planner.global.DataChannel;
 import org.apache.tajo.engine.planner.global.ExecutionBlock;
-import org.apache.tajo.engine.planner.global.GlobalPlanner;
 import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.engine.planner.global.rewriter.rules.GlobalPlanRewriteUtil;
 import org.apache.tajo.engine.utils.TupleUtil;
@@ -83,19 +82,14 @@ public class Repartitioner {
     QueryMasterTask.QueryMasterTaskContext masterContext = stage.getContext();
 
     ScanNode[] scans = execBlock.getScanNodes();
-
-    Path tablePath;
     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
-        FileTablespace storageManager =
-            (FileTablespace) TableSpaceManager.getFileStorageManager(stage.getContext().getConf());
 
-        tablePath = storageManager.getTablePath(scans[i].getTableName());
+      if (tableDesc == null) { // if it is a real table stored on storage
         if (execBlock.getUnionScanMap() != null && !execBlock.getUnionScanMap().isEmpty()) {
           for (Map.Entry<ExecutionBlockId, ExecutionBlockId> unionScanEntry: execBlock.getUnionScanMap().entrySet()) {
             ExecutionBlockId originScanEbId = unionScanEntry.getKey();
@@ -105,25 +99,29 @@ public class Repartitioner {
           ExecutionBlockId scanEBId = TajoIdUtils.createExecutionBlockId(scans[i].getTableName());
           stats[i] = masterContext.getStage(scanEBId).getResultStats().getNumBytes();
         }
-        fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+
+        // TODO - We should remove dummy flagment usages
+        fragments[i] = new FileFragment(scans[i].getCanonicalName(), new Path("/dummy"), 0, 0,
+            new String[]{UNKNOWN_HOST});
+
       } else {
+
         try {
           stats[i] = GlobalPlanRewriteUtil.computeDescendentVolume(scans[i]);
         } catch (PlanningException e) {
           throw new IOException(e);
         }
 
-        Tablespace tablespace =
-            TableSpaceManager.getStorageManager(stage.getContext().getConf(), tableDesc.getMeta().getStoreType());
-
         // if table has no data, tablespace 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<Fragment> fileFragments = tablespace.getSplits(scans[i].getCanonicalName(), tableDesc);
+        Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
+        List<Fragment> fileFragments = space.getSplits(scans[i].getCanonicalName(), tableDesc);
         if (fileFragments.size() > 0) {
           fragments[i] = fileFragments.get(0);
         } else {
-          fragments[i] = new FileFragment(scans[i].getCanonicalName(), new Path(tableDesc.getPath()), 0, 0, new String[]{UNKNOWN_HOST});
+          fragments[i] = new FileFragment(scans[i].getCanonicalName(),
+              new Path(tableDesc.getUri()), 0, 0, new String[]{UNKNOWN_HOST});
         }
       }
     }
@@ -377,26 +375,29 @@ public class Repartitioner {
     if (broadcastFragments != null) {
       //In this phase a ScanNode has a single fragment.
       //If there are more than one data files, that files should be added to fragments or partition path
+
       for (ScanNode eachScan: broadcastScans) {
+
         Path[] partitionScanPaths = null;
         TableDesc tableDesc = masterContext.getTableDescMap().get(eachScan.getCanonicalName());
+        Tablespace space = TableSpaceManager.get(tableDesc.getUri()).get();
+
         if (eachScan.getType() == NodeType.PARTITIONS_SCAN) {
-          FileTablespace storageManager =
-              (FileTablespace) TableSpaceManager.getFileStorageManager(stage.getContext().getConf());
 
           PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)eachScan;
           partitionScanPaths = partitionScan.getInputPaths();
           // set null to inputPaths in getFragmentsFromPartitionedTable()
-          getFragmentsFromPartitionedTable(storageManager, eachScan, tableDesc);
+          getFragmentsFromPartitionedTable((FileTablespace) space, eachScan, tableDesc);
           partitionScan.setInputPaths(partitionScanPaths);
+
         } else {
-          Tablespace tablespace = TableSpaceManager.getStorageManager(stage.getContext().getConf(),
-              tableDesc.getMeta().getStoreType());
-          Collection<Fragment> scanFragments = tablespace.getSplits(eachScan.getCanonicalName(),
+
+          Collection<Fragment> scanFragments = space.getSplits(eachScan.getCanonicalName(),
               tableDesc, eachScan);
           if (scanFragments != null) {
             rightFragments.addAll(scanFragments);
           }
+
         }
       }
     }
@@ -505,18 +506,16 @@ public class Repartitioner {
 
       Collection<Fragment> scanFragments;
       Path[] partitionScanPaths = null;
+
+      FileTablespace space = (FileTablespace) TableSpaceManager.get(desc.getUri()).get();
+
       if (scan.getType() == NodeType.PARTITIONS_SCAN) {
         PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)scan;
         partitionScanPaths = partitionScan.getInputPaths();
         // set null to inputPaths in getFragmentsFromPartitionedTable()
-        FileTablespace storageManager =
-            (FileTablespace) TableSpaceManager.getFileStorageManager(stage.getContext().getConf());
-        scanFragments = getFragmentsFromPartitionedTable(storageManager, scan, desc);
+        scanFragments = getFragmentsFromPartitionedTable(space, scan, desc);
       } else {
-        Tablespace tablespace =
-            TableSpaceManager.getStorageManager(stage.getContext().getConf(), desc.getMeta().getStoreType());
-
-        scanFragments = tablespace.getSplits(scan.getCanonicalName(), desc, scan);
+        scanFragments = space.getSplits(scan.getCanonicalName(), desc, scan);
       }
 
       if (scanFragments != null) {
@@ -618,9 +617,6 @@ public class Repartitioner {
       throws IOException {
     ExecutionBlock execBlock = stage.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
-    Path tablePath;
-    tablePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(stage.getContext().getConf()))
-        .getTablePath(scan.getTableName());
 
     ExecutionBlock sampleChildBlock = masterPlan.getChild(stage.getId(), 0);
     SortNode sortNode = PlannerUtil.findTopNode(sampleChildBlock.getPlan(), NodeType.SORT);
@@ -648,10 +644,15 @@ public class Repartitioner {
         throw new IOException("Can't get table meta data from catalog: " +
             PlannerUtil.getStoreTableName(masterPlan.getLogicalPlan()));
       }
-      ranges = TableSpaceManager.getStorageManager(stage.getContext().getConf(), storeType)
-          .getInsertSortRanges(stage.getContext().getQueryContext(), tableDesc,
-              sortNode.getInSchema(), sortSpecs,
-              mergedRange);
+
+      Tablespace space = TableSpaceManager.getAnyByScheme(storeType).get();
+      ranges = space.getInsertSortRanges(
+          stage.getContext().getQueryContext(),
+          tableDesc,
+          sortNode.getInSchema(),
+          sortSpecs,
+          mergedRange);
+
       determinedTaskNum = ranges.length;
     } else {
       RangePartitionAlgorithm partitioner = new UniformRangePartition(mergedRange, sortSpecs);
@@ -688,7 +689,9 @@ public class Repartitioner {
       }
     }
 
-    FileFragment dummyFragment = new FileFragment(scan.getTableName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+    // TODO - We should remove dummy fragment.
+    FileFragment dummyFragment = new FileFragment(scan.getTableName(), new Path("/dummy"), 0, 0,
+        new String[]{UNKNOWN_HOST});
     Stage.scheduleFragment(stage, dummyFragment);
 
     List<FetchImpl> fetches = new ArrayList<FetchImpl>();
@@ -784,11 +787,9 @@ public class Repartitioner {
                                                  int maxNum) throws IOException {
     ExecutionBlock execBlock = stage.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
-    Path tablePath;
-    tablePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(stage.getContext().getConf()))
-        .getTablePath(scan.getTableName());
 
-    Fragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+    // TODO - We should remove dummy fragment usages
+    Fragment frag = new FileFragment(scan.getCanonicalName(), new Path("/dummy"), 0, 0, new String[]{UNKNOWN_HOST});
     List<Fragment> fragments = new ArrayList<Fragment>();
     fragments.add(frag);
     Stage.scheduleFragments(stage, fragments);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
index 5a0fc38..a7d605c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
@@ -60,8 +60,8 @@ import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.querymaster.Task.IntermediateEntry;
 import org.apache.tajo.storage.FileTablespace;
-import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.Tablespace;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.unit.StorageUnit;
 import org.apache.tajo.util.KeyValueSet;
@@ -1084,18 +1084,18 @@ public class Stage implements EventHandler<StageEvent> {
       Collection<Fragment> fragments;
       TableMeta meta = table.getMeta();
 
+      Tablespace tablespace = TableSpaceManager.get(scan.getTableDesc().getUri()).get();
+
       // Depending on scanner node's type, it creates fragments. If scan is for
       // a partitioned table, It will creates lots fragments for all partitions.
       // Otherwise, it creates at least one fragments for a table, which may
       // span a number of blocks or possibly consists of a number of files.
+      //
+      // Also, we can ensure FileTableSpace if the type of ScanNode is PARTITIONS_SCAN.
       if (scan.getType() == NodeType.PARTITIONS_SCAN) {
         // After calling this method, partition paths are removed from the physical plan.
-        FileTablespace storageManager =
-            (FileTablespace) TableSpaceManager.getFileStorageManager(stage.getContext().getConf());
-        fragments = Repartitioner.getFragmentsFromPartitionedTable(storageManager, scan, table);
+        fragments = Repartitioner.getFragmentsFromPartitionedTable((FileTablespace) tablespace, scan, table);
       } else {
-        Tablespace tablespace =
-            TableSpaceManager.getStorageManager(stage.getContext().getConf(), meta.getStoreType());
         fragments = tablespace.getSplits(scan.getCanonicalName(), table, scan);
       }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
index f265e50..ae22d0d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
+++ b/tajo-core/src/main/java/org/apache/tajo/webapp/QueryExecutorServlet.java
@@ -486,7 +486,7 @@ public class QueryExecutorServlet extends HttpServlet {
       if (resultRows <= 0) {
         resultRows = 1000;
       }
-      LOG.info("Tajo Query Result: " + desc.getPath() + "\n");
+      LOG.info("Tajo Query Result: " + desc.getUri() + "\n");
 
       int numOfColumns = rsmd.getColumnCount();
       for(int i = 0; i < numOfColumns; i++) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
index 0721ef1..0df5d4d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java
@@ -160,7 +160,7 @@ public class LegacyTaskImpl implements Task {
         this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys());
       }
     } else {
-      Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf))
+      Path outFilePath = ((FileTablespace) TableSpaceManager.get(queryContext.getStagingDir().toUri()).get())
           .getAppenderFilePath(getId(), queryContext.getStagingDir());
       LOG.info("Output File Path: " + outFilePath);
       context.setOutputPath(outFilePath);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
index fbd070e..66c8e4a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -40,7 +40,6 @@ import org.apache.tajo.function.FunctionSignature;
 import org.apache.tajo.rpc.RpcClientManager;
 import org.apache.tajo.rpc.RpcConstants;
 import org.apache.tajo.service.ServiceTracker;
-import org.apache.tajo.service.ServiceTrackerException;
 import org.apache.tajo.service.ServiceTrackerFactory;
 import org.apache.tajo.service.TajoMasterInfo;
 import org.apache.tajo.ipc.QueryCoordinatorProtocol.ClusterResourceSummary;
@@ -56,7 +55,7 @@ import org.apache.tajo.rule.EvaluationFailedException;
 import org.apache.tajo.rule.SelfDiagnosisRuleEngine;
 import org.apache.tajo.rule.SelfDiagnosisRuleSession;
 import org.apache.tajo.storage.HashShuffleAppenderManager;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.OldStorageManager;
 import org.apache.tajo.util.*;
 import org.apache.tajo.util.history.HistoryReader;
 import org.apache.tajo.util.history.HistoryWriter;
@@ -370,7 +369,7 @@ public class TajoWorker extends CompositeService {
     }
 
     try {
-      TableSpaceManager.shutdown();
+      OldStorageManager.shutdown();
     } catch (IOException ie) {
       LOG.error(ie.getMessage(), ie);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
index be3960b..5974693 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java
@@ -154,7 +154,7 @@ public class TaskImpl implements Task {
         this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys());
       }
     } else {
-      Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf))
+      Path outFilePath = ((FileTablespace) TableSpaceManager.get(queryContext.getStagingDir().toUri()).get())
           .getAppenderFilePath(getId(), queryContext.getStagingDir());
       LOG.info("Output File Path: " + outFilePath);
       context.setOutputPath(outFilePath);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/catalogview.jsp b/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
index 43ec5ca..6771912 100644
--- a/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
@@ -30,7 +30,6 @@
 <%@ page import="java.util.Collection" %>
 <%@ page import="java.util.List" %>
 <%@ page import="java.util.Map" %>
-<%@ page import="org.apache.tajo.service.ServiceTracker" %>
 <%@ page import="java.net.InetSocketAddress" %>
 <%
   TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
@@ -186,7 +185,7 @@
           <div style='margin-top:10px'>
             <div style=''>Detail</div>
             <table border="1" class='border_table'>
-              <tr><td width='100'>Table path</td><td width='410'><%=tableDesc.getPath()%></td></tr>
+              <tr><td width='100'>Table path</td><td width='410'><%=tableDesc.getUri()%></td></tr>
               <tr><td>Store type</td><td><%=tableDesc.getMeta().getStoreType()%></td></tr>
               <tr><td># rows</td><td><%=(tableDesc.hasStats() ? ("" + tableDesc.getStats().getNumRows()) : "-")%></td></tr>
               <tr><td>Volume</td><td><%=(tableDesc.hasStats() ? FileUtil.humanReadableByteCount(tableDesc.getStats().getNumBytes(),true) : "-")%></td></tr>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/main/resources/webapps/admin/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp
index e0cf876..43bb6c1 100644
--- a/tajo-core/src/main/resources/webapps/admin/index.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/index.jsp
@@ -23,20 +23,19 @@
 <%@ page import="org.apache.tajo.conf.TajoConf" %>
 <%@ page import="org.apache.tajo.ipc.QueryCoordinatorProtocol" %>
 <%@ page import="org.apache.tajo.master.TajoMaster" %>
-<%@ page import="org.apache.tajo.service.ServiceTracker" %>
-<%@ page import="org.apache.tajo.service.TajoMasterInfo" %>
-<%@ page import="org.apache.tajo.master.QueryInProgress" %>
 <%@ page import="org.apache.tajo.master.rm.Worker" %>
 <%@ page import="org.apache.tajo.master.rm.WorkerState" %>
+<%@ page import="org.apache.tajo.service.ServiceTracker" %>
+<%@ page import="org.apache.tajo.service.TajoMasterInfo" %>
+<%@ page import="org.apache.tajo.storage.TableSpaceManager" %>
+<%@ page import="org.apache.tajo.storage.Tablespace" %>
 <%@ page import="org.apache.tajo.util.NetUtils" %>
 <%@ page import="org.apache.tajo.util.TUtil" %>
 <%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
-<%@ page import="java.util.List" %>
-<%@ page import="java.util.Collection" %>
+<%@ page import="java.net.InetSocketAddress" %>
 <%@ page import="java.util.Date" %>
+<%@ page import="java.util.List" %>
 <%@ page import="java.util.Map" %>
-<%@ page import="java.net.InetSocketAddress" %>
-<%@ page import="org.apache.tajo.service.ServiceTracker" %>
 
 <%
   TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
@@ -139,7 +138,15 @@
     <tr><td width='150'>Threads:</td><td><a href='thread.jsp'>thread dump...</a></tr>
   </table>
   <hr/>
-
+  <h3>Tablespaces</h3>
+  <table width="100%" class="border_table" border="1">
+    <tr><th>Tablespace Name</th><th>URI</th><th>Handler</th></tr>
+    <% for (Tablespace space : TableSpaceManager.getAllTablespaces()) {
+      if (space.isVisible()) { %>
+    <tr><td><%=space.getName()%></td><td><%=space.getUri()%></td><td><%=space.getClass().getName()%></td></tr>
+    <% }}%>
+  </table>
+  <hr/>
   <h3>Cluster Summary</h3>
   <table width="100%" class="border_table" border="1">
     <tr><th>Type</th><th>Total</th><th>Live</th><th>Dead</th><th>Running Master</th><th>Memory Resource<br/>(used/total)</th><th>Disk Resource<br/>(used/total)</th></tr>

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
----------------------------------------------------------------------
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 b5be9d0..ca2378b 100644
--- a/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
@@ -46,7 +46,7 @@ public class BackendTestingUtil {
 
   public static void writeTmpTable(TajoConf conf, Path tablePath)
       throws IOException {
-    FileTablespace sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    FileTablespace sm = TableSpaceManager.getDefault();
     Appender appender;
 
     Path filePath = new Path(tablePath, "table.csv");

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
----------------------------------------------------------------------
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 9a92e90..57b1e18 100644
--- a/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/QueryTestCaseBase.java
@@ -945,7 +945,7 @@ public class QueryTestCaseBase {
       return null;
     }
 
-    Path path = new Path(tableDesc.getPath());
+    Path path = new Path(tableDesc.getUri());
     return getTableFileContents(path);
   }
 
@@ -955,7 +955,7 @@ public class QueryTestCaseBase {
       return null;
     }
 
-    Path path = new Path(tableDesc.getPath());
+    Path path = new Path(tableDesc.getUri());
     FileSystem fs = path.getFileSystem(conf);
 
     return listFiles(fs, path);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
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 9b5980b..acdae85 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.client.TajoClientImpl;
 import org.apache.tajo.client.TajoClientUtil;
@@ -48,16 +47,18 @@ import org.apache.tajo.querymaster.QueryMasterTask;
 import org.apache.tajo.querymaster.Stage;
 import org.apache.tajo.querymaster.StageState;
 import org.apache.tajo.service.ServiceTrackerFactory;
+import org.apache.tajo.storage.FileTablespace;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.Pair;
 import org.apache.tajo.worker.TajoWorker;
 
 import java.io.File;
 import java.io.IOException;
 import java.io.Writer;
 import java.net.InetSocketAddress;
-import java.net.URISyntaxException;
 import java.sql.ResultSet;
 import java.util.ArrayList;
 import java.util.List;
@@ -345,10 +346,18 @@ public class TajoTestingCluster {
     LOG.info("derby repository is set to "+conf.get(CatalogConstants.CATALOG_URI));
 
     if (!local) {
-      c.setVar(ConfVars.ROOT_DIR,
-          getMiniDFSCluster().getFileSystem().getUri() + "/tajo");
+      c.setVar(ConfVars.ROOT_DIR, getMiniDFSCluster().getFileSystem().getUri() + "/tajo");
     } else {
-      c.setVar(ConfVars.ROOT_DIR, testBuildDir.getAbsolutePath() + "/tajo");
+      c.setVar(ConfVars.ROOT_DIR, "file://" + testBuildDir.getAbsolutePath() + "/tajo");
+    }
+
+    // Do not need for local file system
+    if (!local) {
+      FileTablespace defaultTableSpace =
+          new FileTablespace(TableSpaceManager.DEFAULT_TABLESPACE_NAME, TajoConf.getWarehouseDir(c).toUri());
+      defaultTableSpace.init(conf);
+
+      TableSpaceManager.addTableSpaceForTest(defaultTableSpace);
     }
 
     setupCatalogForTesting(c, testBuildDir);
@@ -441,13 +450,6 @@ public class TajoTestingCluster {
     }
   }
 
-  public void restartTajoCluster(int numSlaves) throws Exception {
-    tajoMaster.stop();
-    tajoMaster.start();
-
-    LOG.info("Minicluster has been restarted");
-  }
-
   public TajoMaster getMaster() {
     return this.tajoMaster;
   }
@@ -653,7 +655,14 @@ public class TajoTestingCluster {
       if (!fs.exists(rootDir)) {
         fs.mkdirs(rootDir);
       }
-      Path tablePath = new Path(rootDir, tableName);
+      Path tablePath;
+      if (CatalogUtil.isFQTableName(tableName)) {
+        Pair<String, String> name = CatalogUtil.separateQualifierAndName(tableName);
+        tablePath = new Path(rootDir, new Path(name.getFirst(), name.getSecond()));
+      } else {
+        tablePath = new Path(rootDir, tableName);
+      }
+
       fs.mkdirs(tablePath);
       if (tableDatas.length > 0) {
         int recordPerFile = tableDatas.length / numDataFiles;

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java b/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
index 54e50fc..ce951c6 100644
--- a/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
+++ b/tajo-core/src/test/java/org/apache/tajo/cli/tsql/TestTajoCli.java
@@ -32,6 +32,7 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryStatus;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.FileUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -214,10 +215,9 @@ public class TestTajoCli {
 
     String consoleResult = new String(out.toByteArray());
 
-    FileSystem fs = FileSystem.get(testBase.getTestingCluster().getConfiguration());
     if (!cluster.isHiveCatalogStoreRunning()) {
       assertOutputResult(resultFileName, consoleResult, new String[]{"${table.path}"},
-        new String[]{fs.getUri() + "/tajo/warehouse/default/" + tableName});
+        new String[]{TableSpaceManager.getDefault().getTableUri("default", tableName).toString()});
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
index 765a084..73b97fa 100644
--- a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -324,7 +324,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -345,7 +345,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -422,7 +422,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -446,7 +446,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -470,7 +470,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -495,7 +495,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -521,7 +521,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -574,7 +574,7 @@ public class TestTajoClient {
     client.updateQuery(sql);
     assertTrue(client.existTable(tableName));
 
-    Path tablePath = new Path(client.getTableDesc(tableName).getPath());
+    Path tablePath = new Path(client.getTableDesc(tableName).getUri());
     FileSystem hdfs = tablePath.getFileSystem(conf);
     assertTrue(hdfs.exists(tablePath));
 
@@ -704,7 +704,7 @@ public class TestTajoClient {
 
     assertEquals(resultDesc.getMeta().getOption(StorageConstants.TEXT_NULL), "\\\\T");
 
-    Path path = new Path(resultDesc.getPath());
+    Path path = new Path(resultDesc.getUri());
     FileSystem fs = path.getFileSystem(tajoConf);
 
     FileStatus[] files = fs.listStatus(path);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
index 66aedc0..328f883 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java
@@ -50,6 +50,7 @@ import org.apache.tajo.plan.verifier.LogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.PreLogicalPlanVerifier;
 import org.apache.tajo.plan.verifier.VerificationState;
 import org.apache.tajo.storage.LazyTuple;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.VTuple;
 import org.apache.tajo.util.BytesUtils;
@@ -103,7 +104,7 @@ public class ExprTestBase {
 
     analyzer = new SQLAnalyzer();
     preLogicalPlanVerifier = new PreLogicalPlanVerifier(cat);
-    planner = new LogicalPlanner(cat);
+    planner = new LogicalPlanner(cat, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(util.getConfiguration());
     annotatedPlanVerifier = new LogicalPlanVerifier(util.getConfiguration(), cat);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
index 4bfe640..80f3459 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java
@@ -45,6 +45,7 @@ import org.apache.tajo.plan.function.GeneralFunction;
 import org.apache.tajo.plan.logical.GroupbyNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.nameresolver.NameResolvingMode;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.AfterClass;
@@ -116,7 +117,7 @@ public class TestEvalTreeUtil {
     catalog.createFunction(funcMeta);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
 
     String[] QUERIES = {
         "select name, score, age from people where score > 30", // 0

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
index a408fd6..9aa7ddf 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalOptimizer.java
@@ -24,7 +24,6 @@ import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.engine.function.FunctionLoader;
 import org.apache.tajo.engine.function.builtin.SumInt;
@@ -35,6 +34,7 @@ import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;
 import org.junit.AfterClass;
@@ -103,7 +103,7 @@ public class TestLogicalOptimizer {
 
     catalog.createFunction(funcDesc);
     sqlAnalyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(util.getConfiguration());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(util.getConfiguration());

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
index cee1593..3cee816 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
@@ -19,6 +19,7 @@
 package org.apache.tajo.engine.planner;
 
 import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.graph.SimpleDirectedGraph;
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.LogicalPlanner;
@@ -39,7 +40,7 @@ public class TestLogicalPlan {
   public static void setup() throws Exception {
     util = new TajoTestingCluster();
     util.startCatalogCluster();
-    planner = new LogicalPlanner(util.getMiniCatalogCluster().getCatalog());
+    planner = new LogicalPlanner(util.getMiniCatalogCluster().getCatalog(), TableSpaceManager.getInstance());
   }
 
   public static void tearDown() {

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 1feea4c..351a6af 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.QueryVars;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.algebra.Expr;
@@ -29,7 +30,6 @@ import org.apache.tajo.algebra.JoinType;
 import org.apache.tajo.benchmark.TPCH;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
-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.engine.function.FunctionLoader;
@@ -42,6 +42,7 @@ import org.apache.tajo.plan.*;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.KeyValueSet;
@@ -130,7 +131,7 @@ public class TestLogicalPlanner {
 
     catalog.createFunction(funcDesc);
     sqlAnalyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
   }
 
   @AfterClass
@@ -155,6 +156,13 @@ public class TestLogicalPlanner {
       "select length(name), length(deptname), *, empid+10 from employee where empId > 500", // 13
   };
 
+  private static QueryContext createQueryContext() {
+    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    qc.put(QueryVars.DEFAULT_SPACE_URI, "file:/");
+    qc.put(QueryVars.DEFAULT_SPACE_ROOT_URI, "file:/");
+    return qc;
+  }
+
   public static final void testCloneLogicalNode(LogicalNode n1) throws CloneNotSupportedException {
     LogicalNode copy = (LogicalNode) n1.clone();
     assertTrue(n1.deepEquals(copy));
@@ -162,7 +170,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSingleRelation() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(QUERIES[0]);
     LogicalPlan planNode = planner.createPlan(qc, expr);
@@ -196,7 +204,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testImplicityJoinPlan() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[1]);
@@ -285,7 +293,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testNaturalJoinPlan() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
     // two relations
     Expr context = sqlAnalyzer.parse(JOINS[0]);
     LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot();
@@ -317,7 +325,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInnerJoinPlan() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
     // two relations
     Expr expr = sqlAnalyzer.parse(JOINS[1]);
     LogicalPlan plan = planner.createPlan(qc, expr);
@@ -350,7 +358,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testOuterJoinPlan() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     // two relations
     Expr expr = sqlAnalyzer.parse(JOINS[2]);
@@ -385,7 +393,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testGroupby() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     // without 'having clause'
     Expr context = sqlAnalyzer.parse(QUERIES[7]);
@@ -429,7 +437,7 @@ public class TestLogicalPlanner {
   public final void testMultipleJoin() throws IOException, PlanningException {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File("src/test/resources/queries/TestJoinQuery/testTPCHQ2Join.sql")));
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
     LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
     testJsonSerDerObject(plan);
     Schema expected = tpch.getOutSchema("q2");
@@ -488,7 +496,7 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual1.sql")));
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     LogicalPlan plan = planner.createPlan(qc, expr);
     LogicalNode node = plan.getRootBlock().getRoot();
@@ -518,7 +526,7 @@ public class TestLogicalPlanner {
     }
 
     for (Map.Entry<BinaryEval, Boolean> entry : qualMap.entrySet()) {
-      if (!entry.getValue().booleanValue()) {
+      if (!entry.getValue()) {
         Preconditions.checkArgument(false,
             "JoinQual not found. -> required JoinQual:" + entry.getKey().toJson());
       }
@@ -530,7 +538,7 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual2.sql")));
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     LogicalPlan plan = planner.createPlan(qc,expr);
     LogicalNode node = plan.getRootBlock().getRoot();
@@ -559,7 +567,7 @@ public class TestLogicalPlanner {
     }
 
     for (Map.Entry<BinaryEval, Boolean> entry : qualMap.entrySet()) {
-      if (!entry.getValue().booleanValue()) {
+      if (!entry.getValue()) {
         Preconditions.checkArgument(false,
             "SelectionQual not found. -> required JoinQual:" + entry.getKey().toJson());
       }
@@ -571,7 +579,7 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual3.sql")));
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     LogicalPlan plan = planner.createPlan(qc, expr);
     LogicalNode node = plan.getRootBlock().getRoot();
@@ -605,7 +613,7 @@ public class TestLogicalPlanner {
     }
 
     for (Map.Entry<BinaryEval, Boolean> entry : qualMap.entrySet()) {
-      if (!entry.getValue().booleanValue()) {
+      if (!entry.getValue()) {
         Preconditions.checkArgument(false,
             "ScanQual not found. -> required JoinQual:" + entry.getKey().toJson());
       }
@@ -618,7 +626,7 @@ public class TestLogicalPlanner {
     Expr expr = sqlAnalyzer.parse(
         FileUtil.readTextFile(new File
             ("src/test/resources/queries/TestJoinQuery/testJoinWithMultipleJoinQual4.sql")));
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     LogicalPlan plan = planner.createPlan(qc, expr);
     LogicalNode node = plan.getRootBlock().getRoot();
@@ -675,14 +683,14 @@ public class TestLogicalPlanner {
 
 
     for (Map.Entry<BinaryEval, Boolean> entry : joinQualMap.entrySet()) {
-      if (!entry.getValue().booleanValue()) {
+      if (!entry.getValue()) {
         Preconditions.checkArgument(false,
             "JoinQual not found. -> required JoinQual:" + entry.getKey().toJson());
       }
     }
 
     for (Map.Entry<BinaryEval, Boolean> entry : scanMap.entrySet()) {
-      if (!entry.getValue().booleanValue()) {
+      if (!entry.getValue()) {
         Preconditions.checkArgument(false,
             "ScanQual not found. -> required JoinQual:" + entry.getKey().toJson());
       }
@@ -709,7 +717,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testStoreTable() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr context = sqlAnalyzer.parse(QUERIES[8]);
 
@@ -727,7 +735,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testOrderBy() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(QUERIES[4]);
 
@@ -757,7 +765,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testLimit() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(QUERIES[12]);
 
@@ -779,7 +787,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSPJPush() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(QUERIES[5]);
     LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
@@ -801,7 +809,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSPJ() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(QUERIES[6]);
     LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
@@ -811,7 +819,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testJson() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
 	  Expr expr = sqlAnalyzer.parse(QUERIES[9]);
 	  LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
@@ -833,7 +841,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testVisitor() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     // two relations
     Expr expr = sqlAnalyzer.parse(QUERIES[1]);
@@ -860,7 +868,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testExprNode() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(QUERIES[10]);
     LogicalPlan rootNode = planner.createPlan(qc, expr);
@@ -882,7 +890,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testAsterisk() throws CloneNotSupportedException, PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(QUERIES[13]);
     LogicalPlan planNode = planner.createPlan(qc, expr);
@@ -912,7 +920,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testAlias1() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(ALIAS[0]);
     LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
@@ -940,7 +948,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testAlias2() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(ALIAS[1]);
     LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
@@ -961,7 +969,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testCreateTableDef() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(CREATE_TABLE[0]);
     LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
@@ -980,7 +988,7 @@ public class TestLogicalPlanner {
     assertEquals("score", def.getColumn(3).getSimpleName());
     assertEquals(Type.FLOAT4, def.getColumn(3).getDataType().getType());
     assertTrue("CSV".equalsIgnoreCase(createTable.getStorageType()));
-    assertEquals("/tmp/data", createTable.getPath().toString());
+    assertEquals("/tmp/data", createTable.getUri().toString());
     assertTrue(createTable.hasOptions());
     assertEquals("|", createTable.getOptions().get("csv.delimiter"));
   }
@@ -1047,7 +1055,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testSetPlan() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(setStatements[0]);
     LogicalNode plan = planner.createPlan(qc, expr).getRootBlock().getRoot();
@@ -1068,7 +1076,7 @@ public class TestLogicalPlanner {
 
   @Test
   public void testSetQualifier() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr context = sqlAnalyzer.parse(setQualifiers[0]);
     LogicalNode plan = planner.createPlan(qc, context).getRootBlock().getRoot();
@@ -1121,7 +1129,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto0() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(insertStatements[0]);
     LogicalPlan plan = planner.createPlan(qc, expr);
@@ -1134,7 +1142,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto1() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(insertStatements[1]);
     LogicalPlan plan = planner.createPlan(qc, expr);
@@ -1146,7 +1154,7 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto2() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(insertStatements[2]);
     LogicalPlan plan = planner.createPlan(qc, expr);
@@ -1161,19 +1169,19 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto3() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(insertStatements[3]);
     LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertFalse(insertNode.isOverwrite());
-    assertTrue(insertNode.hasPath());
+    assertTrue(insertNode.hasUri());
   }
 
   @Test
   public final void testInsertInto4() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(insertStatements[4]);
     LogicalPlan plan = planner.createPlan(qc, expr);
@@ -1189,19 +1197,19 @@ public class TestLogicalPlanner {
 
   @Test
   public final void testInsertInto5() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(insertStatements[5]);
     LogicalPlan plan = planner.createPlan(qc, expr);
     assertEquals(1, plan.getQueryBlocks().size());
     InsertNode insertNode = getInsertNode(plan);
     assertTrue(insertNode.isOverwrite());
-    assertTrue(insertNode.hasPath());
+    assertTrue(insertNode.hasUri());
   }
 
   @Test
   public final void testInsertInto6() throws PlanningException {
-    QueryContext qc = new QueryContext(util.getConfiguration(), session);
+    QueryContext qc = createQueryContext();
 
     Expr expr = sqlAnalyzer.parse(insertStatements[6]);
     LogicalPlan plan = planner.createPlan(qc, expr);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
----------------------------------------------------------------------
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 0082800..d62eed2 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
@@ -39,6 +39,7 @@ import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.plan.expr.*;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.storage.TableSpaceManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleComparator;
 import org.apache.tajo.storage.VTuple;
@@ -108,7 +109,7 @@ public class TestPlannerUtil {
 
     catalog.createFunction(funcDesc);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
   }
 
   @AfterClass
@@ -337,7 +338,7 @@ public class TestPlannerUtil {
 
     TableDesc tableDesc = new TableDesc();
     tableDesc.setName("Test");
-    tableDesc.setPath(path.toUri());
+    tableDesc.setUri(path.toUri());
 
     FileSystem fs = path.getFileSystem(util.getConfiguration());
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
----------------------------------------------------------------------
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 e7e4f7d..2464fb1 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
@@ -86,7 +86,7 @@ public class TestBNLJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     VTuple tuple = new VTuple(schema.size());
@@ -108,8 +108,7 @@ public class TestBNLJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta("CSV");
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < INNER_TUPLE_NUM; i += 2) {
@@ -125,7 +124,7 @@ public class TestBNLJoinExec {
     people = CatalogUtil.newTableDesc("default.people", peopleSchema, peopleMeta, peoplePath);
     catalog.createTable(people);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
   }
 
   @After
@@ -150,10 +149,10 @@ public class TestBNLJoinExec {
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.BLOCK_NESTED_LOOP_JOIN);
 
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()),
+        new Path(employee.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()),
+        new Path(people.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
     Path workDir = CommonTestingUtil.getTestDir(TajoTestingCluster.DEFAULT_TEST_DIRECTORY + "/testBNLCrossJoin");
@@ -183,9 +182,9 @@ public class TestBNLJoinExec {
         context).getRootBlock().getRoot();
 
     FileFragment[] empFrags = FileTablespace.splitNG(conf, "default.e", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     FileFragment[] peopleFrags = FileTablespace.splitNG(conf, "default.p", people.getMeta(),
-        new Path(people.getPath()), Integer.MAX_VALUE);
+        new Path(people.getUri()), Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(empFrags, peopleFrags);
 
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
----------------------------------------------------------------------
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 5a7ba6a..96a1f36 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
@@ -90,7 +90,7 @@ public class TestBSTIndexExec {
     Path workDir = CommonTestingUtil.getTestDir();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, workDir.toUri().toString());
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
-    sm = (FileTablespace) TableSpaceManager.getFileStorageManager(conf);
+    sm = TableSpaceManager.getLocalFs();
 
     idxPath = new Path(workDir, "test.idx");
 
@@ -144,11 +144,11 @@ public class TestBSTIndexExec {
 
     TableDesc desc = new TableDesc(
         CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta,
-        sm.getTablePath("employee").toUri());
+        sm.getTableUri(TajoConstants.DEFAULT_DATABASE_NAME, "employee"));
     catalog.createTable(desc);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
     optimizer = new LogicalOptimizer(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
----------------------------------------------------------------------
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 8e2f234..d94d3f6 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
@@ -82,7 +82,7 @@ public class TestExternalSortExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta("CSV");
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
@@ -104,7 +104,7 @@ public class TestExternalSortExec {
     employee = new TableDesc("default.employee", schema, employeeMeta, employeePath.toUri());
     catalog.createTable(employee);
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
   }
 
   @After
@@ -120,7 +120,7 @@ public class TestExternalSortExec {
   @Test
   public final void testNext() throws IOException, PlanningException {
     FileFragment[] frags = FileTablespace.splitNG(conf, "default.employee", employee.getMeta(),
-        new Path(employee.getPath()), Integer.MAX_VALUE);
+        new Path(employee.getUri()), Integer.MAX_VALUE);
     Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
     TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf),
         LocalTajoTestingUtility.newTaskAttemptId(), new FileFragment[] { frags[0] }, workDir);

http://git-wip-us.apache.org/repos/asf/tajo/blob/d0f37012/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
----------------------------------------------------------------------
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 fb6fd02..21a101a 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
@@ -104,8 +104,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta("CSV");
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     VTuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -134,8 +133,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta("CSV");
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     VTuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -174,8 +172,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta("CSV");
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
-        .getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     VTuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -227,7 +224,7 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta("CSV");
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = ((FileTablespace) TableSpaceManager.getFileStorageManager(conf))
+    Appender appender5 = ((FileTablespace) TableSpaceManager.getLocalFs())
         .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
@@ -237,7 +234,7 @@ public class TestFullOuterHashJoinExec {
     catalog.createTable(phone3);
 
     analyzer = new SQLAnalyzer();
-    planner = new LogicalPlanner(catalog);
+    planner = new LogicalPlanner(catalog, TableSpaceManager.getInstance());
 
     defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
   }
@@ -266,9 +263,9 @@ public class TestFullOuterHashJoinExec {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] dep3Frags = FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()),
+    FileFragment[] dep3Frags = FileTablespace.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(dep3Frags, emp3Frags);
 
@@ -305,9 +302,9 @@ public class TestFullOuterHashJoinExec {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags);
 
@@ -343,9 +340,9 @@ public class TestFullOuterHashJoinExec {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()),
+    FileFragment[] job3Frags = FileTablespace.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags);
 
@@ -382,9 +379,9 @@ public class TestFullOuterHashJoinExec {
     Enforcer enforcer = new Enforcer();
     enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.IN_MEMORY_HASH_JOIN);
 
-    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()),
+    FileFragment[] emp3Frags = FileTablespace.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getUri()),
         Integer.MAX_VALUE);
-    FileFragment[] phone3Frags = FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()),
+    FileFragment[] phone3Frags = FileTablespace.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getUri()),
         Integer.MAX_VALUE);
     FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags);
 


[10/15] tajo git commit: TAJO-1616: Implement TablespaceManager to load Tablespaces. (missed commits)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java
new file mode 100644
index 0000000..26af769
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java
@@ -0,0 +1,390 @@
+/**
+ * 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.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import net.minidev.json.JSONObject;
+import net.minidev.json.parser.JSONParser;
+import net.minidev.json.parser.ParseException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.Pair;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.URI;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static org.apache.tajo.storage.StorageConstants.LOCAL_FS_URI;
+
+/**
+ * It handles available table spaces and cache TableSpace instances.
+ *
+ * Default tablespace must be a filesystem-based one.
+ * HDFS and S3 can be a default tablespace if a Tajo cluster is in fully distributed mode.
+ * Local file system can be a default tablespace if a Tajo cluster runs on a single machine.
+ */
+public class TablespaceManager implements StorageService {
+  private static final Log LOG = LogFactory.getLog(TablespaceManager.class);
+
+  public static final String DEFAULT_CONFIG_FILE = "storage-default.json";
+  public static final String SITE_CONFIG_FILE = "storage-site.json";
+
+  /** default tablespace name */
+  public static final String DEFAULT_TABLESPACE_NAME = "default";
+
+  private final static TajoConf systemConf = new TajoConf();
+  private final static JSONParser parser = new JSONParser(JSONParser.MODE_JSON_SIMPLE | JSONParser.IGNORE_CONTROL_CHAR);
+
+  // The relation ship among name, URI, Tablespaces must be kept 1:1:1.
+  protected static final Map<String, URI> SPACES_URIS_MAP = Maps.newHashMap();
+  protected static final TreeMap<URI, Tablespace> TABLE_SPACES = Maps.newTreeMap();
+
+  protected static final Map<Class<?>, Constructor<?>> CONSTRUCTORS = Maps.newHashMap();
+  protected static final Map<String, Class<? extends Tablespace>> TABLE_SPACE_HANDLERS = Maps.newHashMap();
+
+  public static final Class [] TABLESPACE_PARAM = new Class [] {String.class, URI.class};
+
+  static {
+    instance = new TablespaceManager();
+  }
+  /**
+   * Singleton instance
+   */
+  private static final TablespaceManager instance;
+
+  private TablespaceManager() {
+    initForDefaultConfig(); // loading storage-default.json
+    initSiteConfig();       // storage-site.json will override the configs of storage-default.json
+    addWarehouseAsSpace();  // adding a warehouse directory for a default tablespace
+    addLocalFsTablespace(); // adding a tablespace using local file system by default
+  }
+
+  private void addWarehouseAsSpace() {
+    Path warehouseDir = TajoConf.getWarehouseDir(systemConf);
+    registerTableSpace(DEFAULT_TABLESPACE_NAME, warehouseDir.toUri(), null, true, false);
+  }
+
+  private void addLocalFsTablespace() {
+    if (TABLE_SPACES.headMap(LOCAL_FS_URI, true).firstEntry() == null) {
+      String tmpName = UUID.randomUUID().toString();
+      registerTableSpace(tmpName, LOCAL_FS_URI, null, false, false);
+    }
+  }
+
+  public static TablespaceManager getInstance() {
+    return instance;
+  }
+
+  private void initForDefaultConfig() {
+    JSONObject json = loadFromConfig(DEFAULT_CONFIG_FILE);
+    if (json == null) {
+      throw new IllegalStateException("There is no " + SITE_CONFIG_FILE);
+    }
+    applyConfig(json, false);
+  }
+
+  private void initSiteConfig() {
+    JSONObject json = loadFromConfig(SITE_CONFIG_FILE);
+
+    // if there is no storage-site.json file, nothing happen.
+    if (json != null) {
+      applyConfig(json, true);
+    }
+  }
+
+  private JSONObject loadFromConfig(String fileName) {
+    String json;
+    try {
+      json = FileUtil.readTextFileFromResource(fileName);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    if (json != null) {
+      return parseJson(json);
+    } else {
+      return null;
+    }
+  }
+
+  private static JSONObject parseJson(String json) {
+    try {
+      return (JSONObject) parser.parse(json);
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private void applyConfig(JSONObject json, boolean override) {
+    loadStorages(json);
+    loadTableSpaces(json, override);
+  }
+
+  private void loadStorages(JSONObject json) {
+    JSONObject spaces = (JSONObject) json.get(KEY_STORAGES);
+
+    if (spaces != null) {
+      Pair<String, Class<? extends Tablespace>> pair = null;
+      for (Map.Entry<String, Object> entry : spaces.entrySet()) {
+
+        try {
+          pair = extractStorage(entry);
+        } catch (ClassNotFoundException e) {
+          LOG.warn(e);
+          continue;
+        }
+
+        TABLE_SPACE_HANDLERS.put(pair.getFirst(), pair.getSecond());
+      }
+    }
+  }
+
+  private Pair<String, Class<? extends Tablespace>> extractStorage(Map.Entry<String, Object> entry)
+      throws ClassNotFoundException {
+
+    String storageType = entry.getKey();
+    JSONObject storageDesc = (JSONObject) entry.getValue();
+    String handlerClass = (String) storageDesc.get(KEY_STORAGE_HANDLER);
+
+    return new Pair<String, Class<? extends Tablespace>>(
+        storageType,(Class<? extends Tablespace>) Class.forName(handlerClass));
+  }
+
+  private void loadTableSpaces(JSONObject json, boolean override) {
+    JSONObject spaces = (JSONObject) json.get(KEY_SPACES);
+
+    if (spaces != null) {
+      for (Map.Entry<String, Object> entry : spaces.entrySet()) {
+        AddTableSpace(entry.getKey(), (JSONObject) entry.getValue(), override);
+      }
+    }
+  }
+
+  public static void AddTableSpace(String spaceName, JSONObject spaceDesc, boolean override) {
+    boolean defaultSpace = Boolean.parseBoolean(spaceDesc.getAsString("default"));
+    URI spaceUri = URI.create(spaceDesc.getAsString("uri"));
+
+    if (defaultSpace) {
+      registerTableSpace(DEFAULT_TABLESPACE_NAME, spaceUri, spaceDesc, true, override);
+    }
+    registerTableSpace(spaceName, spaceUri, spaceDesc, true, override);
+  }
+
+  private static void registerTableSpace(String spaceName, URI uri, JSONObject spaceDesc,
+                                         boolean visible, boolean override) {
+    Tablespace tableSpace = initializeTableSpace(spaceName, uri, visible);
+    tableSpace.setVisible(visible);
+
+    try {
+      tableSpace.init(systemConf);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    putTablespace(tableSpace, override);
+
+    // If the arbitrary path is allowed, root uri is also added as a tablespace
+    if (tableSpace.getProperty().isArbitraryPathAllowed()) {
+      URI rootUri = tableSpace.getRootUri();
+      // if there already exists or the rootUri is 'file:/', it won't overwrite the tablespace.
+      if (!TABLE_SPACES.containsKey(rootUri) && !rootUri.toString().startsWith(LOCAL_FS_URI.toString())) {
+        String tmpName = UUID.randomUUID().toString();
+        registerTableSpace(tmpName, rootUri, spaceDesc, false, override);
+      }
+    }
+  }
+
+  private static void putTablespace(Tablespace space, boolean override) {
+    // It is a device to keep the relationship among name, URI, and tablespace 1:1:1.
+
+    boolean nameExist = SPACES_URIS_MAP.containsKey(space.getName());
+    boolean uriExist = TABLE_SPACES.containsKey(space.uri);
+
+    boolean mismatch = nameExist && !SPACES_URIS_MAP.get(space.getName()).equals(space.getUri());
+    mismatch = mismatch || uriExist && TABLE_SPACES.get(space.uri).equals(space);
+
+    if (!override && mismatch) {
+      throw new RuntimeException("Name or URI of Tablespace must be unique.");
+    }
+
+    SPACES_URIS_MAP.put(space.getName(), space.getUri());
+    // We must guarantee that the same uri results in the same tablespace instance.
+    TABLE_SPACES.put(space.getUri(), space);
+  }
+
+  /**
+   * 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 guessFragmentVolume(TajoConf conf, Fragment fragment) {
+    if (fragment.getLength() == TajoConstants.UNKNOWN_LENGTH) {
+      return conf.getLongVar(TajoConf.ConfVars.FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH);
+    } else {
+      return fragment.getLength();
+    }
+  }
+
+  public static final String KEY_STORAGES = "storages"; // storages
+  public static final String KEY_STORAGE_HANDLER = "handler"; // storages/?/handler
+  public static final String KEY_STORAGE_DEFAULT_FORMAT = "default-format"; // storages/?/default-format
+
+  public static final String KEY_SPACES = "spaces";
+
+  private static Tablespace initializeTableSpace(String spaceName, URI uri, boolean visible) {
+    Preconditions.checkNotNull(uri.getScheme(), "URI must include scheme, but it was " + uri);
+    Class<? extends Tablespace> clazz = TABLE_SPACE_HANDLERS.get(uri.getScheme());
+
+    if (clazz == null) {
+      throw new RuntimeException("There is no tablespace for " + uri.toString());
+    }
+
+    try {
+      Constructor<? extends Tablespace> constructor =
+          (Constructor<? extends Tablespace>) CONSTRUCTORS.get(clazz);
+
+      if (constructor == null) {
+        constructor = clazz.getDeclaredConstructor(TABLESPACE_PARAM);
+        constructor.setAccessible(true);
+        CONSTRUCTORS.put(clazz, constructor);
+      }
+
+      return constructor.newInstance(new Object[]{spaceName, uri});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @VisibleForTesting
+  public static Optional<Tablespace> addTableSpaceForTest(Tablespace space) {
+    Tablespace existing;
+    synchronized (SPACES_URIS_MAP) {
+      // Remove existing one
+      SPACES_URIS_MAP.remove(space.getName());
+      existing = TABLE_SPACES.remove(space.getUri());
+
+      // Add anotherone for test
+      registerTableSpace(space.name, space.uri, null, true, true);
+    }
+    // if there is an existing one, return it.
+    return Optional.fromNullable(existing);
+  }
+
+  public Iterable<String> getSupportSchemes() {
+    return TABLE_SPACE_HANDLERS.keySet();
+  }
+
+  /**
+   * Get tablespace for the given URI. If uri is null, the default tablespace will be returned
+   *
+   * @param uri Table or Table Fragment URI.
+   * @param <T> Tablespace class type
+   * @return Tablespace. If uri is null, the default tablespace will be returned.
+   */
+  public static <T extends Tablespace> Optional<T> get(@Nullable String uri) {
+
+    if (uri == null || uri.isEmpty()) {
+      return (Optional<T>) Optional.of(getDefault());
+    }
+
+    Tablespace lastOne = null;
+
+    // Find the longest matched one. For example, assume that the caller tries to find /x/y/z, and
+    // there are /x and /x/y. In this case, /x/y will be chosen because it is more specific.
+    for (Map.Entry<URI, Tablespace> entry: TABLE_SPACES.headMap(URI.create(uri), true).entrySet()) {
+      if (uri.startsWith(entry.getKey().toString())) {
+        lastOne = entry.getValue();
+      }
+    }
+    return (Optional<T>) Optional.fromNullable(lastOne);
+  }
+
+  /**
+   * Get tablespace for the given URI. If uri is null, the default tablespace will be returned
+   *
+   * @param uri Table or Table Fragment URI.
+   * @param <T> Tablespace class type
+   * @return Tablespace. If uri is null, the default tablespace will be returned.
+   */
+  public static <T extends Tablespace> Optional<T> get(@Nullable URI uri) {
+    if (uri == null) {
+      return (Optional<T>) Optional.of(getDefault());
+    } else {
+      return (Optional<T>) get(uri.toString());
+    }
+  }
+
+  /**
+   * It returns the default tablespace. This method ensures that it always return the tablespace.
+   *
+   * @return
+   */
+  public static <T extends Tablespace> T getDefault() {
+    return (T) getByName(DEFAULT_TABLESPACE_NAME).get();
+  }
+
+  public static <T extends Tablespace> T getLocalFs() {
+    return (T) get(LOCAL_FS_URI).get();
+  }
+
+  public static Optional<? extends Tablespace> getByName(String name) {
+    URI uri = SPACES_URIS_MAP.get(name);
+    if (uri != null) {
+      return Optional.of(TABLE_SPACES.get(uri));
+    } else {
+      return Optional.absent();
+    }
+  }
+
+  public static Optional<? extends Tablespace> getAnyByScheme(String scheme) {
+    for (Map.Entry<URI, Tablespace> entry : TABLE_SPACES.entrySet()) {
+      String uriScheme = entry.getKey().getScheme();
+      if (uriScheme != null && uriScheme.equalsIgnoreCase(scheme)) {
+        return Optional.of(entry.getValue());
+      }
+    }
+
+    return Optional.absent();
+  }
+
+  @Override
+  public URI getTableURI(@Nullable String spaceName, String databaseName, String tableName) {
+    Tablespace space = spaceName == null ? getDefault() : getByName(spaceName).get();
+    return space.getTableUri(databaseName, tableName);
+  }
+
+  public static Iterable<Tablespace> getAllTablespaces() {
+    return TABLE_SPACES.values();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
----------------------------------------------------------------------
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
index 916aae7..7943134 100644
--- 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
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.storage.Tuple;
 
 import java.io.IOException;
@@ -47,7 +47,7 @@ public class HBasePutAppender extends AbstractHBaseAppender {
   public void init() throws IOException {
     super.init();
 
-    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.get(uri).get();
+    HBaseTablespace space = (HBaseTablespace) TablespaceManager.get(uri).get();
     HConnection hconn = space.getConnection();
     htable = hconn.getTable(columnMapping.getHbaseTableName());
     htable.setAutoFlushTo(false);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
----------------------------------------------------------------------
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
index 16f4c14..7369897 100644
--- 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
@@ -133,7 +133,7 @@ public class HBaseScanner implements Scanner {
     rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
     rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
 
-    HBaseTablespace space = (HBaseTablespace) TableSpaceManager.get(fragment.getUri()).get();
+    HBaseTablespace space = (HBaseTablespace) TablespaceManager.get(fragment.getUri()).get();
     hbaseConf = space.getHbaseConf();
     initScanner();
   }
@@ -181,7 +181,7 @@ public class HBaseScanner implements Scanner {
     }
 
     if (htable == null) {
-      HConnection hconn = ((HBaseTablespace) TableSpaceManager.get(fragment.getUri()).get()).getConnection();
+      HConnection hconn = ((HBaseTablespace) TablespaceManager.get(fragment.getUri()).get()).getConnection();
       htable = hconn.getTable(fragment.getHbaseTableName());
     }
     scanner = htable.getScanner(scan);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
index 5fac0cf..18bb7ed 100644
--- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java
@@ -51,10 +51,7 @@ import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.ScanNode;
 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 org.apache.tajo.util.*;
 
 import java.io.BufferedReader;
 import java.io.IOException;
@@ -68,9 +65,9 @@ import java.util.*;
 public class HBaseTablespace extends Tablespace {
   private static final Log LOG = LogFactory.getLog(HBaseTablespace.class);
 
-  public static final StorageProperty HBASE_STORAGE_PROPERTIES = new StorageProperty(false, true, true, false);
-
-  public static final FormatProperty HFILE_FORMAT_PROPERTIES = new FormatProperty(true);
+  public static final StorageProperty HBASE_STORAGE_PROPERTIES = new StorageProperty("hbase", false, true, false);
+  public static final FormatProperty HFILE_FORMAT_PROPERTIES = new FormatProperty(true, false, true);
+  public static final FormatProperty PUT_MODE_PROPERTIES = new FormatProperty(true, true, false);
 
   private Configuration hbaseConf;
 
@@ -572,6 +569,15 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
+  public Appender getAppenderForInsertRow(OverridableConf queryContext,
+                                          TaskAttemptId taskAttemptId,
+                                          TableMeta meta,
+                                          Schema schema,
+                                          Path workDir) throws IOException {
+    return new HBasePutAppender(conf, uri, taskAttemptId, schema, meta, workDir);
+  }
+
+  @Override
   public Appender getAppender(OverridableConf queryContext,
                               TaskAttemptId taskAttemptId, TableMeta meta, Schema schema, Path workDir)
       throws IOException {
@@ -1096,8 +1102,14 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
-  public FormatProperty getFormatProperty(String format) {
-    return HFILE_FORMAT_PROPERTIES;
+  public FormatProperty getFormatProperty(TableMeta meta) {
+    KeyValueSet tableProperty = meta.getOptions();
+    if (tableProperty.isTrue(HBaseStorageConstants.INSERT_PUT_MODE) ||
+        tableProperty.isTrue(StorageConstants.INSERT_DIRECTLY)) {
+      return PUT_MODE_PROPERTIES;
+    } else {
+      return HFILE_FORMAT_PROPERTIES;
+    }
   }
 
   public void prepareTable(LogicalNode node) throws IOException {
@@ -1134,6 +1146,24 @@ public class HBaseTablespace extends Tablespace {
   }
 
   @Override
+  public URI getStagingUri(OverridableConf context, String queryId, TableMeta meta) throws IOException {
+    if (meta.getOptions().isTrue(HBaseStorageConstants.INSERT_PUT_MODE)) {
+      throw new IOException("Staging phase is not supported in this storage.");
+    } else {
+      return TablespaceManager.getDefault().getStagingUri(context, queryId, meta);
+    }
+  }
+
+  public URI prepareStagingSpace(TajoConf conf, String queryId, OverridableConf context,
+                                 TableMeta meta) throws IOException {
+    if (!meta.getOptions().isTrue(HBaseStorageConstants.INSERT_PUT_MODE)) {
+      return TablespaceManager.getDefault().prepareStagingSpace(conf, queryId, context, meta);
+    } else {
+      throw new IOException("Staging phase is not supported in this storage.");
+    }
+  }
+
+  @Override
   public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException  {
     if (tableDesc != null) {
       Schema tableSchema = tableDesc.getSchema();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java
index f7cbb5a..f0c8f15 100644
--- a/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java
+++ b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseTableSpace.java
@@ -25,7 +25,7 @@ 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.TableSpaceManager;
+import org.apache.tajo.storage.TablespaceManager;
 import org.apache.tajo.util.Pair;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -45,7 +45,7 @@ public class TestHBaseTableSpace {
     String tableSpaceUri = "hbase:zk://host1:2171";
     HBaseTablespace hBaseTablespace = new HBaseTablespace("cluster1", URI.create(tableSpaceUri));
     hBaseTablespace.init(new TajoConf());
-    TableSpaceManager.addTableSpaceForTest(hBaseTablespace);
+    TablespaceManager.addTableSpaceForTest(hBaseTablespace);
   }
 
   @Test
@@ -58,8 +58,8 @@ public class TestHBaseTableSpace {
 
   @Test
   public void testTablespaceHandler() throws Exception {
-    assertTrue((TableSpaceManager.getByName("cluster1").get()) instanceof HBaseTablespace);
-    assertTrue((TableSpaceManager.get(URI.create("hbase:zk://host1:2171")).get())
+    assertTrue((TablespaceManager.getByName("cluster1").get()) instanceof HBaseTablespace);
+    assertTrue((TablespaceManager.get(URI.create("hbase:zk://host1:2171")).get())
         instanceof HBaseTablespace);
   }
 
@@ -73,7 +73,7 @@ public class TestHBaseTableSpace {
     EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2);
     scanNode.setQual(evalNodeA);
 
-    HBaseTablespace storageManager = (HBaseTablespace) TableSpaceManager.getByName("cluster1").get();
+    HBaseTablespace storageManager = (HBaseTablespace) TablespaceManager.getByName("cluster1").get();
     List<Set<EvalNode>> indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
     assertNotNull(indexEvals);
     assertEquals(1, indexEvals.size());

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
index 081fa3f..efe2bfd 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
@@ -58,7 +58,7 @@ public abstract class FileAppender implements Appender {
         throw new IllegalArgumentException("Configuration must be an instance of TajoConf");
       }
 
-      Optional<FileTablespace> spaceResult = TableSpaceManager.get(workDir.toUri());
+      Optional<FileTablespace> spaceResult = TablespaceManager.get(workDir.toUri());
 
       if (!spaceResult.isPresent()) {
         throw new IllegalStateException("No TableSpace for " + workDir.toUri());

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
index 2ce1f09..3b63012 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java
@@ -19,14 +19,17 @@
 package org.apache.tajo.storage;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 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.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.tajo.*;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.statistics.TableStats;
@@ -92,8 +95,12 @@ public class FileTablespace extends Tablespace {
         }
       };
 
+  private static final StorageProperty FileStorageProperties = new StorageProperty("TEXT", true, true, true);
+  private static final FormatProperty GeneralFileProperties = new FormatProperty(true, false, true);
+
   protected FileSystem fs;
-  protected Path basePath;
+  protected Path spacePath;
+  protected Path stagingRootPath;
   protected boolean blocksMetadataEnabled;
   private static final HdfsVolumeId zeroVolumeId = new HdfsVolumeId(Bytes.toBytes(0));
 
@@ -103,8 +110,9 @@ public class FileTablespace extends Tablespace {
 
   @Override
   protected void storageInit() throws IOException {
-    this.basePath = new Path(uri);
-    this.fs = basePath.getFileSystem(conf);
+    this.spacePath = new Path(uri);
+    this.fs = spacePath.getFileSystem(conf);
+    this.stagingRootPath = fs.makeQualified(new Path(conf.getVar(TajoConf.ConfVars.STAGING_ROOT_DIR)));
     this.conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, fs.getUri().toString());
 
     this.blocksMetadataEnabled =
@@ -167,7 +175,7 @@ public class FileTablespace extends Tablespace {
 
   @Override
   public URI getTableUri(String databaseName, String tableName) {
-    return StorageUtil.concatPath(basePath, databaseName, tableName).toUri();
+    return StorageUtil.concatPath(spacePath, databaseName, tableName).toUri();
   }
 
   private String partitionPath = "";
@@ -192,12 +200,12 @@ public class FileTablespace extends Tablespace {
   }
 
   public FileFragment[] split(String tableName) throws IOException {
-    Path tablePath = new Path(basePath, tableName);
+    Path tablePath = new Path(spacePath, tableName);
     return split(tableName, tablePath, fs.getDefaultBlockSize());
   }
 
   public FileFragment[] split(String tableName, long fragmentSize) throws IOException {
-    Path tablePath = new Path(basePath, tableName);
+    Path tablePath = new Path(spacePath, tableName);
     return split(tableName, tablePath, fragmentSize);
   }
 
@@ -491,30 +499,6 @@ public class FileTablespace extends Tablespace {
   }
 
   /**
-   * Generate the map of host and make them into Volume Ids.
-   *
-   */
-  private Map<String, Set<Integer>> getVolumeMap(List<FileFragment> frags) {
-    Map<String, Set<Integer>> volumeMap = new HashMap<String, Set<Integer>>();
-    for (FileFragment frag : frags) {
-      String[] hosts = frag.getHosts();
-      int[] diskIds = frag.getDiskIds();
-      for (int i = 0; i < hosts.length; i++) {
-        Set<Integer> volumeList = volumeMap.get(hosts[i]);
-        if (volumeList == null) {
-          volumeList = new HashSet<Integer>();
-          volumeMap.put(hosts[i], volumeList);
-        }
-
-        if (diskIds.length > 0 && diskIds[i] > -1) {
-          volumeList.add(diskIds[i]);
-        }
-      }
-    }
-
-    return volumeMap;
-  }
-  /**
    * Generate the list of files and make them into FileSplits.
    *
    * @throws IOException
@@ -674,7 +658,7 @@ public class FileTablespace extends Tablespace {
       String simpleTableName = splitted[1];
 
       // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} )
-      Path tablePath = StorageUtil.concatPath(basePath, databaseName, simpleTableName);
+      Path tablePath = StorageUtil.concatPath(spacePath, databaseName, simpleTableName);
       tableDesc.setUri(tablePath.toUri());
     } else {
       Preconditions.checkState(tableDesc.getUri() != null, "ERROR: LOCATION must be given.");
@@ -851,22 +835,14 @@ public class FileTablespace extends Tablespace {
     }
   }
 
-  private static final StorageProperty FileStorageProperties = new StorageProperty(true, true, true, true);
-  private static final FormatProperty GeneralFileProperties = new FormatProperty(false);
-  private static final FormatProperty HFileProperties = new FormatProperty(true);
-
   @Override
   public StorageProperty getProperty() {
     return FileStorageProperties;
   }
 
   @Override
-  public FormatProperty getFormatProperty(String format) {
-    if (format.equalsIgnoreCase("hbase")) {
-      return HFileProperties;
-    } else {
-      return GeneralFileProperties;
-    }
+  public FormatProperty getFormatProperty(TableMeta meta) {
+    return GeneralFileProperties;
   }
 
   @Override
@@ -882,6 +858,84 @@ public class FileTablespace extends Tablespace {
   }
 
   @Override
+  public URI getStagingUri(OverridableConf context, String queryId, TableMeta meta) throws IOException {
+    String outputPath = context.get(QueryVars.OUTPUT_TABLE_URI, "");
+
+    Path stagingDir;
+    // The fact that there is no output means that this query is neither CTAS or INSERT (OVERWRITE) INTO
+    // So, this query results won't be materialized as a part of a table.
+    // The result will be temporarily written in the staging directory.
+    if (outputPath.isEmpty()) {
+      // for temporarily written in the storage directory
+      stagingDir = fs.makeQualified(new Path(stagingRootPath, queryId));
+    } else {
+      Optional<Tablespace> spaceResult = TablespaceManager.get(outputPath);
+      if (!spaceResult.isPresent()) {
+        throw new IOException("No registered Tablespace for " + outputPath);
+      }
+
+      Tablespace space = spaceResult.get();
+      if (space.getProperty().isMovable()) { // checking if this tablespace allows MOVE operation
+        // If this space allows move operation, the staging directory will be underneath the final output table uri.
+        stagingDir = fs.makeQualified(StorageUtil.concatPath(outputPath, TMP_STAGING_DIR_PREFIX, queryId));
+      } else {
+        stagingDir = fs.makeQualified(new Path(stagingRootPath, queryId));
+      }
+    }
+
+    return stagingDir.toUri();
+  }
+
+  // query submission directory is private!
+  final public static FsPermission STAGING_DIR_PERMISSION = FsPermission.createImmutable((short) 0700); // rwx--------
+  public static final String TMP_STAGING_DIR_PREFIX = ".staging";
+
+  public URI prepareStagingSpace(TajoConf conf, String queryId, OverridableConf context, TableMeta meta)
+      throws IOException {
+
+    String realUser;
+    String currentUser;
+    UserGroupInformation ugi;
+    ugi = UserGroupInformation.getLoginUser();
+    realUser = ugi.getShortUserName();
+    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+
+    Path stagingDir = new Path(getStagingUri(context, queryId, meta));
+
+    ////////////////////////////////////////////
+    // Create Output Directory
+    ////////////////////////////////////////////
+
+    if (fs.exists(stagingDir)) {
+      throw new IOException("The staging directory '" + stagingDir + "' already exists");
+    }
+    fs.mkdirs(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+    FileStatus fsStatus = fs.getFileStatus(stagingDir);
+    String owner = fsStatus.getOwner();
+
+    if (!owner.isEmpty() && !(owner.equals(currentUser) || owner.equals(realUser))) {
+      throw new IOException("The ownership on the user's query " +
+          "directory " + stagingDir + " is not as expected. " +
+          "It is owned by " + owner + ". The directory must " +
+          "be owned by the submitter " + currentUser + " or " +
+          "by " + realUser);
+    }
+
+    if (!fsStatus.getPermission().equals(STAGING_DIR_PERMISSION)) {
+      LOG.info("Permissions on staging directory " + stagingDir + " are " +
+          "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
+          "to correct value " + STAGING_DIR_PERMISSION);
+      fs.setPermission(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+    }
+
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    fs.mkdirs(stagingResultDir);
+
+    return stagingDir.toUri();
+  }
+
+  @Override
   public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws IOException {
   }
 
@@ -1257,4 +1311,6 @@ public class FileTablespace extends Tablespace {
 
     return retValue;
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
index bd5502d..1d32291 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
@@ -85,7 +85,7 @@ public class HashShuffleAppenderManager {
           fs.mkdirs(dataFile.getParent());
         }
 
-        FileTablespace space = (FileTablespace) TableSpaceManager.get(dataFile.toUri()).get();
+        FileTablespace space = (FileTablespace) TablespaceManager.get(dataFile.toUri()).get();
         FileAppender appender = (FileAppender) space.getAppender(meta, outSchema, dataFile);
         appender.enableStats();
         appender.init();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
index ab63d55..f50a20d 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
@@ -122,7 +122,7 @@ public class TestCompressionStorages {
 
     String fileName = "Compression_" + codec.getSimpleName();
     Path tablePath = new Path(testDir, fileName);
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.enableStats();
 
     appender.init();
@@ -154,7 +154,7 @@ public class TestCompressionStorages {
     FileFragment[] tablets = new FileFragment[1];
     tablets[0] = new FileFragment(fileName, tablePath, 0, fileLen);
 
-    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, tablets[0], schema);
+    Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, schema, tablets[0], schema);
 
     if (storeType.equalsIgnoreCase("CSV")) {
       if (SplittableCompressionCodec.class.isAssignableFrom(codec)) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
index 2d919cd..ca5885c 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java
@@ -103,7 +103,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "-1");
     FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
-    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner =  TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple tuple;
@@ -125,7 +125,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "1");
     FileFragment fragment =  getFileFragment("testErrorTolerance1.json");
-    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner =  TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     assertNotNull(scanner.next());
@@ -147,7 +147,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "0");
     FileFragment fragment =  getFileFragment("testErrorTolerance2.json");
-    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner =  TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     try {
@@ -166,7 +166,7 @@ public class TestDelimitedTextFile {
     TableMeta meta = CatalogUtil.newTableMeta("JSON");
     meta.putOption(StorageUtil.TEXT_ERROR_TOLERANCE_MAXNUM, "1");
     FileFragment fragment = getFileFragment("testErrorTolerance3.json");
-    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     try {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
index 9237e07..1119968 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
@@ -57,7 +57,7 @@ public class TestFileSystems {
   public TestFileSystems(FileSystem fs) throws IOException {
     this.fs = fs;
     this.conf = new TajoConf(fs.getConf());
-    sm = TableSpaceManager.getLocalFs();
+    sm = TablespaceManager.getLocalFs();
     testDir = getTestDir(this.fs, TEST_PATH);
   }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java
index ec3e143..09b91ea 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java
@@ -81,7 +81,7 @@ public class TestFileTablespace {
 
     Path path = StorageUtil.concatPath(testDir, "testGetScannerAndAppender", "table.csv");
     localFs.mkdirs(path.getParent());
-    FileTablespace fileStorageManager = (FileTablespace) TableSpaceManager.getLocalFs();
+    FileTablespace fileStorageManager = (FileTablespace) TablespaceManager.getLocalFs();
     assertEquals(localFs.getUri(), fileStorageManager.getFileSystem().getUri());
 
 		Appender appender = fileStorageManager.getAppender(meta, schema, path);
@@ -224,24 +224,24 @@ public class TestFileTablespace {
     Optional<Tablespace> existingTs = Optional.absent();
     try {
       /* Local FileSystem */
-      FileTablespace space = TableSpaceManager.getLocalFs();
+      FileTablespace space = TablespaceManager.getLocalFs();
       assertEquals(localFs.getUri(), space.getFileSystem().getUri());
 
       FileTablespace distTablespace = new FileTablespace("testGetFileTablespace", uri);
       distTablespace.init(conf);
-      existingTs = TableSpaceManager.addTableSpaceForTest(distTablespace);
+      existingTs = TablespaceManager.addTableSpaceForTest(distTablespace);
 
       /* Distributed FileSystem */
-      space = (FileTablespace) TableSpaceManager.get(uri).get();
+      space = (FileTablespace) TablespaceManager.get(uri).get();
       assertEquals(cluster.getFileSystem().getUri(), space.getFileSystem().getUri());
 
-      space = (FileTablespace) TableSpaceManager.getByName("testGetFileTablespace").get();
+      space = (FileTablespace) TablespaceManager.getByName("testGetFileTablespace").get();
       assertEquals(cluster.getFileSystem().getUri(), space.getFileSystem().getUri());
 
     } finally {
 
       if (existingTs.isPresent()) {
-        TableSpaceManager.addTableSpaceForTest(existingTs.get());
+        TablespaceManager.addTableSpaceForTest(existingTs.get());
       }
 
       cluster.shutdown(true);

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
index c13ce16..7410778 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestLineReader.java
@@ -65,7 +65,7 @@ public class TestLineReader {
 
     TableMeta meta = CatalogUtil.newTableMeta("TEXT");
     Path tablePath = new Path(testDir, "line.data");
-    FileAppender appender = (FileAppender) TableSpaceManager.getLocalFs().getAppender(
+    FileAppender appender = (FileAppender) TablespaceManager.getLocalFs().getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -118,7 +118,7 @@ public class TestLineReader {
     meta.putOption("compression.codec", DeflateCodec.class.getCanonicalName());
 
     Path tablePath = new Path(testDir, "testLineDelimitedReaderWithCompression." + DeflateCodec.class.getSimpleName());
-    FileAppender appender = (FileAppender) (TableSpaceManager.getLocalFs()).getAppender(
+    FileAppender appender = (FileAppender) (TablespaceManager.getLocalFs()).getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -176,7 +176,7 @@ public class TestLineReader {
     TableMeta meta = CatalogUtil.newTableMeta("TEXT");
 
     Path tablePath = new Path(testDir, "testLineDelimitedReader");
-    FileAppender appender = (FileAppender) TableSpaceManager.getLocalFs().getAppender(
+    FileAppender appender = (FileAppender) TablespaceManager.getLocalFs().getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -279,7 +279,7 @@ public class TestLineReader {
 
     TableMeta meta = CatalogUtil.newTableMeta("TEXT");
     Path tablePath = new Path(testDir, "testSeekableByteBufLineReader.data");
-    FileAppender appender = (FileAppender) TableSpaceManager.getLocalFs().getAppender(
+    FileAppender appender = (FileAppender) TablespaceManager.getLocalFs().getAppender(
         null, null, meta, schema, tablePath);
     appender.enableStats();
     appender.init();

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
index 79928ff..331d3e8 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
@@ -94,7 +94,7 @@ public class TestMergeScanner {
     conf.setStrings("tajo.storage.projectable-scanner", "rcfile", "parquet", "avro");
     testDir = CommonTestingUtil.getTestDir(TEST_PATH);
     fs = testDir.getFileSystem(conf);
-    sm = TableSpaceManager.getLocalFs();
+    sm = TablespaceManager.getLocalFs();
   }
 
   @Test
@@ -114,7 +114,7 @@ public class TestMergeScanner {
     }
 
     Path table1Path = new Path(testDir, storeType + "_1.data");
-    Appender appender1 = TableSpaceManager.getLocalFs().getAppender(null, null, meta, schema, table1Path);
+    Appender appender1 = TablespaceManager.getLocalFs().getAppender(null, null, meta, schema, table1Path);
     appender1.enableStats();
     appender1.init();
     int tupleNum = 10000;
@@ -136,7 +136,7 @@ public class TestMergeScanner {
     }
 
     Path table2Path = new Path(testDir, storeType + "_2.data");
-    Appender appender2 = TableSpaceManager.getLocalFs().getAppender(null, null, meta, schema, table2Path);
+    Appender appender2 = TablespaceManager.getLocalFs().getAppender(null, null, meta, schema, table2Path);
     appender2.enableStats();
     appender2.init();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
index ce2a926..dbfdac3 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -155,7 +155,7 @@ public class TestStorages {
 
       TableMeta meta = CatalogUtil.newTableMeta(storeType);
       Path tablePath = new Path(testDir, "Splitable.data");
-      FileTablespace sm = TableSpaceManager.getLocalFs();
+      FileTablespace sm = TablespaceManager.getLocalFs();
       Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.enableStats();
       appender.init();
@@ -210,7 +210,7 @@ public class TestStorages {
 
       TableMeta meta = CatalogUtil.newTableMeta(storeType);
       Path tablePath = new Path(testDir, "Splitable.data");
-      FileTablespace sm = TableSpaceManager.getLocalFs();
+      FileTablespace sm = TablespaceManager.getLocalFs();
       Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.enableStats();
       appender.init();
@@ -271,7 +271,7 @@ public class TestStorages {
     }
 
     Path tablePath = new Path(testDir, "testProjection.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
     int tupleNum = 10000;
@@ -347,7 +347,7 @@ public class TestStorages {
       meta.putOption(StorageConstants.AVRO_SCHEMA_URL, path);
     }
 
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Path tablePath = new Path(testDir, "testVariousTypes.data");
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
@@ -425,7 +425,7 @@ public class TestStorages {
     }
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.init();
 
@@ -469,7 +469,7 @@ public class TestStorages {
 
     FileStatus status = fs.getFileStatus(tablePath);
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -513,7 +513,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.CSVFILE_SERDE, TextSerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -544,7 +544,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner =  TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -582,7 +582,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.RCFILE_SERDE, BinarySerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -613,7 +613,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner =  TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple retrieved;
@@ -651,7 +651,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -682,7 +682,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner =  TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     assertTrue(scanner instanceof SequenceFileScanner);
@@ -724,7 +724,7 @@ public class TestStorages {
     meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, BinarySerializerDeserializer.class.getName());
 
     Path tablePath = new Path(testDir, "testVariousTypes.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -756,7 +756,7 @@ public class TestStorages {
     assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     assertTrue(scanner instanceof SequenceFileScanner);
@@ -786,7 +786,7 @@ public class TestStorages {
       TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
 
       Path tablePath = new Path(testDir, "testTime.data");
-      FileTablespace sm = TableSpaceManager.getLocalFs();
+      FileTablespace sm = TablespaceManager.getLocalFs();
       Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.init();
 
@@ -801,7 +801,7 @@ public class TestStorages {
 
       FileStatus status = fs.getFileStatus(tablePath);
       FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-      Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+      Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
       scanner.init();
 
       Tuple retrieved;
@@ -827,7 +827,7 @@ public class TestStorages {
 
     TableMeta meta = CatalogUtil.newTableMeta(storeType);
     Path tablePath = new Path(testDir, "Seekable.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     FileAppender appender = (FileAppender) sm.getAppender(meta, schema, tablePath);
     appender.enableStats();
     appender.init();
@@ -869,7 +869,7 @@ public class TestStorages {
     long readBytes = 0;
     long readRows = 0;
     for (long offset : offsets) {
-      scanner = TableSpaceManager.getLocalFs().getScanner(meta, schema,
+      scanner = TablespaceManager.getLocalFs().getScanner(meta, schema,
 	        new FileFragment("table", tablePath, prevOffset, offset - prevOffset), schema);
       scanner.init();
 
@@ -917,7 +917,7 @@ public class TestStorages {
       conf.setInt(RawFile.WRITE_BUFFER_SIZE, record + headerSize);
     }
 
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Path tablePath = new Path(testDir, "testMaxValue.data");
     Appender appender = sm.getAppender(meta, schema, tablePath);
 
@@ -972,7 +972,7 @@ public class TestStorages {
     meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
 
     Path tablePath = new Path(testDir, "testLessThanSchemaSize.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, dataSchema, tablePath);
     appender.init();
 
@@ -998,7 +998,7 @@ public class TestStorages {
     inSchema.addColumn("col5", Type.INT8);
 
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = TableSpaceManager.getLocalFs().getScanner(meta, inSchema, fragment);
+    Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, inSchema, fragment);
 
     Schema target = new Schema();
 
@@ -1036,7 +1036,7 @@ public class TestStorages {
     meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
 
     Path tablePath = new Path(testDir, "test_storetype_oversize.data");
-    FileTablespace sm = TableSpaceManager.getLocalFs();
+    FileTablespace sm = TablespaceManager.getLocalFs();
     Appender appender = sm.getAppender(meta, dataSchema, tablePath);
     appender.init();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/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
index 1a62f52..22fb607 100644
--- a/tajo-storage/tajo-storage-hdfs/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 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindValue_" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
@@ -178,7 +178,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testBuildIndexWithAppender_" + storeType);
-    FileAppender appender = (FileAppender) ((FileTablespace)TableSpaceManager.getLocalFs())
+    FileAppender appender = (FileAppender) ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
 
@@ -257,7 +257,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = StorageUtil.concatPath(testDir, "testFindOmittedValue_" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i += 2) {
@@ -327,7 +327,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyValue_" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
@@ -418,7 +418,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyOmittedValue_" + storeType);
-    Appender appender = (((FileTablespace)TableSpaceManager.getLocalFs()))
+    Appender appender = (((FileTablespace) TablespaceManager.getLocalFs()))
         .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
@@ -498,7 +498,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindMinValue" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
 
@@ -582,7 +582,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testMinMax_" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
@@ -687,7 +687,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testConcurrentAccess_" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
 
@@ -768,7 +768,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindValueDescOrder_" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs())
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs())
         .getAppender(meta, schema, tablePath);
     appender.init();
 
@@ -860,7 +860,7 @@ public class TestBSTIndex {
     meta = CatalogUtil.newTableMeta(storeType);
 
     Path tablePath = new Path(testDir, "testFindNextKeyValueDescOrder_" + storeType);
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
 
     Tuple tuple;

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/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
index 5ad7a27..72810fd 100644
--- a/tajo-storage/tajo-storage-hdfs/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
@@ -77,7 +77,7 @@ public class TestSingleCSVFileBSTIndex {
     Path tablePath = StorageUtil.concatPath(testDir, "testFindValueInSingleCSV", "table.csv");
     fs.mkdirs(tablePath.getParent());
 
-    Appender appender = ((FileTablespace)TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for (int i = 0; i < TUPLE_NUM; i++) {
@@ -166,7 +166,7 @@ public class TestSingleCSVFileBSTIndex {
     Path tablePath = StorageUtil.concatPath(testDir, "testFindNextKeyValueInSingleCSV",
         "table1.csv");
     fs.mkdirs(tablePath.getParent());
-    Appender appender = ((FileTablespace) TableSpaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
+    Appender appender = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple;
     for(int i = 0 ; i < TUPLE_NUM; i ++ ) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/90afaa46/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/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
index 2fbf5d6..8095081 100644
--- a/tajo-storage/tajo-storage-hdfs/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
@@ -69,7 +69,7 @@ public class TestJsonSerDe {
     FileSystem fs = FileSystem.getLocal(conf);
     FileStatus status = fs.getFileStatus(tablePath);
     FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  TableSpaceManager.getLocalFs().getScanner(meta, schema, fragment);
+    Scanner scanner =  TablespaceManager.getLocalFs().getScanner(meta, schema, fragment);
     scanner.init();
 
     Tuple tuple = scanner.next();


[13/15] tajo git commit: TAJO-1660: Update copyright year in NOTICE.

Posted by ji...@apache.org.
TAJO-1660: Update copyright year in NOTICE.


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

Branch: refs/heads/index_support
Commit: 03bf8430782a1a57de8b8cdd397fecd61c0a4fe5
Parents: 90afaa4
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Jun 24 18:07:59 2015 -0700
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Jun 24 18:07:59 2015 -0700

----------------------------------------------------------------------
 NOTICE | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/03bf8430/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index af7995f..8cc30e4 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Tajo
-Copyright 2014 The Apache Software Foundation
+Copyright 2013-2015 The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).