You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hj...@apache.org on 2014/12/03 06:30:16 UTC

[01/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Repository: tajo
Updated Branches:
  refs/heads/hbase_storage 87c957e43 -> dfd7f996d


http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
new file mode 100644
index 0000000..517e00e
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
@@ -0,0 +1,130 @@
+/**
+ * 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.parquet;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.junit.Test;
+import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link TajoSchemaConverter}.
+ */
+public class TestSchemaConverter {
+  private static final String ALL_PARQUET_SCHEMA =
+      "message table_schema {\n" +
+      "  optional boolean myboolean;\n" +
+      "  optional int32 myint;\n" +
+      "  optional int64 mylong;\n" +
+      "  optional float myfloat;\n" +
+      "  optional double mydouble;\n" +
+      "  optional binary mybytes;\n" +
+      "  optional binary mystring (UTF8);\n" +
+      "  optional fixed_len_byte_array(1) myfixed;\n" +
+      "}\n";
+
+  private static final String CONVERTED_ALL_PARQUET_SCHEMA =
+      "message table_schema {\n" +
+      "  optional boolean myboolean;\n" +
+      "  optional int32 mybit;\n" +
+      "  optional binary mychar (UTF8);\n" +
+      "  optional int32 myint2;\n" +
+      "  optional int32 myint4;\n" +
+      "  optional int64 myint8;\n" +
+      "  optional float myfloat4;\n" +
+      "  optional double myfloat8;\n" +
+      "  optional binary mytext (UTF8);\n" +
+      "  optional binary myblob;\n" +
+      // NULL_TYPE fields are not encoded.
+      "  optional binary myinet4;\n" +
+      "  optional binary myprotobuf;\n" +
+      "}\n";
+
+  private Schema createAllTypesSchema() {
+    List<Column> columns = new ArrayList<Column>();
+    columns.add(new Column("myboolean", Type.BOOLEAN));
+    columns.add(new Column("mybit", Type.BIT));
+    columns.add(new Column("mychar", Type.CHAR));
+    columns.add(new Column("myint2", Type.INT2));
+    columns.add(new Column("myint4", Type.INT4));
+    columns.add(new Column("myint8", Type.INT8));
+    columns.add(new Column("myfloat4", Type.FLOAT4));
+    columns.add(new Column("myfloat8", Type.FLOAT8));
+    columns.add(new Column("mytext", Type.TEXT));
+    columns.add(new Column("myblob", Type.BLOB));
+    columns.add(new Column("mynull", Type.NULL_TYPE));
+    columns.add(new Column("myinet4", Type.INET4));
+    columns.add(new Column("myprotobuf", Type.PROTOBUF));
+    Column[] columnsArray = new Column[columns.size()];
+    columnsArray = columns.toArray(columnsArray);
+    return new Schema(columnsArray);
+  }
+
+  private Schema createAllTypesConvertedSchema() {
+    List<Column> columns = new ArrayList<Column>();
+    columns.add(new Column("myboolean", Type.BOOLEAN));
+    columns.add(new Column("myint", Type.INT4));
+    columns.add(new Column("mylong", Type.INT8));
+    columns.add(new Column("myfloat", Type.FLOAT4));
+    columns.add(new Column("mydouble", Type.FLOAT8));
+    columns.add(new Column("mybytes", Type.BLOB));
+    columns.add(new Column("mystring", Type.TEXT));
+    columns.add(new Column("myfixed", Type.BLOB));
+    Column[] columnsArray = new Column[columns.size()];
+    columnsArray = columns.toArray(columnsArray);
+    return new Schema(columnsArray);
+  }
+
+  private void testTajoToParquetConversion(
+      Schema tajoSchema, String schemaString) throws Exception {
+    TajoSchemaConverter converter = new TajoSchemaConverter();
+    MessageType schema = converter.convert(tajoSchema);
+    MessageType expected = MessageTypeParser.parseMessageType(schemaString);
+    assertEquals("converting " + schema + " to " + schemaString,
+                 expected.toString(), schema.toString());
+  }
+
+  private void testParquetToTajoConversion(
+      Schema tajoSchema, String schemaString) throws Exception {
+    TajoSchemaConverter converter = new TajoSchemaConverter();
+    Schema schema = converter.convert(
+        MessageTypeParser.parseMessageType(schemaString));
+    assertEquals("converting " + schemaString + " to " + tajoSchema,
+                 tajoSchema.toString(), schema.toString());
+  }
+
+  @Test
+  public void testAllTypesToParquet() throws Exception {
+    Schema schema = createAllTypesSchema();
+    testTajoToParquetConversion(schema, CONVERTED_ALL_PARQUET_SCHEMA);
+  }
+
+  @Test
+  public void testAllTypesToTajo() throws Exception {
+    Schema schema = createAllTypesConvertedSchema();
+    testParquetToTajoConversion(schema, ALL_PARQUET_SCHEMA);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/INode.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/INode.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/INode.java
new file mode 100644
index 0000000..7b09937
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/INode.java
@@ -0,0 +1,124 @@
+/**
+ * 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.s3;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.s3.Block;
+import org.apache.hadoop.io.IOUtils;
+
+import java.io.*;
+
+/**
+ * Holds file metadata including type (regular file, or directory),
+ * and the list of blocks that are pointers to the data.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class INode {
+
+  enum FileType {
+    DIRECTORY, FILE
+  }
+
+  public static final FileType[] FILE_TYPES = {
+      FileType.DIRECTORY,
+      FileType.FILE
+  };
+
+  public static final INode DIRECTORY_INODE = new INode(FileType.DIRECTORY, null);
+
+  private FileType fileType;
+  private Block[] blocks;
+
+  public INode(FileType fileType, Block[] blocks) {
+    this.fileType = fileType;
+    if (isDirectory() && blocks != null) {
+      throw new IllegalArgumentException("A directory cannot contain blocks.");
+    }
+    this.blocks = blocks;
+  }
+
+  public Block[] getBlocks() {
+    return blocks;
+  }
+
+  public FileType getFileType() {
+    return fileType;
+  }
+
+  public boolean isDirectory() {
+    return fileType == FileType.DIRECTORY;
+  }
+
+  public boolean isFile() {
+    return fileType == FileType.FILE;
+  }
+
+  public long getSerializedLength() {
+    return 1L + (blocks == null ? 0 : 4 + blocks.length * 16);
+  }
+
+
+  public InputStream serialize() throws IOException {
+    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(bytes);
+    try {
+      out.writeByte(fileType.ordinal());
+      if (isFile()) {
+        out.writeInt(blocks.length);
+        for (int i = 0; i < blocks.length; i++) {
+          out.writeLong(blocks[i].getId());
+          out.writeLong(blocks[i].getLength());
+        }
+      }
+      out.close();
+      out = null;
+    } finally {
+      IOUtils.closeStream(out);
+    }
+    return new ByteArrayInputStream(bytes.toByteArray());
+  }
+
+  public static INode deserialize(InputStream in) throws IOException {
+    if (in == null) {
+      return null;
+    }
+    DataInputStream dataIn = new DataInputStream(in);
+    FileType fileType = INode.FILE_TYPES[dataIn.readByte()];
+    switch (fileType) {
+      case DIRECTORY:
+        in.close();
+        return INode.DIRECTORY_INODE;
+      case FILE:
+        int numBlocks = dataIn.readInt();
+        Block[] blocks = new Block[numBlocks];
+        for (int i = 0; i < numBlocks; i++) {
+          long id = dataIn.readLong();
+          long length = dataIn.readLong();
+          blocks[i] = new Block(id, length);
+        }
+        in.close();
+        return new INode(fileType, blocks);
+      default:
+        throw new IllegalArgumentException("Cannot deserialize inode.");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
new file mode 100644
index 0000000..f3ac7b6
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
@@ -0,0 +1,176 @@
+/**
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3.Block;
+import org.apache.hadoop.fs.s3.INode;
+import org.apache.hadoop.fs.s3.FileSystemStore;
+import org.apache.hadoop.fs.s3.S3FileSystem;
+import org.apache.tajo.common.exception.NotImplementedException;
+
+import java.io.*;
+import java.net.URI;
+import java.util.*;
+
+/**
+ * A stub implementation of {@link FileSystemStore} for testing
+ * {@link S3FileSystem} without actually connecting to S3.
+ */
+public class InMemoryFileSystemStore implements FileSystemStore {
+
+  private Configuration conf;
+  private SortedMap<Path, INode> inodes = new TreeMap<Path, INode>();
+  private Map<Long, byte[]> blocks = new HashMap<Long, byte[]>();
+
+  @Override
+  public void initialize(URI uri, Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public String getVersion() throws IOException {
+    return "0";
+  }
+
+  @Override
+  public void deleteINode(Path path) throws IOException {
+    inodes.remove(normalize(path));
+  }
+
+  @Override
+  public void deleteBlock(Block block) throws IOException {
+    blocks.remove(block.getId());
+  }
+
+  @Override
+  public boolean inodeExists(Path path) throws IOException {
+    return inodes.containsKey(normalize(path));
+  }
+
+  @Override
+  public boolean blockExists(long blockId) throws IOException {
+    return blocks.containsKey(blockId);
+  }
+
+  @Override
+  public INode retrieveINode(Path path) throws IOException {
+    return inodes.get(normalize(path));
+  }
+
+  @Override
+  public File retrieveBlock(Block block, long byteRangeStart) throws IOException {
+    byte[] data = blocks.get(block.getId());
+    File file = createTempFile();
+    BufferedOutputStream out = null;
+    try {
+      out = new BufferedOutputStream(new FileOutputStream(file));
+      out.write(data, (int) byteRangeStart, data.length - (int) byteRangeStart);
+    } finally {
+      if (out != null) {
+        out.close();
+      }
+    }
+    return file;
+  }
+
+  private File createTempFile() throws IOException {
+    File dir = new File(conf.get("fs.s3.buffer.dir"));
+    if (!dir.exists() && !dir.mkdirs()) {
+      throw new IOException("Cannot create S3 buffer directory: " + dir);
+    }
+    File result = File.createTempFile("test-", ".tmp", dir);
+    result.deleteOnExit();
+    return result;
+  }
+
+  @Override
+  public Set<Path> listSubPaths(Path path) throws IOException {
+    Path normalizedPath = normalize(path);
+    // This is inefficient but more than adequate for testing purposes.
+    Set<Path> subPaths = new LinkedHashSet<Path>();
+    for (Path p : inodes.tailMap(normalizedPath).keySet()) {
+      if (normalizedPath.equals(p.getParent())) {
+        subPaths.add(p);
+      }
+    }
+    return subPaths;
+  }
+
+  @Override
+  public Set<Path> listDeepSubPaths(Path path) throws IOException {
+    Path normalizedPath = normalize(path);
+    String pathString = normalizedPath.toUri().getPath();
+    if (!pathString.endsWith("/")) {
+      pathString += "/";
+    }
+    // This is inefficient but more than adequate for testing purposes.
+    Set<Path> subPaths = new LinkedHashSet<Path>();
+    for (Path p : inodes.tailMap(normalizedPath).keySet()) {
+      if (p.toUri().getPath().startsWith(pathString)) {
+        subPaths.add(p);
+      }
+    }
+    return subPaths;
+  }
+
+  @Override
+  public void storeINode(Path path, INode inode) throws IOException {
+    inodes.put(normalize(path), inode);
+  }
+
+  @Override
+  public void storeBlock(Block block, File file) throws IOException {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    byte[] buf = new byte[8192];
+    int numRead;
+    BufferedInputStream in = null;
+    try {
+      in = new BufferedInputStream(new FileInputStream(file));
+      while ((numRead = in.read(buf)) >= 0) {
+        out.write(buf, 0, numRead);
+      }
+    } finally {
+      if (in != null) {
+        in.close();
+      }
+    }
+    blocks.put(block.getId(), out.toByteArray());
+  }
+
+  private Path normalize(Path path) {
+    if (!path.isAbsolute()) {
+      throw new IllegalArgumentException("Path must be absolute: " + path);
+    }
+    return new Path(path.toUri().getPath());
+  }
+
+  @Override
+  public void purge() throws IOException {
+    inodes.clear();
+    blocks.clear();
+  }
+
+  @Override
+  public void dump() throws IOException {
+    throw new NotImplementedException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
new file mode 100644
index 0000000..d4034b9
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
@@ -0,0 +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.s3;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3.Block;
+import org.apache.hadoop.fs.s3.FileSystemStore;
+import org.apache.hadoop.fs.s3.INode;
+import org.apache.hadoop.util.Progressable;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class S3OutputStream extends OutputStream {
+
+  private Configuration conf;
+
+  private int bufferSize;
+
+  private FileSystemStore store;
+
+  private Path path;
+
+  private long blockSize;
+
+  private File backupFile;
+
+  private OutputStream backupStream;
+
+  private Random r = new Random();
+
+  private boolean closed;
+
+  private int pos = 0;
+
+  private long filePos = 0;
+
+  private int bytesWrittenToBlock = 0;
+
+  private byte[] outBuf;
+
+  private List<Block> blocks = new ArrayList<Block>();
+
+  private Block nextBlock;
+
+  private static final Log LOG =
+      LogFactory.getLog(S3OutputStream.class.getName());
+
+
+  public S3OutputStream(Configuration conf, FileSystemStore store,
+                        Path path, long blockSize, Progressable progress,
+                        int buffersize) throws IOException {
+
+    this.conf = conf;
+    this.store = store;
+    this.path = path;
+    this.blockSize = blockSize;
+    this.backupFile = newBackupFile();
+    this.backupStream = new FileOutputStream(backupFile);
+    this.bufferSize = buffersize;
+    this.outBuf = new byte[bufferSize];
+
+  }
+
+  private File newBackupFile() throws IOException {
+    File dir = new File(conf.get("fs.s3.buffer.dir"));
+    if (!dir.exists() && !dir.mkdirs()) {
+      throw new IOException("Cannot create S3 buffer directory: " + dir);
+    }
+    File result = File.createTempFile("output-", ".tmp", dir);
+    result.deleteOnExit();
+    return result;
+  }
+
+  public long getPos() throws IOException {
+    return filePos;
+  }
+
+  @Override
+  public synchronized void write(int b) throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+
+    if ((bytesWrittenToBlock + pos == blockSize) || (pos >= bufferSize)) {
+      flush();
+    }
+    outBuf[pos++] = (byte) b;
+    filePos++;
+  }
+
+  @Override
+  public synchronized void write(byte b[], int off, int len) throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+    while (len > 0) {
+      int remaining = bufferSize - pos;
+      int toWrite = Math.min(remaining, len);
+      System.arraycopy(b, off, outBuf, pos, toWrite);
+      pos += toWrite;
+      off += toWrite;
+      len -= toWrite;
+      filePos += toWrite;
+
+      if ((bytesWrittenToBlock + pos >= blockSize) || (pos == bufferSize)) {
+        flush();
+      }
+    }
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed");
+    }
+
+    if (bytesWrittenToBlock + pos >= blockSize) {
+      flushData((int) blockSize - bytesWrittenToBlock);
+    }
+    if (bytesWrittenToBlock == blockSize) {
+      endBlock();
+    }
+    flushData(pos);
+  }
+
+  private synchronized void flushData(int maxPos) throws IOException {
+    int workingPos = Math.min(pos, maxPos);
+
+    if (workingPos > 0) {
+      //
+      // To the local block backup, write just the bytes
+      //
+      backupStream.write(outBuf, 0, workingPos);
+
+      //
+      // Track position
+      //
+      bytesWrittenToBlock += workingPos;
+      System.arraycopy(outBuf, workingPos, outBuf, 0, pos - workingPos);
+      pos -= workingPos;
+    }
+  }
+
+  private synchronized void endBlock() throws IOException {
+    //
+    // Done with local copy
+    //
+    backupStream.close();
+
+    //
+    // Send it to S3
+    //
+    // TODO: Use passed in Progressable to report progress.
+    nextBlockOutputStream();
+    store.storeBlock(nextBlock, backupFile);
+    Block[] arr = new Block[blocks.size()];
+    arr = blocks.toArray(arr);
+    store.storeINode(path, new INode(INode.FILE_TYPES[1], arr));
+
+    //
+    // Delete local backup, start new one
+    //
+    boolean b = backupFile.delete();
+    if (!b) {
+      LOG.warn("Ignoring failed delete");
+    }
+    backupFile = newBackupFile();
+    backupStream = new FileOutputStream(backupFile);
+    bytesWrittenToBlock = 0;
+  }
+
+  private synchronized void nextBlockOutputStream() throws IOException {
+    long blockId = r.nextLong();
+    while (store.blockExists(blockId)) {
+      blockId = r.nextLong();
+    }
+    nextBlock = new Block(blockId, bytesWrittenToBlock);
+    blocks.add(nextBlock);
+    bytesWrittenToBlock = 0;
+  }
+
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+
+    flush();
+    if (filePos == 0 || bytesWrittenToBlock != 0) {
+      endBlock();
+    }
+
+    backupStream.close();
+    boolean b = backupFile.delete();
+    if (!b) {
+      LOG.warn("Ignoring failed delete");
+    }
+
+    super.close();
+
+    closed = true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
new file mode 100644
index 0000000..cb97a74
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
@@ -0,0 +1,314 @@
+/**
+ * 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.s3;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3.Block;
+import org.apache.hadoop.fs.s3.INode;
+import org.apache.hadoop.fs.s3.FileSystemStore;
+import org.apache.hadoop.fs.s3.S3FileSystem;
+import org.apache.hadoop.util.Progressable;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+public class SmallBlockS3FileSystem extends S3FileSystem {
+
+  private URI uri;
+
+  private FileSystemStore store;
+
+  private Path workingDir;
+
+  static class Holder {
+    private static InMemoryFileSystemStore s;
+
+    public synchronized static FileSystemStore get() {
+      if(s != null) {
+        return s;
+      }
+      s = new InMemoryFileSystemStore();
+      return s;
+    }
+
+    public synchronized static void set(InMemoryFileSystemStore inMemoryFileSystemStore) {
+      s = inMemoryFileSystemStore;
+    }
+  }
+
+  public SmallBlockS3FileSystem() {
+  }
+
+
+  public SmallBlockS3FileSystem(
+      InMemoryFileSystemStore inMemoryFileSystemStore) {
+    Holder.set(inMemoryFileSystemStore);
+    this.store = inMemoryFileSystemStore;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+  @Override
+  public long getDefaultBlockSize() {
+    return 10;
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    if (store == null) {
+      store = Holder.get();
+    }
+    store.initialize(uri, conf);
+    setConf(conf);
+    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.workingDir =
+        new Path("/user", System.getProperty("user.name")).makeQualified(this);
+  }
+  @Override
+  public boolean isFile(Path path) throws IOException {
+    INode inode = store.retrieveINode(makeAbsolute(path));
+    if (inode == null) {
+      return false;
+    }
+    return inode.isFile();
+  }
+
+  private INode checkFile(Path path) throws IOException {
+    INode inode = store.retrieveINode(makeAbsolute(path));
+    if (inode == null) {
+      throw new IOException("No such file.");
+    }
+    if (inode.isDirectory()) {
+      throw new IOException("Path " + path + " is a directory.");
+    }
+    return inode;
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+    Path absolutePath = makeAbsolute(f);
+    INode inode = store.retrieveINode(absolutePath);
+    if (inode == null) {
+      throw new FileNotFoundException("File " + f + " does not exist.");
+    }
+    if (inode.isFile()) {
+      return new FileStatus[] {
+          new S3FileStatus(f.makeQualified(this), inode)
+      };
+    }
+    ArrayList<FileStatus> ret = new ArrayList<FileStatus>();
+    for (Path p : store.listSubPaths(absolutePath)) {
+      ret.add(getFileStatus(p.makeQualified(this)));
+    }
+    return ret.toArray(new FileStatus[0]);
+  }
+  @Override
+  public FSDataOutputStream create(Path file, FsPermission permission,
+                                   boolean overwrite, int bufferSize,
+                                   short replication, long blockSize, Progressable progress)
+      throws IOException {
+
+    INode inode = store.retrieveINode(makeAbsolute(file));
+    if (inode != null) {
+      if (overwrite) {
+        delete(file, true);
+      } else {
+        throw new IOException("File already exists: " + file);
+      }
+    } else {
+      Path parent = file.getParent();
+      if (parent != null) {
+        if (!mkdirs(parent)) {
+          throw new IOException("Mkdirs failed to create " + parent.toString());
+        }
+      }
+    }
+    return new FSDataOutputStream
+        (new S3OutputStream(getConf(), store, makeAbsolute(file),
+            blockSize, progress, bufferSize),
+            statistics);
+  }
+  @Override
+  public boolean mkdirs(Path path, FsPermission permission) throws IOException {
+    Path absolutePath = makeAbsolute(path);
+    List<Path> paths = new ArrayList<Path>();
+    do {
+      paths.add(0, absolutePath);
+      absolutePath = absolutePath.getParent();
+    } while (absolutePath != null);
+
+    boolean result = true;
+    for (Path p : paths) {
+      result &= mkdir(p);
+    }
+    return result;
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    Path absoluteSrc = makeAbsolute(src);
+    INode srcINode = store.retrieveINode(absoluteSrc);
+    if (srcINode == null) {
+      // src path doesn't exist
+      return false;
+    }
+    Path absoluteDst = makeAbsolute(dst);
+    INode dstINode = store.retrieveINode(absoluteDst);
+    if (dstINode != null && dstINode.isDirectory()) {
+      absoluteDst = new Path(absoluteDst, absoluteSrc.getName());
+      dstINode = store.retrieveINode(absoluteDst);
+    }
+    if (dstINode != null) {
+      // dst path already exists - can't overwrite
+      return false;
+    }
+    Path dstParent = absoluteDst.getParent();
+    if (dstParent != null) {
+      INode dstParentINode = store.retrieveINode(dstParent);
+      if (dstParentINode == null || dstParentINode.isFile()) {
+        // dst parent doesn't exist or is a file
+        return false;
+      }
+    }
+    return renameRecursive(absoluteSrc, absoluteDst);
+  }
+
+  private boolean renameRecursive(Path src, Path dst) throws IOException {
+    INode srcINode = store.retrieveINode(src);
+    store.storeINode(dst, srcINode);
+    store.deleteINode(src);
+    if (srcINode.isDirectory()) {
+      for (Path oldSrc : store.listDeepSubPaths(src)) {
+        INode inode = store.retrieveINode(oldSrc);
+        if (inode == null) {
+          return false;
+        }
+        String oldSrcPath = oldSrc.toUri().getPath();
+        String srcPath = src.toUri().getPath();
+        String dstPath = dst.toUri().getPath();
+        Path newDst = new Path(oldSrcPath.replaceFirst(srcPath, dstPath));
+        store.storeINode(newDst, inode);
+        store.deleteINode(oldSrc);
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public boolean delete(Path path, boolean recursive) throws IOException {
+    Path absolutePath = makeAbsolute(path);
+    INode inode = store.retrieveINode(absolutePath);
+    if (inode == null) {
+      return false;
+    }
+    if (inode.isFile()) {
+      store.deleteINode(absolutePath);
+      for (Block block: inode.getBlocks()) {
+        store.deleteBlock(block);
+      }
+    } else {
+      FileStatus[] contents = null;
+      try {
+        contents = listStatus(absolutePath);
+      } catch(FileNotFoundException fnfe) {
+        return false;
+      }
+
+      if ((contents.length !=0) && (!recursive)) {
+        throw new IOException("Directory " + path.toString()
+            + " is not empty.");
+      }
+      for (FileStatus p:contents) {
+        if (!delete(p.getPath(), recursive)) {
+          return false;
+        }
+      }
+      store.deleteINode(absolutePath);
+    }
+    return true;
+  }
+
+  /**
+   * FileStatus for S3 file systems.
+   */
+  @Override
+  public FileStatus getFileStatus(Path f)  throws IOException {
+    INode inode = store.retrieveINode(makeAbsolute(f));
+    if (inode == null) {
+      throw new FileNotFoundException(f + ": No such file or directory.");
+    }
+    return new S3FileStatus(f.makeQualified(this), inode);
+  }
+  private boolean mkdir(Path path) throws IOException {
+    Path absolutePath = makeAbsolute(path);
+    INode inode = store.retrieveINode(absolutePath);
+    if (inode == null) {
+      store.storeINode(absolutePath, INode.DIRECTORY_INODE);
+    } else if (inode.isFile()) {
+      throw new IOException(String.format(
+          "Can't make directory for path %s since it is a file.",
+          absolutePath));
+    }
+    return true;
+  }
+  private Path makeAbsolute(Path path) {
+    if (path.isAbsolute()) {
+      return path;
+    }
+    return new Path(workingDir, path);
+  }
+
+  private static class S3FileStatus extends FileStatus {
+
+    S3FileStatus(Path f, INode inode) throws IOException {
+      super(findLength(inode), inode.isDirectory(), 1,
+          findBlocksize(inode), 0, f);
+    }
+
+    private static long findLength(INode inode) {
+      if (!inode.isDirectory()) {
+        long length = 0L;
+        for (Block block : inode.getBlocks()) {
+          length += block.getLength();
+        }
+        return length;
+      }
+      return 0;
+    }
+
+    private static long findBlocksize(INode inode) {
+      final Block[] ret = inode.getBlocks();
+      return ret == null ? 0L : ret[0].getLength();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
new file mode 100644
index 0000000..6190d1a
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
@@ -0,0 +1,164 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+  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.
+  -->
+
+<configuration>
+  <property>
+    <name>fs.s3.impl</name>
+    <value>org.apache.tajo.storage.s3.SmallBlockS3FileSystem</value>
+  </property>
+
+  <!-- Storage Manager Configuration -->
+  <property>
+    <name>tajo.storage.manager.hdfs.class</name>
+    <value>org.apache.tajo.storage.FileStorageManager</value>
+  </property>
+  <property>
+    <name>tajo.storage.manager.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
+  </property>
+
+  <!--- Registered Scanner Handler -->
+  <property>
+    <name>tajo.storage.scanner-handler</name>
+    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
+  </property>
+
+  <!--- Fragment Class Configurations -->
+  <property>
+    <name>tajo.storage.fragment.textfile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.csv.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.raw.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.rcfile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.row.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.parquet.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.sequencefile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.avro.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+
+  <!--- Scanner Handler -->
+  <property>
+    <name>tajo.storage.scanner-handler.textfile.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.csv.class</name>
+    <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.raw.class</name>
+    <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.rcfile.class</name>
+    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.rowfile.class</name>
+    <value>org.apache.tajo.storage.RowFile$RowFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.parquet.class</name>
+    <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.sequencefile.class</name>
+    <value>org.apache.tajo.storage.sequencefile.SequenceFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.avro.class</name>
+    <value>org.apache.tajo.storage.avro.AvroScanner</value>
+  </property>
+
+  <!--- Appender Handler -->
+  <property>
+    <name>tajo.storage.appender-handler</name>
+    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.textfile.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.csv.class</name>
+    <value>org.apache.tajo.storage.CSVFile$CSVAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.raw.class</name>
+    <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.rcfile.class</name>
+    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.rowfile.class</name>
+    <value>org.apache.tajo.storage.RowFile$RowFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.parquet.class</name>
+    <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.sequencefile.class</name>
+    <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.avro.class</name>
+    <value>org.apache.tajo.storage.avro.AvroAppender</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/resources/testVariousTypes.avsc
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/testVariousTypes.avsc b/tajo-storage/tajo-storage-hdfs/src/test/resources/testVariousTypes.avsc
new file mode 100644
index 0000000..611b97f
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/testVariousTypes.avsc
@@ -0,0 +1,21 @@
+{
+  "type": "record",
+  "namespace": "org.apache.tajo",
+  "name": "testVariousTypes",
+  "fields": [
+    { "name": "col1", "type": "boolean" },
+    { "name": "col2", "type": "int" },
+    { "name": "col3", "type": "string" },
+    { "name": "col4", "type": "int" },
+    { "name": "col5", "type": "int" },
+    { "name": "col6", "type": "long" },
+    { "name": "col7", "type": "float" },
+    { "name": "col8", "type": "double" },
+    { "name": "col9", "type": "string" },
+    { "name": "col10", "type": "bytes" },
+    { "name": "col11", "type": "bytes" },
+    { "name": "col12", "type": "null" },
+    { "name": "col13", "type": "bytes" }
+  ]
+}
+

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-yarn-pullserver/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-yarn-pullserver/pom.xml b/tajo-yarn-pullserver/pom.xml
index a7644a1..3daec5c 100644
--- a/tajo-yarn-pullserver/pom.xml
+++ b/tajo-yarn-pullserver/pom.xml
@@ -57,7 +57,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
       <scope>provided</scope>
     </dependency>
     <dependency>


[13/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableStatistics.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableStatistics.java
new file mode 100644
index 0000000..a2c08de
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TableStatistics.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+
+/**
+ * This class is not thread-safe.
+ */
+public class TableStatistics {
+  private static final Log LOG = LogFactory.getLog(TableStatistics.class);
+  private Schema schema;
+  private Tuple minValues;
+  private Tuple maxValues;
+  private long [] numNulls;
+  private long numRows = 0;
+  private long numBytes = 0;
+
+  private boolean [] comparable;
+
+  public TableStatistics(Schema schema) {
+    this.schema = schema;
+    minValues = new VTuple(schema.size());
+    maxValues = new VTuple(schema.size());
+
+    numNulls = new long[schema.size()];
+    comparable = new boolean[schema.size()];
+
+    DataType type;
+    for (int i = 0; i < schema.size(); i++) {
+      type = schema.getColumn(i).getDataType();
+      if (type.getType() == Type.PROTOBUF) {
+        comparable[i] = false;
+      } else {
+        comparable[i] = true;
+      }
+    }
+  }
+
+  public Schema getSchema() {
+    return this.schema;
+  }
+
+  public void incrementRow() {
+    numRows++;
+  }
+
+  public long getNumRows() {
+    return this.numRows;
+  }
+
+  public void setNumBytes(long bytes) {
+    this.numBytes = bytes;
+  }
+
+  public long getNumBytes() {
+    return this.numBytes;
+  }
+
+  public void analyzeField(int idx, Datum datum) {
+    if (datum instanceof NullDatum) {
+      numNulls[idx]++;
+      return;
+    }
+
+    if (comparable[idx]) {
+      if (!maxValues.contains(idx) ||
+          maxValues.get(idx).compareTo(datum) < 0) {
+        maxValues.put(idx, datum);
+      }
+      if (!minValues.contains(idx) ||
+          minValues.get(idx).compareTo(datum) > 0) {
+        minValues.put(idx, datum);
+      }
+    }
+  }
+
+  public TableStats getTableStat() {
+    TableStats stat = new TableStats();
+
+    ColumnStats columnStats;
+    for (int i = 0; i < schema.size(); i++) {
+      columnStats = new ColumnStats(schema.getColumn(i));
+      columnStats.setNumNulls(numNulls[i]);
+      if (minValues.get(i) == null || schema.getColumn(i).getDataType().getType() == minValues.get(i).type()) {
+        columnStats.setMinValue(minValues.get(i));
+      } else {
+        LOG.warn("Wrong statistics column type (" + minValues.get(i).type() +
+            ", expected=" + schema.getColumn(i).getDataType().getType() + ")");
+      }
+      if (maxValues.get(i) == null || schema.getColumn(i).getDataType().getType() == maxValues.get(i).type()) {
+        columnStats.setMaxValue(maxValues.get(i));
+      } else {
+        LOG.warn("Wrong statistics column type (" + maxValues.get(i).type() +
+            ", expected=" + schema.getColumn(i).getDataType().getType() + ")");
+      }
+      stat.addColumnStat(columnStats);
+    }
+
+    stat.setNumRows(this.numRows);
+    stat.setNumBytes(this.numBytes);
+
+    return stat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
new file mode 100644
index 0000000..094d285
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
@@ -0,0 +1,226 @@
+/**
+ * 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.protobuf.Message;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
+import org.apache.tajo.util.Bytes;
+import org.apache.tajo.util.NumberUtil;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+//Compatibility with Apache Hive
+@Deprecated
+public class TextSerializerDeserializer implements SerializerDeserializer {
+  public static final byte[] trueBytes = "true".getBytes();
+  public static final byte[] falseBytes = "false".getBytes();
+  private ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
+
+
+  @Override
+  public int serialize(Column col, Datum datum, OutputStream out, byte[] nullCharacters) throws IOException {
+
+    byte[] bytes;
+    int length = 0;
+    TajoDataTypes.DataType dataType = col.getDataType();
+
+    if (datum == null || datum instanceof NullDatum) {
+      switch (dataType.getType()) {
+        case CHAR:
+        case TEXT:
+          length = nullCharacters.length;
+          out.write(nullCharacters);
+          break;
+        default:
+          break;
+      }
+      return length;
+    }
+
+    switch (dataType.getType()) {
+      case BOOLEAN:
+        out.write(datum.asBool() ? trueBytes : falseBytes);
+        length = trueBytes.length;
+        break;
+      case CHAR:
+        byte[] pad = new byte[dataType.getLength() - datum.size()];
+        bytes = datum.asTextBytes();
+        out.write(bytes);
+        out.write(pad);
+        length = bytes.length + pad.length;
+        break;
+      case TEXT:
+      case BIT:
+      case INT2:
+      case INT4:
+      case INT8:
+      case FLOAT4:
+      case FLOAT8:
+      case INET4:
+      case DATE:
+      case INTERVAL:
+        bytes = datum.asTextBytes();
+        length = bytes.length;
+        out.write(bytes);
+        break;
+      case TIME:
+        bytes = ((TimeDatum)datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        length = bytes.length;
+        out.write(bytes);
+        break;
+      case TIMESTAMP:
+        bytes = ((TimestampDatum)datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        length = bytes.length;
+        out.write(bytes);
+        break;
+      case INET6:
+      case BLOB:
+        bytes = Base64.encodeBase64(datum.asByteArray(), false);
+        length = bytes.length;
+        out.write(bytes, 0, length);
+        break;
+      case PROTOBUF:
+        ProtobufDatum protobuf = (ProtobufDatum) datum;
+        byte[] protoBytes = protobufJsonFormat.printToString(protobuf.get()).getBytes();
+        length = protoBytes.length;
+        out.write(protoBytes, 0, protoBytes.length);
+        break;
+      case NULL_TYPE:
+      default:
+        break;
+    }
+    return length;
+  }
+
+  @Override
+  public Datum deserialize(Column col, byte[] bytes, int offset, int length, byte[] nullCharacters) throws IOException {
+
+    Datum datum;
+    switch (col.getDataType().getType()) {
+      case BOOLEAN:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createBool(bytes[offset] == 't' || bytes[offset] == 'T');
+        break;
+      case BIT:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createBit(Byte.parseByte(new String(bytes, offset, length)));
+        break;
+      case CHAR:
+        datum = isNullText(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createChar(new String(bytes, offset, length).trim());
+        break;
+      case INT1:
+      case INT2:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createInt2((short) NumberUtil.parseInt(bytes, offset, length));
+        break;
+      case INT4:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createInt4(NumberUtil.parseInt(bytes, offset, length));
+        break;
+      case INT8:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createInt8(new String(bytes, offset, length));
+        break;
+      case FLOAT4:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createFloat4(new String(bytes, offset, length));
+        break;
+      case FLOAT8:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createFloat8(NumberUtil.parseDouble(bytes, offset, length));
+        break;
+      case TEXT: {
+        byte[] chars = new byte[length];
+        System.arraycopy(bytes, offset, chars, 0, length);
+        datum = isNullText(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createText(chars);
+        break;
+      }
+      case DATE:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createDate(new String(bytes, offset, length));
+        break;
+      case TIME:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createTime(new String(bytes, offset, length));
+        break;
+      case TIMESTAMP:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createTimestamp(new String(bytes, offset, length));
+        break;
+      case INTERVAL:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createInterval(new String(bytes, offset, length));
+        break;
+      case PROTOBUF: {
+        if (isNull(bytes, offset, length, nullCharacters)) {
+          datum = NullDatum.get();
+        } else {
+          ProtobufDatumFactory factory = ProtobufDatumFactory.get(col.getDataType());
+          Message.Builder builder = factory.newBuilder();
+          try {
+            byte[] protoBytes = new byte[length];
+            System.arraycopy(bytes, offset, protoBytes, 0, length);
+            protobufJsonFormat.merge(protoBytes, builder);
+            datum = factory.createDatum(builder.build());
+          } catch (IOException e) {
+            e.printStackTrace();
+            throw new RuntimeException(e);
+          }
+        }
+        break;
+      }
+      case INET4:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createInet4(new String(bytes, offset, length));
+        break;
+      case BLOB: {
+        if (isNull(bytes, offset, length, nullCharacters)) {
+          datum = NullDatum.get();
+        } else {
+          byte[] blob = new byte[length];
+          System.arraycopy(bytes, offset, blob, 0, length);
+          datum = DatumFactory.createBlob(Base64.decodeBase64(blob));
+        }
+        break;
+      }
+      default:
+        datum = NullDatum.get();
+        break;
+    }
+    return datum;
+  }
+
+  private static boolean isNull(byte[] val, int offset, int length, byte[] nullBytes) {
+    return length == 0 || ((length == nullBytes.length)
+        && Bytes.equals(val, offset, length, nullBytes, 0, nullBytes.length));
+  }
+
+  private static boolean isNullText(byte[] val, int offset, int length, byte[] nullBytes) {
+    return length > 0 && length == nullBytes.length
+        && Bytes.equals(val, offset, length, nullBytes, 0, nullBytes.length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleComparator.java
new file mode 100644
index 0000000..8dffd8d
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleComparator.java
@@ -0,0 +1,32 @@
+/***
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.common.ProtoObject;
+
+import java.util.Comparator;
+
+import static org.apache.tajo.index.IndexProtos.TupleComparatorProto;
+
+public abstract class TupleComparator implements Comparator<Tuple>, ProtoObject<TupleComparatorProto> {
+
+  public abstract int compare(Tuple o1, Tuple o2);
+
+  public abstract boolean isAscendingFirstKey();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleRange.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleRange.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleRange.java
new file mode 100644
index 0000000..e824b99
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TupleRange.java
@@ -0,0 +1,112 @@
+/**
+ * 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.Objects;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+
+import java.util.Comparator;
+
+/**
+ * It represents a pair of start and end tuples.
+ */
+public class TupleRange implements Comparable<TupleRange>, Cloneable {
+  private Tuple start;
+  private Tuple end;
+  private final TupleComparator comp;
+
+  public TupleRange(final SortSpec[] sortSpecs, final Tuple start, final Tuple end) {
+    this.comp = new BaseTupleComparator(sortSpecsToSchema(sortSpecs), sortSpecs);
+    // if there is only one value, start == end
+    this.start = start;
+    this.end = end;
+  }
+
+  public static Schema sortSpecsToSchema(SortSpec[] sortSpecs) {
+    Schema schema = new Schema();
+    for (SortSpec spec : sortSpecs) {
+      schema.addColumn(spec.getSortKey());
+    }
+
+    return schema;
+  }
+
+  public void setStart(Tuple tuple) {
+    this.start = tuple;
+  }
+
+  public final Tuple getStart() {
+    return this.start;
+  }
+
+  public void setEnd(Tuple tuple) {
+    this.end = tuple;
+  }
+
+  public final Tuple getEnd() {
+    return this.end;
+  }
+
+  public String toString() {
+    return "[" + this.start + ", " + this.end + ")";
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(start, end);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof TupleRange) {
+      TupleRange other = (TupleRange) obj;
+      return this.start.equals(other.start) && this.end.equals(other.end);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int compareTo(TupleRange o) {
+    // TODO - should handle overlap
+    int cmpVal = comp.compare(this.start, o.start);
+    if (cmpVal != 0) {
+      return cmpVal;
+    } else {
+      return comp.compare(this.end, o.end);
+    }
+  }
+
+  public static class DescendingTupleRangeComparator
+      implements Comparator<TupleRange> {
+
+    @Override
+    public int compare(TupleRange left, TupleRange right) {
+      return right.compareTo(left);
+    }
+  }
+
+  public TupleRange clone() throws CloneNotSupportedException {
+    TupleRange newRange = (TupleRange) super.clone();
+    newRange.setStart(start.clone());
+    newRange.setEnd(end.clone());
+    return newRange;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
new file mode 100644
index 0000000..ad19101
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
@@ -0,0 +1,29 @@
+/**
+ * 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.annotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface ForSplitableStore {
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/compress/CodecPool.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
new file mode 100644
index 0000000..baeda8c
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
@@ -0,0 +1,185 @@
+/**
+ * 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.compress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DoNotPool;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A global compressor/decompressor pool used to save and reuse (possibly
+ * native) compression/decompression codecs.
+ */
+public final class CodecPool {
+  private static final Log LOG = LogFactory.getLog(CodecPool.class);
+
+  /**
+   * A global compressor pool used to save the expensive
+   * construction/destruction of (possibly native) decompression codecs.
+   */
+  private static final Map<Class<Compressor>, List<Compressor>> COMPRESSOR_POOL =
+      new HashMap<Class<Compressor>, List<Compressor>>();
+
+  /**
+   * A global decompressor pool used to save the expensive
+   * construction/destruction of (possibly native) decompression codecs.
+   */
+  private static final Map<Class<Decompressor>, List<Decompressor>> DECOMPRESSOR_POOL =
+      new HashMap<Class<Decompressor>, List<Decompressor>>();
+
+  private static <T> T borrow(Map<Class<T>, List<T>> pool,
+      Class<? extends T> codecClass) {
+    T codec = null;
+
+    // Check if an appropriate codec is available
+    synchronized (pool) {
+      if (pool.containsKey(codecClass)) {
+        List<T> codecList = pool.get(codecClass);
+
+        if (codecList != null) {
+          synchronized (codecList) {
+            if (!codecList.isEmpty()) {
+              codec = codecList.remove(codecList.size() - 1);
+            }
+          }
+        }
+      }
+    }
+
+    return codec;
+  }
+
+  private static <T> void payback(Map<Class<T>, List<T>> pool, T codec) {
+    if (codec != null) {
+      Class<T> codecClass = (Class<T>) codec.getClass();
+      synchronized (pool) {
+        if (!pool.containsKey(codecClass)) {
+          pool.put(codecClass, new ArrayList<T>());
+        }
+
+        List<T> codecList = pool.get(codecClass);
+        synchronized (codecList) {
+          codecList.add(codec);
+        }
+      }
+    }
+  }
+
+  /**
+   * Get a {@link Compressor} for the given {@link CompressionCodec} from the
+   * pool or a new one.
+   *
+   * @param codec
+   *          the <code>CompressionCodec</code> for which to get the
+   *          <code>Compressor</code>
+   * @param conf the <code>Configuration</code> object which contains confs for creating or reinit the compressor
+   * @return <code>Compressor</code> for the given <code>CompressionCodec</code>
+   *         from the pool or a new one
+   */
+  public static Compressor getCompressor(CompressionCodec codec, Configuration conf) {
+    Compressor compressor = borrow(COMPRESSOR_POOL, codec.getCompressorType());
+    if (compressor == null) {
+      compressor = codec.createCompressor();
+      LOG.info("Got brand-new compressor ["+codec.getDefaultExtension()+"]");
+    } else {
+      compressor.reinit(conf);
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Got recycled compressor");
+      }
+    }
+    return compressor;
+  }
+
+  public static Compressor getCompressor(CompressionCodec codec) {
+    return getCompressor(codec, null);
+  }
+
+  /**
+   * Get a {@link Decompressor} for the given {@link CompressionCodec} from the
+   * pool or a new one.
+   *
+   * @param codec
+   *          the <code>CompressionCodec</code> for which to get the
+   *          <code>Decompressor</code>
+   * @return <code>Decompressor</code> for the given
+   *         <code>CompressionCodec</code> the pool or a new one
+   */
+  public static Decompressor getDecompressor(CompressionCodec codec) {
+    Decompressor decompressor = borrow(DECOMPRESSOR_POOL, codec
+        .getDecompressorType());
+    if (decompressor == null) {
+      decompressor = codec.createDecompressor();
+      LOG.info("Got brand-new decompressor ["+codec.getDefaultExtension()+"]");
+    } else {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Got recycled decompressor");
+      }
+    }
+    return decompressor;
+  }
+
+  /**
+   * Return the {@link Compressor} to the pool.
+   *
+   * @param compressor
+   *          the <code>Compressor</code> to be returned to the pool
+   */
+  public static void returnCompressor(Compressor compressor) {
+    if (compressor == null) {
+      return;
+    }
+    // if the compressor can't be reused, don't pool it.
+    if (compressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      return;
+    }
+    compressor.reset();
+    payback(COMPRESSOR_POOL, compressor);
+  }
+
+  /**
+   * Return the {@link Decompressor} to the pool.
+   *
+   * @param decompressor
+   *          the <code>Decompressor</code> to be returned to the pool
+   */
+  public static void returnDecompressor(Decompressor decompressor) {
+    if (decompressor == null) {
+      return;
+    }
+    // if the decompressor can't be reused, don't pool it.
+    if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      return;
+    }
+    decompressor.reset();
+    payback(DECOMPRESSOR_POOL, decompressor);
+  }
+
+  private CodecPool() {
+    // prevent instantiation
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
new file mode 100644
index 0000000..bb035a8
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
@@ -0,0 +1,39 @@
+/**
+ * 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.exception;
+
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+
+public class AlreadyExistsStorageException extends IOException {
+  private static final long serialVersionUID = 965518916144019032L;
+
+
+  public AlreadyExistsStorageException(String path) {
+    super("Error: "+path+" alreay exists");    
+  }
+  
+  public AlreadyExistsStorageException(Path path) {
+    this(path.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
new file mode 100644
index 0000000..a67d1f7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
@@ -0,0 +1,32 @@
+/**
+ * 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.exception;
+
+public class UnknownCodecException extends Exception {
+
+  private static final long serialVersionUID = 4287230843540404529L;
+
+  public UnknownCodecException() {
+
+  }
+
+  public UnknownCodecException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
new file mode 100644
index 0000000..d18b5a0
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
@@ -0,0 +1,32 @@
+/**
+ * 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.exception;
+
+public class UnknownDataTypeException extends Exception {
+
+  private static final long serialVersionUID = -2630390595968966164L;
+
+  public UnknownDataTypeException() {
+
+  }
+
+  public UnknownDataTypeException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
new file mode 100644
index 0000000..8b197d6
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exception;
+
+public class UnsupportedFileTypeException extends RuntimeException {
+	private static final long serialVersionUID = -8160289695849000342L;
+
+	public UnsupportedFileTypeException() {
+	}
+
+	/**
+	 * @param message
+	 */
+	public UnsupportedFileTypeException(String message) {
+		super(message);
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
new file mode 100644
index 0000000..ac43197
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
@@ -0,0 +1,39 @@
+/**
+ * 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.fragment;
+
+import org.apache.tajo.common.ProtoObject;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public interface Fragment extends ProtoObject<FragmentProto> {
+
+  public abstract String getTableName();
+
+  @Override
+  public abstract FragmentProto getProto();
+
+  public abstract long getLength();
+
+  public abstract String getKey();
+
+  public String[] getHosts();
+
+  public abstract boolean isEmpty();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
new file mode 100644
index 0000000..07720c7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
@@ -0,0 +1,129 @@
+/**
+ * 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.fragment;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.annotation.ThreadSafe;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+@ThreadSafe
+public class FragmentConvertor {
+  /**
+   * Cache of fragment classes
+   */
+  protected static final Map<String, Class<? extends Fragment>> CACHED_FRAGMENT_CLASSES = Maps.newConcurrentMap();
+  /**
+   * Cache of constructors for each class.
+   */
+  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
+  /**
+   * default parameter for all constructors
+   */
+  private static final Class<?>[] DEFAULT_FRAGMENT_PARAMS = { ByteString.class };
+
+  public static Class<? extends Fragment> getFragmentClass(Configuration conf, String storeType)
+  throws IOException {
+    Class<? extends Fragment> fragmentClass = CACHED_FRAGMENT_CLASSES.get(storeType.toLowerCase());
+    if (fragmentClass == null) {
+      fragmentClass = conf.getClass(
+          String.format("tajo.storage.fragment.%s.class", storeType.toLowerCase()), null, Fragment.class);
+      CACHED_FRAGMENT_CLASSES.put(storeType.toLowerCase(), fragmentClass);
+    }
+
+    if (fragmentClass == null) {
+      throw new IOException("No such a fragment for " + storeType.toLowerCase());
+    }
+
+    return fragmentClass;
+  }
+
+  public static <T extends Fragment> T convert(Class<T> clazz, FragmentProto fragment) {
+    T result;
+    try {
+      Constructor<T> constructor = (Constructor<T>) CONSTRUCTOR_CACHE.get(clazz);
+      if (constructor == null) {
+        constructor = clazz.getDeclaredConstructor(DEFAULT_FRAGMENT_PARAMS);
+        constructor.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(clazz, constructor);
+      }
+      result = constructor.newInstance(new Object[]{fragment.getContents()});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result;
+  }
+
+  public static <T extends Fragment> T convert(Configuration conf, FragmentProto fragment)
+      throws IOException {
+    Class<T> fragmentClass = (Class<T>) getFragmentClass(conf, fragment.getStoreType().toLowerCase());
+    if (fragmentClass == null) {
+      throw new IOException("No such a fragment class for " + fragment.getStoreType());
+    }
+    return convert(fragmentClass, fragment);
+  }
+
+  public static <T extends Fragment> List<T> convert(Class<T> clazz, FragmentProto...fragments)
+      throws IOException {
+    List<T> list = Lists.newArrayList();
+    if (fragments == null) {
+      return list;
+    }
+    for (FragmentProto proto : fragments) {
+      list.add(convert(clazz, proto));
+    }
+    return list;
+  }
+
+  public static <T extends Fragment> List<T> convert(Configuration conf, FragmentProto...fragments) throws IOException {
+    List<T> list = Lists.newArrayList();
+    if (fragments == null) {
+      return list;
+    }
+    for (FragmentProto proto : fragments) {
+      list.add((T) convert(conf, proto));
+    }
+    return list;
+  }
+
+  public static List<FragmentProto> toFragmentProtoList(Fragment... fragments) {
+    List<FragmentProto> list = Lists.newArrayList();
+    if (fragments == null) {
+      return list;
+    }
+    for (Fragment fragment : fragments) {
+      list.add(fragment.getProto());
+    }
+    return list;
+  }
+
+  public static FragmentProto [] toFragmentProtoArray(Fragment... fragments) {
+    List<FragmentProto> list = toFragmentProtoList(fragments);
+    return list.toArray(new FragmentProto[list.size()]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
new file mode 100644
index 0000000..c1835df
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
@@ -0,0 +1,112 @@
+/***
+ * 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.tuple;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SchemaUtil;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.tuple.offheap.HeapTuple;
+import org.apache.tajo.tuple.offheap.OffHeapRowWriter;
+import org.apache.tajo.tuple.offheap.ZeroCopyTuple;
+import org.apache.tajo.unit.StorageUnit;
+import org.apache.tajo.util.Deallocatable;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.UnsafeUtil;
+import sun.misc.Unsafe;
+import sun.nio.ch.DirectBuffer;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class BaseTupleBuilder extends OffHeapRowWriter implements TupleBuilder, Deallocatable {
+  private static final Log LOG = LogFactory.getLog(BaseTupleBuilder.class);
+
+  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
+
+  // buffer
+  private ByteBuffer buffer;
+  private long address;
+
+  public BaseTupleBuilder(Schema schema) {
+    super(SchemaUtil.toDataTypes(schema));
+    buffer = ByteBuffer.allocateDirect(64 * StorageUnit.KB).order(ByteOrder.nativeOrder());
+    address = UnsafeUtil.getAddress(buffer);
+  }
+
+  @Override
+  public long address() {
+    return address;
+  }
+
+  public void ensureSize(int size) {
+    if (buffer.remaining() - size < 0) { // check the remain size
+      // enlarge new buffer and copy writing data
+      int newBlockSize = UnsafeUtil.alignedSize(buffer.capacity() * 2);
+      ByteBuffer newByteBuf = ByteBuffer.allocateDirect(newBlockSize);
+      long newAddress = ((DirectBuffer)newByteBuf).address();
+      UNSAFE.copyMemory(this.address, newAddress, buffer.limit());
+      LOG.debug("Increase the buffer size to " + FileUtil.humanReadableByteCount(newBlockSize, false));
+
+      // release existing buffer and replace variables
+      UnsafeUtil.free(buffer);
+      buffer = newByteBuf;
+      address = newAddress;
+    }
+  }
+
+  @Override
+  public int position() {
+    return 0;
+  }
+
+  @Override
+  public void forward(int length) {
+  }
+
+  @Override
+  public void endRow() {
+    super.endRow();
+    buffer.position(0).limit(offset());
+  }
+
+  @Override
+  public Tuple build() {
+    return buildToHeapTuple();
+  }
+
+  public HeapTuple buildToHeapTuple() {
+    byte [] bytes = new byte[buffer.limit()];
+    UNSAFE.copyMemory(null, address, bytes, UnsafeUtil.ARRAY_BOOLEAN_BASE_OFFSET, buffer.limit());
+    return new HeapTuple(bytes, dataTypes());
+  }
+
+  public ZeroCopyTuple buildToZeroCopyTuple() {
+    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
+    zcTuple.set(buffer, 0, buffer.limit(), dataTypes());
+    return zcTuple;
+  }
+
+  public void release() {
+    UnsafeUtil.free(buffer);
+    buffer = null;
+    address = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/RowBlockReader.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
new file mode 100644
index 0000000..be734e1
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/RowBlockReader.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.tuple;
+
+import org.apache.tajo.storage.Tuple;
+
+public interface RowBlockReader<T extends Tuple> {
+
+  /**
+   * Return for each tuple
+   *
+   * @return True if tuple block is filled with tuples. Otherwise, It will return false.
+   */
+  public boolean next(T tuple);
+
+  public void reset();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/TupleBuilder.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
new file mode 100644
index 0000000..c43c018
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
@@ -0,0 +1,26 @@
+/***
+ * 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.tuple;
+
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.tuple.offheap.RowWriter;
+
+public interface TupleBuilder extends RowWriter {
+  public Tuple build();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
new file mode 100644
index 0000000..9662d5a
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
@@ -0,0 +1,41 @@
+/*
+ * 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.tuple.offheap;
+
+import org.apache.tajo.util.Deallocatable;
+import org.apache.tajo.util.UnsafeUtil;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class DirectBufTuple extends UnSafeTuple implements Deallocatable {
+  private ByteBuffer bb;
+
+  public DirectBufTuple(int length, DataType[] types) {
+    bb = ByteBuffer.allocateDirect(length).order(ByteOrder.nativeOrder());
+    set(bb, 0, length, types);
+  }
+
+  @Override
+  public void release() {
+    UnsafeUtil.free(bb);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
new file mode 100644
index 0000000..a327123
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
@@ -0,0 +1,32 @@
+/*
+ * 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.tuple.offheap;
+
+/**
+ * Fixed size limit specification
+ */
+public class FixedSizeLimitSpec extends ResizableLimitSpec {
+  public FixedSizeLimitSpec(long size) {
+    super(size, size);
+  }
+
+  public FixedSizeLimitSpec(long size, float allowedOverflowRatio) {
+    super(size, size, allowedOverflowRatio);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
new file mode 100644
index 0000000..33f9f1c
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
@@ -0,0 +1,272 @@
+/***
+ * 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.tuple.offheap;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+
+import org.apache.tajo.datum.*;
+import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.util.SizeOf;
+import org.apache.tajo.util.StringUtils;
+import org.apache.tajo.util.UnsafeUtil;
+
+import sun.misc.Unsafe;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class HeapTuple implements Tuple {
+  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
+  private static final long BASE_OFFSET = UnsafeUtil.ARRAY_BYTE_BASE_OFFSET;
+
+  private final byte [] data;
+  private final DataType [] types;
+
+  public HeapTuple(final byte [] bytes, final DataType [] types) {
+    this.data = bytes;
+    this.types = types;
+  }
+
+  @Override
+  public int size() {
+    return data.length;
+  }
+
+  public ByteBuffer nioBuffer() {
+    return ByteBuffer.wrap(data);
+  }
+
+  private int getFieldOffset(int fieldId) {
+    return UNSAFE.getInt(data, BASE_OFFSET + SizeOf.SIZE_OF_INT + (fieldId * SizeOf.SIZE_OF_INT));
+  }
+
+  private int checkNullAndGetOffset(int fieldId) {
+    int offset = getFieldOffset(fieldId);
+    if (offset == OffHeapRowBlock.NULL_FIELD_OFFSET) {
+      throw new RuntimeException("Invalid Field Access: " + fieldId);
+    }
+    return offset;
+  }
+
+  @Override
+  public boolean contains(int fieldid) {
+    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
+  }
+
+  @Override
+  public boolean isNull(int fieldid) {
+    return getFieldOffset(fieldid) == OffHeapRowBlock.NULL_FIELD_OFFSET;
+  }
+
+  @Override
+  public boolean isNotNull(int fieldid) {
+    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
+  }
+
+  @Override
+  public void clear() {
+    // nothing to do
+  }
+
+  @Override
+  public void put(int fieldId, Datum value) {
+    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum).");
+  }
+
+  @Override
+  public void put(int fieldId, Datum[] values) {
+    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum []).");
+  }
+
+  @Override
+  public void put(int fieldId, Tuple tuple) {
+    throw new UnsupportedException("UnSafeTuple does not support put(int, Tuple).");
+  }
+
+  @Override
+  public void put(Datum[] values) {
+    throw new UnsupportedException("UnSafeTuple does not support put(Datum []).");
+  }
+
+  @Override
+  public Datum get(int fieldId) {
+    if (isNull(fieldId)) {
+      return NullDatum.get();
+    }
+
+    switch (types[fieldId].getType()) {
+    case BOOLEAN:
+      return DatumFactory.createBool(getBool(fieldId));
+    case INT1:
+    case INT2:
+      return DatumFactory.createInt2(getInt2(fieldId));
+    case INT4:
+      return DatumFactory.createInt4(getInt4(fieldId));
+    case INT8:
+      return DatumFactory.createInt8(getInt4(fieldId));
+    case FLOAT4:
+      return DatumFactory.createFloat4(getFloat4(fieldId));
+    case FLOAT8:
+      return DatumFactory.createFloat8(getFloat8(fieldId));
+    case TEXT:
+      return DatumFactory.createText(getText(fieldId));
+    case TIMESTAMP:
+      return DatumFactory.createTimestamp(getInt8(fieldId));
+    case DATE:
+      return DatumFactory.createDate(getInt4(fieldId));
+    case TIME:
+      return DatumFactory.createTime(getInt8(fieldId));
+    case INTERVAL:
+      return getInterval(fieldId);
+    case INET4:
+      return DatumFactory.createInet4(getInt4(fieldId));
+    case PROTOBUF:
+      return getProtobufDatum(fieldId);
+    default:
+      throw new UnsupportedException("Unknown type: " + types[fieldId]);
+    }
+  }
+
+  @Override
+  public void setOffset(long offset) {
+  }
+
+  @Override
+  public long getOffset() {
+    return 0;
+  }
+
+  @Override
+  public boolean getBool(int fieldId) {
+    return UNSAFE.getByte(data, BASE_OFFSET + checkNullAndGetOffset(fieldId)) == 0x01;
+  }
+
+  @Override
+  public byte getByte(int fieldId) {
+    return UNSAFE.getByte(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
+  }
+
+  @Override
+  public char getChar(int fieldId) {
+    return UNSAFE.getChar(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
+  }
+
+  @Override
+  public byte[] getBytes(int fieldId) {
+    long pos = checkNullAndGetOffset(fieldId);
+    int len = UNSAFE.getInt(data, BASE_OFFSET + pos);
+    pos += SizeOf.SIZE_OF_INT;
+
+    byte [] bytes = new byte[len];
+    UNSAFE.copyMemory(data, BASE_OFFSET + pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
+    return bytes;
+  }
+
+  @Override
+  public short getInt2(int fieldId) {
+    return UNSAFE.getShort(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
+  }
+
+  @Override
+  public int getInt4(int fieldId) {
+    return UNSAFE.getInt(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
+  }
+
+  @Override
+  public long getInt8(int fieldId) {
+    return UNSAFE.getLong(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
+  }
+
+  @Override
+  public float getFloat4(int fieldId) {
+    return UNSAFE.getFloat(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
+  }
+
+  @Override
+  public double getFloat8(int fieldId) {
+    return UNSAFE.getDouble(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
+  }
+
+  @Override
+  public String getText(int fieldId) {
+    return new String(getBytes(fieldId));
+  }
+
+  public IntervalDatum getInterval(int fieldId) {
+    long pos = checkNullAndGetOffset(fieldId);
+    int months = UNSAFE.getInt(data, BASE_OFFSET + pos);
+    pos += SizeOf.SIZE_OF_INT;
+    long millisecs = UNSAFE.getLong(data, BASE_OFFSET + pos);
+    return new IntervalDatum(months, millisecs);
+  }
+
+  @Override
+  public Datum getProtobufDatum(int fieldId) {
+    byte [] bytes = getBytes(fieldId);
+
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(types[fieldId].getCode());
+    Message.Builder builder = factory.newBuilder();
+    try {
+      builder.mergeFrom(bytes);
+    } catch (InvalidProtocolBufferException e) {
+      return NullDatum.get();
+    }
+
+    return new ProtobufDatum(builder.build());
+  }
+
+  @Override
+  public char[] getUnicodeChars(int fieldId) {
+    long pos = checkNullAndGetOffset(fieldId);
+    int len = UNSAFE.getInt(data, BASE_OFFSET + pos);
+    pos += SizeOf.SIZE_OF_INT;
+
+    byte [] bytes = new byte[len];
+    UNSAFE.copyMemory(data, BASE_OFFSET + pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
+    return StringUtils.convertBytesToChars(bytes, Charset.forName("UTF-8"));
+  }
+
+  @Override
+  public Tuple clone() throws CloneNotSupportedException {
+    return this;
+  }
+
+  @Override
+  public Datum[] getValues() {
+    Datum [] datums = new Datum[size()];
+    for (int i = 0; i < size(); i++) {
+      if (contains(i)) {
+        datums[i] = get(i);
+      } else {
+        datums[i] = NullDatum.get();
+      }
+    }
+    return datums;
+  }
+
+  @Override
+  public String toString() {
+    return VTuple.toDisplayString(getValues());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
new file mode 100644
index 0000000..2f8e349
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
@@ -0,0 +1,102 @@
+/**
+ * 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.tuple.offheap;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.util.Deallocatable;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.UnsafeUtil;
+import sun.misc.Unsafe;
+import sun.nio.ch.DirectBuffer;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class OffHeapMemory implements Deallocatable {
+  private static final Log LOG = LogFactory.getLog(OffHeapMemory.class);
+
+  protected static final Unsafe UNSAFE = UnsafeUtil.unsafe;
+
+  protected ByteBuffer buffer;
+  protected int memorySize;
+  protected ResizableLimitSpec limitSpec;
+  protected long address;
+
+  @VisibleForTesting
+  protected OffHeapMemory(ByteBuffer buffer, ResizableLimitSpec limitSpec) {
+    this.buffer = buffer;
+    this.address = ((DirectBuffer) buffer).address();
+    this.memorySize = buffer.limit();
+    this.limitSpec = limitSpec;
+  }
+
+  public OffHeapMemory(ResizableLimitSpec limitSpec) {
+    this(ByteBuffer.allocateDirect((int) limitSpec.initialSize()).order(ByteOrder.nativeOrder()), limitSpec);
+  }
+
+  public long address() {
+    return address;
+  }
+
+  public long size() {
+    return memorySize;
+  }
+
+  public void resize(int newSize) {
+    Preconditions.checkArgument(newSize > 0, "Size must be greater than 0 bytes");
+
+    if (newSize > limitSpec.limit()) {
+      throw new RuntimeException("Resize cannot exceed the size limit");
+    }
+
+    if (newSize < memorySize) {
+      LOG.warn("The size reduction is ignored.");
+    }
+
+    int newBlockSize = UnsafeUtil.alignedSize(newSize);
+    ByteBuffer newByteBuf = ByteBuffer.allocateDirect(newBlockSize);
+    long newAddress = ((DirectBuffer)newByteBuf).address();
+
+    UNSAFE.copyMemory(this.address, newAddress, memorySize);
+
+    UnsafeUtil.free(buffer);
+    this.memorySize = newSize;
+    this.buffer = newByteBuf;
+    this.address = newAddress;
+  }
+
+  public java.nio.Buffer nioBuffer() {
+    return (ByteBuffer) buffer.position(0).limit(memorySize);
+  }
+
+  @Override
+  public void release() {
+    UnsafeUtil.free(this.buffer);
+    this.buffer = null;
+    this.address = 0;
+    this.memorySize = 0;
+  }
+
+  public String toString() {
+    return "memory=" + FileUtil.humanReadableByteCount(memorySize, false) + "," + limitSpec;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
new file mode 100644
index 0000000..689efb7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
@@ -0,0 +1,176 @@
+/***
+ * 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.tuple.offheap;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SchemaUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.util.Deallocatable;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.SizeOf;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class OffHeapRowBlock extends OffHeapMemory implements Deallocatable {
+  private static final Log LOG = LogFactory.getLog(OffHeapRowBlock.class);
+
+  public static final int NULL_FIELD_OFFSET = -1;
+
+  DataType [] dataTypes;
+
+  // Basic States
+  private int maxRowNum = Integer.MAX_VALUE; // optional
+  private int rowNum;
+  protected int position = 0;
+
+  private OffHeapRowBlockWriter builder;
+
+  private OffHeapRowBlock(ByteBuffer buffer, Schema schema, ResizableLimitSpec limitSpec) {
+    super(buffer, limitSpec);
+    initialize(schema);
+  }
+
+  public OffHeapRowBlock(Schema schema, ResizableLimitSpec limitSpec) {
+    super(limitSpec);
+    initialize(schema);
+  }
+
+  private void initialize(Schema schema) {
+    dataTypes = SchemaUtil.toDataTypes(schema);
+
+    this.builder = new OffHeapRowBlockWriter(this);
+  }
+
+  @VisibleForTesting
+  public OffHeapRowBlock(Schema schema, int bytes) {
+    this(schema, new ResizableLimitSpec(bytes));
+  }
+
+  @VisibleForTesting
+  public OffHeapRowBlock(Schema schema, ByteBuffer buffer) {
+    this(buffer, schema, ResizableLimitSpec.DEFAULT_LIMIT);
+  }
+
+  public void position(int pos) {
+    this.position = pos;
+  }
+
+  public void clear() {
+    this.position = 0;
+    this.rowNum = 0;
+
+    builder.clear();
+  }
+
+  @Override
+  public ByteBuffer nioBuffer() {
+    return (ByteBuffer) buffer.position(0).limit(position);
+  }
+
+  public int position() {
+    return position;
+  }
+
+  public long usedMem() {
+    return position;
+  }
+
+  /**
+   * Ensure that this buffer has enough remaining space to add the size.
+   * Creates and copies to a new buffer if necessary
+   *
+   * @param size Size to add
+   */
+  public void ensureSize(int size) {
+    if (remain() - size < 0) {
+      if (!limitSpec.canIncrease(memorySize)) {
+        throw new RuntimeException("Cannot increase RowBlock anymore.");
+      }
+
+      int newBlockSize = limitSpec.increasedSize(memorySize);
+      resize(newBlockSize);
+      LOG.info("Increase DirectRowBlock to " + FileUtil.humanReadableByteCount(newBlockSize, false));
+    }
+  }
+
+  public long remain() {
+    return memorySize - position - builder.offset();
+  }
+
+  public int maxRowNum() {
+    return maxRowNum;
+  }
+  public int rows() {
+    return rowNum;
+  }
+
+  public void setRows(int rowNum) {
+    this.rowNum = rowNum;
+  }
+
+  public boolean copyFromChannel(FileChannel channel, TableStats stats) throws IOException {
+    if (channel.position() < channel.size()) {
+      clear();
+
+      buffer.clear();
+      channel.read(buffer);
+      memorySize = buffer.position();
+
+      while (position < memorySize) {
+        long recordPtr = address + position;
+
+        if (remain() < SizeOf.SIZE_OF_INT) {
+          channel.position(channel.position() - remain());
+          memorySize = (int) (memorySize - remain());
+          return true;
+        }
+
+        int recordSize = UNSAFE.getInt(recordPtr);
+
+        if (remain() < recordSize) {
+          channel.position(channel.position() - remain());
+          memorySize = (int) (memorySize - remain());
+          return true;
+        }
+
+        position += recordSize;
+        rowNum++;
+      }
+
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public RowWriter getWriter() {
+    return builder;
+  }
+
+  public OffHeapRowBlockReader getReader() {
+    return new OffHeapRowBlockReader(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
new file mode 100644
index 0000000..4a9313f
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
@@ -0,0 +1,63 @@
+/***
+ * 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.tuple.offheap;
+
+import org.apache.tajo.tuple.RowBlockReader;
+import org.apache.tajo.util.UnsafeUtil;
+import sun.misc.Unsafe;
+
+public class OffHeapRowBlockReader implements RowBlockReader<ZeroCopyTuple> {
+  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
+  OffHeapRowBlock rowBlock;
+
+  // Read States
+  private int curRowIdxForRead;
+  private int curPosForRead;
+
+  public OffHeapRowBlockReader(OffHeapRowBlock rowBlock) {
+    this.rowBlock = rowBlock;
+  }
+
+  public long remainForRead() {
+    return rowBlock.memorySize - curPosForRead;
+  }
+
+  @Override
+  public boolean next(ZeroCopyTuple tuple) {
+    if (curRowIdxForRead < rowBlock.rows()) {
+
+      long recordStartPtr = rowBlock.address() + curPosForRead;
+      int recordLen = UNSAFE.getInt(recordStartPtr);
+      tuple.set(rowBlock.buffer, curPosForRead, recordLen, rowBlock.dataTypes);
+
+      curPosForRead += recordLen;
+      curRowIdxForRead++;
+
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void reset() {
+    curPosForRead = 0;
+    curRowIdxForRead = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
new file mode 100644
index 0000000..dbc3188
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tuple.offheap;
+
+import com.google.common.collect.Lists;
+import org.apache.tajo.storage.Tuple;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+public class OffHeapRowBlockUtils {
+
+  public static List<Tuple> sort(OffHeapRowBlock rowBlock, Comparator<Tuple> comparator) {
+    List<Tuple> tupleList = Lists.newArrayList();
+    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
+    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
+    while(reader.next(zcTuple)) {
+      tupleList.add(zcTuple);
+      zcTuple = new ZeroCopyTuple();
+    }
+    Collections.sort(tupleList, comparator);
+    return tupleList;
+  }
+
+  public static Tuple[] sortToArray(OffHeapRowBlock rowBlock, Comparator<Tuple> comparator) {
+    Tuple[] tuples = new Tuple[rowBlock.rows()];
+    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
+    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
+    for (int i = 0; i < rowBlock.rows() && reader.next(zcTuple); i++) {
+      tuples[i] = zcTuple;
+      zcTuple = new ZeroCopyTuple();
+    }
+    Arrays.sort(tuples, comparator);
+    return tuples;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
new file mode 100644
index 0000000..d177e0c
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.tuple.offheap;
+
+import org.apache.tajo.common.TajoDataTypes;
+
+public class OffHeapRowBlockWriter extends OffHeapRowWriter {
+  OffHeapRowBlock rowBlock;
+
+  OffHeapRowBlockWriter(OffHeapRowBlock rowBlock) {
+    super(rowBlock.dataTypes);
+    this.rowBlock = rowBlock;
+  }
+
+  public long address() {
+    return rowBlock.address();
+  }
+
+  public int position() {
+    return rowBlock.position();
+  }
+
+  @Override
+  public void forward(int length) {
+    rowBlock.position(position() + length);
+  }
+
+  public void ensureSize(int size) {
+    rowBlock.ensureSize(size);
+  }
+
+  @Override
+  public void endRow() {
+    super.endRow();
+    rowBlock.setRows(rowBlock.rows() + 1);
+  }
+
+  @Override
+  public TajoDataTypes.DataType[] dataTypes() {
+    return rowBlock.dataTypes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
new file mode 100644
index 0000000..85c7e0b
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
@@ -0,0 +1,232 @@
+/**
+ * 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.tuple.offheap;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.IntervalDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.util.SizeOf;
+import org.apache.tajo.util.UnsafeUtil;
+
+/**
+ *
+ * Row Record Structure
+ *
+ * | row length (4 bytes) | field 1 offset | field 2 offset | ... | field N offset| field 1 | field 2| ... | field N |
+ *                              4 bytes          4 bytes               4 bytes
+ *
+ */
+public abstract class OffHeapRowWriter implements RowWriter {
+  /** record size + offset list */
+  private final int headerSize;
+  /** field offsets */
+  private final int [] fieldOffsets;
+  private final TajoDataTypes.DataType [] dataTypes;
+
+  private int curFieldIdx;
+  private int curOffset;
+
+  public OffHeapRowWriter(final TajoDataTypes.DataType [] dataTypes) {
+    this.dataTypes = dataTypes;
+    fieldOffsets = new int[dataTypes.length];
+    headerSize = SizeOf.SIZE_OF_INT * (dataTypes.length + 1);
+  }
+
+  public void clear() {
+    curOffset = 0;
+    curFieldIdx = 0;
+  }
+
+  public long recordStartAddr() {
+    return address() + position();
+  }
+
+  public abstract long address();
+
+  public abstract void ensureSize(int size);
+
+  public int offset() {
+    return curOffset;
+  }
+
+  /**
+   * Current position
+   *
+   * @return The position
+   */
+  public abstract int position();
+
+  /**
+   * Forward the address;
+   *
+   * @param length Length to be forwarded
+   */
+  public abstract void forward(int length);
+
+  @Override
+  public TajoDataTypes.DataType[] dataTypes() {
+    return dataTypes;
+  }
+
+  public boolean startRow() {
+    curOffset = headerSize;
+    curFieldIdx = 0;
+    return true;
+  }
+
+  public void endRow() {
+    long rowHeaderPos = address() + position();
+    OffHeapMemory.UNSAFE.putInt(rowHeaderPos, curOffset);
+    rowHeaderPos += SizeOf.SIZE_OF_INT;
+
+    for (int i = 0; i < curFieldIdx; i++) {
+      OffHeapMemory.UNSAFE.putInt(rowHeaderPos, fieldOffsets[i]);
+      rowHeaderPos += SizeOf.SIZE_OF_INT;
+    }
+    for (int i = curFieldIdx; i < dataTypes.length; i++) {
+      OffHeapMemory.UNSAFE.putInt(rowHeaderPos, OffHeapRowBlock.NULL_FIELD_OFFSET);
+      rowHeaderPos += SizeOf.SIZE_OF_INT;
+    }
+
+    // rowOffset is equivalent to a byte length of this row.
+    forward(curOffset);
+  }
+
+  public void skipField() {
+    fieldOffsets[curFieldIdx++] = OffHeapRowBlock.NULL_FIELD_OFFSET;
+  }
+
+  private void forwardField() {
+    fieldOffsets[curFieldIdx++] = curOffset;
+  }
+
+  public void putBool(boolean val) {
+    ensureSize(SizeOf.SIZE_OF_BOOL);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putByte(recordStartAddr() + curOffset, (byte) (val ? 0x01 : 0x00));
+
+    curOffset += SizeOf.SIZE_OF_BOOL;
+  }
+
+  public void putInt2(short val) {
+    ensureSize(SizeOf.SIZE_OF_SHORT);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putShort(recordStartAddr() + curOffset, val);
+    curOffset += SizeOf.SIZE_OF_SHORT;
+  }
+
+  public void putInt4(int val) {
+    ensureSize(SizeOf.SIZE_OF_INT);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putInt(recordStartAddr() + curOffset, val);
+    curOffset += SizeOf.SIZE_OF_INT;
+  }
+
+  public void putInt8(long val) {
+    ensureSize(SizeOf.SIZE_OF_LONG);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putLong(recordStartAddr() + curOffset, val);
+    curOffset += SizeOf.SIZE_OF_LONG;
+  }
+
+  public void putFloat4(float val) {
+    ensureSize(SizeOf.SIZE_OF_FLOAT);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putFloat(recordStartAddr() + curOffset, val);
+    curOffset += SizeOf.SIZE_OF_FLOAT;
+  }
+
+  public void putFloat8(double val) {
+    ensureSize(SizeOf.SIZE_OF_DOUBLE);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putDouble(recordStartAddr() + curOffset, val);
+    curOffset += SizeOf.SIZE_OF_DOUBLE;
+  }
+
+  public void putText(String val) {
+    byte[] bytes = val.getBytes(TextDatum.DEFAULT_CHARSET);
+    putText(bytes);
+  }
+
+  public void putText(byte[] val) {
+    int bytesLen = val.length;
+
+    ensureSize(SizeOf.SIZE_OF_INT + bytesLen);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putInt(recordStartAddr() + curOffset, bytesLen);
+    curOffset += SizeOf.SIZE_OF_INT;
+
+    OffHeapMemory.UNSAFE.copyMemory(val, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, null,
+        recordStartAddr() + curOffset, bytesLen);
+    curOffset += bytesLen;
+  }
+
+  public void putBlob(byte[] val) {
+    int bytesLen = val.length;
+
+    ensureSize(SizeOf.SIZE_OF_INT + bytesLen);
+    forwardField();
+
+    OffHeapMemory.UNSAFE.putInt(recordStartAddr() + curOffset, bytesLen);
+    curOffset += SizeOf.SIZE_OF_INT;
+
+    OffHeapMemory.UNSAFE.copyMemory(val, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, null,
+        recordStartAddr() + curOffset, bytesLen);
+    curOffset += bytesLen;
+  }
+
+  public void putTimestamp(long val) {
+    putInt8(val);
+  }
+
+  public void putDate(int val) {
+    putInt4(val);
+  }
+
+  public void putTime(long val) {
+    putInt8(val);
+  }
+
+  public void putInterval(IntervalDatum val) {
+    ensureSize(SizeOf.SIZE_OF_INT + SizeOf.SIZE_OF_LONG);
+    forwardField();
+
+    long offset = recordStartAddr() + curOffset;
+    OffHeapMemory.UNSAFE.putInt(offset, val.getMonths());
+    offset += SizeOf.SIZE_OF_INT;
+    OffHeapMemory.UNSAFE.putLong(offset, val.getMilliSeconds());
+    curOffset += SizeOf.SIZE_OF_INT + SizeOf.SIZE_OF_LONG;
+  }
+
+  public void putInet4(int val) {
+    putInt4(val);
+  }
+
+  public void putProtoDatum(ProtobufDatum val) {
+    putBlob(val.asByteArray());
+  }
+}


[08/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
new file mode 100644
index 0000000..cb86f35
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java
@@ -0,0 +1,773 @@
+/**
+ * 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.protobuf.Message;
+import io.netty.buffer.ByteBuf;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.ProtobufDatumFactory;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.unit.StorageUnit;
+import org.apache.tajo.util.BitArray;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+public class RawFile {
+  private static final Log LOG = LogFactory.getLog(RawFile.class);
+
+  public static class RawFileScanner extends FileScanner implements SeekableScanner {
+    private FileChannel channel;
+    private DataType[] columnTypes;
+
+    private ByteBuffer buffer;
+    private ByteBuf buf;
+    private Tuple tuple;
+
+    private int headerSize = 0; // Header size of a tuple
+    private BitArray nullFlags;
+    private static final int RECORD_SIZE = 4;
+    private boolean eos = false;
+    private long startOffset;
+    private long endOffset;
+    private FileInputStream fis;
+    private long recordCount;
+    private long totalReadBytes;
+    private long filePosition;
+    private boolean forceFillBuffer;
+
+    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
+      super(conf, schema, meta, fragment);
+    }
+
+    public void init() throws IOException {
+      File file;
+      try {
+        if (fragment.getPath().toUri().getScheme() != null) {
+          file = new File(fragment.getPath().toUri());
+        } else {
+          file = new File(fragment.getPath().toString());
+        }
+      } catch (IllegalArgumentException iae) {
+        throw new IOException(iae);
+      }
+      fis = new FileInputStream(file);
+      channel = fis.getChannel();
+      filePosition = startOffset = fragment.getStartKey();
+      endOffset = fragment.getStartKey() + fragment.getLength();
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RawFileScanner open:" + fragment + "," + channel.position() + ", file size :" + channel.size()
+            + ", fragment length :" + fragment.getLength());
+      }
+
+      buf = BufferPool.directBuffer(64 * StorageUnit.KB);
+      buffer = buf.nioBuffer(0, buf.capacity());
+
+      columnTypes = new DataType[schema.size()];
+      for (int i = 0; i < schema.size(); i++) {
+        columnTypes[i] = schema.getColumn(i).getDataType();
+      }
+
+      tuple = new VTuple(columnTypes.length);
+      nullFlags = new BitArray(schema.size());
+      headerSize = RECORD_SIZE + 2 + nullFlags.bytesLength(); // The middle 2 bytes is for NullFlagSize
+
+      // initial set position
+      if (fragment.getStartKey() > 0) {
+        channel.position(fragment.getStartKey());
+      }
+
+      forceFillBuffer = true;
+      super.init();
+    }
+
+    @Override
+    public long getNextOffset() throws IOException {
+      return filePosition - (forceFillBuffer ? 0 : buffer.remaining());
+    }
+
+    @Override
+    public void seek(long offset) throws IOException {
+      eos = false;
+      filePosition = channel.position();
+
+      // do not fill the buffer if the offset is already included in the buffer.
+      if(!forceFillBuffer && filePosition > offset && offset > filePosition - buffer.limit()){
+        buffer.position((int)(offset - (filePosition - buffer.limit())));
+      } else {
+        if(offset < startOffset || offset > startOffset + fragment.getLength()){
+          throw new IndexOutOfBoundsException(String.format("range(%d, %d), offset: %d",
+              startOffset, startOffset + fragment.getLength(), offset));
+        }
+        channel.position(offset);
+        filePosition = offset;
+        buffer.clear();
+        forceFillBuffer = true;
+        fillBuffer();
+      }
+    }
+
+    private boolean fillBuffer() throws IOException {
+      if(!forceFillBuffer) buffer.compact();
+
+      int bytesRead = channel.read(buffer);
+      forceFillBuffer = false;
+      if (bytesRead == -1) {
+        eos = true;
+        return false;
+      } else {
+        buffer.flip(); //The limit is set to the current filePosition and then the filePosition is set to zero
+        filePosition += bytesRead;
+        totalReadBytes += bytesRead;
+        return true;
+      }
+    }
+
+    /**
+     * Decode a ZigZag-encoded 32-bit value.  ZigZag encodes signed integers
+     * into values that can be efficiently encoded with varint.  (Otherwise,
+     * negative values must be sign-extended to 64 bits to be varint encoded,
+     * thus always taking 10 bytes on the wire.)
+     *
+     * @param n An unsigned 32-bit integer, stored in a signed int because
+     *          Java has no explicit unsigned support.
+     * @return A signed 32-bit integer.
+     */
+    public static int decodeZigZag32(final int n) {
+      return (n >>> 1) ^ -(n & 1);
+    }
+
+    /**
+     * Decode a ZigZag-encoded 64-bit value.  ZigZag encodes signed integers
+     * into values that can be efficiently encoded with varint.  (Otherwise,
+     * negative values must be sign-extended to 64 bits to be varint encoded,
+     * thus always taking 10 bytes on the wire.)
+     *
+     * @param n An unsigned 64-bit integer, stored in a signed int because
+     *          Java has no explicit unsigned support.
+     * @return A signed 64-bit integer.
+     */
+    public static long decodeZigZag64(final long n) {
+      return (n >>> 1) ^ -(n & 1);
+    }
+
+
+    /**
+     * Read a raw Varint from the stream.  If larger than 32 bits, discard the
+     * upper bits.
+     */
+    public int readRawVarint32() throws IOException {
+      byte tmp = buffer.get();
+      if (tmp >= 0) {
+        return tmp;
+      }
+      int result = tmp & 0x7f;
+      if ((tmp = buffer.get()) >= 0) {
+        result |= tmp << 7;
+      } else {
+        result |= (tmp & 0x7f) << 7;
+        if ((tmp = buffer.get()) >= 0) {
+          result |= tmp << 14;
+        } else {
+          result |= (tmp & 0x7f) << 14;
+          if ((tmp = buffer.get()) >= 0) {
+            result |= tmp << 21;
+          } else {
+            result |= (tmp & 0x7f) << 21;
+            result |= (tmp = buffer.get()) << 28;
+            if (tmp < 0) {
+              // Discard upper 32 bits.
+              for (int i = 0; i < 5; i++) {
+                if (buffer.get() >= 0) {
+                  return result;
+                }
+              }
+              throw new IOException("Invalid Variable int32");
+            }
+          }
+        }
+      }
+      return result;
+    }
+
+    /** Read a raw Varint from the stream. */
+    public long readRawVarint64() throws IOException {
+      int shift = 0;
+      long result = 0;
+      while (shift < 64) {
+        final byte b = buffer.get();
+        result |= (long)(b & 0x7F) << shift;
+        if ((b & 0x80) == 0) {
+          return result;
+        }
+        shift += 7;
+      }
+      throw new IOException("Invalid Variable int64");
+    }
+
+    @Override
+    public Tuple next() throws IOException {
+      if(eos) return null;
+
+      if (forceFillBuffer || buffer.remaining() < headerSize) {
+        if (!fillBuffer()) {
+          return null;
+        }
+      }
+
+      // backup the buffer state
+      int bufferLimit = buffer.limit();
+      int recordSize = buffer.getInt();
+      int nullFlagSize = buffer.getShort();
+
+      buffer.limit(buffer.position() + nullFlagSize);
+      nullFlags.fromByteBuffer(buffer);
+      // restore the start of record contents
+      buffer.limit(bufferLimit);
+      if (buffer.remaining() < (recordSize - headerSize)) {
+
+        //if the buffer reaches the writable size, the buffer increase the record size
+        reSizeBuffer(recordSize);
+
+        if (!fillBuffer()) {
+          return null;
+        }
+      }
+
+      for (int i = 0; i < columnTypes.length; i++) {
+        // check if the i'th column is null
+        if (nullFlags.get(i)) {
+          tuple.put(i, DatumFactory.createNullDatum());
+          continue;
+        }
+
+        switch (columnTypes[i].getType()) {
+          case BOOLEAN :
+            tuple.put(i, DatumFactory.createBool(buffer.get()));
+            break;
+
+          case BIT :
+            tuple.put(i, DatumFactory.createBit(buffer.get()));
+            break;
+
+          case CHAR :
+            int realLen = readRawVarint32();
+            byte[] buf = new byte[realLen];
+            buffer.get(buf);
+            tuple.put(i, DatumFactory.createChar(buf));
+            break;
+
+          case INT2 :
+            tuple.put(i, DatumFactory.createInt2(buffer.getShort()));
+            break;
+
+          case INT4 :
+            tuple.put(i, DatumFactory.createInt4(decodeZigZag32(readRawVarint32())));
+            break;
+
+          case INT8 :
+            tuple.put(i, DatumFactory.createInt8(decodeZigZag64(readRawVarint64())));
+            break;
+
+          case FLOAT4 :
+            tuple.put(i, DatumFactory.createFloat4(buffer.getFloat()));
+            break;
+
+          case FLOAT8 :
+            tuple.put(i, DatumFactory.createFloat8(buffer.getDouble()));
+            break;
+
+          case TEXT : {
+            int len = readRawVarint32();
+            byte [] strBytes = new byte[len];
+            buffer.get(strBytes);
+            tuple.put(i, DatumFactory.createText(strBytes));
+            break;
+          }
+
+          case BLOB : {
+            int len = readRawVarint32();
+            byte [] rawBytes = new byte[len];
+            buffer.get(rawBytes);
+            tuple.put(i, DatumFactory.createBlob(rawBytes));
+            break;
+          }
+
+          case PROTOBUF: {
+            int len = readRawVarint32();
+            byte [] rawBytes = new byte[len];
+            buffer.get(rawBytes);
+
+            ProtobufDatumFactory factory = ProtobufDatumFactory.get(columnTypes[i]);
+            Message.Builder builder = factory.newBuilder();
+            builder.mergeFrom(rawBytes);
+            tuple.put(i, factory.createDatum(builder.build()));
+            break;
+          }
+
+          case INET4 :
+            byte [] ipv4Bytes = new byte[4];
+            buffer.get(ipv4Bytes);
+            tuple.put(i, DatumFactory.createInet4(ipv4Bytes));
+            break;
+
+          case DATE: {
+            int val = buffer.getInt();
+            if (val < Integer.MIN_VALUE + 1) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            } else {
+              tuple.put(i, DatumFactory.createFromInt4(columnTypes[i], val));
+            }
+            break;
+          }
+          case TIME:
+          case TIMESTAMP: {
+            long val = buffer.getLong();
+            if (val < Long.MIN_VALUE + 1) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            } else {
+              tuple.put(i, DatumFactory.createFromInt8(columnTypes[i], val));
+            }
+            break;
+          }
+          case NULL_TYPE:
+            tuple.put(i, NullDatum.get());
+            break;
+
+          default:
+        }
+      }
+
+      recordCount++;
+
+      if(filePosition - buffer.remaining() >= endOffset){
+        eos = true;
+      }
+      return new VTuple(tuple);
+    }
+
+    private void reSizeBuffer(int writableBytes){
+      if (buffer.capacity() - buffer.remaining()  <  writableBytes) {
+        buf.setIndex(buffer.position(), buffer.limit());
+        buf.markReaderIndex();
+        buf.discardSomeReadBytes();
+        buf.ensureWritable(writableBytes);
+        buffer = buf.nioBuffer(0, buf.capacity());
+        buffer.limit(buf.writerIndex());
+      }
+    }
+
+    @Override
+    public void reset() throws IOException {
+      // reset the buffer
+      buffer.clear();
+      forceFillBuffer = true;
+      filePosition = fragment.getStartKey();
+      channel.position(filePosition);
+      eos = false;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if(buf != null){
+        buffer.clear();
+        buffer = null;
+
+        buf.release();
+        buf = null;
+      }
+
+      IOUtils.cleanup(LOG, channel, fis);
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return false;
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public boolean isSplittable(){
+      return false;
+    }
+
+    @Override
+    public TableStats getInputStats() {
+      if(tableStats != null){
+        tableStats.setNumRows(recordCount);
+        tableStats.setReadBytes(totalReadBytes); // actual read bytes (scan + rescan * n)
+        tableStats.setNumBytes(fragment.getLength());
+      }
+      return tableStats;
+    }
+
+    @Override
+    public float getProgress() {
+      if(eos) {
+        return 1.0f;
+      }
+
+      if (filePosition - startOffset == 0) {
+        return 0.0f;
+      } else {
+        return Math.min(1.0f, ((float) filePosition / endOffset));
+      }
+    }
+  }
+
+  public static class RawFileAppender extends FileAppender {
+    private FileChannel channel;
+    private RandomAccessFile randomAccessFile;
+    private DataType[] columnTypes;
+
+    private ByteBuffer buffer;
+    private ByteBuf buf;
+    private BitArray nullFlags;
+    private int headerSize = 0;
+    private static final int RECORD_SIZE = 4;
+    private long pos;
+
+    private TableStatistics stats;
+
+    public RawFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                           Schema schema, TableMeta meta, Path workDir) throws IOException {
+      super(conf, taskAttemptId, schema, meta, workDir);
+    }
+
+    public void init() throws IOException {
+      File file;
+      try {
+        if (path.toUri().getScheme() != null) {
+          file = new File(path.toUri());
+        } else {
+          file = new File(path.toString());
+        }
+      } catch (IllegalArgumentException iae) {
+        throw new IOException(iae);
+      }
+
+      randomAccessFile = new RandomAccessFile(file, "rw");
+      channel = randomAccessFile.getChannel();
+      pos = 0;
+
+      columnTypes = new DataType[schema.size()];
+      for (int i = 0; i < schema.size(); i++) {
+        columnTypes[i] = schema.getColumn(i).getDataType();
+      }
+
+      buf = BufferPool.directBuffer(64 * StorageUnit.KB);
+      buffer = buf.nioBuffer(0, buf.capacity());
+
+      // comput the number of bytes, representing the null flags
+
+      nullFlags = new BitArray(schema.size());
+      headerSize = RECORD_SIZE + 2 + nullFlags.bytesLength();
+
+      if (enabledStats) {
+        this.stats = new TableStatistics(this.schema);
+      }
+
+      super.init();
+    }
+
+    @Override
+    public long getOffset() throws IOException {
+      return pos;
+    }
+
+    private void flushBuffer() throws IOException {
+      buffer.flip();
+      channel.write(buffer);
+      buffer.clear();
+    }
+
+    private boolean flushBufferAndReplace(int recordOffset, int sizeToBeWritten)
+        throws IOException {
+
+      // if the buffer reaches the limit,
+      // write the bytes from 0 to the previous record.
+      if (buffer.remaining() < sizeToBeWritten) {
+
+        int limit = buffer.position();
+        buffer.limit(recordOffset);
+        buffer.flip();
+        channel.write(buffer);
+        buffer.position(recordOffset);
+        buffer.limit(limit);
+        buffer.compact();
+
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    /**
+     * Encode a ZigZag-encoded 32-bit value.  ZigZag encodes signed integers
+     * into values that can be efficiently encoded with varint.  (Otherwise,
+     * negative values must be sign-extended to 64 bits to be varint encoded,
+     * thus always taking 10 bytes on the wire.)
+     *
+     * @param n A signed 32-bit integer.
+     * @return An unsigned 32-bit integer, stored in a signed int because
+     *         Java has no explicit unsigned support.
+     */
+    public static int encodeZigZag32(final int n) {
+      // Note:  the right-shift must be arithmetic
+      return (n << 1) ^ (n >> 31);
+    }
+
+    /**
+     * Encode a ZigZag-encoded 64-bit value.  ZigZag encodes signed integers
+     * into values that can be efficiently encoded with varint.  (Otherwise,
+     * negative values must be sign-extended to 64 bits to be varint encoded,
+     * thus always taking 10 bytes on the wire.)
+     *
+     * @param n A signed 64-bit integer.
+     * @return An unsigned 64-bit integer, stored in a signed int because
+     *         Java has no explicit unsigned support.
+     */
+    public static long encodeZigZag64(final long n) {
+      // Note:  the right-shift must be arithmetic
+      return (n << 1) ^ (n >> 63);
+    }
+
+    /**
+     * Encode and write a varint.  {@code value} is treated as
+     * unsigned, so it won't be sign-extended if negative.
+     */
+    public void writeRawVarint32(int value) throws IOException {
+      while (true) {
+        if ((value & ~0x7F) == 0) {
+          buffer.put((byte) value);
+          return;
+        } else {
+          buffer.put((byte) ((value & 0x7F) | 0x80));
+          value >>>= 7;
+        }
+      }
+    }
+
+    /**
+     * Compute the number of bytes that would be needed to encode a varint.
+     * {@code value} is treated as unsigned, so it won't be sign-extended if
+     * negative.
+     */
+    public static int computeRawVarint32Size(final int value) {
+      if ((value & (0xffffffff <<  7)) == 0) return 1;
+      if ((value & (0xffffffff << 14)) == 0) return 2;
+      if ((value & (0xffffffff << 21)) == 0) return 3;
+      if ((value & (0xffffffff << 28)) == 0) return 4;
+      return 5;
+    }
+
+    /** Encode and write a varint. */
+    public void writeRawVarint64(long value) throws IOException {
+      while (true) {
+        if ((value & ~0x7FL) == 0) {
+          buffer.put((byte) value);
+          return;
+        } else {
+          buffer.put((byte) ((value & 0x7F) | 0x80));
+          value >>>= 7;
+        }
+      }
+    }
+
+    @Override
+    public void addTuple(Tuple t) throws IOException {
+
+      if (buffer.remaining() < headerSize) {
+        flushBuffer();
+      }
+
+      // skip the row header
+      int recordOffset = buffer.position();
+      buffer.position(recordOffset + headerSize);
+      // reset the null flags
+      nullFlags.clear();
+      for (int i = 0; i < schema.size(); i++) {
+        if (enabledStats) {
+          stats.analyzeField(i, t.get(i));
+        }
+
+        if (t.isNull(i)) {
+          nullFlags.set(i);
+          continue;
+        }
+
+        // 8 is the maximum bytes size of all types
+        if (flushBufferAndReplace(recordOffset, 8)) {
+          recordOffset = 0;
+        }
+
+        switch(columnTypes[i].getType()) {
+          case NULL_TYPE:
+            nullFlags.set(i);
+            continue;
+
+          case BOOLEAN:
+          case BIT:
+            buffer.put(t.getByte(i));
+            break;
+
+          case INT2 :
+            buffer.putShort(t.getInt2(i));
+            break;
+
+          case INT4 :
+            writeRawVarint32(encodeZigZag32(t.getInt4(i)));
+            break;
+
+          case INT8 :
+            writeRawVarint64(encodeZigZag64(t.getInt8(i)));
+            break;
+
+          case FLOAT4 :
+            buffer.putFloat(t.getFloat4(i));
+            break;
+
+          case FLOAT8 :
+            buffer.putDouble(t.getFloat8(i));
+            break;
+
+          case CHAR:
+          case TEXT: {
+            byte [] strBytes = t.getBytes(i);
+            if (flushBufferAndReplace(recordOffset, strBytes.length + computeRawVarint32Size(strBytes.length))) {
+              recordOffset = 0;
+            }
+            writeRawVarint32(strBytes.length);
+            buffer.put(strBytes);
+            break;
+          }
+
+        case DATE:
+          buffer.putInt(t.getInt4(i));
+          break;
+
+        case TIME:
+        case TIMESTAMP:
+          buffer.putLong(t.getInt8(i));
+          break;
+
+          case BLOB : {
+            byte [] rawBytes = t.getBytes(i);
+            if (flushBufferAndReplace(recordOffset, rawBytes.length + computeRawVarint32Size(rawBytes.length))) {
+              recordOffset = 0;
+            }
+            writeRawVarint32(rawBytes.length);
+            buffer.put(rawBytes);
+            break;
+          }
+
+          case PROTOBUF: {
+            byte [] rawBytes = t.getBytes(i);
+            if (flushBufferAndReplace(recordOffset, rawBytes.length + computeRawVarint32Size(rawBytes.length))) {
+              recordOffset = 0;
+            }
+            writeRawVarint32(rawBytes.length);
+            buffer.put(rawBytes);
+            break;
+          }
+
+          case INET4 :
+            buffer.put(t.getBytes(i));
+            break;
+
+          default:
+            throw new IOException("Cannot support data type: " + columnTypes[i].getType());
+        }
+      }
+
+      // write a record header
+      int bufferPos = buffer.position();
+      buffer.position(recordOffset);
+      buffer.putInt(bufferPos - recordOffset);
+      byte [] flags = nullFlags.toArray();
+      buffer.putShort((short) flags.length);
+      buffer.put(flags);
+
+      pos += bufferPos - recordOffset;
+      buffer.position(bufferPos);
+
+      if (enabledStats) {
+        stats.incrementRow();
+      }
+    }
+
+    @Override
+    public void flush() throws IOException {
+      if(buffer != null){
+        flushBuffer();
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      flush();
+      if (enabledStats) {
+        stats.setNumBytes(getOffset());
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RawFileAppender written: " + getOffset() + " bytes, path: " + path);
+      }
+
+      if(buf != null){
+        buffer.clear();
+        buffer = null;
+
+        buf.release();
+        buf = null;
+      }
+
+      IOUtils.cleanup(LOG, channel, randomAccessFile);
+    }
+
+    @Override
+    public TableStats getStats() {
+      if (enabledStats) {
+        stats.setNumBytes(pos);
+        return stats.getTableStat();
+      } else {
+        return null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java
new file mode 100644
index 0000000..8da6ada
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java
@@ -0,0 +1,498 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.BitArray;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+
+public class RowFile {
+  public static final Log LOG = LogFactory.getLog(RowFile.class);
+
+  private static final int SYNC_ESCAPE = -1;
+  private static final int SYNC_HASH_SIZE = 16;
+  private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE;
+  private final static int DEFAULT_BUFFER_SIZE = 65535;
+  public static int SYNC_INTERVAL;
+
+  public static class RowFileScanner extends FileScanner {
+    private FileSystem fs;
+    private FSDataInputStream in;
+    private Tuple tuple;
+
+    private byte[] sync = new byte[SYNC_HASH_SIZE];
+    private byte[] checkSync = new byte[SYNC_HASH_SIZE];
+    private long start, end;
+
+    private ByteBuffer buffer;
+    private final int tupleHeaderSize;
+    private BitArray nullFlags;
+    private long bufferStartPos;
+
+    public RowFileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment)
+        throws IOException {
+      super(conf, schema, meta, fragment);
+
+      SYNC_INTERVAL = conf.getInt(ConfVars.ROWFILE_SYNC_INTERVAL.varname,
+          ConfVars.ROWFILE_SYNC_INTERVAL.defaultIntVal) * SYNC_SIZE;
+
+      nullFlags = new BitArray(schema.size());
+      tupleHeaderSize = nullFlags.bytesLength() + (2 * Short.SIZE / 8);
+      this.start = this.fragment.getStartKey();
+      this.end = this.start + this.fragment.getLength();
+    }
+
+    public void init() throws IOException {
+      // set default page size.
+      fs = fragment.getPath().getFileSystem(conf);
+      in = fs.open(fragment.getPath());
+      buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE * schema.size());
+      buffer.flip();
+
+      readHeader();
+
+      // find the correct position from the start
+      if (this.start > in.getPos()) {
+        long realStart = start > SYNC_SIZE ? (start-SYNC_SIZE) : 0;
+        in.seek(realStart);
+      }
+      bufferStartPos = in.getPos();
+      fillBuffer();
+
+      if (start != 0) {
+        // TODO: improve
+        boolean syncFound = false;
+        while (!syncFound) {
+          if (buffer.remaining() < SYNC_SIZE) {
+            fillBuffer();
+          }
+          buffer.mark();
+          syncFound = checkSync();
+          if (!syncFound) {
+            buffer.reset();
+            buffer.get(); // proceed one byte
+          }
+        }
+        bufferStartPos += buffer.position();
+        buffer.compact();
+        buffer.flip();
+      }
+
+      super.init();
+    }
+
+    private void readHeader() throws IOException {
+      SYNC_INTERVAL = in.readInt();
+      StorageUtil.readFully(in, this.sync, 0, SYNC_HASH_SIZE);
+    }
+
+    /**
+     * Find the sync from the front of the buffer
+     *
+     * @return return true if it succeeds to find the sync.
+     * @throws java.io.IOException
+     */
+    private boolean checkSync() throws IOException {
+      buffer.getInt();                           // escape
+      buffer.get(checkSync, 0, SYNC_HASH_SIZE);  // sync
+      return Arrays.equals(checkSync, sync);
+    }
+
+    private int fillBuffer() throws IOException {
+      bufferStartPos += buffer.position();
+      buffer.compact();
+      int remain = buffer.remaining();
+      int read = in.read(buffer);
+      if (read == -1) {
+        buffer.flip();
+        return read;
+      } else {
+        int totalRead = read;
+        if (remain > totalRead) {
+          read = in.read(buffer);
+          totalRead += read > 0 ? read : 0;
+        }
+        buffer.flip();
+        return totalRead;
+      }
+    }
+
+    @Override
+    public Tuple next() throws IOException {
+      while (buffer.remaining() < SYNC_SIZE) {
+        if (fillBuffer() < 0) {
+          return null;
+        }
+      }
+
+      buffer.mark();
+      if (!checkSync()) {
+        buffer.reset();
+      } else {
+        if (bufferStartPos + buffer.position() > end) {
+          return null;
+        }
+      }
+
+      while (buffer.remaining() < tupleHeaderSize) {
+        if (fillBuffer() < 0) {
+          return null;
+        }
+      }
+
+      int i;
+      tuple = new VTuple(schema.size());
+
+      int nullFlagSize = buffer.getShort();
+      byte[] nullFlagBytes = new byte[nullFlagSize];
+      buffer.get(nullFlagBytes, 0, nullFlagSize);
+      nullFlags = new BitArray(nullFlagBytes);
+      int tupleSize = buffer.getShort();
+
+      while (buffer.remaining() < (tupleSize)) {
+        if (fillBuffer() < 0) {
+          return null;
+        }
+      }
+
+      Datum datum;
+      Column col;
+      for (i = 0; i < schema.size(); i++) {
+        if (!nullFlags.get(i)) {
+          col = schema.getColumn(i);
+          switch (col.getDataType().getType()) {
+            case BOOLEAN :
+              datum = DatumFactory.createBool(buffer.get());
+              tuple.put(i, datum);
+              break;
+
+            case BIT:
+              datum = DatumFactory.createBit(buffer.get());
+              tuple.put(i, datum );
+              break;
+
+            case CHAR :
+              int realLen = buffer.getInt();
+              byte[] buf = new byte[col.getDataType().getLength()];
+              buffer.get(buf);
+              byte[] charBuf = Arrays.copyOf(buf, realLen);
+              tuple.put(i, DatumFactory.createChar(charBuf));
+              break;
+
+            case INT2 :
+              datum = DatumFactory.createInt2(buffer.getShort());
+              tuple.put(i, datum );
+              break;
+
+            case INT4 :
+              datum = DatumFactory.createInt4(buffer.getInt());
+              tuple.put(i, datum );
+              break;
+
+            case INT8 :
+              datum = DatumFactory.createInt8(buffer.getLong());
+              tuple.put(i, datum );
+              break;
+
+            case FLOAT4 :
+              datum = DatumFactory.createFloat4(buffer.getFloat());
+              tuple.put(i, datum);
+              break;
+
+            case FLOAT8 :
+              datum = DatumFactory.createFloat8(buffer.getDouble());
+              tuple.put(i, datum);
+              break;
+
+            case TEXT:
+              short bytelen = buffer.getShort();
+              byte[] strbytes = new byte[bytelen];
+              buffer.get(strbytes, 0, bytelen);
+              datum = DatumFactory.createText(strbytes);
+              tuple.put(i, datum);
+              break;
+
+            case BLOB:
+              short bytesLen = buffer.getShort();
+              byte [] bytesBuf = new byte[bytesLen];
+              buffer.get(bytesBuf);
+              datum = DatumFactory.createBlob(bytesBuf);
+              tuple.put(i, datum);
+              break;
+
+            case INET4 :
+              byte[] ipv4 = new byte[4];
+              buffer.get(ipv4, 0, 4);
+              datum = DatumFactory.createInet4(ipv4);
+              tuple.put(i, datum);
+              break;
+
+            default:
+              break;
+          }
+        } else {
+          tuple.put(i, DatumFactory.createNullDatum());
+        }
+      }
+      return tuple;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      init();
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (in != null) {
+        in.close();
+      }
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return false;
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public boolean isSplittable(){
+      return true;
+    }
+  }
+
+  public static class RowFileAppender extends FileAppender {
+    private FSDataOutputStream out;
+    private long lastSyncPos;
+    private FileSystem fs;
+    private byte[] sync;
+    private ByteBuffer buffer;
+
+    private BitArray nullFlags;
+    // statistics
+    private TableStatistics stats;
+
+    public RowFileAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
+                           final Schema schema, final TableMeta meta, final Path workDir)
+        throws IOException {
+      super(conf, taskAttemptId, schema, meta, workDir);
+    }
+
+    public void init() throws IOException {
+      SYNC_INTERVAL = conf.getInt(ConfVars.ROWFILE_SYNC_INTERVAL.varname,
+          ConfVars.ROWFILE_SYNC_INTERVAL.defaultIntVal);
+      fs = path.getFileSystem(conf);
+
+      if (!fs.exists(path.getParent())) {
+        throw new FileNotFoundException(path.toString());
+      }
+
+      if (fs.exists(path)) {
+        throw new AlreadyExistsStorageException(path);
+      }
+
+      sync = new byte[SYNC_HASH_SIZE];
+      lastSyncPos = 0;
+
+      out = fs.create(path);
+
+      MessageDigest md;
+      try {
+        md = MessageDigest.getInstance("MD5");
+        md.update((path.toString()+System.currentTimeMillis()).getBytes());
+        sync = md.digest();
+      } catch (NoSuchAlgorithmException e) {
+        LOG.error(e);
+      }
+
+      writeHeader();
+
+      buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
+
+      nullFlags = new BitArray(schema.size());
+
+      if (enabledStats) {
+        this.stats = new TableStatistics(this.schema);
+      }
+    }
+
+    private void writeHeader() throws IOException {
+      out.writeInt(SYNC_INTERVAL);
+      out.write(sync);
+      out.flush();
+      lastSyncPos = out.getPos();
+    }
+
+    @Override
+    public void addTuple(Tuple t) throws IOException {
+      checkAndWriteSync();
+      Column col;
+
+      buffer.clear();
+      nullFlags.clear();
+
+      for (int i = 0; i < schema.size(); i++) {
+        if (enabledStats) {
+          stats.analyzeField(i, t.get(i));
+        }
+
+        if (t.isNull(i)) {
+          nullFlags.set(i);
+        } else {
+          col = schema.getColumn(i);
+          switch (col.getDataType().getType()) {
+            case BOOLEAN:
+              buffer.put(t.get(i).asByte());
+              break;
+            case BIT:
+              buffer.put(t.get(i).asByte());
+              break;
+            case CHAR:
+              byte[] src = t.get(i).asByteArray();
+              byte[] dst = Arrays.copyOf(src, col.getDataType().getLength());
+              buffer.putInt(src.length);
+              buffer.put(dst);
+              break;
+            case TEXT:
+              byte [] strbytes = t.get(i).asByteArray();
+              buffer.putShort((short)strbytes.length);
+              buffer.put(strbytes, 0, strbytes.length);
+              break;
+            case INT2:
+              buffer.putShort(t.get(i).asInt2());
+              break;
+            case INT4:
+              buffer.putInt(t.get(i).asInt4());
+              break;
+            case INT8:
+              buffer.putLong(t.get(i).asInt8());
+              break;
+            case FLOAT4:
+              buffer.putFloat(t.get(i).asFloat4());
+              break;
+            case FLOAT8:
+              buffer.putDouble(t.get(i).asFloat8());
+              break;
+            case BLOB:
+              byte [] bytes = t.get(i).asByteArray();
+              buffer.putShort((short)bytes.length);
+              buffer.put(bytes);
+              break;
+            case INET4:
+              buffer.put(t.get(i).asByteArray());
+              break;
+            case INET6:
+              buffer.put(t.get(i).asByteArray());
+              break;
+            case NULL_TYPE:
+              nullFlags.set(i);
+              break;
+            default:
+              break;
+          }
+        }
+      }
+
+      byte[] bytes = nullFlags.toArray();
+      out.writeShort(bytes.length);
+      out.write(bytes);
+
+      bytes = buffer.array();
+      int dataLen = buffer.position();
+      out.writeShort(dataLen);
+      out.write(bytes, 0, dataLen);
+
+      // Statistical section
+      if (enabledStats) {
+        stats.incrementRow();
+      }
+    }
+
+    @Override
+    public long getOffset() throws IOException {
+      return out.getPos();
+    }
+
+    @Override
+    public void flush() throws IOException {
+      out.flush();
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (out != null) {
+        if (enabledStats) {
+          stats.setNumBytes(out.getPos());
+        }
+        sync();
+        out.flush();
+        out.close();
+      }
+    }
+
+    private void sync() throws IOException {
+      if (lastSyncPos != out.getPos()) {
+        out.writeInt(SYNC_ESCAPE);
+        out.write(sync);
+        lastSyncPos = out.getPos();
+      }
+    }
+
+    private void checkAndWriteSync() throws IOException {
+      if (out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
+        sync();
+      }
+    }
+
+    @Override
+    public TableStats getStats() {
+      if (enabledStats) {
+        return stats.getTableStat();
+      } else {
+        return null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/SplitLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/SplitLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/SplitLineReader.java
new file mode 100644
index 0000000..3579674
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/SplitLineReader.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class SplitLineReader extends LineReader {
+  public SplitLineReader(InputStream in, byte[] recordDelimiterBytes) {
+    super(in, recordDelimiterBytes);
+  }
+
+  public SplitLineReader(InputStream in, Configuration conf,
+                         byte[] recordDelimiterBytes) throws IOException {
+    super(in, conf, recordDelimiterBytes);
+  }
+
+  public boolean needAdditionalRecordAfterSplit() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
new file mode 100644
index 0000000..85f91cc
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
@@ -0,0 +1,219 @@
+/**
+ * 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.FileAppender;
+import org.apache.tajo.storage.TableStatistics;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * FileAppender for writing to Avro files.
+ */
+public class AvroAppender extends FileAppender {
+  private TableStatistics stats;
+  private Schema avroSchema;
+  private List<Schema.Field> avroFields;
+  private DataFileWriter<GenericRecord> dataFileWriter;
+
+  /**
+   * Creates a new AvroAppender.
+   *
+   * @param conf Configuration properties.
+   * @param schema The table schema.
+   * @param meta The table metadata.
+   * @param workDir The path of the Parquet file to write to.
+   */
+  public AvroAppender(Configuration conf,
+                      QueryUnitAttemptId taskAttemptId,
+                      org.apache.tajo.catalog.Schema schema,
+                      TableMeta meta, Path workDir) throws IOException {
+    super(conf, taskAttemptId, schema, meta, workDir);
+  }
+
+  /**
+   * Initializes the Appender.
+   */
+  public void init() throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+    if (!fs.exists(path.getParent())) {
+      throw new FileNotFoundException(path.toString());
+    }
+    FSDataOutputStream outputStream = fs.create(path);
+
+    avroSchema = AvroUtil.getAvroSchema(meta, conf);
+    avroFields = avroSchema.getFields();
+
+    DatumWriter<GenericRecord> datumWriter =
+        new GenericDatumWriter<GenericRecord>(avroSchema);
+    dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
+    dataFileWriter.create(avroSchema, outputStream);
+
+    if (enabledStats) {
+      this.stats = new TableStatistics(schema);
+    }
+    super.init();
+  }
+
+  /**
+   * Gets the current offset. Tracking offsets is currenly not implemented, so
+   * this method always returns 0.
+   *
+   * @return 0
+   */
+  @Override
+  public long getOffset() throws IOException {
+    return 0;
+  }
+
+  private Object getPrimitive(Tuple tuple, int i, Schema.Type avroType) {
+    if (tuple.get(i) instanceof NullDatum) {
+      return null;
+    }
+    switch (avroType) {
+      case NULL:
+        return null;
+      case BOOLEAN:
+        return tuple.getBool(i);
+      case INT:
+        return tuple.getInt4(i);
+      case LONG:
+        return tuple.getInt8(i);
+      case FLOAT:
+        return tuple.getFloat4(i);
+      case DOUBLE:
+        return tuple.getFloat8(i);
+      case BYTES:
+      case FIXED:
+        return ByteBuffer.wrap(tuple.getBytes(i));
+      case STRING:
+        return tuple.getText(i);
+      default:
+        throw new RuntimeException("Unknown primitive type.");
+    }
+  }
+
+  /**
+   * Write a Tuple to the Avro file.
+   *
+   * @param tuple The Tuple to write.
+   */
+  @Override
+  public void addTuple(Tuple tuple) throws IOException {
+    GenericRecord record = new GenericData.Record(avroSchema);
+    for (int i = 0; i < schema.size(); ++i) {
+      Column column = schema.getColumn(i);
+      if (enabledStats) {
+        stats.analyzeField(i, tuple.get(i));
+      }
+      Object value;
+      Schema.Field avroField = avroFields.get(i);
+      Schema.Type avroType = avroField.schema().getType();
+      switch (avroType) {
+        case NULL:
+        case BOOLEAN:
+        case INT:
+        case LONG:
+        case FLOAT:
+        case DOUBLE:
+        case BYTES:
+        case STRING:
+        case FIXED:
+          value = getPrimitive(tuple, i, avroType);
+          break;
+        case RECORD:
+          throw new RuntimeException("Avro RECORD not supported.");
+        case ENUM:
+          throw new RuntimeException("Avro ENUM not supported.");
+        case MAP:
+          throw new RuntimeException("Avro MAP not supported.");
+        case UNION:
+          List<Schema> schemas = avroField.schema().getTypes();
+          if (schemas.size() != 2) {
+            throw new RuntimeException("Avro UNION not supported.");
+          }
+          if (schemas.get(0).getType().equals(Schema.Type.NULL)) {
+            value = getPrimitive(tuple, i, schemas.get(1).getType());
+          } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) {
+            value = getPrimitive(tuple, i, schemas.get(0).getType());
+          } else {
+            throw new RuntimeException("Avro UNION not supported.");
+          }
+          break;
+        default:
+          throw new RuntimeException("Unknown type: " + avroType);
+      }
+      record.put(i, value);
+    }
+    dataFileWriter.append(record);
+
+    if (enabledStats) {
+      stats.incrementRow();
+    }
+  }
+
+  /**
+   * Flushes the current state of the file.
+   */
+  @Override
+  public void flush() throws IOException {
+    dataFileWriter.flush();
+  }
+
+  /**
+   * Closes the Appender.
+   */
+  @Override
+  public void close() throws IOException {
+    dataFileWriter.close();
+  }
+
+  /**
+   * If table statistics is enabled, retrieve the table statistics.
+   *
+   * @return Table statistics if enabled or null otherwise.
+   */
+  @Override
+  public TableStats getStats() {
+    if (enabledStats) {
+      return stats.getTableStat();
+    } else {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
new file mode 100644
index 0000000..51594df
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
@@ -0,0 +1,286 @@
+/**
+ * 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.avro;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.mapred.FsInput;
+import org.apache.avro.util.Utf8;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.storage.FileScanner;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * FileScanner for reading Avro files
+ */
+public class AvroScanner extends FileScanner {
+  private Schema avroSchema;
+  private List<Schema.Field> avroFields;
+  private DataFileReader<GenericRecord> dataFileReader;
+  private int[] projectionMap;
+
+  /**
+   * Creates a new AvroScanner.
+   *
+   * @param conf
+   * @param schema
+   * @param meta
+   * @param fragment
+   */
+  public AvroScanner(Configuration conf,
+                     final org.apache.tajo.catalog.Schema schema,
+                     final TableMeta meta, final Fragment fragment) {
+    super(conf, schema, meta, fragment);
+  }
+
+  /**
+   * Initializes the AvroScanner.
+   */
+  @Override
+  public void init() throws IOException {
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+    prepareProjection(targets);
+
+    avroSchema = AvroUtil.getAvroSchema(meta, conf);
+    avroFields = avroSchema.getFields();
+
+    DatumReader<GenericRecord> datumReader =
+        new GenericDatumReader<GenericRecord>(avroSchema);
+    SeekableInput input = new FsInput(fragment.getPath(), conf);
+    dataFileReader = new DataFileReader<GenericRecord>(input, datumReader);
+    super.init();
+  }
+
+  private void prepareProjection(Column[] targets) {
+    projectionMap = new int[targets.length];
+    for (int i = 0; i < targets.length; ++i) {
+      projectionMap[i] = schema.getColumnId(targets[i].getQualifiedName());
+    }
+  }
+
+  private static String fromAvroString(Object value) {
+    if (value instanceof Utf8) {
+      Utf8 utf8 = (Utf8)value;
+      return utf8.toString();
+    }
+    return value.toString();
+  }
+
+  private static Schema getNonNull(Schema schema) {
+    if (!schema.getType().equals(Schema.Type.UNION)) {
+      return schema;
+    }
+    List<Schema> schemas = schema.getTypes();
+    if (schemas.size() != 2) {
+      return schema;
+    }
+    if (schemas.get(0).getType().equals(Schema.Type.NULL)) {
+      return schemas.get(1);
+    } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) {
+      return schemas.get(0);
+    } else {
+      return schema;
+    }
+  }
+
+  private Datum convertInt(Object value, TajoDataTypes.Type tajoType) {
+    int intValue = (Integer)value;
+    switch (tajoType) {
+      case BIT:
+        return DatumFactory.createBit((byte)(intValue & 0xff));
+      case INT2:
+        return DatumFactory.createInt2((short)intValue);
+      default:
+        return DatumFactory.createInt4(intValue);
+    }
+  }
+
+  private Datum convertBytes(Object value, TajoDataTypes.Type tajoType,
+                             DataType dataType) {
+    ByteBuffer buffer = (ByteBuffer)value;
+    byte[] bytes = new byte[buffer.capacity()];
+    buffer.get(bytes, 0, bytes.length);
+    switch (tajoType) {
+      case INET4:
+        return DatumFactory.createInet4(bytes);
+      case PROTOBUF:
+        try {
+          ProtobufDatumFactory factory =
+              ProtobufDatumFactory.get(dataType.getCode());
+          Message.Builder builder = factory.newBuilder();
+          builder.mergeFrom(bytes);
+          return factory.createDatum(builder);
+        } catch (InvalidProtocolBufferException e) {
+          throw new RuntimeException(e);
+        }
+      default:
+        return new BlobDatum(bytes);
+    }
+  }
+
+  private Datum convertString(Object value, TajoDataTypes.Type tajoType) {
+    switch (tajoType) {
+      case CHAR:
+        return DatumFactory.createChar(fromAvroString(value));
+      default:
+        return DatumFactory.createText(fromAvroString(value));
+    }
+  }
+
+  /**
+   * Reads the next Tuple from the Avro file.
+   *
+   * @return The next Tuple from the Avro file or null if end of file is
+   *         reached.
+   */
+  @Override
+  public Tuple next() throws IOException {
+    if (!dataFileReader.hasNext()) {
+      return null;
+    }
+
+    Tuple tuple = new VTuple(schema.size());
+    GenericRecord record = dataFileReader.next();
+    for (int i = 0; i < projectionMap.length; ++i) {
+      int columnIndex = projectionMap[i];
+      Object value = record.get(columnIndex);
+      if (value == null) {
+        tuple.put(columnIndex, NullDatum.get());
+        continue;
+      }
+
+      // Get Avro type.
+      Schema.Field avroField = avroFields.get(columnIndex);
+      Schema nonNullAvroSchema = getNonNull(avroField.schema());
+      Schema.Type avroType = nonNullAvroSchema.getType();
+
+      // Get Tajo type.
+      Column column = schema.getColumn(columnIndex);
+      DataType dataType = column.getDataType();
+      TajoDataTypes.Type tajoType = dataType.getType();
+      switch (avroType) {
+        case NULL:
+          tuple.put(columnIndex, NullDatum.get());
+          break;
+        case BOOLEAN:
+          tuple.put(columnIndex, DatumFactory.createBool((Boolean)value));
+          break;
+        case INT:
+          tuple.put(columnIndex, convertInt(value, tajoType));
+          break;
+        case LONG:
+          tuple.put(columnIndex, DatumFactory.createInt8((Long)value));
+          break;
+        case FLOAT:
+          tuple.put(columnIndex, DatumFactory.createFloat4((Float)value));
+          break;
+        case DOUBLE:
+          tuple.put(columnIndex, DatumFactory.createFloat8((Double)value));
+          break;
+        case BYTES:
+          tuple.put(columnIndex, convertBytes(value, tajoType, dataType));
+          break;
+        case STRING:
+          tuple.put(columnIndex, convertString(value, tajoType));
+          break;
+        case RECORD:
+          throw new RuntimeException("Avro RECORD not supported.");
+        case ENUM:
+          throw new RuntimeException("Avro ENUM not supported.");
+        case MAP:
+          throw new RuntimeException("Avro MAP not supported.");
+        case UNION:
+          throw new RuntimeException("Avro UNION not supported.");
+        case FIXED:
+          tuple.put(columnIndex, new BlobDatum(((GenericFixed)value).bytes()));
+          break;
+        default:
+          throw new RuntimeException("Unknown type.");
+      }
+    }
+    return tuple;
+  }
+
+  /**
+   * Resets the scanner
+   */
+  @Override
+  public void reset() throws IOException {
+  }
+
+  /**
+   * Closes the scanner.
+   */
+  @Override
+  public void close() throws IOException {
+    if (dataFileReader != null) {
+      dataFileReader.close();
+    }
+  }
+
+  /**
+   * Returns whether this scanner is projectable.
+   *
+   * @return true
+   */
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  /**
+   * Returns whether this scanner is selectable.
+   *
+   * @return false
+   */
+  @Override
+  public boolean isSelectable() {
+    return false;
+  }
+
+  /**
+   * Returns whether this scanner is splittable.
+   *
+   * @return false
+   */
+  @Override
+  public boolean isSplittable() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
new file mode 100644
index 0000000..0d14c3d
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
@@ -0,0 +1,77 @@
+/**
+ * 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.StorageConstants;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+
+public class AvroUtil {
+  public static Schema getAvroSchema(TableMeta meta, Configuration conf)
+      throws IOException {
+
+    boolean isSchemaLiteral = meta.containsOption(StorageConstants.AVRO_SCHEMA_LITERAL);
+    boolean isSchemaUrl = meta.containsOption(StorageConstants.AVRO_SCHEMA_URL);
+    if (!isSchemaLiteral && !isSchemaUrl) {
+      throw new RuntimeException("No Avro schema for table.");
+    }
+    if (isSchemaLiteral) {
+      String schema = meta.getOption(StorageConstants.AVRO_SCHEMA_LITERAL);
+      return new Schema.Parser().parse(schema);
+    }
+
+    String schemaURL = meta.getOption(StorageConstants.AVRO_SCHEMA_URL);
+    if (schemaURL.toLowerCase().startsWith("http")) {
+      return getAvroSchemaFromHttp(schemaURL);
+    } else {
+      return getAvroSchemaFromFileSystem(schemaURL, conf);
+    }
+  }
+
+  public static Schema getAvroSchemaFromHttp(String schemaURL) throws IOException {
+    InputStream inputStream = new URL(schemaURL).openStream();
+
+    try {
+      return new Schema.Parser().parse(inputStream);
+    } finally {
+      IOUtils.closeStream(inputStream);
+    }
+  }
+
+  public static Schema getAvroSchemaFromFileSystem(String schemaURL, Configuration conf) throws IOException {
+    Path schemaPath = new Path(schemaURL);
+    FileSystem fs = schemaPath.getFileSystem(conf);
+    FSDataInputStream inputStream = fs.open(schemaPath);
+
+    try {
+      return new Schema.Parser().parse(inputStream);
+    } finally {
+      IOUtils.closeStream(inputStream);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/package-info.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/package-info.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/package-info.java
new file mode 100644
index 0000000..40d1545
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/package-info.java
@@ -0,0 +1,85 @@
+/**
+ * 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.
+ */
+/**
+ * <p>
+ * Provides read and write support for Avro files. Avro schemas are
+ * converted to Tajo schemas according to the following mapping of Avro
+ * and Tajo types:
+ * </p>
+ *
+ * <table>
+ *   <tr>
+ *     <th>Avro type</th>
+ *     <th>Tajo type</th>
+ *   </tr>
+ *   <tr>
+ *     <td>NULL</td>
+ *     <td>NULL_TYPE</td>
+ *   </tr>
+ *   <tr>
+ *     <td>BOOLEAN</td>
+ *     <td>BOOLEAN</td>
+ *   </tr>
+ *   <tr>
+ *     <td>INT</td>
+ *     <td>INT4</td>
+ *   </tr>
+ *   <tr>
+ *     <td>LONG</td>
+ *     <td>INT8</td>
+ *   </tr>
+ *   <tr>
+ *     <td>FLOAT</td>
+ *     <td>FLOAT4</td>
+ *   </tr>
+ *   <tr>
+ *     <td>DOUBLE</td>
+ *     <td>FLOAT8</td>
+ *   </tr>
+ *   <tr>
+ *     <td>BYTES</td>
+ *     <td>BLOB</td>
+ *   </tr>
+ *   <tr>
+ *     <td>STRING</td>
+ *     <td>TEXT</td>
+ *   </tr>
+ *   <tr>
+ *     <td>FIXED</td>
+ *     <td>BLOB</td>
+ *   </tr>
+ *   <tr>
+ *     <td>RECORD</td>
+ *     <td>Not currently supported</td>
+ *   </tr>
+ *   <tr>
+ *     <td>ENUM</td>
+ *     <td>Not currently supported.</td>
+ *   </tr>
+ *   <tr>
+ *     <td>MAP</td>
+ *     <td>Not currently supported.</td>
+ *   </tr>
+ *   <tr>
+ *     <td>UNION</td>
+ *     <td>Not currently supported.</td>
+ *   </tr>
+ * </table>
+ */
+
+package org.apache.tajo.storage.avro;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
new file mode 100644
index 0000000..ac413ca
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
@@ -0,0 +1,237 @@
+/**
+ * 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.fragment;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.storage.StorageFragmentProtos.*;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public class FileFragment implements Fragment, Comparable<FileFragment>, Cloneable {
+  @Expose private String tableName; // required
+  @Expose private Path uri; // required
+  @Expose public Long startOffset; // required
+  @Expose public Long length; // required
+
+  private String[] hosts; // Datanode hostnames
+  @Expose private int[] diskIds;
+
+  public FileFragment(ByteString raw) throws InvalidProtocolBufferException {
+    FileFragmentProto.Builder builder = FileFragmentProto.newBuilder();
+    builder.mergeFrom(raw);
+    builder.build();
+    init(builder.build());
+  }
+
+  public FileFragment(String tableName, Path uri, BlockLocation blockLocation)
+      throws IOException {
+    this.set(tableName, uri, blockLocation.getOffset(), blockLocation.getLength(), blockLocation.getHosts(), null);
+  }
+
+  public FileFragment(String tableName, Path uri, long start, long length, String[] hosts, int[] diskIds) {
+    this.set(tableName, uri, start, length, hosts, diskIds);
+  }
+  // Non splittable
+  public FileFragment(String tableName, Path uri, long start, long length, String[] hosts) {
+    this.set(tableName, uri, start, length, hosts, null);
+  }
+
+  public FileFragment(String fragmentId, Path path, long start, long length) {
+    this.set(fragmentId, path, start, length, null, null);
+  }
+
+  public FileFragment(FileFragmentProto proto) {
+    init(proto);
+  }
+
+  private void init(FileFragmentProto proto) {
+    int[] diskIds = new int[proto.getDiskIdsList().size()];
+    int i = 0;
+    for(Integer eachValue: proto.getDiskIdsList()) {
+      diskIds[i++] = eachValue;
+    }
+    this.set(proto.getId(), new Path(proto.getPath()),
+        proto.getStartOffset(), proto.getLength(),
+        proto.getHostsList().toArray(new String[]{}),
+        diskIds);
+  }
+
+  private void set(String tableName, Path path, long start,
+      long length, String[] hosts, int[] diskIds) {
+    this.tableName = tableName;
+    this.uri = path;
+    this.startOffset = start;
+    this.length = length;
+    this.hosts = hosts;
+    this.diskIds = diskIds;
+  }
+
+
+  /**
+   * Get the list of hosts (hostname) hosting this block
+   */
+  public String[] getHosts() {
+    if (hosts == null) {
+      this.hosts = new String[0];
+    }
+    return hosts;
+  }
+
+  /**
+   * Get the list of Disk Ids
+   * Unknown disk is -1. Others 0 ~ N
+   */
+  public int[] getDiskIds() {
+    if (diskIds == null) {
+      this.diskIds = new int[getHosts().length];
+      Arrays.fill(this.diskIds, -1);
+    }
+    return diskIds;
+  }
+
+  public void setDiskIds(int[] diskIds){
+    this.diskIds = diskIds;
+  }
+
+  @Override
+  public String getTableName() {
+    return this.tableName;
+  }
+
+  public Path getPath() {
+    return this.uri;
+  }
+
+  public void setPath(Path path) {
+    this.uri = path;
+  }
+
+  public Long getStartKey() {
+    return this.startOffset;
+  }
+
+  @Override
+  public String getKey() {
+    return this.uri.toString();
+  }
+
+  @Override
+  public long getLength() {
+    return this.length;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return this.length <= 0;
+  }
+  /**
+   * 
+   * The offset range of tablets <b>MUST NOT</b> be overlapped.
+   * 
+   * @param t
+   * @return If the table paths are not same, return -1.
+   */
+  @Override
+  public int compareTo(FileFragment t) {
+    if (getPath().equals(t.getPath())) {
+      long diff = this.getStartKey() - t.getStartKey();
+      if (diff < 0) {
+        return -1;
+      } else if (diff > 0) {
+        return 1;
+      } else {
+        return 0;
+      }
+    } else {
+      return -1;
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof FileFragment) {
+      FileFragment t = (FileFragment) o;
+      if (getPath().equals(t.getPath())
+          && TUtil.checkEquals(t.getStartKey(), this.getStartKey())
+          && TUtil.checkEquals(t.getLength(), this.getLength())) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(tableName, uri, startOffset, length);
+  }
+  
+  public Object clone() throws CloneNotSupportedException {
+    FileFragment frag = (FileFragment) super.clone();
+    frag.tableName = tableName;
+    frag.uri = uri;
+    frag.diskIds = diskIds;
+    frag.hosts = hosts;
+
+    return frag;
+  }
+
+  @Override
+  public String toString() {
+    return "\"fragment\": {\"id\": \""+ tableName +"\", \"path\": "
+    		+getPath() + "\", \"start\": " + this.getStartKey() + ",\"length\": "
+        + getLength() + "}" ;
+  }
+
+  public FragmentProto getProto() {
+    FileFragmentProto.Builder builder = FileFragmentProto.newBuilder();
+    builder.setId(this.tableName);
+    builder.setStartOffset(this.startOffset);
+    builder.setLength(this.length);
+    builder.setPath(this.uri.toString());
+    if(diskIds != null) {
+      List<Integer> idList = new ArrayList<Integer>();
+      for(int eachId: diskIds) {
+        idList.add(eachId);
+      }
+      builder.addAllDiskIds(idList);
+    }
+
+    if(hosts != null) {
+      builder.addAllHosts(TUtil.newList(hosts));
+    }
+
+    FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder();
+    fragmentBuilder.setId(this.tableName);
+    fragmentBuilder.setStoreType(StoreType.CSV.name());
+    fragmentBuilder.setContents(builder.buildPartial().toByteString());
+    return fragmentBuilder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexMethod.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
new file mode 100644
index 0000000..a6af19b
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
@@ -0,0 +1,32 @@
+/**
+ * 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.index;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.TupleComparator;
+
+import java.io.IOException;
+
+public interface IndexMethod {
+  IndexWriter getIndexWriter(final Path fileName, int level, Schema keySchema,
+                             TupleComparator comparator) throws IOException;
+  IndexReader getIndexReader(final Path fileName, Schema keySchema,
+                             TupleComparator comparator) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexReader.java
new file mode 100644
index 0000000..3ae5c9d
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexReader.java
@@ -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.
+ */
+
+package org.apache.tajo.storage.index;
+
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public interface IndexReader {
+  
+  /**
+   * Find the offset corresponding to key which is equal to a given key.
+   * 
+   * @param key
+   * @return
+   * @throws java.io.IOException
+   */
+  public long find(Tuple key) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
new file mode 100644
index 0000000..04738f8
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * 
+ */
+package org.apache.tajo.storage.index;
+
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public abstract class IndexWriter {
+  
+  public abstract void write(Tuple key, long offset) throws IOException;
+  
+  public abstract void close() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
new file mode 100644
index 0000000..0c07b4a
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
@@ -0,0 +1,45 @@
+/**
+ * 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.index;
+
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public interface OrderIndexReader extends IndexReader {
+  /**
+   * Find the offset corresponding to key which is equal to or greater than 
+   * a given key.
+   * 
+   * @param key to find
+   * @return
+   * @throws java.io.IOException
+   */
+  public long find(Tuple key, boolean nextKey) throws IOException;
+
+  /**
+   * Return the next offset from the latest find or next offset
+   * @return
+   * @throws java.io.IOException
+   */
+  public long next() throws IOException;
+}


[04/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
new file mode 100644
index 0000000..1448885
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
@@ -0,0 +1,154 @@
+/**
+ * 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.text;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.util.CharsetUtil;
+import org.apache.tajo.storage.BufferPool;
+import org.apache.tajo.storage.ByteBufInputChannel;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class ByteBufLineReader implements Closeable {
+  private static int DEFAULT_BUFFER = 64 * 1024;
+
+  private int bufferSize;
+  private long readBytes;
+  private ByteBuf buffer;
+  private final ByteBufInputChannel channel;
+  private final AtomicInteger tempReadBytes = new AtomicInteger();
+  private final LineSplitProcessor processor = new LineSplitProcessor();
+
+  public ByteBufLineReader(ByteBufInputChannel channel) {
+    this(channel, BufferPool.directBuffer(DEFAULT_BUFFER));
+  }
+
+  public ByteBufLineReader(ByteBufInputChannel channel, ByteBuf buf) {
+    this.readBytes = 0;
+    this.channel = channel;
+    this.buffer = buf;
+    this.bufferSize = buf.capacity();
+  }
+
+  public long readBytes() {
+    return readBytes - buffer.readableBytes();
+  }
+
+  public long available() throws IOException {
+    return channel.available() + buffer.readableBytes();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.buffer.refCnt() > 0) {
+      this.buffer.release();
+    }
+    this.channel.close();
+  }
+
+  public String readLine() throws IOException {
+    ByteBuf buf = readLineBuf(tempReadBytes);
+    if (buf != null) {
+      return buf.toString(CharsetUtil.UTF_8);
+    }
+    return null;
+  }
+
+  private void fillBuffer() throws IOException {
+
+    int tailBytes = 0;
+    if (this.readBytes > 0) {
+      this.buffer.markReaderIndex();
+      this.buffer.discardSomeReadBytes();  // compact the buffer
+      tailBytes = this.buffer.writerIndex();
+      if (!this.buffer.isWritable()) {
+        // a line bytes is large than the buffer
+        BufferPool.ensureWritable(buffer, bufferSize);
+        this.bufferSize = buffer.capacity();
+      }
+    }
+
+    boolean release = true;
+    try {
+      int readBytes = tailBytes;
+      for (; ; ) {
+        int localReadBytes = buffer.writeBytes(channel, bufferSize - readBytes);
+        if (localReadBytes < 0) {
+          break;
+        }
+        readBytes += localReadBytes;
+        if (readBytes == bufferSize) {
+          break;
+        }
+      }
+      this.readBytes += (readBytes - tailBytes);
+      release = false;
+      this.buffer.readerIndex(this.buffer.readerIndex() + tailBytes); //skip past buffer (tail)
+    } finally {
+      if (release) {
+        buffer.release();
+      }
+    }
+  }
+
+  /**
+   * Read a line terminated by one of CR, LF, or CRLF.
+   */
+  public ByteBuf readLineBuf(AtomicInteger reads) throws IOException {
+    int startIndex = buffer.readerIndex();
+    int readBytes;
+    int readable;
+    int newlineLength; //length of terminating newline
+
+    loop:
+    while (true) {
+      readable = buffer.readableBytes();
+      if (readable <= 0) {
+        buffer.readerIndex(startIndex);
+        fillBuffer(); //compact and fill buffer
+        if (!buffer.isReadable()) {
+          return null;
+        } else {
+          startIndex = 0; // reset the line start position
+        }
+        readable = buffer.readableBytes();
+      }
+
+      int endIndex = buffer.forEachByte(buffer.readerIndex(), readable, processor);
+      if (endIndex < 0) {
+        buffer.readerIndex(buffer.writerIndex());
+      } else {
+        buffer.readerIndex(endIndex + 1);
+        readBytes = buffer.readerIndex() - startIndex;
+        if (processor.isPrevCharCR() && buffer.isReadable()
+            && buffer.getByte(buffer.readerIndex()) == LineSplitProcessor.LF) {
+          buffer.skipBytes(1);
+          newlineLength = 2;
+        } else {
+          newlineLength = 1;
+        }
+        break loop;
+      }
+    }
+    reads.set(readBytes);
+    return buffer.slice(startIndex, readBytes - newlineLength);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
new file mode 100644
index 0000000..10d86bd
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
@@ -0,0 +1,157 @@
+/**
+ * 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.text;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.SplittableCompressionCodec;
+import org.apache.tajo.common.exception.NotImplementedException;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.BufferPool;
+import org.apache.tajo.storage.ByteBufInputChannel;
+import org.apache.tajo.storage.FileScanner;
+import org.apache.tajo.storage.compress.CodecPool;
+import org.apache.tajo.storage.fragment.FileFragment;
+
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class DelimitedLineReader implements Closeable {
+  private static final Log LOG = LogFactory.getLog(DelimitedLineReader.class);
+  private final static int DEFAULT_PAGE_SIZE = 128 * 1024;
+
+  private FileSystem fs;
+  private FSDataInputStream fis;
+  private InputStream is; //decompressd stream
+  private CompressionCodecFactory factory;
+  private CompressionCodec codec;
+  private Decompressor decompressor;
+
+  private long startOffset, end, pos;
+  private boolean eof = true;
+  private ByteBufLineReader lineReader;
+  private AtomicInteger tempReadBytes = new AtomicInteger();
+  private FileFragment fragment;
+  private Configuration conf;
+
+  public DelimitedLineReader(Configuration conf, final FileFragment fragment) throws IOException {
+    this.fragment = fragment;
+    this.conf = conf;
+    this.factory = new CompressionCodecFactory(conf);
+    this.codec = factory.getCodec(fragment.getPath());
+    if (this.codec instanceof SplittableCompressionCodec) {
+      throw new NotImplementedException(); // bzip2 does not support multi-thread model
+    }
+  }
+
+  public void init() throws IOException {
+    if (fs == null) {
+      fs = FileScanner.getFileSystem((TajoConf) conf, fragment.getPath());
+    }
+    if (fis == null) fis = fs.open(fragment.getPath());
+    pos = startOffset = fragment.getStartKey();
+    end = startOffset + fragment.getLength();
+
+    if (codec != null) {
+      decompressor = CodecPool.getDecompressor(codec);
+      is = new DataInputStream(codec.createInputStream(fis, decompressor));
+      ByteBufInputChannel channel = new ByteBufInputChannel(is);
+      lineReader = new ByteBufLineReader(channel, BufferPool.directBuffer(DEFAULT_PAGE_SIZE));
+    } else {
+      fis.seek(startOffset);
+      is = fis;
+
+      ByteBufInputChannel channel = new ByteBufInputChannel(is);
+      lineReader = new ByteBufLineReader(channel,
+          BufferPool.directBuffer((int) Math.min(DEFAULT_PAGE_SIZE, end)));
+    }
+    eof = false;
+  }
+
+  public long getCompressedPosition() throws IOException {
+    long retVal;
+    if (isCompressed()) {
+      retVal = fis.getPos();
+    } else {
+      retVal = pos;
+    }
+    return retVal;
+  }
+
+  public long getUnCompressedPosition() throws IOException {
+    return pos;
+  }
+
+  public long getReadBytes() {
+    return pos - startOffset;
+  }
+
+  public boolean isReadable() {
+    return !eof;
+  }
+
+  public ByteBuf readLine() throws IOException {
+    if (eof) {
+      return null;
+    }
+
+    ByteBuf buf = lineReader.readLineBuf(tempReadBytes);
+    if (buf == null) {
+      eof = true;
+    } else {
+      pos += tempReadBytes.get();
+    }
+
+    if (!isCompressed() && getCompressedPosition() > end) {
+      eof = true;
+    }
+    return buf;
+  }
+
+  public boolean isCompressed() {
+    return codec != null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.cleanup(LOG, lineReader, is, fis);
+      fs = null;
+      is = null;
+      fis = null;
+      lineReader = null;
+    } finally {
+      if (decompressor != null) {
+        CodecPool.returnDecompressor(decompressor);
+        decompressor = null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
new file mode 100644
index 0000000..a337509
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
@@ -0,0 +1,468 @@
+/**
+ * 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.text;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.compress.CodecPool;
+import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+
+public class DelimitedTextFile {
+
+  public static final byte LF = '\n';
+  public static int EOF = -1;
+
+  private static final Log LOG = LogFactory.getLog(DelimitedTextFile.class);
+
+  public static class DelimitedTextFileAppender extends FileAppender {
+    private final TableMeta meta;
+    private final Schema schema;
+    private final int columnNum;
+    private final FileSystem fs;
+    private FSDataOutputStream fos;
+    private DataOutputStream outputStream;
+    private CompressionOutputStream deflateFilter;
+    private char delimiter;
+    private TableStatistics stats = null;
+    private Compressor compressor;
+    private CompressionCodecFactory codecFactory;
+    private CompressionCodec codec;
+    private Path compressedPath;
+    private byte[] nullChars;
+    private int BUFFER_SIZE = 128 * 1024;
+    private int bufferedBytes = 0;
+    private long pos = 0;
+
+    private NonSyncByteArrayOutputStream os;
+    private FieldSerializerDeserializer serde;
+
+    public DelimitedTextFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                                     final Schema schema, final TableMeta meta, final Path path)
+        throws IOException {
+      super(conf, taskAttemptId, schema, meta, path);
+      this.fs = path.getFileSystem(conf);
+      this.meta = meta;
+      this.schema = schema;
+      this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_DELIMITER,
+          StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
+      this.columnNum = schema.size();
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_NULL,
+          NullDatum.DEFAULT_TEXT));
+      if (StringUtils.isEmpty(nullCharacters)) {
+        nullChars = NullDatum.get().asTextBytes();
+      } else {
+        nullChars = nullCharacters.getBytes();
+      }
+    }
+
+    @Override
+    public void init() throws IOException {
+      if (!fs.exists(path.getParent())) {
+        throw new FileNotFoundException(path.toString());
+      }
+
+      if (this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
+        String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
+        codecFactory = new CompressionCodecFactory(conf);
+        codec = codecFactory.getCodecByClassName(codecName);
+        compressor = CodecPool.getCompressor(codec);
+        if (compressor != null) compressor.reset();  //builtin gzip is null
+
+        String extension = codec.getDefaultExtension();
+        compressedPath = path.suffix(extension);
+
+        if (fs.exists(compressedPath)) {
+          throw new AlreadyExistsStorageException(compressedPath);
+        }
+
+        fos = fs.create(compressedPath);
+        deflateFilter = codec.createOutputStream(fos, compressor);
+        outputStream = new DataOutputStream(deflateFilter);
+
+      } else {
+        if (fs.exists(path)) {
+          throw new AlreadyExistsStorageException(path);
+        }
+        fos = fs.create(path);
+        outputStream = new DataOutputStream(new BufferedOutputStream(fos));
+      }
+
+      if (enabledStats) {
+        this.stats = new TableStatistics(this.schema);
+      }
+
+      serde = new TextFieldSerializerDeserializer();
+
+      if (os == null) {
+        os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
+      }
+
+      os.reset();
+      pos = fos.getPos();
+      bufferedBytes = 0;
+      super.init();
+    }
+
+
+    @Override
+    public void addTuple(Tuple tuple) throws IOException {
+      Datum datum;
+      int rowBytes = 0;
+
+      for (int i = 0; i < columnNum; i++) {
+        datum = tuple.get(i);
+        rowBytes += serde.serialize(os, datum, schema.getColumn(i), i, nullChars);
+
+        if (columnNum - 1 > i) {
+          os.write((byte) delimiter);
+          rowBytes += 1;
+        }
+      }
+      os.write(LF);
+      rowBytes += 1;
+
+      pos += rowBytes;
+      bufferedBytes += rowBytes;
+      if (bufferedBytes > BUFFER_SIZE) {
+        flushBuffer();
+      }
+      // Statistical section
+      if (enabledStats) {
+        stats.incrementRow();
+      }
+    }
+
+    private void flushBuffer() throws IOException {
+      if (os.getLength() > 0) {
+        os.writeTo(outputStream);
+        os.reset();
+        bufferedBytes = 0;
+      }
+    }
+
+    @Override
+    public long getOffset() throws IOException {
+      return pos;
+    }
+
+    @Override
+    public void flush() throws IOException {
+      flushBuffer();
+      outputStream.flush();
+    }
+
+    @Override
+    public void close() throws IOException {
+
+      try {
+        if(outputStream != null){
+          flush();
+        }
+
+        // Statistical section
+        if (enabledStats) {
+          stats.setNumBytes(getOffset());
+        }
+
+        if (deflateFilter != null) {
+          deflateFilter.finish();
+          deflateFilter.resetState();
+          deflateFilter = null;
+        }
+
+        os.close();
+      } finally {
+        IOUtils.cleanup(LOG, fos);
+        if (compressor != null) {
+          CodecPool.returnCompressor(compressor);
+          compressor = null;
+        }
+      }
+    }
+
+    @Override
+    public TableStats getStats() {
+      if (enabledStats) {
+        return stats.getTableStat();
+      } else {
+        return null;
+      }
+    }
+
+    public boolean isCompress() {
+      return compressor != null;
+    }
+
+    public String getExtension() {
+      return codec != null ? codec.getDefaultExtension() : "";
+    }
+  }
+
+  public static class DelimitedTextFileScanner extends FileScanner {
+
+    private boolean splittable = false;
+    private final long startOffset;
+    private final long endOffset;
+
+    private int recordCount = 0;
+    private int[] targetColumnIndexes;
+
+    private ByteBuf nullChars;
+    private FieldSerializerDeserializer serde;
+    private DelimitedLineReader reader;
+    private FieldSplitProcessor processor;
+
+    public DelimitedTextFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
+                                    final Fragment fragment)
+        throws IOException {
+      super(conf, schema, meta, fragment);
+      reader = new DelimitedLineReader(conf, this.fragment);
+      if (!reader.isCompressed()) {
+        splittable = true;
+      }
+
+      startOffset = this.fragment.getStartKey();
+      endOffset = startOffset + fragment.getLength();
+
+      //Delimiter
+      String delim = meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+      this.processor = new FieldSplitProcessor(StringEscapeUtils.unescapeJava(delim).charAt(0));
+    }
+
+    @Override
+    public void init() throws IOException {
+      if (nullChars != null) {
+        nullChars.release();
+      }
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_NULL,
+          NullDatum.DEFAULT_TEXT));
+      byte[] bytes;
+      if (StringUtils.isEmpty(nullCharacters)) {
+        bytes = NullDatum.get().asTextBytes();
+      } else {
+        bytes = nullCharacters.getBytes();
+      }
+
+      nullChars = BufferPool.directBuffer(bytes.length, bytes.length);
+      nullChars.writeBytes(bytes);
+
+      if (reader != null) {
+        reader.close();
+      }
+      reader = new DelimitedLineReader(conf, fragment);
+      reader.init();
+      recordCount = 0;
+
+      if (targets == null) {
+        targets = schema.toArray();
+      }
+
+      targetColumnIndexes = new int[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
+      }
+
+      serde = new TextFieldSerializerDeserializer();
+
+      super.init();
+      Arrays.sort(targetColumnIndexes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("DelimitedTextFileScanner open:" + fragment.getPath() + "," + startOffset + "," + endOffset);
+      }
+
+      if (startOffset > 0) {
+        reader.readLine();  // skip first line;
+      }
+    }
+
+    public ByteBuf readLine() throws IOException {
+      ByteBuf buf = reader.readLine();
+      if (buf == null) {
+        return null;
+      } else {
+        recordCount++;
+      }
+
+      return buf;
+    }
+
+    @Override
+    public float getProgress() {
+      try {
+        if (!reader.isReadable()) {
+          return 1.0f;
+        }
+        long filePos = reader.getCompressedPosition();
+        if (startOffset == filePos) {
+          return 0.0f;
+        } else {
+          long readBytes = filePos - startOffset;
+          long remainingBytes = Math.max(endOffset - filePos, 0);
+          return Math.min(1.0f, (float) (readBytes) / (float) (readBytes + remainingBytes));
+        }
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+        return 0.0f;
+      }
+    }
+
+    @Override
+    public Tuple next() throws IOException {
+      try {
+        if (!reader.isReadable()) return null;
+
+        ByteBuf buf = readLine();
+        if (buf == null) return null;
+
+        if (targets.length == 0) {
+          return EmptyTuple.get();
+        }
+
+        VTuple tuple = new VTuple(schema.size());
+        fillTuple(schema, tuple, buf, targetColumnIndexes);
+        return tuple;
+      } catch (Throwable t) {
+        LOG.error("Tuple list current index: " + recordCount + " file offset:" + reader.getCompressedPosition(), t);
+        throw new IOException(t);
+      }
+    }
+
+    private void fillTuple(Schema schema, Tuple dst, ByteBuf lineBuf, int[] target) throws IOException {
+      int[] projection = target;
+      if (lineBuf == null || target == null || target.length == 0) {
+        return;
+      }
+
+      final int rowLength = lineBuf.readableBytes();
+      int start = 0, fieldLength = 0, end = 0;
+
+      //Projection
+      int currentTarget = 0;
+      int currentIndex = 0;
+
+      while (end != -1) {
+        end = lineBuf.forEachByte(start, rowLength - start, processor);
+
+        if (end < 0) {
+          fieldLength = rowLength - start;
+        } else {
+          fieldLength = end - start;
+        }
+
+        if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {
+          lineBuf.setIndex(start, start + fieldLength);
+          Datum datum = serde.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars);
+          dst.put(currentIndex, datum);
+          currentTarget++;
+        }
+
+        if (projection.length == currentTarget) {
+          break;
+        }
+
+        start = end + 1;
+        currentIndex++;
+      }
+    }
+
+    @Override
+    public void reset() throws IOException {
+      init();
+    }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        if (nullChars != null) {
+          nullChars.release();
+          nullChars = null;
+        }
+
+        if (tableStats != null && reader != null) {
+          tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
+          tableStats.setNumRows(recordCount);
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("DelimitedTextFileScanner processed record:" + recordCount);
+        }
+      } finally {
+        IOUtils.cleanup(LOG, reader);
+        reader = null;
+      }
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return true;
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public void setSearchCondition(Object expr) {
+    }
+
+    @Override
+    public boolean isSplittable() {
+      return splittable;
+    }
+
+    @Override
+    public TableStats getInputStats() {
+      if (tableStats != null && reader != null) {
+        tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
+        tableStats.setNumRows(recordCount);
+        tableStats.setNumBytes(fragment.getLength());
+      }
+      return tableStats;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
new file mode 100644
index 0000000..a5ac142
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
@@ -0,0 +1,38 @@
+/**
+ * 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.text;
+
+import io.netty.buffer.ByteBufProcessor;
+
+public class FieldSplitProcessor implements ByteBufProcessor {
+  private char delimiter; //the ascii separate character
+
+  public FieldSplitProcessor(char recordDelimiterByte) {
+    this.delimiter = recordDelimiterByte;
+  }
+
+  @Override
+  public boolean process(byte value) throws Exception {
+    return delimiter != value;
+  }
+
+  public char getDelimiter() {
+    return delimiter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
new file mode 100644
index 0000000..a130527
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
@@ -0,0 +1,45 @@
+/**
+ * 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.text;
+
+import io.netty.buffer.ByteBufProcessor;
+
+public class LineSplitProcessor implements ByteBufProcessor {
+  public static final byte CR = '\r';
+  public static final byte LF = '\n';
+  private boolean prevCharCR = false; //true of prev char was CR
+
+  @Override
+  public boolean process(byte value) throws Exception {
+    switch (value) {
+      case LF:
+        return false;
+      case CR:
+        prevCharCR = true;
+        return false;
+      default:
+        prevCharCR = false;
+        return true;
+    }
+  }
+
+  public boolean isPrevCharCR() {
+    return prevCharCR;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
new file mode 100644
index 0000000..9722959
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.text;
+
+import com.google.protobuf.Message;
+import io.netty.buffer.ByteBuf;
+import io.netty.util.CharsetUtil;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
+import org.apache.tajo.storage.FieldSerializerDeserializer;
+import org.apache.tajo.util.NumberUtil;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.CharsetDecoder;
+
+//Compatibility with Apache Hive
+public class TextFieldSerializerDeserializer implements FieldSerializerDeserializer {
+  public static final byte[] trueBytes = "true".getBytes();
+  public static final byte[] falseBytes = "false".getBytes();
+  private ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
+  private final CharsetDecoder decoder = CharsetUtil.getDecoder(CharsetUtil.UTF_8);
+
+  private static boolean isNull(ByteBuf val, ByteBuf nullBytes) {
+    return !val.isReadable() || nullBytes.equals(val);
+  }
+
+  private static boolean isNullText(ByteBuf val, ByteBuf nullBytes) {
+    return val.readableBytes() > 0 && nullBytes.equals(val);
+  }
+
+  @Override
+  public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) throws IOException {
+    byte[] bytes;
+    int length = 0;
+    TajoDataTypes.DataType dataType = col.getDataType();
+
+    if (datum == null || datum instanceof NullDatum) {
+      switch (dataType.getType()) {
+        case CHAR:
+        case TEXT:
+          length = nullChars.length;
+          out.write(nullChars);
+          break;
+        default:
+          break;
+      }
+      return length;
+    }
+
+    switch (dataType.getType()) {
+      case BOOLEAN:
+        out.write(datum.asBool() ? trueBytes : falseBytes);
+        length = trueBytes.length;
+        break;
+      case CHAR:
+        byte[] pad = new byte[dataType.getLength() - datum.size()];
+        bytes = datum.asTextBytes();
+        out.write(bytes);
+        out.write(pad);
+        length = bytes.length + pad.length;
+        break;
+      case TEXT:
+      case BIT:
+      case INT2:
+      case INT4:
+      case INT8:
+      case FLOAT4:
+      case FLOAT8:
+      case INET4:
+      case DATE:
+      case INTERVAL:
+        bytes = datum.asTextBytes();
+        length = bytes.length;
+        out.write(bytes);
+        break;
+      case TIME:
+        bytes = ((TimeDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        length = bytes.length;
+        out.write(bytes);
+        break;
+      case TIMESTAMP:
+        bytes = ((TimestampDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
+        length = bytes.length;
+        out.write(bytes);
+        break;
+      case INET6:
+      case BLOB:
+        bytes = Base64.encodeBase64(datum.asByteArray(), false);
+        length = bytes.length;
+        out.write(bytes, 0, length);
+        break;
+      case PROTOBUF:
+        ProtobufDatum protobuf = (ProtobufDatum) datum;
+        byte[] protoBytes = protobufJsonFormat.printToString(protobuf.get()).getBytes();
+        length = protoBytes.length;
+        out.write(protoBytes, 0, protoBytes.length);
+        break;
+      case NULL_TYPE:
+      default:
+        break;
+    }
+    return length;
+  }
+
+  @Override
+  public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars) throws IOException {
+    Datum datum;
+    TajoDataTypes.Type type = col.getDataType().getType();
+    boolean nullField;
+    if (type == TajoDataTypes.Type.TEXT || type == TajoDataTypes.Type.CHAR) {
+      nullField = isNullText(buf, nullChars);
+    } else {
+      nullField = isNull(buf, nullChars);
+    }
+
+    if (nullField) {
+      datum = NullDatum.get();
+    } else {
+      switch (type) {
+        case BOOLEAN:
+          byte bool = buf.readByte();
+          datum = DatumFactory.createBool(bool == 't' || bool == 'T');
+          break;
+        case BIT:
+          datum = DatumFactory.createBit(Byte.parseByte(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()));
+          break;
+        case CHAR:
+          datum = DatumFactory.createChar(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString().trim());
+          break;
+        case INT1:
+        case INT2:
+          datum = DatumFactory.createInt2((short) NumberUtil.parseInt(buf));
+          break;
+        case INT4:
+          datum = DatumFactory.createInt4(NumberUtil.parseInt(buf));
+          break;
+        case INT8:
+          datum = DatumFactory.createInt8(NumberUtil.parseLong(buf));
+          break;
+        case FLOAT4:
+          datum = DatumFactory.createFloat4(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          break;
+        case FLOAT8:
+          datum = DatumFactory.createFloat8(NumberUtil.parseDouble(buf));
+          break;
+        case TEXT: {
+          byte[] bytes = new byte[buf.readableBytes()];
+          buf.readBytes(bytes);
+          datum = DatumFactory.createText(bytes);
+          break;
+        }
+        case DATE:
+          datum = DatumFactory.createDate(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          break;
+        case TIME:
+          datum = DatumFactory.createTime(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          break;
+        case TIMESTAMP:
+          datum = DatumFactory.createTimestamp(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          break;
+        case INTERVAL:
+          datum = DatumFactory.createInterval(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          break;
+        case PROTOBUF: {
+          ProtobufDatumFactory factory = ProtobufDatumFactory.get(col.getDataType());
+          Message.Builder builder = factory.newBuilder();
+          try {
+            byte[] bytes = new byte[buf.readableBytes()];
+            buf.readBytes(bytes);
+            protobufJsonFormat.merge(bytes, builder);
+            datum = factory.createDatum(builder.build());
+          } catch (IOException e) {
+            e.printStackTrace();
+            throw new RuntimeException(e);
+          }
+          break;
+        }
+        case INET4:
+          datum = DatumFactory.createInet4(
+              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
+          break;
+        case BLOB: {
+          byte[] bytes = new byte[buf.readableBytes()];
+          buf.readBytes(bytes);
+          datum = DatumFactory.createBlob(Base64.decodeBase64(bytes));
+          break;
+        }
+        default:
+          datum = NullDatum.get();
+          break;
+      }
+    }
+    return datum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
new file mode 100644
index 0000000..f76593e
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
@@ -0,0 +1,190 @@
+/**
+ * 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.thirdparty.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.*;
+import org.apache.hadoop.util.ReflectionUtils;
+import parquet.bytes.BytesInput;
+import parquet.hadoop.BadConfigurationException;
+import parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+class CodecFactory {
+
+  public class BytesDecompressor {
+
+    private final CompressionCodec codec;
+    private final Decompressor decompressor;
+
+    public BytesDecompressor(CompressionCodec codec) {
+      this.codec = codec;
+      if (codec != null) {
+        decompressor = CodecPool.getDecompressor(codec);
+      } else {
+        decompressor = null;
+      }
+    }
+
+    public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException {
+      final BytesInput decompressed;
+      if (codec != null) {
+        decompressor.reset();
+        InputStream is = codec.createInputStream(new ByteArrayInputStream(bytes.toByteArray()), decompressor);
+        decompressed = BytesInput.from(is, uncompressedSize);
+      } else {
+        decompressed = bytes;
+      }
+      return decompressed;
+    }
+
+    private void release() {
+      if (decompressor != null) {
+        CodecPool.returnDecompressor(decompressor);
+      }
+    }
+  }
+
+  /**
+   * Encapsulates the logic around hadoop compression
+   *
+   * @author Julien Le Dem
+   *
+   */
+  public static class BytesCompressor {
+
+    private final CompressionCodec codec;
+    private final Compressor compressor;
+    private final ByteArrayOutputStream compressedOutBuffer;
+    private final CompressionCodecName codecName;
+
+    public BytesCompressor(CompressionCodecName codecName, CompressionCodec codec, int pageSize) {
+      this.codecName = codecName;
+      this.codec = codec;
+      if (codec != null) {
+        this.compressor = CodecPool.getCompressor(codec);
+        this.compressedOutBuffer = new ByteArrayOutputStream(pageSize);
+      } else {
+        this.compressor = null;
+        this.compressedOutBuffer = null;
+      }
+    }
+
+    public BytesInput compress(BytesInput bytes) throws IOException {
+      final BytesInput compressedBytes;
+      if (codec == null) {
+        compressedBytes = bytes;
+      } else {
+        compressedOutBuffer.reset();
+        if (compressor != null) {
+          // null compressor for non-native gzip
+          compressor.reset();
+        }
+        CompressionOutputStream cos = codec.createOutputStream(compressedOutBuffer, compressor);
+        bytes.writeAllTo(cos);
+        cos.finish();
+        cos.close();
+        compressedBytes = BytesInput.from(compressedOutBuffer);
+      }
+      return compressedBytes;
+    }
+
+    private void release() {
+      if (compressor != null) {
+        CodecPool.returnCompressor(compressor);
+      }
+    }
+
+    public CompressionCodecName getCodecName() {
+      return codecName;
+    }
+
+  }
+
+  private final Map<CompressionCodecName, BytesCompressor> compressors = new HashMap<CompressionCodecName, BytesCompressor>();
+  private final Map<CompressionCodecName, BytesDecompressor> decompressors = new HashMap<CompressionCodecName, BytesDecompressor>();
+  private final Map<String, CompressionCodec> codecByName = new HashMap<String, CompressionCodec>();
+  private final Configuration configuration;
+
+  public CodecFactory(Configuration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   *
+   * @param codecName the requested codec
+   * @return the corresponding hadoop codec. null if UNCOMPRESSED
+   */
+  private CompressionCodec getCodec(CompressionCodecName codecName) {
+    String codecClassName = codecName.getHadoopCompressionCodecClassName();
+    if (codecClassName == null) {
+      return null;
+    }
+    CompressionCodec codec = codecByName.get(codecClassName);
+    if (codec != null) {
+      return codec;
+    }
+
+    try {
+      Class<?> codecClass = Class.forName(codecClassName);
+      codec = (CompressionCodec)ReflectionUtils.newInstance(codecClass, configuration);
+      codecByName.put(codecClassName, codec);
+      return codec;
+    } catch (ClassNotFoundException e) {
+      throw new BadConfigurationException("Class " + codecClassName + " was not found", e);
+    }
+  }
+
+  public BytesCompressor getCompressor(CompressionCodecName codecName, int pageSize) {
+    BytesCompressor comp = compressors.get(codecName);
+    if (comp == null) {
+      CompressionCodec codec = getCodec(codecName);
+      comp = new BytesCompressor(codecName, codec, pageSize);
+      compressors.put(codecName, comp);
+    }
+    return comp;
+  }
+
+  public BytesDecompressor getDecompressor(CompressionCodecName codecName) {
+    BytesDecompressor decomp = decompressors.get(codecName);
+    if (decomp == null) {
+      CompressionCodec codec = getCodec(codecName);
+      decomp = new BytesDecompressor(codec);
+      decompressors.put(codecName, decomp);
+    }
+    return decomp;
+  }
+
+  public void release() {
+    for (BytesCompressor compressor : compressors.values()) {
+      compressor.release();
+    }
+    compressors.clear();
+    for (BytesDecompressor decompressor : decompressors.values()) {
+      decompressor.release();
+    }
+    decompressors.clear();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
new file mode 100644
index 0000000..0dedd9b
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
@@ -0,0 +1,206 @@
+/**
+ * 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.thirdparty.parquet;
+
+import parquet.Log;
+import parquet.bytes.BytesInput;
+import parquet.bytes.CapacityByteArrayOutputStream;
+import parquet.column.ColumnDescriptor;
+import parquet.column.Encoding;
+import parquet.column.page.DictionaryPage;
+import parquet.column.page.PageWriteStore;
+import parquet.column.page.PageWriter;
+import parquet.column.statistics.BooleanStatistics;
+import parquet.column.statistics.Statistics;
+import parquet.format.converter.ParquetMetadataConverter;
+import parquet.io.ParquetEncodingException;
+import parquet.schema.MessageType;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
+import static parquet.Log.INFO;
+
+class ColumnChunkPageWriteStore implements PageWriteStore {
+  private static final Log LOG = Log.getLog(ColumnChunkPageWriteStore.class);
+
+  private static ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
+
+  private static final class ColumnChunkPageWriter implements PageWriter {
+
+    private final ColumnDescriptor path;
+    private final BytesCompressor compressor;
+
+    private final CapacityByteArrayOutputStream buf;
+    private DictionaryPage dictionaryPage;
+
+    private long uncompressedLength;
+    private long compressedLength;
+    private long totalValueCount;
+    private int pageCount;
+
+    private Set<Encoding> encodings = new HashSet<Encoding>();
+
+    private Statistics totalStatistics;
+
+    private ColumnChunkPageWriter(ColumnDescriptor path, BytesCompressor compressor, int initialSize) {
+      this.path = path;
+      this.compressor = compressor;
+      this.buf = new CapacityByteArrayOutputStream(initialSize);
+      this.totalStatistics = Statistics.getStatsBasedOnType(this.path.getType());
+    }
+
+    @Deprecated
+    @Override
+    public void writePage(BytesInput bytes,
+                          int valueCount,
+                          Encoding rlEncoding,
+                          Encoding dlEncoding,
+                          Encoding valuesEncoding) throws IOException {
+      long uncompressedSize = bytes.size();
+      BytesInput compressedBytes = compressor.compress(bytes);
+      long compressedSize = compressedBytes.size();
+      BooleanStatistics statistics = new BooleanStatistics(); // dummy stats object
+      parquetMetadataConverter.writeDataPageHeader(
+          (int)uncompressedSize,
+          (int)compressedSize,
+          valueCount,
+          statistics,
+          rlEncoding,
+          dlEncoding,
+          valuesEncoding,
+          buf);
+      this.uncompressedLength += uncompressedSize;
+      this.compressedLength += compressedSize;
+      this.totalValueCount += valueCount;
+      this.pageCount += 1;
+      compressedBytes.writeAllTo(buf);
+      encodings.add(rlEncoding);
+      encodings.add(dlEncoding);
+      encodings.add(valuesEncoding);
+    }
+
+    @Override
+    public void writePage(BytesInput bytes,
+                          int valueCount,
+                          Statistics statistics,
+                          Encoding rlEncoding,
+                          Encoding dlEncoding,
+                          Encoding valuesEncoding) throws IOException {
+      long uncompressedSize = bytes.size();
+      BytesInput compressedBytes = compressor.compress(bytes);
+      long compressedSize = compressedBytes.size();
+      parquetMetadataConverter.writeDataPageHeader(
+          (int)uncompressedSize,
+          (int)compressedSize,
+          valueCount,
+          statistics,
+          rlEncoding,
+          dlEncoding,
+          valuesEncoding,
+          buf);
+      this.uncompressedLength += uncompressedSize;
+      this.compressedLength += compressedSize;
+      this.totalValueCount += valueCount;
+      this.pageCount += 1;
+      this.totalStatistics.mergeStatistics(statistics);
+      compressedBytes.writeAllTo(buf);
+      encodings.add(rlEncoding);
+      encodings.add(dlEncoding);
+      encodings.add(valuesEncoding);
+    }
+
+    @Override
+    public long getMemSize() {
+      return buf.size();
+    }
+
+    public void writeToFileWriter(ParquetFileWriter writer) throws IOException {
+      writer.startColumn(path, totalValueCount, compressor.getCodecName());
+      if (dictionaryPage != null) {
+        writer.writeDictionaryPage(dictionaryPage);
+        encodings.add(dictionaryPage.getEncoding());
+      }
+      writer.writeDataPages(BytesInput.from(buf), uncompressedLength, compressedLength, totalStatistics, new ArrayList<Encoding>(encodings));
+      writer.endColumn();
+      if (INFO) {
+        LOG.info(
+            String.format(
+                "written %,dB for %s: %,d values, %,dB raw, %,dB comp, %d pages, encodings: %s",
+                buf.size(), path, totalValueCount, uncompressedLength, compressedLength, pageCount, encodings)
+                + (dictionaryPage != null ? String.format(
+                ", dic { %,d entries, %,dB raw, %,dB comp}",
+                dictionaryPage.getDictionarySize(), dictionaryPage.getUncompressedSize(), dictionaryPage.getDictionarySize())
+                : ""));
+      }
+      encodings.clear();
+      pageCount = 0;
+    }
+
+    @Override
+    public long allocatedSize() {
+      return buf.getCapacity();
+    }
+
+    @Override
+    public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException {
+      if (this.dictionaryPage != null) {
+        throw new ParquetEncodingException("Only one dictionary page is allowed");
+      }
+      BytesInput dictionaryBytes = dictionaryPage.getBytes();
+      int uncompressedSize = (int)dictionaryBytes.size();
+      BytesInput compressedBytes = compressor.compress(dictionaryBytes);
+      this.dictionaryPage = new DictionaryPage(BytesInput.copy(compressedBytes), uncompressedSize, dictionaryPage.getDictionarySize(), dictionaryPage.getEncoding());
+    }
+
+    @Override
+    public String memUsageString(String prefix) {
+      return buf.memUsageString(prefix + " ColumnChunkPageWriter");
+    }
+  }
+
+  private final Map<ColumnDescriptor, ColumnChunkPageWriter> writers = new HashMap<ColumnDescriptor, ColumnChunkPageWriter>();
+  private final MessageType schema;
+  private final BytesCompressor compressor;
+  private final int initialSize;
+
+  public ColumnChunkPageWriteStore(BytesCompressor compressor, MessageType schema, int initialSize) {
+    this.compressor = compressor;
+    this.schema = schema;
+    this.initialSize = initialSize;
+  }
+
+  @Override
+  public PageWriter getPageWriter(ColumnDescriptor path) {
+    if (!writers.containsKey(path)) {
+      writers.put(path,  new ColumnChunkPageWriter(path, compressor, initialSize));
+    }
+    return writers.get(path);
+  }
+
+  public void flushToFileWriter(ParquetFileWriter writer) throws IOException {
+    List<ColumnDescriptor> columns = schema.getColumns();
+    for (ColumnDescriptor columnDescriptor : columns) {
+      ColumnChunkPageWriter pageWriter = writers.get(columnDescriptor);
+      pageWriter.writeToFileWriter(writer);
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
new file mode 100644
index 0000000..6bbd7b5
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
@@ -0,0 +1,188 @@
+/**
+ * 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.thirdparty.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import parquet.Log;
+import parquet.column.ColumnDescriptor;
+import parquet.column.page.PageReadStore;
+import parquet.filter.UnboundRecordFilter;
+import parquet.hadoop.ParquetFileReader;
+import parquet.hadoop.api.ReadSupport;
+import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.util.counters.BenchmarkCounter;
+import parquet.io.ColumnIOFactory;
+import parquet.io.MessageColumnIO;
+import parquet.io.ParquetDecodingException;
+import parquet.io.api.RecordMaterializer;
+import parquet.schema.GroupType;
+import parquet.schema.MessageType;
+import parquet.schema.Type;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import static java.lang.String.format;
+import static parquet.Log.DEBUG;
+
+class InternalParquetRecordReader<T> {
+  private static final Log LOG = Log.getLog(InternalParquetRecordReader.class);
+
+  private final ColumnIOFactory columnIOFactory = new ColumnIOFactory();
+
+  private MessageType requestedSchema;
+  private MessageType fileSchema;
+  private int columnCount;
+  private final ReadSupport<T> readSupport;
+
+  private RecordMaterializer<T> recordConverter;
+
+  private T currentValue;
+  private long total;
+  private int current = 0;
+  private int currentBlock = -1;
+  private ParquetFileReader reader;
+  private parquet.io.RecordReader<T> recordReader;
+  private UnboundRecordFilter recordFilter;
+
+  private long totalTimeSpentReadingBytes;
+  private long totalTimeSpentProcessingRecords;
+  private long startedAssemblingCurrentBlockAt;
+
+  private long totalCountLoadedSoFar = 0;
+
+  private Path file;
+
+  /**
+   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
+   */
+  public InternalParquetRecordReader(ReadSupport<T> readSupport) {
+    this(readSupport, null);
+  }
+
+  /**
+   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
+   * @param filter Optional filter for only returning matching records.
+   */
+  public InternalParquetRecordReader(ReadSupport<T> readSupport, UnboundRecordFilter
+      filter) {
+    this.readSupport = readSupport;
+    this.recordFilter = filter;
+  }
+
+  private void checkRead() throws IOException {
+    if (current == totalCountLoadedSoFar) {
+      if (current != 0) {
+        long timeAssembling = System.currentTimeMillis() - startedAssemblingCurrentBlockAt;
+        totalTimeSpentProcessingRecords += timeAssembling;
+        LOG.info("Assembled and processed " + totalCountLoadedSoFar + " records from " + columnCount + " columns in " + totalTimeSpentProcessingRecords + " ms: "+((float)totalCountLoadedSoFar / totalTimeSpentProcessingRecords) + " rec/ms, " + ((float)totalCountLoadedSoFar * columnCount / totalTimeSpentProcessingRecords) + " cell/ms");
+        long totalTime = totalTimeSpentProcessingRecords + totalTimeSpentReadingBytes;
+        long percentReading = 100 * totalTimeSpentReadingBytes / totalTime;
+        long percentProcessing = 100 * totalTimeSpentProcessingRecords / totalTime;
+        LOG.info("time spent so far " + percentReading + "% reading ("+totalTimeSpentReadingBytes+" ms) and " + percentProcessing + "% processing ("+totalTimeSpentProcessingRecords+" ms)");
+      }
+
+      LOG.info("at row " + current + ". reading next block");
+      long t0 = System.currentTimeMillis();
+      PageReadStore pages = reader.readNextRowGroup();
+      if (pages == null) {
+        throw new IOException("expecting more rows but reached last block. Read " + current + " out of " + total);
+      }
+      long timeSpentReading = System.currentTimeMillis() - t0;
+      totalTimeSpentReadingBytes += timeSpentReading;
+      BenchmarkCounter.incrementTime(timeSpentReading);
+      LOG.info("block read in memory in " + timeSpentReading + " ms. row count = " + pages.getRowCount());
+      if (Log.DEBUG) LOG.debug("initializing Record assembly with requested schema " + requestedSchema);
+      MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema, fileSchema);
+      recordReader = columnIO.getRecordReader(pages, recordConverter, recordFilter);
+      startedAssemblingCurrentBlockAt = System.currentTimeMillis();
+      totalCountLoadedSoFar += pages.getRowCount();
+      ++ currentBlock;
+    }
+  }
+
+  public void close() throws IOException {
+    reader.close();
+  }
+
+  public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  public T getCurrentValue() throws IOException,
+      InterruptedException {
+    return currentValue;
+  }
+
+  public float getProgress() throws IOException, InterruptedException {
+    return (float) current / total;
+  }
+
+  public void initialize(MessageType requestedSchema, MessageType fileSchema,
+                         Map<String, String> extraMetadata, Map<String, String> readSupportMetadata,
+                         Path file, List<BlockMetaData> blocks, Configuration configuration)
+      throws IOException {
+    this.requestedSchema = requestedSchema;
+    this.fileSchema = fileSchema;
+    this.file = file;
+    this.columnCount = this.requestedSchema.getPaths().size();
+    this.recordConverter = readSupport.prepareForRead(
+        configuration, extraMetadata, fileSchema,
+        new ReadSupport.ReadContext(requestedSchema, readSupportMetadata));
+
+    List<ColumnDescriptor> columns = requestedSchema.getColumns();
+    reader = new ParquetFileReader(configuration, file, blocks, columns);
+    for (BlockMetaData block : blocks) {
+      total += block.getRowCount();
+    }
+    LOG.info("RecordReader initialized will read a total of " + total + " records.");
+  }
+
+  private boolean contains(GroupType group, String[] path, int index) {
+    if (index == path.length) {
+      return false;
+    }
+    if (group.containsField(path[index])) {
+      Type type = group.getType(path[index]);
+      if (type.isPrimitive()) {
+        return index + 1 == path.length;
+      } else {
+        return contains(type.asGroupType(), path, index + 1);
+      }
+    }
+    return false;
+  }
+
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+    if (current < total) {
+      try {
+        checkRead();
+        currentValue = recordReader.read();
+        if (DEBUG) LOG.debug("read value: " + currentValue);
+        current ++;
+      } catch (RuntimeException e) {
+        throw new ParquetDecodingException(format("Can not read value at %d in block %d in file %s", current, currentBlock, file), e);
+      }
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
new file mode 100644
index 0000000..532d9a2
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
@@ -0,0 +1,160 @@
+/**
+ * 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.thirdparty.parquet;
+
+import parquet.Log;
+import parquet.column.ParquetProperties.WriterVersion;
+import parquet.column.impl.ColumnWriteStoreImpl;
+import parquet.hadoop.api.WriteSupport;
+import parquet.io.ColumnIOFactory;
+import parquet.io.MessageColumnIO;
+import parquet.schema.MessageType;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.String.format;
+import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
+import static parquet.Log.DEBUG;
+import static parquet.Preconditions.checkNotNull;
+
+class InternalParquetRecordWriter<T> {
+  private static final Log LOG = Log.getLog(InternalParquetRecordWriter.class);
+
+  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
+  private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
+  private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
+
+  private final ParquetFileWriter w;
+  private final WriteSupport<T> writeSupport;
+  private final MessageType schema;
+  private final Map<String, String> extraMetaData;
+  private final int blockSize;
+  private final int pageSize;
+  private final BytesCompressor compressor;
+  private final int dictionaryPageSize;
+  private final boolean enableDictionary;
+  private final boolean validating;
+  private final WriterVersion writerVersion;
+
+  private long recordCount = 0;
+  private long recordCountForNextMemCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
+
+  private ColumnWriteStoreImpl store;
+  private ColumnChunkPageWriteStore pageStore;
+
+  /**
+   * @param w the file to write to
+   * @param writeSupport the class to convert incoming records
+   * @param schema the schema of the records
+   * @param extraMetaData extra meta data to write in the footer of the file
+   * @param blockSize the size of a block in the file (this will be approximate)
+   * @param codec the codec used to compress
+   */
+  public InternalParquetRecordWriter(
+      ParquetFileWriter w,
+      WriteSupport<T> writeSupport,
+      MessageType schema,
+      Map<String, String> extraMetaData,
+      int blockSize,
+      int pageSize,
+      BytesCompressor compressor,
+      int dictionaryPageSize,
+      boolean enableDictionary,
+      boolean validating,
+      WriterVersion writerVersion) {
+    this.w = w;
+    this.writeSupport = checkNotNull(writeSupport, "writeSupport");
+    this.schema = schema;
+    this.extraMetaData = extraMetaData;
+    this.blockSize = blockSize;
+    this.pageSize = pageSize;
+    this.compressor = compressor;
+    this.dictionaryPageSize = dictionaryPageSize;
+    this.enableDictionary = enableDictionary;
+    this.validating = validating;
+    this.writerVersion = writerVersion;
+    initStore();
+  }
+
+  private void initStore() {
+    // we don't want this number to be too small
+    // ideally we divide the block equally across the columns
+    // it is unlikely all columns are going to be the same size.
+    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize / schema.getColumns().size() / 5);
+    pageStore = new ColumnChunkPageWriteStore(compressor, schema, initialBlockBufferSize);
+    // we don't want this number to be too small either
+    // ideally, slightly bigger than the page size, but not bigger than the block buffer
+    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize + pageSize / 10, initialBlockBufferSize));
+    store = new ColumnWriteStoreImpl(pageStore, pageSize, initialPageBufferSize, dictionaryPageSize, enableDictionary, writerVersion);
+    MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(schema);
+    writeSupport.prepareForWrite(columnIO.getRecordWriter(store));
+  }
+
+  public void close() throws IOException, InterruptedException {
+    flushStore();
+    w.end(extraMetaData);
+  }
+
+  public void write(T value) throws IOException, InterruptedException {
+    writeSupport.write(value);
+    ++ recordCount;
+    checkBlockSizeReached();
+  }
+
+  private void checkBlockSizeReached() throws IOException {
+    if (recordCount >= recordCountForNextMemCheck) { // checking the memory size is relatively expensive, so let's not do it for every record.
+      long memSize = store.memSize();
+      if (memSize > blockSize) {
+        LOG.info(format("mem size %,d > %,d: flushing %,d records to disk.", memSize, blockSize, recordCount));
+        flushStore();
+        initStore();
+        recordCountForNextMemCheck = min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2), MAXIMUM_RECORD_COUNT_FOR_CHECK);
+      } else {
+        float recordSize = (float) memSize / recordCount;
+        recordCountForNextMemCheck = min(
+            max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(blockSize / recordSize)) / 2), // will check halfway
+            recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead
+        );
+        if (DEBUG) LOG.debug(format("Checked mem at %,d will check again at: %,d ", recordCount, recordCountForNextMemCheck));
+      }
+    }
+  }
+
+  public long getEstimatedWrittenSize() throws IOException {
+    return w.getPos() + store.memSize();
+  }
+
+  private void flushStore()
+      throws IOException {
+    LOG.info(format("Flushing mem store to file. allocated memory: %,d", store.allocatedSize()));
+    if (store.allocatedSize() > 3 * blockSize) {
+      LOG.warn("Too much memory used: " + store.memUsageString());
+    }
+    w.startBlock(recordCount);
+    store.flush();
+    pageStore.flushToFileWriter(w);
+    recordCount = 0;
+    w.endBlock();
+    store = null;
+    pageStore = null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
new file mode 100644
index 0000000..f1c5368
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
@@ -0,0 +1,492 @@
+/**
+ * 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.thirdparty.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import parquet.Log;
+import parquet.Version;
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+import parquet.column.ColumnDescriptor;
+import parquet.column.page.DictionaryPage;
+import parquet.column.statistics.Statistics;
+import parquet.format.converter.ParquetMetadataConverter;
+import parquet.hadoop.Footer;
+import parquet.hadoop.metadata.*;
+import parquet.io.ParquetEncodingException;
+import parquet.schema.MessageType;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.*;
+import java.util.Map.Entry;
+
+import static parquet.Log.DEBUG;
+import static parquet.format.Util.writeFileMetaData;
+
+/**
+ * Internal implementation of the Parquet file writer as a block container
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class ParquetFileWriter {
+  private static final Log LOG = Log.getLog(ParquetFileWriter.class);
+
+  public static final String PARQUET_METADATA_FILE = "_metadata";
+  public static final byte[] MAGIC = "PAR1".getBytes(Charset.forName("ASCII"));
+  public static final int CURRENT_VERSION = 1;
+
+  private static final ParquetMetadataConverter metadataConverter = new ParquetMetadataConverter();
+
+  private final MessageType schema;
+  private final FSDataOutputStream out;
+  private BlockMetaData currentBlock;
+  private ColumnChunkMetaData currentColumn;
+  private long currentRecordCount;
+  private List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
+  private long uncompressedLength;
+  private long compressedLength;
+  private Set<parquet.column.Encoding> currentEncodings;
+
+  private CompressionCodecName currentChunkCodec;
+  private ColumnPath currentChunkPath;
+  private PrimitiveTypeName currentChunkType;
+  private long currentChunkFirstDataPage;
+  private long currentChunkDictionaryPageOffset;
+  private long currentChunkValueCount;
+
+  private Statistics currentStatistics;
+
+  /**
+   * Captures the order in which methods should be called
+   *
+   * @author Julien Le Dem
+   *
+   */
+  private enum STATE {
+    NOT_STARTED {
+      STATE start() {
+        return STARTED;
+      }
+    },
+    STARTED {
+      STATE startBlock() {
+        return BLOCK;
+      }
+      STATE end() {
+        return ENDED;
+      }
+    },
+    BLOCK  {
+      STATE startColumn() {
+        return COLUMN;
+      }
+      STATE endBlock() {
+        return STARTED;
+      }
+    },
+    COLUMN {
+      STATE endColumn() {
+        return BLOCK;
+      };
+      STATE write() {
+        return this;
+      }
+    },
+    ENDED;
+
+    STATE start() throws IOException { return error(); }
+    STATE startBlock() throws IOException { return error(); }
+    STATE startColumn() throws IOException { return error(); }
+    STATE write() throws IOException { return error(); }
+    STATE endColumn() throws IOException { return error(); }
+    STATE endBlock() throws IOException { return error(); }
+    STATE end() throws IOException { return error(); }
+
+    private final STATE error() throws IOException {
+      throw new IOException("The file being written is in an invalid state. Probably caused by an error thrown previously. Current state: " + this.name());
+    }
+  }
+
+  private STATE state = STATE.NOT_STARTED;
+
+  /**
+   *
+   * @param schema the schema of the data
+   * @param out the file to write to
+   * @param codec the codec to use to compress blocks
+   * @throws java.io.IOException if the file can not be created
+   */
+  public ParquetFileWriter(Configuration configuration, MessageType schema, Path file) throws IOException {
+    super();
+    this.schema = schema;
+    FileSystem fs = file.getFileSystem(configuration);
+    this.out = fs.create(file, false);
+  }
+
+  /**
+   * start the file
+   * @throws java.io.IOException
+   */
+  public void start() throws IOException {
+    state = state.start();
+    if (DEBUG) LOG.debug(out.getPos() + ": start");
+    out.write(MAGIC);
+  }
+
+  /**
+   * start a block
+   * @param recordCount the record count in this block
+   * @throws java.io.IOException
+   */
+  public void startBlock(long recordCount) throws IOException {
+    state = state.startBlock();
+    if (DEBUG) LOG.debug(out.getPos() + ": start block");
+//    out.write(MAGIC); // TODO: add a magic delimiter
+    currentBlock = new BlockMetaData();
+    currentRecordCount = recordCount;
+  }
+
+  /**
+   * start a column inside a block
+   * @param descriptor the column descriptor
+   * @param valueCount the value count in this column
+   * @param statistics the statistics in this column
+   * @param compressionCodecName
+   * @throws java.io.IOException
+   */
+  public void startColumn(ColumnDescriptor descriptor,
+                          long valueCount,
+                          CompressionCodecName compressionCodecName) throws IOException {
+    state = state.startColumn();
+    if (DEBUG) LOG.debug(out.getPos() + ": start column: " + descriptor + " count=" + valueCount);
+    currentEncodings = new HashSet<parquet.column.Encoding>();
+    currentChunkPath = ColumnPath.get(descriptor.getPath());
+    currentChunkType = descriptor.getType();
+    currentChunkCodec = compressionCodecName;
+    currentChunkValueCount = valueCount;
+    currentChunkFirstDataPage = out.getPos();
+    compressedLength = 0;
+    uncompressedLength = 0;
+    // need to know what type of stats to initialize to
+    // better way to do this?
+    currentStatistics = Statistics.getStatsBasedOnType(currentChunkType);
+  }
+
+  /**
+   * writes a dictionary page page
+   * @param dictionaryPage the dictionary page
+   */
+  public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException {
+    state = state.write();
+    if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page: " + dictionaryPage.getDictionarySize() + " values");
+    currentChunkDictionaryPageOffset = out.getPos();
+    int uncompressedSize = dictionaryPage.getUncompressedSize();
+    int compressedPageSize = (int)dictionaryPage.getBytes().size(); // TODO: fix casts
+    metadataConverter.writeDictionaryPageHeader(
+        uncompressedSize,
+        compressedPageSize,
+        dictionaryPage.getDictionarySize(),
+        dictionaryPage.getEncoding(),
+        out);
+    long headerSize = out.getPos() - currentChunkDictionaryPageOffset;
+    this.uncompressedLength += uncompressedSize + headerSize;
+    this.compressedLength += compressedPageSize + headerSize;
+    if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page content " + compressedPageSize);
+    dictionaryPage.getBytes().writeAllTo(out);
+    currentEncodings.add(dictionaryPage.getEncoding());
+  }
+
+
+  /**
+   * writes a single page
+   * @param valueCount count of values
+   * @param uncompressedPageSize the size of the data once uncompressed
+   * @param bytes the compressed data for the page without header
+   * @param rlEncoding encoding of the repetition level
+   * @param dlEncoding encoding of the definition level
+   * @param valuesEncoding encoding of values
+   */
+  @Deprecated
+  public void writeDataPage(
+      int valueCount, int uncompressedPageSize,
+      BytesInput bytes,
+      parquet.column.Encoding rlEncoding,
+      parquet.column.Encoding dlEncoding,
+      parquet.column.Encoding valuesEncoding) throws IOException {
+    state = state.write();
+    long beforeHeader = out.getPos();
+    if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values");
+    int compressedPageSize = (int)bytes.size();
+    metadataConverter.writeDataPageHeader(
+        uncompressedPageSize, compressedPageSize,
+        valueCount,
+        rlEncoding,
+        dlEncoding,
+        valuesEncoding,
+        out);
+    long headerSize = out.getPos() - beforeHeader;
+    this.uncompressedLength += uncompressedPageSize + headerSize;
+    this.compressedLength += compressedPageSize + headerSize;
+    if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize);
+    bytes.writeAllTo(out);
+    currentEncodings.add(rlEncoding);
+    currentEncodings.add(dlEncoding);
+    currentEncodings.add(valuesEncoding);
+  }
+
+  /**
+   * writes a single page
+   * @param valueCount count of values
+   * @param uncompressedPageSize the size of the data once uncompressed
+   * @param bytes the compressed data for the page without header
+   * @param rlEncoding encoding of the repetition level
+   * @param dlEncoding encoding of the definition level
+   * @param valuesEncoding encoding of values
+   */
+  public void writeDataPage(
+      int valueCount, int uncompressedPageSize,
+      BytesInput bytes,
+      Statistics statistics,
+      parquet.column.Encoding rlEncoding,
+      parquet.column.Encoding dlEncoding,
+      parquet.column.Encoding valuesEncoding) throws IOException {
+    state = state.write();
+    long beforeHeader = out.getPos();
+    if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values");
+    int compressedPageSize = (int)bytes.size();
+    metadataConverter.writeDataPageHeader(
+        uncompressedPageSize, compressedPageSize,
+        valueCount,
+        statistics,
+        rlEncoding,
+        dlEncoding,
+        valuesEncoding,
+        out);
+    long headerSize = out.getPos() - beforeHeader;
+    this.uncompressedLength += uncompressedPageSize + headerSize;
+    this.compressedLength += compressedPageSize + headerSize;
+    if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize);
+    bytes.writeAllTo(out);
+    currentStatistics.mergeStatistics(statistics);
+    currentEncodings.add(rlEncoding);
+    currentEncodings.add(dlEncoding);
+    currentEncodings.add(valuesEncoding);
+  }
+
+  /**
+   * writes a number of pages at once
+   * @param bytes bytes to be written including page headers
+   * @param uncompressedTotalPageSize total uncompressed size (without page headers)
+   * @param compressedTotalPageSize total compressed size (without page headers)
+   * @throws java.io.IOException
+   */
+  void writeDataPages(BytesInput bytes,
+                      long uncompressedTotalPageSize,
+                      long compressedTotalPageSize,
+                      Statistics totalStats,
+                      List<parquet.column.Encoding> encodings) throws IOException {
+    state = state.write();
+    if (DEBUG) LOG.debug(out.getPos() + ": write data pages");
+    long headersSize = bytes.size() - compressedTotalPageSize;
+    this.uncompressedLength += uncompressedTotalPageSize + headersSize;
+    this.compressedLength += compressedTotalPageSize + headersSize;
+    if (DEBUG) LOG.debug(out.getPos() + ": write data pages content");
+    bytes.writeAllTo(out);
+    currentEncodings.addAll(encodings);
+    currentStatistics = totalStats;
+  }
+
+  /**
+   * end a column (once all rep, def and data have been written)
+   * @throws java.io.IOException
+   */
+  public void endColumn() throws IOException {
+    state = state.endColumn();
+    if (DEBUG) LOG.debug(out.getPos() + ": end column");
+    currentBlock.addColumn(ColumnChunkMetaData.get(
+        currentChunkPath,
+        currentChunkType,
+        currentChunkCodec,
+        currentEncodings,
+        currentStatistics,
+        currentChunkFirstDataPage,
+        currentChunkDictionaryPageOffset,
+        currentChunkValueCount,
+        compressedLength,
+        uncompressedLength));
+    if (DEBUG) LOG.info("ended Column chumk: " + currentColumn);
+    currentColumn = null;
+    this.currentBlock.setTotalByteSize(currentBlock.getTotalByteSize() + uncompressedLength);
+    this.uncompressedLength = 0;
+    this.compressedLength = 0;
+  }
+
+  /**
+   * ends a block once all column chunks have been written
+   * @throws java.io.IOException
+   */
+  public void endBlock() throws IOException {
+    state = state.endBlock();
+    if (DEBUG) LOG.debug(out.getPos() + ": end block");
+    currentBlock.setRowCount(currentRecordCount);
+    blocks.add(currentBlock);
+    currentBlock = null;
+  }
+
+  /**
+   * ends a file once all blocks have been written.
+   * closes the file.
+   * @param extraMetaData the extra meta data to write in the footer
+   * @throws java.io.IOException
+   */
+  public void end(Map<String, String> extraMetaData) throws IOException {
+    state = state.end();
+    if (DEBUG) LOG.debug(out.getPos() + ": end");
+    ParquetMetadata footer = new ParquetMetadata(new FileMetaData(schema, extraMetaData, Version.FULL_VERSION), blocks);
+    serializeFooter(footer, out);
+    out.close();
+  }
+
+  private static void serializeFooter(ParquetMetadata footer, FSDataOutputStream out) throws IOException {
+    long footerIndex = out.getPos();
+    parquet.format.FileMetaData parquetMetadata = new ParquetMetadataConverter().toParquetMetadata(CURRENT_VERSION, footer);
+    writeFileMetaData(parquetMetadata, out);
+    if (DEBUG) LOG.debug(out.getPos() + ": footer length = " + (out.getPos() - footerIndex));
+    BytesUtils.writeIntLittleEndian(out, (int)(out.getPos() - footerIndex));
+    out.write(MAGIC);
+  }
+
+  /**
+   * writes a _metadata file
+   * @param configuration the configuration to use to get the FileSystem
+   * @param outputPath the directory to write the _metadata file to
+   * @param footers the list of footers to merge
+   * @throws java.io.IOException
+   */
+  public static void writeMetadataFile(Configuration configuration, Path outputPath, List<Footer> footers) throws IOException {
+    Path metaDataPath = new Path(outputPath, PARQUET_METADATA_FILE);
+    FileSystem fs = outputPath.getFileSystem(configuration);
+    outputPath = outputPath.makeQualified(fs);
+    FSDataOutputStream metadata = fs.create(metaDataPath);
+    metadata.write(MAGIC);
+    ParquetMetadata metadataFooter = mergeFooters(outputPath, footers);
+    serializeFooter(metadataFooter, metadata);
+    metadata.close();
+  }
+
+  private static ParquetMetadata mergeFooters(Path root, List<Footer> footers) {
+    String rootPath = root.toString();
+    GlobalMetaData fileMetaData = null;
+    List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
+    for (Footer footer : footers) {
+      String path = footer.getFile().toString();
+      if (!path.startsWith(rootPath)) {
+        throw new ParquetEncodingException(path + " invalid: all the files must be contained in the root " + root);
+      }
+      path = path.substring(rootPath.length());
+      while (path.startsWith("/")) {
+        path = path.substring(1);
+      }
+      fileMetaData = mergeInto(footer.getParquetMetadata().getFileMetaData(), fileMetaData);
+      for (BlockMetaData block : footer.getParquetMetadata().getBlocks()) {
+        block.setPath(path);
+        blocks.add(block);
+      }
+    }
+    return new ParquetMetadata(fileMetaData.merge(), blocks);
+  }
+
+  /**
+   * @return the current position in the underlying file
+   * @throws java.io.IOException
+   */
+  public long getPos() throws IOException {
+    return out.getPos();
+  }
+
+  /**
+   * Will merge the metadata of all the footers together
+   * @param footers the list files footers to merge
+   * @return the global meta data for all the footers
+   */
+  static GlobalMetaData getGlobalMetaData(List<Footer> footers) {
+    GlobalMetaData fileMetaData = null;
+    for (Footer footer : footers) {
+      ParquetMetadata currentMetadata = footer.getParquetMetadata();
+      fileMetaData = mergeInto(currentMetadata.getFileMetaData(), fileMetaData);
+    }
+    return fileMetaData;
+  }
+
+  /**
+   * Will return the result of merging toMerge into mergedMetadata
+   * @param toMerge the metadata toMerge
+   * @param mergedMetadata the reference metadata to merge into
+   * @return the result of the merge
+   */
+  static GlobalMetaData mergeInto(
+      FileMetaData toMerge,
+      GlobalMetaData mergedMetadata) {
+    MessageType schema = null;
+    Map<String, Set<String>> newKeyValues = new HashMap<String, Set<String>>();
+    Set<String> createdBy = new HashSet<String>();
+    if (mergedMetadata != null) {
+      schema = mergedMetadata.getSchema();
+      newKeyValues.putAll(mergedMetadata.getKeyValueMetaData());
+      createdBy.addAll(mergedMetadata.getCreatedBy());
+    }
+    if ((schema == null && toMerge.getSchema() != null)
+        || (schema != null && !schema.equals(toMerge.getSchema()))) {
+      schema = mergeInto(toMerge.getSchema(), schema);
+    }
+    for (Entry<String, String> entry : toMerge.getKeyValueMetaData().entrySet()) {
+      Set<String> values = newKeyValues.get(entry.getKey());
+      if (values == null) {
+        values = new HashSet<String>();
+        newKeyValues.put(entry.getKey(), values);
+      }
+      values.add(entry.getValue());
+    }
+    createdBy.add(toMerge.getCreatedBy());
+    return new GlobalMetaData(
+        schema,
+        newKeyValues,
+        createdBy);
+  }
+
+  /**
+   * will return the result of merging toMerge into mergedSchema
+   * @param toMerge the schema to merge into mergedSchema
+   * @param mergedSchema the schema to append the fields to
+   * @return the resulting schema
+   */
+  static MessageType mergeInto(MessageType toMerge, MessageType mergedSchema) {
+    if (mergedSchema == null) {
+      return toMerge;
+    }
+    return mergedSchema.union(toMerge);
+  }
+
+}


[29/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 f807987..8a61cab 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
@@ -70,7 +70,7 @@ public class TestSortExec {
     util = TpchTestBase.getInstance().getTestingCluster();
     catalog = util.getMaster().getCatalog();
     workDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    sm = StorageManager.getFileStorageManager(conf, workDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, workDir);
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -82,7 +82,8 @@ public class TestSortExec {
     tablePath = StorageUtil.concatPath(workDir, "employee", "table1");
     sm.getFileSystem().mkdirs(tablePath.getParent());
 
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, schema, tablePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, tablePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < 100; i++) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
index b74f634..68b3fb3 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
@@ -565,8 +565,8 @@ public class TestJoinBroadcast extends QueryTestCaseBase {
         }
         Path dataPath = new Path(table.getPath().toString(), fileIndex + ".csv");
         fileIndex++;
-        appender = StorageManager.getFileStorageManager(conf).getAppender(tableMeta, schema,
-            dataPath);
+        appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+            .getAppender(tableMeta, schema, dataPath);
         appender.init();
       }
       String[] columnDatas = rows[i].split("\\|");

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 ac5ff13..b8f3ef7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
+++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java
@@ -63,7 +63,7 @@ public class TestResultSet {
   public static void setup() throws Exception {
     util = TpchTestBase.getInstance().getTestingCluster();
     conf = util.getConfiguration();
-    sm = StorageManager.getFileStorageManager(conf);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
 
     scoreSchema = new Schema();
     scoreSchema.addColumn("deptname", Type.TEXT);
@@ -73,8 +73,7 @@ public class TestResultSet {
 
     Path p = sm.getTablePath("score");
     sm.getFileSystem().mkdirs(p);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(scoreMeta, scoreSchema,
-        new Path(p, "score"));
+    Appender appender = sm.getAppender(scoreMeta, scoreSchema, new Path(p, "score"));
     appender.init();
     int deptSize = 100;
     int tupleNum = 10000;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 2aa56db..f36ff24 100644
--- a/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java
+++ b/tajo-core/src/test/java/org/apache/tajo/storage/TestRowFile.java
@@ -36,7 +36,6 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
 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.FileUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -70,7 +69,8 @@ public class TestRowFile {
 
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.ROWFILE);
 
-    FileStorageManager sm = StorageManager.getFileStorageManager(conf, new Path(conf.getVar(ConfVars.ROOT_DIR)));
+    FileStorageManager sm =
+        (FileStorageManager)StorageManager.getFileStorageManager(conf, new Path(conf.getVar(ConfVars.ROOT_DIR)));
 
     Path tablePath = new Path("/test");
     Path metaPath = new Path(tablePath, ".meta");
@@ -80,7 +80,7 @@ public class TestRowFile {
 
     FileUtil.writeProto(fs, metaPath, meta.getProto());
 
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, dataPath);
+    Appender appender = sm.getAppender(meta, schema, dataPath);
     appender.enableStats();
     appender.init();
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
index 517f425..5a93538 100644
--- a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
@@ -185,7 +185,7 @@ public class TestRangeRetrieverHandler {
     reader.open();
 
     TableMeta meta = CatalogUtil.newTableMeta(StoreType.RAW, new KeyValueSet());
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema,
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema,
         StorageUtil.concatPath(testDir, "output", "output"));
 
     scanner.init();
@@ -308,7 +308,7 @@ public class TestRangeRetrieverHandler {
         new Path(testDir, "output/index"), keySchema, comp);
     reader.open();
     TableMeta outputMeta = CatalogUtil.newTableMeta(StoreType.RAW, new KeyValueSet());
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, outputMeta, schema,
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, outputMeta, schema,
         StorageUtil.concatPath(testDir, "output", "output"));
     scanner.init();
     int cnt = 0;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index d350889..eb8ada9 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -75,7 +75,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
       <scope>provided</scope>
     </dependency>
   </dependencies>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-jdbc/pom.xml b/tajo-jdbc/pom.xml
index 20cdf16..1c3c410 100644
--- a/tajo-jdbc/pom.xml
+++ b/tajo-jdbc/pom.xml
@@ -102,7 +102,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index a82aa46..82ccbdc 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -744,6 +744,22 @@
         <groupId>org.apache.tajo</groupId>
         <artifactId>tajo-storage</artifactId>
         <version>${tajo.version}</version>
+        <type>pom</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-storage-common</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-storage-hdfs</artifactId>
+        <version>${tajo.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tajo</groupId>
+        <artifactId>tajo-storage-hbase</artifactId>
+        <version>${tajo.version}</version>
       </dependency>
       <dependency>
         <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/pom.xml b/tajo-storage/pom.xml
index dee429f..8acb1a9 100644
--- a/tajo-storage/pom.xml
+++ b/tajo-storage/pom.xml
@@ -1,5 +1,5 @@
-<!--
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
   Copyright 2012 Database Lab., Korea Univ.
 
   Licensed under the Apache License, Version 2.0 (the "License");
@@ -16,328 +16,47 @@
   -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
   <parent>
     <artifactId>tajo-project</artifactId>
     <groupId>org.apache.tajo</groupId>
     <version>0.9.1-SNAPSHOT</version>
     <relativePath>../tajo-project</relativePath>
   </parent>
-
+  <modelVersion>4.0.0</modelVersion>
   <artifactId>tajo-storage</artifactId>
-  <packaging>jar</packaging>
+  <packaging>pom</packaging>
   <name>Tajo Storage</name>
-  <description>Tajo Storage Package</description>
-
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-    <parquet.version>1.5.0</parquet.version>
-    <parquet.format.version>2.1.0</parquet.format.version>
   </properties>
 
-  <repositories>
-    <repository>
-      <id>repository.jboss.org</id>
-      <url>https://repository.jboss.org/nexus/content/repositories/releases/
-            </url>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </repository>
-  </repositories>
+  <modules>
+    <module>tajo-storage-common</module>
+    <module>tajo-storage-hdfs</module>
+    <module>tajo-storage-hbase</module>
+  </modules>
 
   <build>
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
-          <encoding>${project.build.sourceEncoding}</encoding>
-        </configuration>
-      </plugin>
-      <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>verify</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <excludes>
-            <exclude>src/test/resources/testVariousTypes.avsc</exclude>
-          </excludes>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <systemProperties>
-            <tajo.test>TRUE</tajo.test>
-          </systemProperties>
-          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
-        </configuration>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <version>2.4</version>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
+        <artifactId>maven-surefire-report-plugin</artifactId>
       </plugin>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>create-protobuf-generated-sources-directory</id>
-            <phase>initialize</phase>
-            <configuration>
-              <target>
-                <mkdir dir="target/generated-sources/proto" />
-              </target>
-            </configuration>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <version>1.2</version>
-        <executions>
-          <execution>
-            <id>generate-sources</id>
-            <phase>generate-sources</phase>
-            <configuration>
-              <executable>protoc</executable>
-              <arguments>
-                <argument>-Isrc/main/proto/</argument>
-                <argument>--proto_path=../tajo-common/src/main/proto</argument>
-                <argument>--proto_path=../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
-                <argument>--java_out=target/generated-sources/proto</argument>
-                <argument>src/main/proto/IndexProtos.proto</argument>
-                <argument>src/main/proto/StorageFragmentProtos.proto</argument>
-              </arguments>
-            </configuration>
-            <goals>
-              <goal>exec</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>build-helper-maven-plugin</artifactId>
-        <version>1.5</version>
-        <executions>
-          <execution>
-            <id>add-source</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>add-source</goal>
-            </goals>
-            <configuration>
-              <sources>
-                <source>target/generated-sources/proto</source>
-              </sources>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-pmd-plugin</artifactId>
-        <version>2.7.1</version>
+        <artifactId>maven-deploy-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
       </plugin>
     </plugins>
   </build>
 
 
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-catalog-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-plan</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-mapred</artifactId>
-      <version>1.7.7</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-core</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <artifactId>zookeeper</artifactId>
-          <groupId>org.apache.zookeeper</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>slf4j-api</artifactId>
-          <groupId>org.slf4j</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>jersey-json</artifactId>
-          <groupId>com.sun.jersey</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <scope>provided</scope>
-      <exclusions>
-      <exclusion>
-        <groupId>commons-el</groupId>
-        <artifactId>commons-el</artifactId>
-      </exclusion>
-      <exclusion>
-        <groupId>tomcat</groupId>
-        <artifactId>jasper-runtime</artifactId>
-      </exclusion>
-      <exclusion>
-        <groupId>tomcat</groupId>
-        <artifactId>jasper-compiler</artifactId>
-      </exclusion>
-      <exclusion>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jsp-2.1-jetty</artifactId>
-      </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.jersey-test-framework</groupId>
-          <artifactId>jersey-test-framework-grizzly2</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-minicluster</artifactId>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>commons-el</groupId>
-          <artifactId>commons-el</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-runtime</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-compiler</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jsp-2.1-jetty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jersey.jersey-test-framework</groupId>
-          <artifactId>jersey-test-framework-grizzly2</artifactId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-server-tests</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-app</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-yarn-api</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-hs</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-        <exclusion>
-          <artifactId>hadoop-mapreduce-client-core</artifactId>
-          <groupId>org.apache.hadoop</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-core</artifactId>
-      <version>${hadoop.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>parquet-column</artifactId>
-      <version>${parquet.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>parquet-hadoop</artifactId>
-      <version>${parquet.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>parquet-format</artifactId>
-      <version>${parquet.format.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-server</artifactId>
-      <version>${hbase.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-client</artifactId>
-      <version>${hbase.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-buffer</artifactId>
-    </dependency>
-  </dependencies>
-
   <profiles>
     <profile>
       <id>docs</id>
@@ -382,7 +101,7 @@
             <executions>
               <execution>
                 <id>dist</id>
-                <phase>package</phase>
+                <phase>prepare-package</phase>
                 <goals>
                   <goal>run</goal>
                 </goals>
@@ -405,12 +124,15 @@
                       echo
                       echo "Current directory `pwd`"
                       echo
-                      run rm -rf ${project.artifactId}-${project.version}
-                      run mkdir ${project.artifactId}-${project.version}
-                      run cd ${project.artifactId}-${project.version}
-                      run cp -r ${basedir}/target/${project.artifactId}-${project.version}*.jar .
+                      run rm -rf tajo-storage-${project.version}
+                      run mkdir tajo-storage-${project.version}
+                      run cd tajo-storage-${project.version}
+                      run cp -r ${basedir}/tajo-storage-common/target/tajo-storage-common-${project.version}*.jar .
+                      run cp -r ${basedir}/tajo-storage-hdfs/target/tajo-storage-hdfs-${project.version}*.jar .
+                      run cp -r ${basedir}/tajo-storage-hbase/target/tajo-storage-hbase-${project.version}*.jar .
+
                       echo
-                      echo "Tajo Storage dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo "Tajo Storage dist layout available at: ${project.build.directory}/tajo-storage-${project.version}"
                       echo
                     </echo>
                     <exec executable="sh" dir="${project.build.directory}" failonerror="true">
@@ -430,11 +152,7 @@
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-project-info-reports-plugin</artifactId>
-        <version>2.4</version>
-        <configuration>
-          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
-        </configuration>
+        <artifactId>maven-surefire-report-plugin</artifactId>
       </plugin>
     </plugins>
   </reporting>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
deleted file mode 100644
index c5e96ac..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.catalog.statistics.TableStats;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public interface Appender extends Closeable {
-
-  void init() throws IOException;
-
-  void addTuple(Tuple t) throws IOException;
-  
-  void flush() throws IOException;
-
-  long getEstimatedOutputSize() throws IOException;
-  
-  void close() throws IOException;
-
-  void enableStats();
-  
-  TableStats getStats();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java b/tajo-storage/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
deleted file mode 100644
index b829f60..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
+++ /dev/null
@@ -1,206 +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.base.Objects;
-import com.google.common.base.Preconditions;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SortSpec;
-import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.datum.Datum;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.TupleComparatorSpecProto;
-import static org.apache.tajo.index.IndexProtos.TupleComparatorProto;
-
-/**
- * The Comparator class for Tuples
- * 
- * @see Tuple
- */
-public class BaseTupleComparator extends TupleComparator implements ProtoObject<TupleComparatorProto> {
-  private final Schema schema;
-  private final SortSpec [] sortSpecs;
-  private final int[] sortKeyIds;
-  private final boolean[] asc;
-  @SuppressWarnings("unused")
-  private final boolean[] nullFirsts;  
-
-  private Datum left;
-  private Datum right;
-  private int compVal;
-
-  /**
-   * @param schema The schema of input tuples
-   * @param sortKeys The description of sort keys
-   */
-  public BaseTupleComparator(Schema schema, SortSpec[] sortKeys) {
-    Preconditions.checkArgument(sortKeys.length > 0, 
-        "At least one sort key must be specified.");
-
-    this.schema = schema;
-    this.sortSpecs = sortKeys;
-    this.sortKeyIds = new int[sortKeys.length];
-    this.asc = new boolean[sortKeys.length];
-    this.nullFirsts = new boolean[sortKeys.length];
-    for (int i = 0; i < sortKeys.length; i++) {
-      if (sortKeys[i].getSortKey().hasQualifier()) {
-        this.sortKeyIds[i] = schema.getColumnId(sortKeys[i].getSortKey().getQualifiedName());
-      } else {
-        this.sortKeyIds[i] = schema.getColumnIdByName(sortKeys[i].getSortKey().getSimpleName());
-      }
-          
-      this.asc[i] = sortKeys[i].isAscending();
-      this.nullFirsts[i]= sortKeys[i].isNullFirst();
-    }
-  }
-
-  public BaseTupleComparator(TupleComparatorProto proto) {
-    this.schema = new Schema(proto.getSchema());
-
-    this.sortSpecs = new SortSpec[proto.getSortSpecsCount()];
-    for (int i = 0; i < proto.getSortSpecsCount(); i++) {
-      sortSpecs[i] = new SortSpec(proto.getSortSpecs(i));
-    }
-
-    this.sortKeyIds = new int[proto.getCompSpecsCount()];
-    this.asc = new boolean[proto.getCompSpecsCount()];
-    this.nullFirsts = new boolean[proto.getCompSpecsCount()];
-
-    for (int i = 0; i < proto.getCompSpecsCount(); i++) {
-      TupleComparatorSpecProto sortSepcProto = proto.getCompSpecs(i);
-      sortKeyIds[i] = sortSepcProto.getColumnId();
-      asc[i] = sortSepcProto.getAscending();
-      nullFirsts[i] = sortSepcProto.getNullFirst();
-    }
-  }
-
-  public Schema getSchema() {
-    return schema;
-  }
-
-  public SortSpec [] getSortSpecs() {
-    return sortSpecs;
-  }
-
-  public int [] getSortKeyIds() {
-    return sortKeyIds;
-  }
-
-  @Override
-  public boolean isAscendingFirstKey() {
-    return this.asc[0];
-  }
-
-  @Override
-  public int compare(Tuple tuple1, Tuple tuple2) {
-    for (int i = 0; i < sortKeyIds.length; i++) {
-      left = tuple1.get(sortKeyIds[i]);
-      right = tuple2.get(sortKeyIds[i]);
-
-      if (left.isNull() || right.isNull()) {
-        if (!left.equals(right)) {
-          if (left.isNull()) {
-            compVal = 1;
-          } else if (right.isNull()) {
-            compVal = -1;
-          }
-          if (nullFirsts[i]) {
-            if (compVal != 0) {
-              compVal *= -1;
-            }
-          }
-        } else {
-          compVal = 0;
-        }
-      } else {
-        if (asc[i]) {
-          compVal = left.compareTo(right);
-        } else {
-          compVal = right.compareTo(left);
-        }
-      }
-
-      if (compVal < 0 || compVal > 0) {
-        return compVal;
-      }
-    }
-    return 0;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(sortKeyIds);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof BaseTupleComparator) {
-      BaseTupleComparator other = (BaseTupleComparator) obj;
-      if (sortKeyIds.length != other.sortKeyIds.length) {
-        return false;
-      }
-
-      for (int i = 0; i < sortKeyIds.length; i++) {
-        if (sortKeyIds[i] != other.sortKeyIds[i] ||
-            asc[i] != other.asc[i] ||
-            nullFirsts[i] != other.nullFirsts[i]) {
-          return false;
-        }
-      }
-
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public TupleComparatorProto getProto() {
-    TupleComparatorProto.Builder builder = TupleComparatorProto.newBuilder();
-    builder.setSchema(schema.getProto());
-    for (int i = 0; i < sortSpecs.length; i++) {
-      builder.addSortSpecs(sortSpecs[i].getProto());
-    }
-
-    TupleComparatorSpecProto.Builder sortSpecBuilder;
-    for (int i = 0; i < sortKeyIds.length; i++) {
-      sortSpecBuilder = TupleComparatorSpecProto.newBuilder();
-      sortSpecBuilder.setColumnId(sortKeyIds[i]);
-      sortSpecBuilder.setAscending(asc[i]);
-      sortSpecBuilder.setNullFirst(nullFirsts[i]);
-      builder.addCompSpecs(sortSpecBuilder);
-    }
-
-    return builder.build();
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-
-    String prefix = "";
-    for (int i = 0; i < sortKeyIds.length; i++) {
-      sb.append(prefix).append("SortKeyId=").append(sortKeyIds[i])
-        .append(",Asc=").append(asc[i])
-        .append(",NullFirst=").append(nullFirsts[i]);
-      prefix = " ,";
-    }
-    return sb.toString();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
deleted file mode 100644
index 00112e7..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
+++ /dev/null
@@ -1,258 +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.base.Preconditions;
-import com.google.protobuf.Message;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.datum.*;
-import org.apache.tajo.util.Bytes;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-@Deprecated
-public class BinarySerializerDeserializer implements SerializerDeserializer {
-
-  static final byte[] INVALID_UTF__SINGLE_BYTE = {(byte) Integer.parseInt("10111111", 2)};
-
-  @Override
-  public int serialize(Column col, Datum datum, OutputStream out, byte[] nullCharacters)
-      throws IOException {
-    byte[] bytes;
-    int length = 0;
-    if (datum == null || datum instanceof NullDatum) {
-      return 0;
-    }
-
-    switch (col.getDataType().getType()) {
-      case BOOLEAN:
-      case BIT:
-      case CHAR:
-        bytes = datum.asByteArray();
-        length = bytes.length;
-        out.write(bytes, 0, length);
-        break;
-      case INT2:
-        length = writeShort(out, datum.asInt2());
-        break;
-      case INT4:
-        length = writeVLong(out, datum.asInt4());
-        break;
-      case INT8:
-        length = writeVLong(out, datum.asInt8());
-        break;
-      case FLOAT4:
-        length = writeFloat(out, datum.asFloat4());
-        break;
-      case FLOAT8:
-        length = writeDouble(out, datum.asFloat8());
-        break;
-      case TEXT: {
-        bytes = datum.asTextBytes();
-        length = datum.size();
-        if (length == 0) {
-          bytes = INVALID_UTF__SINGLE_BYTE;
-          length = INVALID_UTF__SINGLE_BYTE.length;
-        }
-        out.write(bytes, 0, bytes.length);
-        break;
-      }
-      case BLOB:
-      case INET4:
-      case INET6:
-        bytes = datum.asByteArray();
-        length = bytes.length;
-        out.write(bytes, 0, length);
-        break;
-      case PROTOBUF:
-        ProtobufDatum protobufDatum = (ProtobufDatum) datum;
-        bytes = protobufDatum.asByteArray();
-        length = bytes.length;
-        out.write(bytes, 0, length);
-        break;
-      case NULL_TYPE:
-        break;
-      default:
-        throw new IOException("Does not support type");
-    }
-    return length;
-  }
-
-  @Override
-  public Datum deserialize(Column col, byte[] bytes, int offset, int length, byte[] nullCharacters) throws IOException {
-    if (length == 0) return NullDatum.get();
-
-    Datum datum;
-    switch (col.getDataType().getType()) {
-      case BOOLEAN:
-        datum = DatumFactory.createBool(bytes[offset]);
-        break;
-      case BIT:
-        datum = DatumFactory.createBit(bytes[offset]);
-        break;
-      case CHAR: {
-        byte[] chars = new byte[length];
-        System.arraycopy(bytes, offset, chars, 0, length);
-        datum = DatumFactory.createChar(chars);
-        break;
-      }
-      case INT2:
-        datum = DatumFactory.createInt2(Bytes.toShort(bytes, offset, length));
-        break;
-      case INT4:
-        datum = DatumFactory.createInt4((int) Bytes.readVLong(bytes, offset));
-        break;
-      case INT8:
-        datum = DatumFactory.createInt8(Bytes.readVLong(bytes, offset));
-        break;
-      case FLOAT4:
-        datum = DatumFactory.createFloat4(toFloat(bytes, offset, length));
-        break;
-      case FLOAT8:
-        datum = DatumFactory.createFloat8(toDouble(bytes, offset, length));
-        break;
-      case TEXT: {
-        byte[] chars = new byte[length];
-        System.arraycopy(bytes, offset, chars, 0, length);
-
-        if (Bytes.equals(INVALID_UTF__SINGLE_BYTE, chars)) {
-          datum = DatumFactory.createText(new byte[0]);
-        } else {
-          datum = DatumFactory.createText(chars);
-        }
-        break;
-      }
-      case PROTOBUF: {
-        ProtobufDatumFactory factory = ProtobufDatumFactory.get(col.getDataType().getCode());
-        Message.Builder builder = factory.newBuilder();
-        builder.mergeFrom(bytes, offset, length);
-        datum = factory.createDatum(builder);
-        break;
-      }
-      case INET4:
-        datum = DatumFactory.createInet4(bytes, offset, length);
-        break;
-      case BLOB:
-        datum = DatumFactory.createBlob(bytes, offset, length);
-        break;
-      default:
-        datum = NullDatum.get();
-    }
-    return datum;
-  }
-
-  private byte[] shortBytes = new byte[2];
-
-  public int writeShort(OutputStream out, short val) throws IOException {
-    shortBytes[0] = (byte) (val >> 8);
-    shortBytes[1] = (byte) val;
-    out.write(shortBytes, 0, 2);
-    return 2;
-  }
-
-  public float toFloat(byte[] bytes, int offset, int length) {
-    Preconditions.checkArgument(length == 4);
-
-    int val = ((bytes[offset] & 0x000000FF) << 24) +
-        ((bytes[offset + 1] & 0x000000FF) << 16) +
-        ((bytes[offset + 2] & 0x000000FF) << 8) +
-        (bytes[offset + 3] & 0x000000FF);
-    return Float.intBitsToFloat(val);
-  }
-
-  private byte[] floatBytes = new byte[4];
-
-  public int writeFloat(OutputStream out, float f) throws IOException {
-    int val = Float.floatToIntBits(f);
-
-    floatBytes[0] = (byte) (val >> 24);
-    floatBytes[1] = (byte) (val >> 16);
-    floatBytes[2] = (byte) (val >> 8);
-    floatBytes[3] = (byte) val;
-    out.write(floatBytes, 0, 4);
-    return floatBytes.length;
-  }
-
-  public double toDouble(byte[] bytes, int offset, int length) {
-    Preconditions.checkArgument(length == 8);
-    long val = ((long) (bytes[offset] & 0x00000000000000FF) << 56) +
-        ((long) (bytes[offset + 1] & 0x00000000000000FF) << 48) +
-        ((long) (bytes[offset + 2] & 0x00000000000000FF) << 40) +
-        ((long) (bytes[offset + 3] & 0x00000000000000FF) << 32) +
-        ((long) (bytes[offset + 4] & 0x00000000000000FF) << 24) +
-        ((long) (bytes[offset + 5] & 0x00000000000000FF) << 16) +
-        ((long) (bytes[offset + 6] & 0x00000000000000FF) << 8) +
-        (long) (bytes[offset + 7] & 0x00000000000000FF);
-    return Double.longBitsToDouble(val);
-  }
-
-  private byte[] doubleBytes = new byte[8];
-
-  public int writeDouble(OutputStream out, double d) throws IOException {
-    long val = Double.doubleToLongBits(d);
-
-    doubleBytes[0] = (byte) (val >> 56);
-    doubleBytes[1] = (byte) (val >> 48);
-    doubleBytes[2] = (byte) (val >> 40);
-    doubleBytes[3] = (byte) (val >> 32);
-    doubleBytes[4] = (byte) (val >> 24);
-    doubleBytes[5] = (byte) (val >> 16);
-    doubleBytes[6] = (byte) (val >> 8);
-    doubleBytes[7] = (byte) val;
-    out.write(doubleBytes, 0, 8);
-    return doubleBytes.length;
-  }
-
-  private byte[] vLongBytes = new byte[9];
-
-  public static int writeVLongToByteArray(byte[] bytes, int offset, long l) {
-    if (l >= -112 && l <= 127) {
-      bytes[offset] = (byte) l;
-      return 1;
-    }
-
-    int len = -112;
-    if (l < 0) {
-      l ^= -1L; // take one's complement'
-      len = -120;
-    }
-
-    long tmp = l;
-    while (tmp != 0) {
-      tmp = tmp >> 8;
-      len--;
-    }
-
-    bytes[offset++] = (byte) len;
-    len = (len < -120) ? -(len + 120) : -(len + 112);
-
-    for (int idx = len; idx != 0; idx--) {
-      int shiftbits = (idx - 1) * 8;
-      bytes[offset++] = (byte) ((l & (0xFFL << shiftbits)) >> shiftbits);
-    }
-    return 1 + len;
-  }
-
-  public int writeVLong(OutputStream out, long l) throws IOException {
-    int len = writeVLongToByteArray(vLongBytes, 0, l);
-    out.write(vLongBytes, 0, len);
-    return len;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/BufferPool.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/BufferPool.java b/tajo-storage/src/main/java/org/apache/tajo/storage/BufferPool.java
deleted file mode 100644
index 85c79fa..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/BufferPool.java
+++ /dev/null
@@ -1,74 +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 io.netty.buffer.ByteBuf;
-import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.util.internal.PlatformDependent;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/* this class is PooledBuffer holder */
-public class BufferPool {
-
-  private static final PooledByteBufAllocator allocator;
-
-  private BufferPool() {
-  }
-
-  static {
-    //TODO we need determine the default params
-    allocator = new PooledByteBufAllocator(PlatformDependent.directBufferPreferred());
-
-    /* if you are finding memory leak, please enable this line */
-    //ResourceLeakDetector.setLevel(ResourceLeakDetector.Level.ADVANCED);
-  }
-
-  public static long maxDirectMemory() {
-    return PlatformDependent.maxDirectMemory();
-  }
-
-
-  public synchronized static ByteBuf directBuffer(int size) {
-    return allocator.directBuffer(size);
-  }
-
-  /**
-   *
-   * @param size the initial capacity
-   * @param max the max capacity
-   * @return allocated ByteBuf from pool
-   */
-  public static ByteBuf directBuffer(int size, int max) {
-    return allocator.directBuffer(size, max);
-  }
-
-  @InterfaceStability.Unstable
-  public static void forceRelease(ByteBuf buf) {
-    buf.release(buf.refCnt());
-  }
-
-  /**
-   * the ByteBuf will increase to writable size
-   * @param buf
-   * @param minWritableBytes required minimum writable size
-   */
-  public static void ensureWritable(ByteBuf buf, int minWritableBytes) {
-    buf.ensureWritable(minWritableBytes);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java b/tajo-storage/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
deleted file mode 100644
index b1b6d65..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.hdfs.DFSInputStream;
-import org.apache.hadoop.io.IOUtils;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.ScatteringByteChannel;
-import java.nio.channels.spi.AbstractInterruptibleChannel;
-
-public class ByteBufInputChannel extends AbstractInterruptibleChannel implements ScatteringByteChannel {
-
-  ByteBufferReadable byteBufferReadable;
-  ReadableByteChannel channel;
-  InputStream inputStream;
-
-  public ByteBufInputChannel(InputStream inputStream) {
-    if (inputStream instanceof DFSInputStream && inputStream instanceof ByteBufferReadable) {
-      this.byteBufferReadable = (ByteBufferReadable) inputStream;
-    } else {
-      this.channel = Channels.newChannel(inputStream);
-    }
-
-    this.inputStream = inputStream;
-  }
-
-  @Override
-  public long read(ByteBuffer[] dsts, int offset, int length) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long read(ByteBuffer[] dsts) {
-    return read(dsts, 0, dsts.length);
-  }
-
-  @Override
-  public int read(ByteBuffer dst) throws IOException {
-    if (byteBufferReadable != null) {
-      return byteBufferReadable.read(dst);
-    } else {
-      return channel.read(dst);
-    }
-  }
-
-  @Override
-  protected void implCloseChannel() throws IOException {
-    IOUtils.cleanup(null, channel, inputStream);
-  }
-
-  public int available() throws IOException {
-    return inputStream.available();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
deleted file mode 100644
index 1e2b0f3..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ /dev/null
@@ -1,588 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.commons.lang.StringUtils;
-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.FileSystem;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.*;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.exception.UnsupportedException;
-import org.apache.tajo.storage.compress.CodecPool;
-import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
-import org.apache.tajo.util.BytesUtils;
-
-import java.io.*;
-import java.util.ArrayList;
-import java.util.Arrays;
-
-public class CSVFile {
-
-  public static final byte LF = '\n';
-  public static int EOF = -1;
-
-  private static final Log LOG = LogFactory.getLog(CSVFile.class);
-
-  public static class CSVAppender extends FileAppender {
-    private final TableMeta meta;
-    private final Schema schema;
-    private final int columnNum;
-    private final FileSystem fs;
-    private FSDataOutputStream fos;
-    private DataOutputStream outputStream;
-    private CompressionOutputStream deflateFilter;
-    private char delimiter;
-    private TableStatistics stats = null;
-    private Compressor compressor;
-    private CompressionCodecFactory codecFactory;
-    private CompressionCodec codec;
-    private Path compressedPath;
-    private byte[] nullChars;
-    private int BUFFER_SIZE = 128 * 1024;
-    private int bufferedBytes = 0;
-    private long pos = 0;
-    private boolean isShuffle;
-
-    private NonSyncByteArrayOutputStream os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
-    private SerializerDeserializer serde;
-
-    public CSVAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
-                       final Schema schema, final TableMeta meta, final Path workDir) throws IOException {
-      super(conf, taskAttemptId, schema, meta, workDir);
-      this.fs = workDir.getFileSystem(conf);
-      this.meta = meta;
-      this.schema = schema;
-      this.delimiter = StringEscapeUtils.unescapeJava(
-          this.meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
-
-      this.columnNum = schema.size();
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(
-          this.meta.getOption(StorageConstants.TEXT_NULL, NullDatum.DEFAULT_TEXT));
-
-      if (StringUtils.isEmpty(nullCharacters)) {
-        nullChars = NullDatum.get().asTextBytes();
-      } else {
-        nullChars = nullCharacters.getBytes();
-      }
-    }
-
-    @Override
-    public void init() throws IOException {
-      if (!fs.exists(path.getParent())) {
-        throw new FileNotFoundException(path.getParent().toString());
-      }
-
-      //determine the intermediate file type
-      String store = conf.get(TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.varname,
-          TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.defaultVal);
-      if (enabledStats && CatalogProtos.StoreType.CSV == CatalogProtos.StoreType.valueOf(store.toUpperCase())) {
-        isShuffle = true;
-      } else {
-        isShuffle = false;
-      }
-
-      if(this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
-        String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
-        codecFactory = new CompressionCodecFactory(conf);
-        codec = codecFactory.getCodecByClassName(codecName);
-        compressor =  CodecPool.getCompressor(codec);
-        if(compressor != null) compressor.reset();  //builtin gzip is null
-
-        String extension = codec.getDefaultExtension();
-        compressedPath = path.suffix(extension);
-
-        if (fs.exists(compressedPath)) {
-          throw new AlreadyExistsStorageException(compressedPath);
-        }
-
-        fos = fs.create(compressedPath);
-        deflateFilter = codec.createOutputStream(fos, compressor);
-        outputStream = new DataOutputStream(deflateFilter);
-
-      } else {
-        if (fs.exists(path)) {
-          throw new AlreadyExistsStorageException(path);
-        }
-        fos = fs.create(path);
-        outputStream = new DataOutputStream(new BufferedOutputStream(fos));
-      }
-
-      if (enabledStats) {
-        this.stats = new TableStatistics(this.schema);
-      }
-
-      try {
-        //It will be remove, because we will add custom serde in textfile
-        String serdeClass = this.meta.getOption(StorageConstants.CSVFILE_SERDE,
-            TextSerializerDeserializer.class.getName());
-        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
-        throw new IOException(e);
-      }
-
-      os.reset();
-      pos = fos.getPos();
-      bufferedBytes = 0;
-      super.init();
-    }
-
-
-    @Override
-    public void addTuple(Tuple tuple) throws IOException {
-      Datum datum;
-      int rowBytes = 0;
-
-      for (int i = 0; i < columnNum; i++) {
-        datum = tuple.get(i);
-        rowBytes += serde.serialize(schema.getColumn(i), datum, os, nullChars);
-
-        if(columnNum - 1 > i){
-          os.write((byte) delimiter);
-          rowBytes += 1;
-        }
-        if (isShuffle) {
-          // it is to calculate min/max values, and it is only used for the intermediate file.
-          stats.analyzeField(i, datum);
-        }
-      }
-      os.write(LF);
-      rowBytes += 1;
-
-      pos += rowBytes;
-      bufferedBytes += rowBytes;
-      if(bufferedBytes > BUFFER_SIZE){
-        flushBuffer();
-      }
-      // Statistical section
-      if (enabledStats) {
-        stats.incrementRow();
-      }
-    }
-
-    private void flushBuffer() throws IOException {
-      if(os.getLength() > 0) {
-        os.writeTo(outputStream);
-        os.reset();
-        bufferedBytes = 0;
-      }
-    }
-    @Override
-    public long getOffset() throws IOException {
-      return pos;
-    }
-
-    @Override
-    public void flush() throws IOException {
-      flushBuffer();
-      outputStream.flush();
-    }
-
-    @Override
-    public void close() throws IOException {
-
-      try {
-        flush();
-
-        // Statistical section
-        if (enabledStats) {
-          stats.setNumBytes(getOffset());
-        }
-
-        if(deflateFilter != null) {
-          deflateFilter.finish();
-          deflateFilter.resetState();
-          deflateFilter = null;
-        }
-
-        os.close();
-      } finally {
-        IOUtils.cleanup(LOG, fos);
-        if (compressor != null) {
-          CodecPool.returnCompressor(compressor);
-          compressor = null;
-        }
-      }
-    }
-
-    @Override
-    public TableStats getStats() {
-      if (enabledStats) {
-        return stats.getTableStat();
-      } else {
-        return null;
-      }
-    }
-
-    public boolean isCompress() {
-      return compressor != null;
-    }
-
-    public String getExtension() {
-      return codec != null ? codec.getDefaultExtension() : "";
-    }
-  }
-
-  public static class CSVScanner extends FileScanner implements SeekableScanner {
-    public CSVScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment)
-        throws IOException {
-      super(conf, schema, meta, fragment);
-      factory = new CompressionCodecFactory(conf);
-      codec = factory.getCodec(this.fragment.getPath());
-      if (codec == null || codec instanceof SplittableCompressionCodec) {
-        splittable = true;
-      }
-
-      //Delimiter
-      this.delimiter = StringEscapeUtils.unescapeJava(
-          meta.getOption(StorageConstants.TEXT_DELIMITER,
-          meta.getOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER))).charAt(0);
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(
-          meta.getOption(StorageConstants.TEXT_NULL,
-          meta.getOption(StorageConstants.CSVFILE_NULL, NullDatum.DEFAULT_TEXT)));
-
-      if (StringUtils.isEmpty(nullCharacters)) {
-        nullChars = NullDatum.get().asTextBytes();
-      } else {
-        nullChars = nullCharacters.getBytes();
-      }
-    }
-
-    private final static int DEFAULT_PAGE_SIZE = 256 * 1024;
-    private char delimiter;
-    private FileSystem fs;
-    private FSDataInputStream fis;
-    private InputStream is; //decompressd stream
-    private CompressionCodecFactory factory;
-    private CompressionCodec codec;
-    private Decompressor decompressor;
-    private Seekable filePosition;
-    private boolean splittable = false;
-    private long startOffset, end, pos;
-    private int currentIdx = 0, validIdx = 0, recordCount = 0;
-    private int[] targetColumnIndexes;
-    private boolean eof = false;
-    private final byte[] nullChars;
-    private SplitLineReader reader;
-    private ArrayList<Long> fileOffsets;
-    private ArrayList<Integer> rowLengthList;
-    private ArrayList<Integer> startOffsets;
-    private NonSyncByteArrayOutputStream buffer;
-    private SerializerDeserializer serde;
-
-    @Override
-    public void init() throws IOException {
-      fileOffsets = new ArrayList<Long>();
-      rowLengthList = new ArrayList<Integer>();
-      startOffsets = new ArrayList<Integer>();
-      buffer = new NonSyncByteArrayOutputStream(DEFAULT_PAGE_SIZE);
-
-      // FileFragment information
-      if(fs == null) {
-        fs = FileScanner.getFileSystem((TajoConf)conf, fragment.getPath());
-      }
-      if(fis == null) fis = fs.open(fragment.getPath());
-
-      recordCount = 0;
-      pos = startOffset = fragment.getStartKey();
-      end = startOffset + fragment.getLength();
-
-      if (codec != null) {
-        decompressor = CodecPool.getDecompressor(codec);
-        if (codec instanceof SplittableCompressionCodec) {
-          SplitCompressionInputStream cIn = ((SplittableCompressionCodec) codec).createInputStream(
-              fis, decompressor, startOffset, end,
-              SplittableCompressionCodec.READ_MODE.BYBLOCK);
-
-          reader = new CompressedSplitLineReader(cIn, conf, null);
-          startOffset = cIn.getAdjustedStart();
-          end = cIn.getAdjustedEnd();
-          filePosition = cIn;
-          is = cIn;
-        } else {
-          is = new DataInputStream(codec.createInputStream(fis, decompressor));
-          reader = new SplitLineReader(is, null);
-          filePosition = fis;
-        }
-      } else {
-        fis.seek(startOffset);
-        filePosition = fis;
-        is = fis;
-        reader = new SplitLineReader(is, null);
-      }
-
-      if (targets == null) {
-        targets = schema.toArray();
-      }
-
-      targetColumnIndexes = new int[targets.length];
-      for (int i = 0; i < targets.length; i++) {
-        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
-      }
-
-      try {
-        //FIXME
-        String serdeClass = this.meta.getOption(StorageConstants.CSVFILE_SERDE,
-            TextSerializerDeserializer.class.getName());
-        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
-        throw new IOException(e);
-      }
-
-      super.init();
-      Arrays.sort(targetColumnIndexes);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("CSVScanner open:" + fragment.getPath() + "," + startOffset + "," + end +
-            "," + fs.getFileStatus(fragment.getPath()).getLen());
-      }
-
-      if (startOffset != 0) {
-        pos += reader.readLine(new Text(), 0, maxBytesToConsume(pos));
-      }
-      eof = false;
-      page();
-    }
-
-    private int maxBytesToConsume(long pos) {
-      return isCompress() ? Integer.MAX_VALUE : (int) Math.min(Integer.MAX_VALUE, end - pos);
-    }
-
-    private long fragmentable() throws IOException {
-      return end - getFilePosition();
-    }
-
-    private long getFilePosition() throws IOException {
-      long retVal;
-      if (isCompress()) {
-        retVal = filePosition.getPos();
-      } else {
-        retVal = pos;
-      }
-      return retVal;
-    }
-
-    private void page() throws IOException {
-//      // Index initialization
-      currentIdx = 0;
-      validIdx = 0;
-      int currentBufferPos = 0;
-      int bufferedSize = 0;
-
-      buffer.reset();
-      startOffsets.clear();
-      rowLengthList.clear();
-      fileOffsets.clear();
-
-      if(eof) {
-        return;
-      }
-
-      while (DEFAULT_PAGE_SIZE >= bufferedSize){
-
-        int ret = reader.readDefaultLine(buffer, rowLengthList, Integer.MAX_VALUE, Integer.MAX_VALUE);
-
-        if(ret == 0){
-          break;
-        } else {
-          fileOffsets.add(pos);
-          pos += ret;
-          startOffsets.add(currentBufferPos);
-          currentBufferPos += rowLengthList.get(rowLengthList.size() - 1);
-          bufferedSize += ret;
-          validIdx++;
-          recordCount++;
-        }
-
-        if(getFilePosition() > end && !reader.needAdditionalRecordAfterSplit()){
-          eof = true;
-          break;
-        }
-      }
-      if (tableStats != null) {
-        tableStats.setReadBytes(pos - startOffset);
-        tableStats.setNumRows(recordCount);
-      }
-    }
-
-    @Override
-    public float getProgress() {
-      try {
-        if(eof) {
-          return 1.0f;
-        }
-        long filePos = getFilePosition();
-        if (startOffset == filePos) {
-          return 0.0f;
-        } else {
-          long readBytes = filePos - startOffset;
-          long remainingBytes = Math.max(end - filePos, 0);
-          return Math.min(1.0f, (float)(readBytes) / (float)(readBytes + remainingBytes));
-        }
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-        return 0.0f;
-      }
-    }
-
-    @Override
-    public Tuple next() throws IOException {
-      try {
-        if (currentIdx == validIdx) {
-          if (eof) {
-            return null;
-          } else {
-            page();
-
-            if(currentIdx == validIdx){
-              return null;
-            }
-          }
-        }
-
-        long offset = -1;
-        if(!isCompress()){
-          offset = fileOffsets.get(currentIdx);
-        }
-
-        byte[][] cells = BytesUtils.splitPreserveAllTokens(buffer.getData(), startOffsets.get(currentIdx),
-            rowLengthList.get(currentIdx), delimiter, targetColumnIndexes);
-        currentIdx++;
-        return new LazyTuple(schema, cells, offset, nullChars, serde);
-      } catch (Throwable t) {
-        LOG.error("Tuple list length: " + (fileOffsets != null ? fileOffsets.size() : 0), t);
-        LOG.error("Tuple list current index: " + currentIdx, t);
-        throw new IOException(t);
-      }
-    }
-
-    private boolean isCompress() {
-      return codec != null;
-    }
-
-    @Override
-    public void reset() throws IOException {
-      if (decompressor != null) {
-        CodecPool.returnDecompressor(decompressor);
-        decompressor = null;
-      }
-
-      init();
-    }
-
-    @Override
-    public void close() throws IOException {
-      try {
-        if (tableStats != null) {
-          tableStats.setReadBytes(pos - startOffset);  //Actual Processed Bytes. (decompressed bytes + overhead)
-          tableStats.setNumRows(recordCount);
-        }
-
-        IOUtils.cleanup(LOG, reader, is, fis);
-        fs = null;
-        is = null;
-        fis = null;
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("CSVScanner processed record:" + recordCount);
-        }
-      } finally {
-        if (decompressor != null) {
-          CodecPool.returnDecompressor(decompressor);
-          decompressor = null;
-        }
-      }
-    }
-
-    @Override
-    public boolean isProjectable() {
-      return true;
-    }
-
-    @Override
-    public boolean isSelectable() {
-      return false;
-    }
-
-    @Override
-    public void setSearchCondition(Object expr) {
-    }
-
-    @Override
-    public void seek(long offset) throws IOException {
-      if(isCompress()) throw new UnsupportedException();
-
-      int tupleIndex = Arrays.binarySearch(fileOffsets.toArray(), offset);
-
-      if (tupleIndex > -1) {
-        this.currentIdx = tupleIndex;
-      } else if (isSplittable() && end >= offset || startOffset <= offset) {
-        eof = false;
-        fis.seek(offset);
-        pos = offset;
-        reader.reset();
-        this.currentIdx = 0;
-        this.validIdx = 0;
-        // pageBuffer();
-      } else {
-        throw new IOException("invalid offset " +
-            " < start : " +  startOffset + " , " +
-            "  end : " + end + " , " +
-            "  filePos : " + filePosition.getPos() + " , " +
-            "  input offset : " + offset + " >");
-      }
-    }
-
-    @Override
-    public long getNextOffset() throws IOException {
-      if(isCompress()) throw new UnsupportedException();
-
-      if (this.currentIdx == this.validIdx) {
-        if (fragmentable() <= 0) {
-          return -1;
-        } else {
-          page();
-          if(currentIdx == validIdx) return -1;
-        }
-      }
-      return fileOffsets.get(currentIdx);
-    }
-
-    @Override
-    public boolean isSplittable(){
-      return splittable;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
deleted file mode 100644
index 4f58e68..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.SplitCompressionInputStream;
-import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-
-/**
- * Line reader for compressed splits
- *
- * Reading records from a compressed split is tricky, as the
- * LineRecordReader is using the reported compressed input stream
- * position directly to determine when a split has ended.  In addition the
- * compressed input stream is usually faking the actual byte position, often
- * updating it only after the first compressed block after the split is
- * accessed.
- *
- * Depending upon where the last compressed block of the split ends relative
- * to the record delimiters it can be easy to accidentally drop the last
- * record or duplicate the last record between this split and the next.
- *
- * Split end scenarios:
- *
- * 1) Last block of split ends in the middle of a record
- *      Nothing special that needs to be done here, since the compressed input
- *      stream will report a position after the split end once the record
- *      is fully read.  The consumer of the next split will discard the
- *      partial record at the start of the split normally, and no data is lost
- *      or duplicated between the splits.
- *
- * 2) Last block of split ends in the middle of a delimiter
- *      The line reader will continue to consume bytes into the next block to
- *      locate the end of the delimiter.  If a custom delimiter is being used
- *      then the next record must be read by this split or it will be dropped.
- *      The consumer of the next split will not recognize the partial
- *      delimiter at the beginning of its split and will discard it along with
- *      the next record.
- *
- *      However for the default delimiter processing there is a special case
- *      because CR, LF, and CRLF are all valid record delimiters.  If the
- *      block ends with a CR then the reader must peek at the next byte to see
- *      if it is an LF and therefore part of the same record delimiter.
- *      Peeking at the next byte is an access to the next block and triggers
- *      the stream to report the end of the split.  There are two cases based
- *      on the next byte:
- *
- *      A) The next byte is LF
- *           The split needs to end after the current record is returned.  The
- *           consumer of the next split will discard the first record, which
- *           is degenerate since LF is itself a delimiter, and start consuming
- *           records after that byte.  If the current split tries to read
- *           another record then the record will be duplicated between splits.
- *
- *      B) The next byte is not LF
- *           The current record will be returned but the stream will report
- *           the split has ended due to the peek into the next block.  If the
- *           next record is not read then it will be lost, as the consumer of
- *           the next split will discard it before processing subsequent
- *           records.  Therefore the next record beyond the reported split end
- *           must be consumed by this split to avoid data loss.
- *
- * 3) Last block of split ends at the beginning of a delimiter
- *      This is equivalent to case 1, as the reader will consume bytes into
- *      the next block and trigger the end of the split.  No further records
- *      should be read as the consumer of the next split will discard the
- *      (degenerate) record at the beginning of its split.
- *
- * 4) Last block of split ends at the end of a delimiter
- *      Nothing special needs to be done here. The reader will not start
- *      examining the bytes into the next block until the next record is read,
- *      so the stream will not report the end of the split just yet.  Once the
- *      next record is read then the next block will be accessed and the
- *      stream will indicate the end of the split.  The consumer of the next
- *      split will correctly discard the first record of its split, and no
- *      data is lost or duplicated.
- *
- *      If the default delimiter is used and the block ends at a CR then this
- *      is treated as case 2 since the reader does not yet know without
- *      looking at subsequent bytes whether the delimiter has ended.
- *
- * NOTE: It is assumed that compressed input streams *never* return bytes from
- *       multiple compressed blocks from a single read.  Failure to do so will
- *       violate the buffering performed by this class, as it will access
- *       bytes into the next block after the split before returning all of the
- *       records from the previous block.
- */
-
-public class CompressedSplitLineReader extends SplitLineReader {
-  SplitCompressionInputStream scin;
-  private boolean usingCRLF;
-  private boolean needAdditionalRecord = false;
-  private boolean finished = false;
-
-  public CompressedSplitLineReader(SplitCompressionInputStream in,
-                                   Configuration conf,
-                                   byte[] recordDelimiterBytes)
-      throws IOException {
-    super(in, conf, recordDelimiterBytes);
-    scin = in;
-    usingCRLF = (recordDelimiterBytes == null);
-  }
-
-  @Override
-  protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter)
-      throws IOException {
-    int bytesRead = in.read(buffer);
-
-    // If the split ended in the middle of a record delimiter then we need
-    // to read one additional record, as the consumer of the next split will
-    // not recognize the partial delimiter as a record.
-    // However if using the default delimiter and the next character is a
-    // linefeed then next split will treat it as a delimiter all by itself
-    // and the additional record read should not be performed.
-    if (inDelimiter && bytesRead > 0) {
-      if (usingCRLF) {
-        needAdditionalRecord = (buffer[0] != '\n');
-      } else {
-        needAdditionalRecord = true;
-      }
-    }
-    return bytesRead;
-  }
-
-  @Override
-  public int readLine(Text str, int maxLineLength, int maxBytesToConsume)
-      throws IOException {
-    int bytesRead = 0;
-    if (!finished) {
-      // only allow at most one more record to be read after the stream
-      // reports the split ended
-      if (scin.getPos() > scin.getAdjustedEnd()) {
-        finished = true;
-      }
-
-      bytesRead = super.readLine(str, maxLineLength, maxBytesToConsume);
-    }
-    return bytesRead;
-  }
-
-  @Override
-  public int readDefaultLine(NonSyncByteArrayOutputStream str, ArrayList<Integer> offsets, int maxLineLength
-      , int maxBytesToConsume) throws IOException {
-    int bytesRead = 0;
-    if (!finished) {
-      // only allow at most one more record to be read after the stream
-      // reports the split ended
-      if (scin.getPos() > scin.getAdjustedEnd()) {
-        finished = true;
-      }
-
-      bytesRead = super.readDefaultLine(str, offsets, maxLineLength, maxBytesToConsume);
-    }
-    return bytesRead;
-  }
-
-  @Override
-  public boolean needAdditionalRecordAfterSplit() {
-    return !finished && needAdditionalRecord;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/DataLocation.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DataLocation.java b/tajo-storage/src/main/java/org/apache/tajo/storage/DataLocation.java
deleted file mode 100644
index 8841a31..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/DataLocation.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-public class DataLocation {
-  private String host;
-  private int volumeId;
-
-  public DataLocation(String host, int volumeId) {
-    this.host = host;
-    this.volumeId = volumeId;
-  }
-
-  public String getHost() {
-    return host;
-  }
-
-  public int getVolumeId() {
-    return volumeId;
-  }
-
-  @Override
-  public String toString() {
-    return "DataLocation{" +
-        "host=" + host +
-        ", volumeId=" + volumeId +
-        '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java b/tajo-storage/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
deleted file mode 100644
index 2396349..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
+++ /dev/null
@@ -1,62 +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 java.util.ArrayList;
-import java.util.List;
-
-public class DiskDeviceInfo {
-	private int id;
-	private String name;
-	
-	private List<DiskMountInfo> mountInfos = new ArrayList<DiskMountInfo>();
-
-	public DiskDeviceInfo(int id) {
-		this.id = id;
-	}
-	
-	public int getId() {
-		return id;
-	}
-
-	public String getName() {
-		return name;
-	}
-
-	public void setName(String name) {
-		this.name = name;
-	}
-	
-	@Override
-	public String toString() {
-		return id + "," + name;
-	}
-
-	public void addMountPath(DiskMountInfo diskMountInfo) {
-		mountInfos.add(diskMountInfo);
-	}
-
-	public List<DiskMountInfo> getMountInfos() {
-		return mountInfos;
-	}
-
-	public void setMountInfos(List<DiskMountInfo> mountInfos) {
-		this.mountInfos = mountInfos;
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/DiskInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskInfo.java b/tajo-storage/src/main/java/org/apache/tajo/storage/DiskInfo.java
deleted file mode 100644
index 22f18ba..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskInfo.java
+++ /dev/null
@@ -1,75 +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;
-
-public class DiskInfo {
-	private int id;
-	private String partitionName;
-	private String mountPath;
-	
-	private long capacity;
-	private long used;
-	
-	public DiskInfo(int id, String partitionName) {
-		this.id = id;
-		this.partitionName = partitionName;
-	}
-
-	public int getId() {
-		return id;
-	}
-
-	public void setId(int id) {
-		this.id = id;
-	}
-
-	public String getPartitionName() {
-		return partitionName;
-	}
-
-	public void setPartitionName(String partitionName) {
-		this.partitionName = partitionName;
-	}
-
-	public String getMountPath() {
-		return mountPath;
-	}
-
-	public void setMountPath(String mountPath) {
-		this.mountPath = mountPath;
-	}
-
-	public long getCapacity() {
-		return capacity;
-	}
-
-	public void setCapacity(long capacity) {
-		this.capacity = capacity;
-	}
-
-	public long getUsed() {
-		return used;
-	}
-
-	public void setUsed(long used) {
-		this.used = used;
-	}
-	
-	
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskMountInfo.java b/tajo-storage/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
deleted file mode 100644
index aadb0e7..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
+++ /dev/null
@@ -1,101 +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.base.Objects;
-
-public class DiskMountInfo implements Comparable<DiskMountInfo> {
-	private String mountPath;
-	
-	private long capacity;
-	private long used;
-	
-	private int deviceId;
-	
-	public DiskMountInfo(int deviceId, String mountPath) {
-		this.mountPath = mountPath;
-	}
-
-	public String getMountPath() {
-		return mountPath;
-	}
-
-	public void setMountPath(String mountPath) {
-		this.mountPath = mountPath;
-	}
-
-	public long getCapacity() {
-		return capacity;
-	}
-
-	public void setCapacity(long capacity) {
-		this.capacity = capacity;
-	}
-
-	public long getUsed() {
-		return used;
-	}
-
-	public void setUsed(long used) {
-		this.used = used;
-	}
-
-	public int getDeviceId() {
-		return deviceId;
-	}
-
-  @Override
-  public boolean equals(Object obj){
-    if (!(obj instanceof DiskMountInfo)) return false;
-
-    if (compareTo((DiskMountInfo) obj) == 0) return true;
-    else return false;
-  }
-
-  @Override
-  public int hashCode(){
-    return Objects.hashCode(mountPath);
-  }
-
-	@Override
-	public int compareTo(DiskMountInfo other) {
-		String path1 = mountPath;
-		String path2 = other.mountPath;
-		
-		int path1Depth = "/".equals(path1) ? 0 : path1.split("/", -1).length - 1 ;
-		int path2Depth = "/".equals(path2) ? 0 : path2.split("/", -1).length - 1 ;
-		
-		if(path1Depth > path2Depth) {
-			return -1;
-		} else if(path1Depth < path2Depth) {
-			return 1;
-		} else {
-			int path1Length = path1.length();
-			int path2Length = path2.length();
-			
-			if(path1Length < path2Length) {
-				return 1;
-			} else if(path1Length > path2Length) {
-				return -1;
-			} else {
-				return path1.compareTo(path2);
-			}
-		}
-	}
-}


[11/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..7ddf09a
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
@@ -0,0 +1,236 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.util.BytesUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ColumnMapping {
+  private TableMeta tableMeta;
+  private Schema schema;
+  private char rowKeyDelimiter;
+
+  private String hbaseTableName;
+
+  private int[] rowKeyFieldIndexes;
+  private boolean[] isRowKeyMappings;
+  private boolean[] isBinaryColumns;
+  private boolean[] isColumnKeys;
+  private boolean[] isColumnValues;
+
+  // schema order -> 0: cf name, 1: column name -> name bytes
+  private byte[][][] mappingColumns;
+
+  private int numRowKeys;
+
+  public ColumnMapping(Schema schema, TableMeta tableMeta) throws IOException {
+    this.schema = schema;
+    this.tableMeta = tableMeta;
+
+    init();
+  }
+
+  public void init() throws IOException {
+    hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY);
+    String delim = tableMeta.getOption(HBaseStorageConstants.META_ROWKEY_DELIMITER, "").trim();
+    if (delim.length() > 0) {
+      rowKeyDelimiter = delim.charAt(0);
+    }
+    isRowKeyMappings = new boolean[schema.size()];
+    rowKeyFieldIndexes = new int[schema.size()];
+    isBinaryColumns = new boolean[schema.size()];
+    isColumnKeys = new boolean[schema.size()];
+    isColumnValues = new boolean[schema.size()];
+
+    mappingColumns = new byte[schema.size()][][];
+
+    for (int i = 0; i < schema.size(); i++) {
+      rowKeyFieldIndexes[i] = -1;
+    }
+
+    String columnMapping = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
+    if (columnMapping == null || columnMapping.isEmpty()) {
+      throw new IOException("'columns' property is required.");
+    }
+
+    String[] columnMappingTokens = columnMapping.split(",");
+
+    if (columnMappingTokens.length != schema.getColumns().size()) {
+      throw new IOException("The number of mapped HBase columns is great than the number of Tajo table columns");
+    }
+
+    int index = 0;
+    for (String eachToken: columnMappingTokens) {
+      mappingColumns[index] = new byte[2][];
+
+      byte[][] mappingTokens = BytesUtils.splitPreserveAllTokens(eachToken.trim().getBytes(), ':');
+
+      if (mappingTokens.length == 3) {
+        if (mappingTokens[0].length == 0) {
+          // cfname
+          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
+              "or '<cfname>:value:' or '<cfname>:value:#b'");
+        }
+        //<cfname>:key: or <cfname>:value:
+        if (mappingTokens[2].length != 0) {
+          String binaryOption = new String(mappingTokens[2]);
+          if ("#b".equals(binaryOption)) {
+            isBinaryColumns[index] = true;
+          } else {
+            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
+                "or '<cfname>:value:' or '<cfname>:value:#b'");
+          }
+        }
+        mappingColumns[index][0] = mappingTokens[0];
+        String keyOrValue = new String(mappingTokens[1]);
+        if (HBaseStorageConstants.KEY_COLUMN_MAPPING.equalsIgnoreCase(keyOrValue)) {
+          isColumnKeys[index] = true;
+        } else if (HBaseStorageConstants.VALUE_COLUMN_MAPPING.equalsIgnoreCase(keyOrValue)) {
+          isColumnValues[index] = true;
+        } else {
+          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
+        }
+      } else if (mappingTokens.length == 2) {
+        //<cfname>: or <cfname>:<qualifier> or :key
+        String cfName = new String(mappingTokens[0]);
+        String columnName = new String(mappingTokens[1]);
+        RowKeyMapping rowKeyMapping = getRowKeyMapping(cfName, columnName);
+        if (rowKeyMapping != null) {
+          isRowKeyMappings[index] = true;
+          numRowKeys++;
+          isBinaryColumns[index] = rowKeyMapping.isBinary();
+          if (!cfName.isEmpty()) {
+            if (rowKeyDelimiter == 0) {
+              throw new IOException("hbase.rowkey.delimiter is required.");
+            }
+            rowKeyFieldIndexes[index] = Integer.parseInt(cfName);
+          } else {
+            rowKeyFieldIndexes[index] = -1; //rowkey is mapped a single column.
+          }
+        } else {
+          if (cfName.isEmpty()) {
+            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
+          }
+          if (cfName != null) {
+            mappingColumns[index][0] = Bytes.toBytes(cfName);
+          }
+
+          if (columnName != null && !columnName.isEmpty()) {
+            String[] columnNameTokens = columnName.split("#");
+            if (columnNameTokens[0].isEmpty()) {
+              mappingColumns[index][1] = null;
+            } else {
+              mappingColumns[index][1] = Bytes.toBytes(columnNameTokens[0]);
+            }
+            if (columnNameTokens.length == 2 && "b".equals(columnNameTokens[1])) {
+              isBinaryColumns[index] = true;
+            }
+          }
+        }
+      } else {
+        throw new IOException(eachToken + " 'column' attribute '[cfname]:[qualfier]:'");
+      }
+
+      index++;
+    } // for loop
+  }
+
+  public List<String> getColumnFamilyNames() {
+    List<String> cfNames = new ArrayList<String>();
+
+    for (byte[][] eachCfName: mappingColumns) {
+      if (eachCfName != null && eachCfName.length > 0 && eachCfName[0] != null) {
+        String cfName = new String(eachCfName[0]);
+        if (!cfNames.contains(cfName)) {
+          cfNames.add(cfName);
+        }
+      }
+    }
+
+    return cfNames;
+  }
+
+  private RowKeyMapping getRowKeyMapping(String cfName, String columnName) {
+    if (columnName == null || columnName.isEmpty()) {
+      return null;
+    }
+
+    String[] tokens = columnName.split("#");
+    if (!tokens[0].equalsIgnoreCase(HBaseStorageConstants.KEY_COLUMN_MAPPING)) {
+      return null;
+    }
+
+    RowKeyMapping rowKeyMapping = new RowKeyMapping();
+
+    if (tokens.length == 2 && "b".equals(tokens[1])) {
+      rowKeyMapping.setBinary(true);
+    }
+
+    if (cfName != null && !cfName.isEmpty()) {
+      rowKeyMapping.setKeyFieldIndex(Integer.parseInt(cfName));
+    }
+    return rowKeyMapping;
+  }
+
+  public char getRowKeyDelimiter() {
+    return rowKeyDelimiter;
+  }
+
+  public int[] getRowKeyFieldIndexes() {
+    return rowKeyFieldIndexes;
+  }
+
+  public boolean[] getIsRowKeyMappings() {
+    return isRowKeyMappings;
+  }
+
+  public byte[][][] getMappingColumns() {
+    return mappingColumns;
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  public boolean[] getIsBinaryColumns() {
+    return isBinaryColumns;
+  }
+
+  public String getHbaseTableName() {
+    return hbaseTableName;
+  }
+
+  public boolean[] getIsColumnKeys() {
+    return isColumnKeys;
+  }
+
+  public int getNumRowKeys() {
+    return numRowKeys;
+  }
+
+  public boolean[] getIsColumnValues() {
+    return isColumnValues;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
new file mode 100644
index 0000000..c05c5bb
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.util.Bytes;
+
+import java.io.IOException;
+
+public class HBaseBinarySerializerDeserializer {
+
+  public static Datum deserialize(Column col, byte[] bytes) throws IOException {
+    Datum datum;
+    switch (col.getDataType().getType()) {
+      case INT1:
+      case INT2:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt2(Bytes.toShort(bytes));
+        break;
+      case INT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt4(Bytes.toInt(bytes));
+        break;
+      case INT8:
+        if (bytes.length == 4) {
+          datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt8(Bytes.toInt(bytes));
+        } else {
+          datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt8(Bytes.toLong(bytes));
+        }
+        break;
+      case FLOAT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createFloat4(Bytes.toFloat(bytes));
+        break;
+      case FLOAT8:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createFloat8(Bytes.toDouble(bytes));
+        break;
+      case TEXT:
+        datum = bytes == null ? NullDatum.get() : DatumFactory.createText(bytes);
+        break;
+      default:
+        datum = NullDatum.get();
+        break;
+    }
+    return datum;
+  }
+
+  public static byte[] serialize(Column col, Datum datum) throws IOException {
+    if (datum == null || datum instanceof NullDatum) {
+      return null;
+    }
+
+    byte[] bytes;
+    switch (col.getDataType().getType()) {
+      case INT1:
+      case INT2:
+        bytes = Bytes.toBytes(datum.asInt2());
+        break;
+      case INT4:
+        bytes = Bytes.toBytes(datum.asInt4());
+        break;
+      case INT8:
+        bytes = Bytes.toBytes(datum.asInt8());
+        break;
+      case FLOAT4:
+        bytes = Bytes.toBytes(datum.asFloat4());
+        break;
+      case FLOAT8:
+        bytes = Bytes.toBytes(datum.asFloat8());
+        break;
+      case TEXT:
+        bytes = Bytes.toBytes(datum.asChars());
+        break;
+      default:
+        bytes = null;
+        break;
+    }
+
+    return bytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..2674511
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.hbase.StorageFragmentProtos.*;
+
+public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Cloneable {
+  @Expose
+  private String tableName;
+  @Expose
+  private String hbaseTableName;
+  @Expose
+  private byte[] startRow;
+  @Expose
+  private byte[] stopRow;
+  @Expose
+  private String regionLocation;
+  @Expose
+  private boolean last;
+  @Expose
+  private long length;
+
+  public HBaseFragment(String tableName, String hbaseTableName, byte[] startRow, byte[] stopRow, String regionLocation) {
+    this.tableName = tableName;
+    this.hbaseTableName = hbaseTableName;
+    this.startRow = startRow;
+    this.stopRow = stopRow;
+    this.regionLocation = regionLocation;
+    this.last = false;
+  }
+
+  public HBaseFragment(ByteString raw) throws InvalidProtocolBufferException {
+    HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder();
+    builder.mergeFrom(raw);
+    builder.build();
+    init(builder.build());
+  }
+
+  private void init(HBaseFragmentProto proto) {
+    this.tableName = proto.getTableName();
+    this.hbaseTableName = proto.getHbaseTableName();
+    this.startRow = proto.getStartRow().toByteArray();
+    this.stopRow = proto.getStopRow().toByteArray();
+    this.regionLocation = proto.getRegionLocation();
+    this.length = proto.getLength();
+    this.last = proto.getLast();
+  }
+
+  @Override
+  public int compareTo(HBaseFragment t) {
+    return Bytes.compareTo(startRow, t.startRow);
+  }
+
+  @Override
+  public String getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public String getKey() {
+    return new String(startRow);
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return startRow == null || stopRow == null;
+  }
+
+  @Override
+  public long getLength() {
+    return length;
+  }
+
+  public void setLength(long length) {
+    this.length = length;
+  }
+
+  @Override
+  public String[] getHosts() {
+    return new String[] {regionLocation};
+  }
+
+  public Object clone() throws CloneNotSupportedException {
+    HBaseFragment frag = (HBaseFragment) super.clone();
+    frag.tableName = tableName;
+    frag.hbaseTableName = hbaseTableName;
+    frag.startRow = startRow;
+    frag.stopRow = stopRow;
+    frag.regionLocation = regionLocation;
+    frag.last = last;
+    frag.length = length;
+    return frag;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof HBaseFragment) {
+      HBaseFragment t = (HBaseFragment) o;
+      if (tableName.equals(t.tableName)
+          && Bytes.equals(startRow, t.startRow)
+          && Bytes.equals(stopRow, t.stopRow)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(tableName, hbaseTableName, startRow, stopRow);
+  }
+
+  @Override
+  public String toString() {
+    return "\"fragment\": {\"tableName\": \""+ tableName + "\", hbaseTableName\": \"" + hbaseTableName + "\"" +
+        ", \"startRow\": \"" + new String(startRow) + "\"" +
+        ", \"stopRow\": \"" + new String(stopRow) + "\"" +
+        ", \"length\": \"" + length + "\"}" ;
+  }
+
+  @Override
+  public FragmentProto getProto() {
+    HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder();
+    builder.setTableName(tableName)
+        .setHbaseTableName(hbaseTableName)
+        .setStartRow(ByteString.copyFrom(startRow))
+        .setStopRow(ByteString.copyFrom(stopRow))
+        .setLast(last)
+        .setLength(length)
+        .setRegionLocation(regionLocation);
+
+    FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder();
+    fragmentBuilder.setId(this.tableName);
+    fragmentBuilder.setContents(builder.buildPartial().toByteString());
+    fragmentBuilder.setStoreType(StoreType.HBASE.name());
+    return fragmentBuilder.build();
+  }
+
+  public byte[] getStartRow() {
+    return startRow;
+  }
+
+  public byte[] getStopRow() {
+    return stopRow;
+  }
+
+  public String getRegionLocation() {
+    return regionLocation;
+  }
+
+  public boolean isLast() {
+    return last;
+  }
+
+  public void setLast(boolean last) {
+    this.last = last;
+  }
+
+  public String getHbaseTableName() {
+    return hbaseTableName;
+  }
+
+  public void setHbaseTableName(String hbaseTableName) {
+    this.hbaseTableName = hbaseTableName;
+  }
+
+  public void setStartRow(byte[] startRow) {
+    this.startRow = startRow;
+  }
+
+  public void setStopRow(byte[] stopRow) {
+    this.stopRow = stopRow;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..50f61a8
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+public class HBasePutAppender extends AbstractHBaseAppender {
+  private HTableInterface htable;
+  private long totalNumBytes;
+
+  public HBasePutAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                          Schema schema, TableMeta meta, Path stagingDir) {
+    super(conf, taskAttemptId, schema, meta, stagingDir);
+  }
+
+  @Override
+  public void init() throws IOException {
+    super.init();
+
+    Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(conf, meta);
+    HConnection hconn = ((HBaseStorageManager) StorageManager.getStorageManager((TajoConf)conf, StoreType.HBASE))
+        .getConnection(hbaseConf);
+    htable = hconn.getTable(columnMapping.getHbaseTableName());
+    htable.setAutoFlushTo(false);
+    htable.setWriteBufferSize(5 * 1024 * 1024);
+  }
+
+  @Override
+  public void addTuple(Tuple tuple) throws IOException {
+    byte[] rowkey = getRowKeyBytes(tuple);
+    totalNumBytes += rowkey.length;
+    Put put = new Put(rowkey);
+    readKeyValues(tuple, rowkey);
+
+    for (int i = 0; i < columnNum; i++) {
+      if (isRowKeyMappings[i]) {
+        continue;
+      }
+      Datum datum = tuple.get(i);
+      byte[] value;
+      if (isBinaryColumns[i]) {
+        value = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(i), datum);
+      } else {
+        value = HBaseTextSerializerDeserializer.serialize(schema.getColumn(i), datum);
+      }
+
+      if (isColumnKeys[i]) {
+        columnKeyDatas[columnKeyValueDataIndexes[i]] = value;
+      } else if (isColumnValues[i]) {
+        columnValueDatas[columnKeyValueDataIndexes[i]] = value;
+      } else {
+        put.add(mappingColumnFamilies[i][0], mappingColumnFamilies[i][1], value);
+        totalNumBytes += value.length;
+      }
+    }
+
+    for (int i = 0; i < columnKeyDatas.length; i++) {
+     put.add(columnKeyCfNames[i], columnKeyDatas[i], columnValueDatas[i]);
+      totalNumBytes += columnKeyDatas[i].length + columnValueDatas[i].length;
+    }
+
+    htable.put(put);
+
+    if (enabledStats) {
+      stats.incrementRow();
+      stats.setNumBytes(totalNumBytes);
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    htable.flushCommits();
+  }
+
+  @Override
+  public long getEstimatedOutputSize() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (htable != null) {
+      htable.flushCommits();
+      htable.close();
+    }
+    if (enabledStats) {
+      stats.setNumBytes(totalNumBytes);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..5cae077
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
@@ -0,0 +1,449 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.BytesUtils;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class HBaseScanner implements Scanner {
+  private static final Log LOG = LogFactory.getLog(HBaseScanner.class);
+  private static final int DEFAULT_FETCH_SIZE = 1000;
+  private static final int MAX_LIST_SIZE = 100;
+
+  protected boolean inited = false;
+  private TajoConf conf;
+  private Schema schema;
+  private TableMeta meta;
+  private HBaseFragment fragment;
+  private Scan scan;
+  private HTableInterface htable;
+  private Configuration hbaseConf;
+  private Column[] targets;
+  private TableStats tableStats;
+  private ResultScanner scanner;
+  private AtomicBoolean finished = new AtomicBoolean(false);
+  private float progress = 0.0f;
+  private int scanFetchSize;
+  private Result[] scanResults;
+  private int scanResultIndex = -1;
+  private Column[] schemaColumns;
+
+  private ColumnMapping columnMapping;
+  private int[] targetIndexes;
+
+  private int numRows = 0;
+
+  private byte[][][] mappingColumnFamilies;
+  private boolean[] isRowKeyMappings;
+  private boolean[] isBinaryColumns;
+  private boolean[] isColumnKeys;
+  private boolean[] isColumnValues;
+
+  private int[] rowKeyFieldIndexes;
+  private char rowKeyDelimiter;
+
+  public HBaseScanner (Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
+    this.conf = (TajoConf)conf;
+    this.schema = schema;
+    this.meta = meta;
+    this.fragment = (HBaseFragment)fragment;
+    this.tableStats = new TableStats();
+  }
+
+  @Override
+  public void init() throws IOException {
+    inited = true;
+    schemaColumns = schema.toArray();
+    if (fragment != null) {
+      tableStats.setNumBytes(0);
+      tableStats.setNumBlocks(1);
+    }
+    if (schema != null) {
+      for(Column eachColumn: schema.getColumns()) {
+        ColumnStats columnStats = new ColumnStats(eachColumn);
+        tableStats.addColumnStat(columnStats);
+      }
+    }
+
+    scanFetchSize = Integer.parseInt(
+        meta.getOption(HBaseStorageConstants.META_FETCH_ROWNUM_KEY, "" + DEFAULT_FETCH_SIZE));
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+
+    columnMapping = new ColumnMapping(schema, meta);
+    targetIndexes = new int[targets.length];
+    int index = 0;
+    for (Column eachTargetColumn: targets) {
+      targetIndexes[index++] = schema.getColumnId(eachTargetColumn.getQualifiedName());
+    }
+
+    mappingColumnFamilies = columnMapping.getMappingColumns();
+    isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    isBinaryColumns = columnMapping.getIsBinaryColumns();
+    isColumnKeys = columnMapping.getIsColumnKeys();
+    isColumnValues = columnMapping.getIsColumnValues();
+
+    rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
+    rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
+
+    hbaseConf = HBaseStorageManager.getHBaseConfiguration(conf, meta);
+
+    initScanner();
+  }
+
+  private void initScanner() throws IOException {
+    scan = new Scan();
+    scan.setBatch(scanFetchSize);
+    scan.setCacheBlocks(false);
+    scan.setCaching(scanFetchSize);
+
+    FilterList filters = null;
+    if (targetIndexes == null || targetIndexes.length == 0) {
+      filters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
+      filters.addFilter(new FirstKeyOnlyFilter());
+      filters.addFilter(new KeyOnlyFilter());
+    } else {
+      boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+      for (int eachIndex : targetIndexes) {
+        if (isRowKeyMappings[eachIndex]) {
+          continue;
+        }
+        byte[][] mappingColumn = columnMapping.getMappingColumns()[eachIndex];
+        if (mappingColumn[1] == null) {
+          scan.addFamily(mappingColumn[0]);
+        } else {
+          scan.addColumn(mappingColumn[0], mappingColumn[1]);
+        }
+      }
+    }
+
+    scan.setStartRow(fragment.getStartRow());
+    if (fragment.isLast() && fragment.getStopRow() != null &&
+        fragment.getStopRow().length > 0) {
+      // last and stopRow is not empty
+      if (filters == null) {
+        filters = new FilterList();
+      }
+      filters.addFilter(new InclusiveStopFilter(fragment.getStopRow()));
+    } else {
+      scan.setStopRow(fragment.getStopRow());
+    }
+
+    if (filters != null) {
+      scan.setFilter(filters);
+    }
+
+    if (htable == null) {
+      HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
+          .getConnection(hbaseConf);
+      htable = hconn.getTable(fragment.getHbaseTableName());
+    }
+    scanner = htable.getScanner(scan);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (finished.get()) {
+      return null;
+    }
+
+    if (scanResults == null || scanResultIndex >= scanResults.length) {
+      scanResults = scanner.next(scanFetchSize);
+      if (scanResults == null || scanResults.length == 0) {
+        finished.set(true);
+        progress = 1.0f;
+        return null;
+      }
+      scanResultIndex = 0;
+    }
+
+    Result result = scanResults[scanResultIndex++];
+    Tuple resultTuple = new VTuple(schema.size());
+    for (int i = 0; i < targetIndexes.length; i++) {
+      resultTuple.put(targetIndexes[i], getDatum(result, targetIndexes[i]));
+    }
+    numRows++;
+    return resultTuple;
+  }
+
+  private Datum getDatum(Result result, int fieldId) throws IOException {
+    byte[] value = null;
+    if (isRowKeyMappings[fieldId]) {
+      value = result.getRow();
+      if (!isBinaryColumns[fieldId] && rowKeyFieldIndexes[fieldId] >= 0) {
+        int rowKeyFieldIndex = rowKeyFieldIndexes[fieldId];
+
+        byte[][] rowKeyFields = BytesUtils.splitPreserveAllTokens(value, rowKeyDelimiter);
+
+        if (rowKeyFields.length < rowKeyFieldIndex) {
+          return NullDatum.get();
+        } else {
+          value = rowKeyFields[rowKeyFieldIndex];
+        }
+      }
+    } else {
+      if (isColumnKeys[fieldId]) {
+        NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
+        if (cfMap != null) {
+          Set<byte[]> keySet = cfMap.keySet();
+          if (keySet.size() == 1) {
+            try {
+              return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], keySet.iterator().next());
+            } catch (Exception e) {
+              LOG.error(e.getMessage(), e);
+              throw new RuntimeException(e.getMessage(), e);
+            }
+          } else {
+            StringBuilder sb = new StringBuilder();
+            sb.append("[");
+            int count = 0;
+            for (byte[] eachKey : keySet) {
+              if (count > 0) {
+                sb.append(", ");
+              }
+              Datum datum = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], eachKey);
+              sb.append("\"").append(datum.asChars()).append("\"");
+              count++;
+              if (count > MAX_LIST_SIZE) {
+                break;
+              }
+            }
+            sb.append("]");
+            return new TextDatum(sb.toString());
+          }
+        }
+      } else if (isColumnValues[fieldId]) {
+        NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
+        if (cfMap != null) {
+          Collection<byte[]> valueList = cfMap.values();
+          if (valueList.size() == 1) {
+            try {
+              return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], valueList.iterator().next());
+            } catch (Exception e) {
+              LOG.error(e.getMessage(), e);
+              throw new RuntimeException(e.getMessage(), e);
+            }
+          } else {
+            StringBuilder sb = new StringBuilder();
+            sb.append("[");
+            int count = 0;
+            for (byte[] eachValue : valueList) {
+              if (count > 0) {
+                sb.append(", ");
+              }
+              Datum datum = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], eachValue);
+              sb.append("\"").append(datum.asChars()).append("\"");
+              count++;
+              if (count > MAX_LIST_SIZE) {
+                break;
+              }
+            }
+            sb.append("]");
+            return new TextDatum(sb.toString());
+          }
+        }
+      } else {
+        if (mappingColumnFamilies[fieldId][1] == null) {
+          NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
+          if (cfMap != null && !cfMap.isEmpty()) {
+            int count = 0;
+            String delim = "";
+
+            if (cfMap.size() == 0) {
+              return NullDatum.get();
+            } else if (cfMap.size() == 1) {
+              // If a column family is mapped without column name like "cf1:" and the number of cells is one,
+              // return value is flat format not json format.
+              NavigableMap.Entry<byte[], byte[]> entry = cfMap.entrySet().iterator().next();
+              byte[] entryKey = entry.getKey();
+              byte[] entryValue = entry.getValue();
+              if (entryKey == null || entryKey.length == 0) {
+                try {
+                  if (isBinaryColumns[fieldId]) {
+                    return HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue);
+                  } else {
+                    return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue);
+                  }
+                } catch (Exception e) {
+                  LOG.error(e.getMessage(), e);
+                  throw new RuntimeException(e.getMessage(), e);
+                }
+              }
+            }
+            StringBuilder sb = new StringBuilder();
+            sb.append("{");
+            for (NavigableMap.Entry<byte[], byte[]> entry : cfMap.entrySet()) {
+              byte[] entryKey = entry.getKey();
+              byte[] entryValue = entry.getValue();
+
+              String keyText = new String(entryKey);
+              String valueText = null;
+              if (entryValue != null) {
+                try {
+                  if (isBinaryColumns[fieldId]) {
+                    valueText = HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue).asChars();
+                  } else {
+                    valueText = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue).asChars();
+                  }
+                } catch (Exception e) {
+                  LOG.error(e.getMessage(), e);
+                  throw new RuntimeException(e.getMessage(), e);
+                }
+              }
+              sb.append(delim).append("\"").append(keyText).append("\":\"").append(valueText).append("\"");
+              delim = ", ";
+              count++;
+              if (count > MAX_LIST_SIZE) {
+                break;
+              }
+            } //end of for
+            sb.append("}");
+            return new TextDatum(sb.toString());
+          } else {
+            value = null;
+          }
+        } else {
+          value = result.getValue(mappingColumnFamilies[fieldId][0], mappingColumnFamilies[fieldId][1]);
+        }
+      }
+    }
+
+    if (value == null) {
+      return NullDatum.get();
+    } else {
+      try {
+        if (isBinaryColumns[fieldId]) {
+          return HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], value);
+        } else {
+          return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], value);
+        }
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new RuntimeException(e.getMessage(), e);
+      }
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    progress = 0.0f;
+    scanResultIndex = -1;
+    scanResults = null;
+    finished.set(false);
+    tableStats = new TableStats();
+
+    if (scanner != null) {
+      scanner.close();
+      scanner = null;
+    }
+
+    initScanner();
+  }
+
+  @Override
+  public void close() throws IOException {
+    progress = 1.0f;
+    finished.set(true);
+    if (scanner != null) {
+      try {
+        scanner.close();
+        scanner = null;
+      } catch (Exception e) {
+        LOG.warn("Error while closing hbase scanner: " + e.getMessage(), e);
+      }
+    }
+    if (htable != null) {
+      htable.close();
+      htable = null;
+    }
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  @Override
+  public void setTarget(Column[] targets) {
+    if (inited) {
+      throw new IllegalStateException("Should be called before init()");
+    }
+    this.targets = targets;
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return false;
+  }
+
+  @Override
+  public void setSearchCondition(Object expr) {
+    // TODO implements adding column filter to scanner.
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return true;
+  }
+
+  @Override
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    tableStats.setNumRows(numRows);
+    return tableStats;
+  }
+
+  @Override
+  public Schema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
new file mode 100644
index 0000000..2c525a1
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+public interface HBaseStorageConstants {
+  public static final String KEY_COLUMN_MAPPING = "key";
+  public static final String VALUE_COLUMN_MAPPING = "value";
+  public static final String META_FETCH_ROWNUM_KEY = "fetch.rownum";
+  public static final String META_TABLE_KEY = "table";
+  public static final String META_COLUMNS_KEY = "columns";
+  public static final String META_SPLIT_ROW_KEYS_KEY = "hbase.split.rowkeys";
+  public static final String META_SPLIT_ROW_KEYS_FILE_KEY = "hbase.split.rowkeys.file";
+  public static final String META_ZK_QUORUM_KEY = "hbase.zookeeper.quorum";
+  public static final String META_ROWKEY_DELIMITER = "hbase.rowkey.delimiter";
+
+  public static final String INSERT_PUT_MODE = "tajo.hbase.insert.put.mode";
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
new file mode 100644
index 0000000..a6e7a81
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
@@ -0,0 +1,1135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.expr.*;
+import org.apache.tajo.plan.logical.CreateTableNode;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.NodeType;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.plan.rewrite.RewriteRule;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.Bytes;
+import org.apache.tajo.util.BytesUtils;
+import org.apache.tajo.util.Pair;
+import org.apache.tajo.util.TUtil;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.*;
+
+/**
+ * StorageManager for HBase table.
+ */
+public class HBaseStorageManager extends StorageManager {
+  private static final Log LOG = LogFactory.getLog(HBaseStorageManager.class);
+
+  private Map<HConnectionKey, HConnection> connMap = new HashMap<HConnectionKey, HConnection>();
+
+  public HBaseStorageManager (StoreType storeType) {
+    super(storeType);
+  }
+
+  @Override
+  public void storageInit() throws IOException {
+  }
+
+  @Override
+  public void closeStorageManager() {
+    synchronized (connMap) {
+      for (HConnection eachConn: connMap.values()) {
+        try {
+          eachConn.close();
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
+    createTable(tableDesc.getMeta(), tableDesc.getSchema(), tableDesc.isExternal(), ifNotExists);
+    TableStats stats = new TableStats();
+    stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
+    tableDesc.setStats(stats);
+  }
+
+  private void createTable(TableMeta tableMeta, Schema schema,
+                           boolean isExternal, boolean ifNotExists) throws IOException {
+    String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
+    if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
+      throw new IOException("HBase mapped table is required a '" +
+          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
+    }
+    TableName hTableName = TableName.valueOf(hbaseTableName);
+
+    String mappedColumns = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
+    if (mappedColumns != null && mappedColumns.split(",").length > schema.size()) {
+      throw new IOException("Columns property has more entry than Tajo table columns");
+    }
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+    int numRowKeys = 0;
+    boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (isRowKeyMappings[i]) {
+        numRowKeys++;
+      }
+    }
+    if (numRowKeys > 1) {
+      for (int i = 0; i < isRowKeyMappings.length; i++) {
+        if (isRowKeyMappings[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
+          throw new IOException("Key field type should be TEXT type.");
+        }
+      }
+    }
+
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (columnMapping.getIsColumnKeys()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
+        throw new IOException("Column key field('<cfname>:key:') type should be TEXT type.");
+      }
+      if (columnMapping.getIsColumnValues()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
+        throw new IOException("Column value field(('<cfname>:value:') type should be TEXT type.");
+      }
+    }
+
+    Configuration hConf = getHBaseConfiguration(conf, tableMeta);
+    HBaseAdmin hAdmin =  new HBaseAdmin(hConf);
+
+    try {
+      if (isExternal) {
+        // If tajo table is external table, only check validation.
+        if (mappedColumns == null || mappedColumns.isEmpty()) {
+          throw new IOException("HBase mapped table is required a '" +
+              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
+        }
+        if (!hAdmin.tableExists(hTableName)) {
+          throw new IOException("HBase table [" + hbaseTableName + "] not exists. " +
+              "External table should be a existed table.");
+        }
+        HTableDescriptor hTableDescriptor = hAdmin.getTableDescriptor(hTableName);
+        Set<String> tableColumnFamilies = new HashSet<String>();
+        for (HColumnDescriptor eachColumn : hTableDescriptor.getColumnFamilies()) {
+          tableColumnFamilies.add(eachColumn.getNameAsString());
+        }
+
+        Collection<String> mappingColumnFamilies =columnMapping.getColumnFamilyNames();
+        if (mappingColumnFamilies.isEmpty()) {
+          throw new IOException("HBase mapped table is required a '" +
+              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
+        }
+
+        for (String eachMappingColumnFamily : mappingColumnFamilies) {
+          if (!tableColumnFamilies.contains(eachMappingColumnFamily)) {
+            throw new IOException("There is no " + eachMappingColumnFamily + " column family in " + hbaseTableName);
+          }
+        }
+      } else {
+        if (hAdmin.tableExists(hbaseTableName)) {
+          if (ifNotExists) {
+            return;
+          } else {
+            throw new IOException("HBase table [" + hbaseTableName + "] already exists.");
+          }
+        }
+        // Creating hbase table
+        HTableDescriptor hTableDescriptor = parseHTableDescriptor(tableMeta, schema);
+
+        byte[][] splitKeys = getSplitKeys(conf, schema, tableMeta);
+        if (splitKeys == null) {
+          hAdmin.createTable(hTableDescriptor);
+        } else {
+          hAdmin.createTable(hTableDescriptor, splitKeys);
+        }
+      }
+    } finally {
+      hAdmin.close();
+    }
+  }
+
+  /**
+   * Returns initial region split keys.
+   *
+   * @param conf
+   * @param schema
+   * @param meta
+   * @return
+   * @throws java.io.IOException
+   */
+  private byte[][] getSplitKeys(TajoConf conf, Schema schema, TableMeta meta) throws IOException {
+    String splitRowKeys = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_KEY, "");
+    String splitRowKeysFile = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_FILE_KEY, "");
+
+    if ((splitRowKeys == null || splitRowKeys.isEmpty()) &&
+        (splitRowKeysFile == null || splitRowKeysFile.isEmpty())) {
+      return null;
+    }
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, meta);
+    boolean[] isBinaryColumns = columnMapping.getIsBinaryColumns();
+    boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
+
+    boolean rowkeyBinary = false;
+    int numRowKeys = 0;
+    Column rowKeyColumn = null;
+    for (int i = 0; i < isBinaryColumns.length; i++) {
+      if (isBinaryColumns[i] && isRowKeys[i]) {
+        rowkeyBinary = true;
+      }
+      if (isRowKeys[i]) {
+        numRowKeys++;
+        rowKeyColumn = schema.getColumn(i);
+      }
+    }
+
+    if (rowkeyBinary && numRowKeys > 1) {
+      throw new IOException("If rowkey is mapped to multi column and a rowkey is binary, " +
+          "Multiple region for creation is not support.");
+    }
+
+    if (splitRowKeys != null && !splitRowKeys.isEmpty()) {
+      String[] splitKeyTokens = splitRowKeys.split(",");
+      byte[][] splitKeys = new byte[splitKeyTokens.length][];
+      for (int i = 0; i < splitKeyTokens.length; i++) {
+        if (numRowKeys == 1 && rowkeyBinary) {
+          splitKeys[i] = HBaseBinarySerializerDeserializer.serialize(rowKeyColumn, new TextDatum(splitKeyTokens[i]));
+        } else {
+          splitKeys[i] = HBaseTextSerializerDeserializer.serialize(rowKeyColumn, new TextDatum(splitKeyTokens[i]));
+        }
+      }
+      return splitKeys;
+    }
+
+    if (splitRowKeysFile != null && !splitRowKeysFile.isEmpty()) {
+      // If there is many split keys, Tajo allows to define in the file.
+      Path path = new Path(splitRowKeysFile);
+      FileSystem fs = path.getFileSystem(conf);
+      if (!fs.exists(path)) {
+        throw new IOException("hbase.split.rowkeys.file=" + path.toString() + " not exists.");
+      }
+
+      SortedSet<String> splitKeySet = new TreeSet<String>();
+      BufferedReader reader = null;
+      try {
+        reader = new BufferedReader(new InputStreamReader(fs.open(path)));
+        String line = null;
+        while ( (line = reader.readLine()) != null ) {
+          if (line.isEmpty()) {
+            continue;
+          }
+          splitKeySet.add(line);
+        }
+      } finally {
+        if (reader != null) {
+          reader.close();
+        }
+      }
+
+      if (splitKeySet.isEmpty()) {
+        return null;
+      }
+
+      byte[][] splitKeys = new byte[splitKeySet.size()][];
+      int index = 0;
+      for (String eachKey: splitKeySet) {
+        if (numRowKeys == 1 && rowkeyBinary) {
+          splitKeys[index++] = HBaseBinarySerializerDeserializer.serialize(rowKeyColumn, new TextDatum(eachKey));
+        } else {
+          splitKeys[index++] = HBaseTextSerializerDeserializer.serialize(rowKeyColumn, new TextDatum(eachKey));
+        }
+      }
+
+      return splitKeys;
+    }
+
+    return null;
+  }
+
+  /**
+   * Creates Configuration instance and sets with hbase connection options.
+   *
+   * @param conf
+   * @param tableMeta
+   * @return
+   * @throws java.io.IOException
+   */
+  public static Configuration getHBaseConfiguration(Configuration conf, TableMeta tableMeta) throws IOException {
+    String zkQuorum = tableMeta.getOption(HBaseStorageConstants.META_ZK_QUORUM_KEY, "");
+    if (zkQuorum == null || zkQuorum.trim().isEmpty()) {
+      throw new IOException("HBase mapped table is required a '" +
+          HBaseStorageConstants.META_ZK_QUORUM_KEY + "' attribute.");
+    }
+
+    Configuration hbaseConf = (conf == null) ? HBaseConfiguration.create() : HBaseConfiguration.create(conf);
+    hbaseConf.set(HConstants.ZOOKEEPER_QUORUM, zkQuorum);
+
+    for (Map.Entry<String, String> eachOption: tableMeta.getOptions().getAllKeyValus().entrySet()) {
+      String key = eachOption.getKey();
+      if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) {
+        hbaseConf.set(key, eachOption.getValue());
+      }
+    }
+    return hbaseConf;
+  }
+
+  /**
+   * Creates HTableDescription using table meta data.
+   *
+   * @param tableMeta
+   * @param schema
+   * @return
+   * @throws java.io.IOException
+   */
+  public static HTableDescriptor parseHTableDescriptor(TableMeta tableMeta, Schema schema) throws IOException {
+    String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
+    if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
+      throw new IOException("HBase mapped table is required a '" +
+          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
+    }
+    TableName hTableName = TableName.valueOf(hbaseTableName);
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+
+    HTableDescriptor hTableDescriptor = new HTableDescriptor(hTableName);
+
+    Collection<String> columnFamilies = columnMapping.getColumnFamilyNames();
+    //If 'columns' attribute is empty, Tajo table columns are mapped to all HBase table column.
+    if (columnFamilies.isEmpty()) {
+      for (Column eachColumn: schema.getColumns()) {
+        columnFamilies.add(eachColumn.getSimpleName());
+      }
+    }
+
+    for (String eachColumnFamily: columnFamilies) {
+      hTableDescriptor.addFamily(new HColumnDescriptor(eachColumnFamily));
+    }
+
+    return hTableDescriptor;
+  }
+
+  @Override
+  public void purgeTable(TableDesc tableDesc) throws IOException {
+    HBaseAdmin hAdmin =  new HBaseAdmin(getHBaseConfiguration(conf, tableDesc.getMeta()));
+
+    try {
+      HTableDescriptor hTableDesc = parseHTableDescriptor(tableDesc.getMeta(), tableDesc.getSchema());
+      LOG.info("Deleting hbase table: " + new String(hTableDesc.getName()));
+      hAdmin.disableTable(hTableDesc.getName());
+      hAdmin.deleteTable(hTableDesc.getName());
+    } finally {
+      hAdmin.close();
+    }
+  }
+
+  /**
+   * Returns columns which are mapped to the rowkey of the hbase table.
+   *
+   * @param tableDesc
+   * @return
+   * @throws java.io.IOException
+   */
+  private Column[] getIndexableColumns(TableDesc tableDesc) throws IOException {
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+    boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    int[] rowKeyIndexes = columnMapping.getRowKeyFieldIndexes();
+
+    Column indexColumn = null;
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (isRowKeyMappings[i]) {
+        if (columnMapping.getNumRowKeys() == 1 ||
+            rowKeyIndexes[i] == 0) {
+          indexColumn = tableDesc.getSchema().getColumn(i);
+        }
+      }
+    }
+    return new Column[]{indexColumn};
+  }
+
+  @Override
+  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc, ScanNode scanNode) throws IOException {
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+
+    List<IndexPredication> indexPredications = getIndexPredications(columnMapping, tableDesc, scanNode);
+    Configuration hconf = getHBaseConfiguration(conf, tableDesc.getMeta());
+    HTable htable = null;
+    HBaseAdmin hAdmin = null;
+
+    try {
+      htable = new HTable(hconf, tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY));
+
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
+        HRegionLocation regLoc = htable.getRegionLocation(HConstants.EMPTY_BYTE_ARRAY, false);
+        if (null == regLoc) {
+          throw new IOException("Expecting at least one region.");
+        }
+        List<Fragment> fragments = new ArrayList<Fragment>(1);
+        Fragment fragment = new HBaseFragment(fragmentId, htable.getName().getNameAsString(),
+            HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, regLoc.getHostname());
+        fragments.add(fragment);
+        return fragments;
+      }
+
+      List<byte[]> startRows;
+      List<byte[]> stopRows;
+
+      if (indexPredications != null && !indexPredications.isEmpty()) {
+        // indexPredications is Disjunctive set
+        startRows = new ArrayList<byte[]>();
+        stopRows = new ArrayList<byte[]>();
+        for (IndexPredication indexPredication: indexPredications) {
+          byte[] startRow;
+          byte[] stopRow;
+          if (indexPredication.getStartValue() != null) {
+            startRow = serialize(columnMapping, indexPredication, indexPredication.getStartValue());
+          } else {
+            startRow = HConstants.EMPTY_START_ROW;
+          }
+          if (indexPredication.getStopValue() != null) {
+            stopRow = serialize(columnMapping, indexPredication, indexPredication.getStopValue());
+          } else {
+            stopRow = HConstants.EMPTY_END_ROW;
+          }
+          startRows.add(startRow);
+          stopRows.add(stopRow);
+        }
+      } else {
+        startRows = TUtil.newList(HConstants.EMPTY_START_ROW);
+        stopRows = TUtil.newList(HConstants.EMPTY_END_ROW);
+      }
+
+      hAdmin =  new HBaseAdmin(hconf);
+      Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
+
+      // region startkey -> HBaseFragment
+      Map<byte[], HBaseFragment> fragmentMap = new HashMap<byte[], HBaseFragment>();
+      for (int i = 0; i < keys.getFirst().length; i++) {
+        HRegionLocation location = htable.getRegionLocation(keys.getFirst()[i], false);
+
+        byte[] regionStartKey = keys.getFirst()[i];
+        byte[] regionStopKey = keys.getSecond()[i];
+
+        int startRowsSize = startRows.size();
+        for (int j = 0; j < startRowsSize; j++) {
+          byte[] startRow = startRows.get(j);
+          byte[] stopRow = stopRows.get(j);
+          // determine if the given start an stop key fall into the region
+          if ((startRow.length == 0 || regionStopKey.length == 0 || Bytes.compareTo(startRow, regionStopKey) < 0)
+              && (stopRow.length == 0 || Bytes.compareTo(stopRow, regionStartKey) > 0)) {
+            byte[] fragmentStart = (startRow.length == 0 || Bytes.compareTo(regionStartKey, startRow) >= 0) ?
+                regionStartKey : startRow;
+
+            byte[] fragmentStop = (stopRow.length == 0 || Bytes.compareTo(regionStopKey, stopRow) <= 0) &&
+                regionStopKey.length > 0 ? regionStopKey : stopRow;
+
+            String regionName = location.getRegionInfo().getRegionNameAsString();
+
+            ServerLoad serverLoad = serverLoadMap.get(location.getServerName());
+            if (serverLoad == null) {
+              serverLoad = hAdmin.getClusterStatus().getLoad(location.getServerName());
+              serverLoadMap.put(location.getServerName(), serverLoad);
+            }
+
+            if (fragmentMap.containsKey(regionStartKey)) {
+              HBaseFragment prevFragment = fragmentMap.get(regionStartKey);
+              if (Bytes.compareTo(fragmentStart, prevFragment.getStartRow()) < 0) {
+                prevFragment.setStartRow(fragmentStart);
+              }
+              if (Bytes.compareTo(fragmentStop, prevFragment.getStopRow()) > 0) {
+                prevFragment.setStopRow(fragmentStop);
+              }
+            } else {
+              HBaseFragment fragment = new HBaseFragment(fragmentId, htable.getName().getNameAsString(),
+                  fragmentStart, fragmentStop, location.getHostname());
+
+              // get region size
+              boolean foundLength = false;
+              for (Map.Entry<byte[], RegionLoad> entry : serverLoad.getRegionsLoad().entrySet()) {
+                if (regionName.equals(Bytes.toString(entry.getKey()))) {
+                  RegionLoad regionLoad = entry.getValue();
+                  long storeFileSize = (regionLoad.getStorefileSizeMB() + regionLoad.getMemStoreSizeMB()) * 1024L * 1024L;
+                  fragment.setLength(storeFileSize);
+                  foundLength = true;
+                  break;
+                }
+              }
+
+              if (!foundLength) {
+                fragment.setLength(TajoConstants.UNKNOWN_LENGTH);
+              }
+
+              fragmentMap.put(regionStartKey, fragment);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("getFragments: fragment -> " + i + " -> " + fragment);
+              }
+            }
+          }
+        }
+      }
+
+      List<HBaseFragment> fragments = new ArrayList<HBaseFragment>(fragmentMap.values());
+      Collections.sort(fragments);
+      if (!fragments.isEmpty()) {
+        fragments.get(fragments.size() - 1).setLast(true);
+      }
+      return (ArrayList<Fragment>) (ArrayList) fragments;
+    } finally {
+      if (htable != null) {
+        htable.close();
+      }
+      if (hAdmin != null) {
+        hAdmin.close();
+      }
+    }
+  }
+
+  private byte[] serialize(ColumnMapping columnMapping,
+                           IndexPredication indexPredication, Datum datum) throws IOException {
+    if (columnMapping.getIsBinaryColumns()[indexPredication.getColumnId()]) {
+      return HBaseBinarySerializerDeserializer.serialize(indexPredication.getColumn(), datum);
+    } else {
+      return HBaseTextSerializerDeserializer.serialize(indexPredication.getColumn(), datum);
+    }
+  }
+
+  @Override
+  public Appender getAppender(OverridableConf queryContext,
+                              QueryUnitAttemptId taskAttemptId, TableMeta meta, Schema schema, Path workDir)
+      throws IOException {
+    if ("true".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
+      return new HBasePutAppender(conf, taskAttemptId, schema, meta, workDir);
+    } else {
+      return super.getAppender(queryContext, taskAttemptId, meta, schema, workDir);
+    }
+  }
+
+  @Override
+  public List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments)
+      throws IOException {
+    Configuration hconf = getHBaseConfiguration(conf, tableDesc.getMeta());
+    HTable htable = null;
+    HBaseAdmin hAdmin = null;
+    try {
+      htable = new HTable(hconf, tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY));
+
+      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
+      if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
+        return new ArrayList<Fragment>(1);
+      }
+      hAdmin =  new HBaseAdmin(hconf);
+      Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
+
+      List<Fragment> fragments = new ArrayList<Fragment>(keys.getFirst().length);
+
+      int start = currentPage * numFragments;
+      if (start >= keys.getFirst().length) {
+        return new ArrayList<Fragment>(1);
+      }
+      int end = (currentPage + 1) * numFragments;
+      if (end > keys.getFirst().length) {
+        end = keys.getFirst().length;
+      }
+      for (int i = start; i < end; i++) {
+        HRegionLocation location = htable.getRegionLocation(keys.getFirst()[i], false);
+
+        String regionName = location.getRegionInfo().getRegionNameAsString();
+        ServerLoad serverLoad = serverLoadMap.get(location.getServerName());
+        if (serverLoad == null) {
+          serverLoad = hAdmin.getClusterStatus().getLoad(location.getServerName());
+          serverLoadMap.put(location.getServerName(), serverLoad);
+        }
+
+        HBaseFragment fragment = new HBaseFragment(tableDesc.getName(), htable.getName().getNameAsString(),
+            location.getRegionInfo().getStartKey(), location.getRegionInfo().getEndKey(), location.getHostname());
+
+        // get region size
+        boolean foundLength = false;
+        for (Map.Entry<byte[], RegionLoad> entry : serverLoad.getRegionsLoad().entrySet()) {
+          if (regionName.equals(Bytes.toString(entry.getKey()))) {
+            RegionLoad regionLoad = entry.getValue();
+            long storeLength = (regionLoad.getStorefileSizeMB() + regionLoad.getMemStoreSizeMB()) * 1024L * 1024L;
+            if (storeLength == 0) {
+              // If store size is smaller than 1 MB, storeLength is zero
+              storeLength = 1 * 1024 * 1024;  //default 1MB
+            }
+            fragment.setLength(storeLength);
+            foundLength = true;
+            break;
+          }
+        }
+
+        if (!foundLength) {
+          fragment.setLength(TajoConstants.UNKNOWN_LENGTH);
+        }
+
+        fragments.add(fragment);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("getFragments: fragment -> " + i + " -> " + fragment);
+        }
+      }
+
+      if (!fragments.isEmpty()) {
+        ((HBaseFragment) fragments.get(fragments.size() - 1)).setLast(true);
+      }
+      return fragments;
+    } finally {
+      if (htable != null) {
+        htable.close();
+      }
+      if (hAdmin != null) {
+        hAdmin.close();
+      }
+    }
+  }
+
+  public HConnection getConnection(Configuration hbaseConf) throws IOException {
+    synchronized(connMap) {
+      HConnectionKey key = new HConnectionKey(hbaseConf);
+      HConnection conn = connMap.get(key);
+      if (conn == null) {
+        conn = HConnectionManager.createConnection(hbaseConf);
+        connMap.put(key, conn);
+      }
+
+      return conn;
+    }
+  }
+
+  static class HConnectionKey {
+    final static String[] CONNECTION_PROPERTIES = new String[] {
+        HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT,
+        HConstants.ZOOKEEPER_CLIENT_PORT,
+        HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
+        HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+        HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
+        HConstants.HBASE_META_SCANNER_CACHING,
+        HConstants.HBASE_CLIENT_INSTANCE_ID,
+        HConstants.RPC_CODEC_CONF_KEY };
+
+    private Map<String, String> properties;
+    private String username;
+
+    HConnectionKey(Configuration conf) {
+      Map<String, String> m = new HashMap<String, String>();
+      if (conf != null) {
+        for (String property : CONNECTION_PROPERTIES) {
+          String value = conf.get(property);
+          if (value != null) {
+            m.put(property, value);
+          }
+        }
+      }
+      this.properties = Collections.unmodifiableMap(m);
+
+      try {
+        UserProvider provider = UserProvider.instantiate(conf);
+        User currentUser = provider.getCurrent();
+        if (currentUser != null) {
+          username = currentUser.getName();
+        }
+      } catch (IOException ioe) {
+        LOG.warn("Error obtaining current user, skipping username in HConnectionKey", ioe);
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      if (username != null) {
+        result = username.hashCode();
+      }
+      for (String property : CONNECTION_PROPERTIES) {
+        String value = properties.get(property);
+        if (value != null) {
+          result = prime * result + value.hashCode();
+        }
+      }
+
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      HConnectionKey that = (HConnectionKey) obj;
+      if (this.username != null && !this.username.equals(that.username)) {
+        return false;
+      } else if (this.username == null && that.username != null) {
+        return false;
+      }
+      if (this.properties == null) {
+        if (that.properties != null) {
+          return false;
+        }
+      } else {
+        if (that.properties == null) {
+          return false;
+        }
+        for (String property : CONNECTION_PROPERTIES) {
+          String thisValue = this.properties.get(property);
+          String thatValue = that.properties.get(property);
+          //noinspection StringEquality
+          if (thisValue == thatValue) {
+            continue;
+          }
+          if (thisValue == null || !thisValue.equals(thatValue)) {
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "HConnectionKey{" +
+          "properties=" + properties +
+          ", username='" + username + '\'' +
+          '}';
+    }
+  }
+
+  public List<IndexPredication> getIndexPredications(ColumnMapping columnMapping,
+                                                     TableDesc tableDesc, ScanNode scanNode) throws IOException {
+    List<IndexPredication> indexPredications = new ArrayList<IndexPredication>();
+    Column[] indexableColumns = getIndexableColumns(tableDesc);
+    if (indexableColumns != null && indexableColumns.length == 1) {
+      // Currently supports only single index column.
+      List<Set<EvalNode>> indexablePredicateList = findIndexablePredicateSet(scanNode, indexableColumns);
+      for (Set<EvalNode> eachEvalSet: indexablePredicateList) {
+        Pair<Datum, Datum> indexPredicationValues = getIndexablePredicateValue(columnMapping, eachEvalSet);
+        if (indexPredicationValues != null) {
+          IndexPredication indexPredication = new IndexPredication();
+          indexPredication.setColumn(indexableColumns[0]);
+          indexPredication.setColumnId(tableDesc.getLogicalSchema().getColumnId(indexableColumns[0].getQualifiedName()));
+          indexPredication.setStartValue(indexPredicationValues.getFirst());
+          indexPredication.setStopValue(indexPredicationValues.getSecond());
+
+          indexPredications.add(indexPredication);
+        }
+      }
+    }
+    return indexPredications;
+  }
+
+  public List<Set<EvalNode>> findIndexablePredicateSet(ScanNode scanNode, Column[] indexableColumns) throws IOException {
+    List<Set<EvalNode>> indexablePredicateList = new ArrayList<Set<EvalNode>>();
+
+    // if a query statement has a search condition, try to find indexable predicates
+    if (indexableColumns != null && scanNode.getQual() != null) {
+      EvalNode[] disjunctiveForms = AlgebraicUtil.toDisjunctiveNormalFormArray(scanNode.getQual());
+
+      // add qualifier to schema for qual
+      for (Column column : indexableColumns) {
+        for (EvalNode disjunctiveExpr : disjunctiveForms) {
+          EvalNode[] conjunctiveForms = AlgebraicUtil.toConjunctiveNormalFormArray(disjunctiveExpr);
+          Set<EvalNode> indexablePredicateSet = Sets.newHashSet();
+          for (EvalNode conjunctiveExpr : conjunctiveForms) {
+            if (checkIfIndexablePredicateOnTargetColumn(conjunctiveExpr, column)) {
+              indexablePredicateSet.add(conjunctiveExpr);
+            }
+          }
+          if (!indexablePredicateSet.isEmpty()) {
+            indexablePredicateList.add(indexablePredicateSet);
+          }
+        }
+      }
+    }
+
+    return indexablePredicateList;
+  }
+
+  private boolean checkIfIndexablePredicateOnTargetColumn(EvalNode evalNode, Column targetColumn) {
+    if (checkIfIndexablePredicate(evalNode) || checkIfConjunctiveButOneVariable(evalNode)) {
+      Set<Column> variables = EvalTreeUtil.findUniqueColumns(evalNode);
+      // if it contains only single variable matched to a target column
+      return variables.size() == 1 && variables.contains(targetColumn);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   *
+   * @param evalNode The expression to be checked
+   * @return true if an conjunctive expression, consisting of indexable expressions
+   */
+  private boolean checkIfConjunctiveButOneVariable(EvalNode evalNode) {
+    if (evalNode.getType() == EvalType.AND) {
+      BinaryEval orEval = (BinaryEval) evalNode;
+      boolean indexable =
+          checkIfIndexablePredicate(orEval.getLeftExpr()) &&
+              checkIfIndexablePredicate(orEval.getRightExpr());
+
+      boolean sameVariable =
+          EvalTreeUtil.findUniqueColumns(orEval.getLeftExpr())
+              .equals(EvalTreeUtil.findUniqueColumns(orEval.getRightExpr()));
+
+      return indexable && sameVariable;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Check if an expression consists of one variable and one constant and
+   * the expression is a comparison operator.
+   *
+   * @param evalNode The expression to be checked
+   * @return true if an expression consists of one variable and one constant
+   * and the expression is a comparison operator. Other, false.
+   */
+  private boolean checkIfIndexablePredicate(EvalNode evalNode) {
+    return AlgebraicUtil.containSingleVar(evalNode) && isIndexableOperator(evalNode);
+  }
+
+  public static boolean isIndexableOperator(EvalNode expr) {
+    return expr.getType() == EvalType.EQUAL ||
+        expr.getType() == EvalType.LEQ ||
+        expr.getType() == EvalType.LTH ||
+        expr.getType() == EvalType.GEQ ||
+        expr.getType() == EvalType.GTH ||
+        expr.getType() == EvalType.BETWEEN;
+  }
+
+  public Pair<Datum, Datum> getIndexablePredicateValue(ColumnMapping columnMapping,
+                                                       Set<EvalNode> evalNodes) {
+    Datum startDatum = null;
+    Datum endDatum = null;
+    for (EvalNode evalNode: evalNodes) {
+      if (evalNode instanceof BinaryEval) {
+        BinaryEval binaryEval = (BinaryEval) evalNode;
+        EvalNode left = binaryEval.getLeftExpr();
+        EvalNode right = binaryEval.getRightExpr();
+
+        Datum constValue = null;
+        if (left.getType() == EvalType.CONST) {
+          constValue = ((ConstEval) left).getValue();
+        } else if (right.getType() == EvalType.CONST) {
+          constValue = ((ConstEval) right).getValue();
+        }
+
+        if (constValue != null) {
+          if (evalNode.getType() == EvalType.EQUAL ||
+              evalNode.getType() == EvalType.GEQ ||
+              evalNode.getType() == EvalType.GTH) {
+            if (startDatum != null) {
+              if (constValue.compareTo(startDatum) > 0) {
+                startDatum = constValue;
+              }
+            } else {
+              startDatum = constValue;
+            }
+          }
+
+          if (evalNode.getType() == EvalType.EQUAL ||
+              evalNode.getType() == EvalType.LEQ ||
+              evalNode.getType() == EvalType.LTH) {
+            if (endDatum != null) {
+              if (constValue.compareTo(endDatum) < 0) {
+                endDatum = constValue;
+              }
+            } else {
+              endDatum = constValue;
+            }
+          }
+        }
+      } else if (evalNode instanceof BetweenPredicateEval) {
+        BetweenPredicateEval betweenEval = (BetweenPredicateEval) evalNode;
+        if (betweenEval.getBegin().getType() == EvalType.CONST && betweenEval.getEnd().getType() == EvalType.CONST) {
+          Datum value = ((ConstEval) betweenEval.getBegin()).getValue();
+          if (startDatum != null) {
+            if (value.compareTo(startDatum) > 0) {
+              startDatum = value;
+            }
+          } else {
+            startDatum = value;
+          }
+
+          value = ((ConstEval) betweenEval.getEnd()).getValue();
+          if (endDatum != null) {
+            if (value.compareTo(endDatum) < 0) {
+              endDatum = value;
+            }
+          } else {
+            endDatum = value;
+          }
+        }
+      }
+    }
+
+    if (endDatum != null && columnMapping != null && columnMapping.getNumRowKeys() > 1) {
+      endDatum = new TextDatum(endDatum.asChars() +
+          new String(new char[]{columnMapping.getRowKeyDelimiter(), Character.MAX_VALUE}));
+    }
+    if (startDatum != null || endDatum != null) {
+      return new Pair<Datum, Datum>(startDatum, endDatum);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
+                               LogicalPlan plan, Schema schema,
+                               TableDesc tableDesc) throws IOException {
+    if (tableDesc == null) {
+      throw new IOException("TableDesc is null while calling loadIncrementalHFiles: " + finalEbId);
+    }
+    Path stagingDir = new Path(queryContext.get(QueryVars.STAGING_DIR));
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+
+    Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
+    hbaseConf.set("hbase.loadincremental.threads.max", "2");
+
+    JobContextImpl jobContext = new JobContextImpl(hbaseConf,
+        new JobID(finalEbId.getQueryId().toString(), finalEbId.getId()));
+
+    FileOutputCommitter committer = new FileOutputCommitter(stagingResultDir, jobContext);
+    Path jobAttemptPath = committer.getJobAttemptPath(jobContext);
+    FileSystem fs = jobAttemptPath.getFileSystem(queryContext.getConf());
+    if (!fs.exists(jobAttemptPath) || fs.listStatus(jobAttemptPath) == null) {
+      LOG.warn("No query attempt file in " + jobAttemptPath);
+      return stagingResultDir;
+    }
+    committer.commitJob(jobContext);
+
+    if (tableDesc.getName() == null && tableDesc.getPath() != null) {
+
+      // insert into location
+      return super.commitOutputData(queryContext, finalEbId, plan, schema, tableDesc, false);
+    } else {
+      // insert into table
+      String tableName = tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY);
+
+      HTable htable = new HTable(hbaseConf, tableName);
+      try {
+        LoadIncrementalHFiles loadIncrementalHFiles = null;
+        try {
+          loadIncrementalHFiles = new LoadIncrementalHFiles(hbaseConf);
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+          throw new IOException(e.getMessage(), e);
+        }
+        loadIncrementalHFiles.doBulkLoad(stagingResultDir, htable);
+
+        return stagingResultDir;
+      } finally {
+        htable.close();
+      }
+    }
+  }
+
+  @Override
+  public TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
+                                          Schema inputSchema, SortSpec[] sortSpecs, TupleRange dataRange)
+      throws IOException {
+    try {
+      int[] sortKeyIndexes = new int[sortSpecs.length];
+      for (int i = 0; i < sortSpecs.length; i++) {
+        sortKeyIndexes[i] = inputSchema.getColumnId(sortSpecs[i].getSortKey().getQualifiedName());
+      }
+
+      ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+      Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
+
+      HTable htable = new HTable(hbaseConf, columnMapping.getHbaseTableName());
+      try {
+        byte[][] endKeys = htable.getEndKeys();
+        if (endKeys.length == 1) {
+          return new TupleRange[]{dataRange};
+        }
+        List<TupleRange> tupleRanges = new ArrayList<TupleRange>(endKeys.length);
+
+        TupleComparator comparator = new BaseTupleComparator(inputSchema, sortSpecs);
+        Tuple previousTuple = dataRange.getStart();
+
+        for (byte[] eachEndKey : endKeys) {
+          Tuple endTuple = new VTuple(sortSpecs.length);
+          byte[][] rowKeyFields;
+          if (sortSpecs.length > 1) {
+            byte[][] splitValues = BytesUtils.splitPreserveAllTokens(eachEndKey, columnMapping.getRowKeyDelimiter());
+            if (splitValues.length == sortSpecs.length) {
+              rowKeyFields = splitValues;
+            } else {
+              rowKeyFields = new byte[sortSpecs.length][];
+              for (int j = 0; j < sortSpecs.length; j++) {
+                if (j < splitValues.length) {
+                  rowKeyFields[j] = splitValues[j];
+                } else {
+                  rowKeyFields[j] = null;
+                }
+              }
+            }
+
+          } else {
+            rowKeyFields = new byte[1][];
+            rowKeyFields[0] = eachEndKey;
+          }
+
+          for (int i = 0; i < sortSpecs.length; i++) {
+            if (columnMapping.getIsBinaryColumns()[sortKeyIndexes[i]]) {
+              endTuple.put(i,
+                  HBaseBinarySerializerDeserializer.deserialize(inputSchema.getColumn(sortKeyIndexes[i]),
+                      rowKeyFields[i]));
+            } else {
+              endTuple.put(i,
+                  HBaseTextSerializerDeserializer.deserialize(inputSchema.getColumn(sortKeyIndexes[i]),
+                      rowKeyFields[i]));
+            }
+          }
+          tupleRanges.add(new TupleRange(sortSpecs, previousTuple, endTuple));
+          previousTuple = endTuple;
+        }
+
+        // Last region endkey is empty. Tajo ignores empty key, so endkey is replaced with max data value.
+        if (comparator.compare(dataRange.getEnd(), tupleRanges.get(tupleRanges.size() - 1).getStart()) >= 0) {
+          tupleRanges.get(tupleRanges.size() - 1).setEnd(dataRange.getEnd());
+        } else {
+          tupleRanges.remove(tupleRanges.size() - 1);
+        }
+        return tupleRanges.toArray(new TupleRange[]{});
+      } finally {
+        htable.close();
+      }
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+      throw new IOException(t.getMessage(), t);
+    }
+  }
+
+  public List<RewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
+    if ("false".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
+      List<RewriteRule> rules = new ArrayList<RewriteRule>();
+      rules.add(new AddSortForInsertRewriter(tableDesc, getIndexColumns(tableDesc)));
+      return rules;
+    } else {
+      return null;
+    }
+  }
+
+  private Column[] getIndexColumns(TableDesc tableDesc) throws IOException {
+    List<Column> indexColumns = new ArrayList<Column>();
+
+    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
+
+    boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
+    for (int i = 0; i < isRowKeys.length; i++) {
+      if (isRowKeys[i]) {
+        indexColumns.add(tableDesc.getSchema().getColumn(i));
+      }
+    }
+
+    return indexColumns.toArray(new Column[]{});
+  }
+
+  @Override
+  public StorageProperty getStorageProperty() {
+    StorageProperty storageProperty = new StorageProperty();
+    storageProperty.setSortedInsert(true);
+    storageProperty.setSupportsInsertInto(true);
+    return storageProperty;
+  }
+
+  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
+    if (node.getType() == NodeType.CREATE_TABLE) {
+      CreateTableNode cNode = (CreateTableNode)node;
+      if (!cNode.isExternal()) {
+        TableMeta tableMeta = new TableMeta(cNode.getStorageType(), cNode.getOptions());
+        createTable(tableMeta, cNode.getTableSchema(), cNode.isExternal(), cNode.isIfNotExists());
+      }
+    }
+  }
+
+  @Override
+  public void rollbackOutputCommit(LogicalNode node) throws IOException {
+    if (node.getType() == NodeType.CREATE_TABLE) {
+      CreateTableNode cNode = (CreateTableNode)node;
+      if (cNode.isExternal()) {
+        return;
+      }
+      TableMeta tableMeta = new TableMeta(cNode.getStorageType(), cNode.getOptions());
+      HBaseAdmin hAdmin =  new HBaseAdmin(getHBaseConfiguration(conf, tableMeta));
+
+      try {
+        HTableDescriptor hTableDesc = parseHTableDescriptor(tableMeta, cNode.getTableSchema());
+        LOG.info("Delete table cause query failed:" + hTableDesc.getName());
+        hAdmin.disableTable(hTableDesc.getName());
+        hAdmin.deleteTable(hTableDesc.getName());
+      } finally {
+        hAdmin.close();
+      }
+    }
+  }
+
+  @Override
+  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException  {
+    if (tableDesc != null) {
+      Schema tableSchema = tableDesc.getSchema();
+      if (tableSchema.size() != outSchema.size()) {
+        throw new IOException("The number of table columns is different from SELECT columns");
+      }
+
+      for (int i = 0; i < tableSchema.size(); i++) {
+        if (!tableSchema.getColumn(i).getDataType().equals(outSchema.getColumn(i).getDataType())) {
+          throw new IOException(outSchema.getColumn(i).getQualifiedName() +
+              "(" + outSchema.getColumn(i).getDataType().getType() + ")" +
+              " is different column type with " + tableSchema.getColumn(i).getSimpleName() +
+              "(" + tableSchema.getColumn(i).getDataType().getType() + ")");
+        }
+      }
+    }
+  }
+}


[06/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
new file mode 100644
index 0000000..352776f
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
@@ -0,0 +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.rcfile;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+
+import java.util.ArrayList;
+
+/**
+ * ColumnProjectionUtils.
+ *
+ */
+public final class ColumnProjectionUtils {
+
+  public static final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
+
+  /**
+   * Sets read columns' ids(start from zero) for RCFile's Reader. Once a column
+   * is included in the list, RCFile's reader will not skip its value.
+   *
+   */
+  public static void setReadColumnIDs(Configuration conf, ArrayList<Integer> ids) {
+    String id = toReadColumnIDString(ids);
+    setReadColumnIDConf(conf, id);
+  }
+
+  /**
+   * Sets read columns' ids(start from zero) for RCFile's Reader. Once a column
+   * is included in the list, RCFile's reader will not skip its value.
+   *
+   */
+  public static void appendReadColumnIDs(Configuration conf,
+                                         ArrayList<Integer> ids) {
+    String id = toReadColumnIDString(ids);
+    if (id != null) {
+      String old = conf.get(READ_COLUMN_IDS_CONF_STR, null);
+      String newConfStr = id;
+      if (old != null) {
+        newConfStr = newConfStr + StringUtils.COMMA_STR + old;
+      }
+
+      setReadColumnIDConf(conf, newConfStr);
+    }
+  }
+
+  private static void setReadColumnIDConf(Configuration conf, String id) {
+    if (id == null || id.length() <= 0) {
+      conf.set(READ_COLUMN_IDS_CONF_STR, "");
+      return;
+    }
+
+    conf.set(READ_COLUMN_IDS_CONF_STR, id);
+  }
+
+  private static String toReadColumnIDString(ArrayList<Integer> ids) {
+    String id = null;
+    if (ids != null) {
+      for (int i = 0; i < ids.size(); i++) {
+        if (i == 0) {
+          id = "" + ids.get(i);
+        } else {
+          id = id + StringUtils.COMMA_STR + ids.get(i);
+        }
+      }
+    }
+    return id;
+  }
+
+  /**
+   * Returns an array of column ids(start from zero) which is set in the given
+   * parameter <tt>conf</tt>.
+   */
+  public static ArrayList<Integer> getReadColumnIDs(Configuration conf) {
+    if (conf == null) {
+      return new ArrayList<Integer>(0);
+    }
+    String skips = conf.get(READ_COLUMN_IDS_CONF_STR, "");
+    String[] list = StringUtils.split(skips);
+    ArrayList<Integer> result = new ArrayList<Integer>(list.length);
+    for (String element : list) {
+      // it may contain duplicates, remove duplicates
+      Integer toAdd = Integer.parseInt(element);
+      if (!result.contains(toAdd)) {
+        result.add(toAdd);
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Clears the read column ids set in the conf, and will read all columns.
+   */
+  public static void setFullyReadColumns(Configuration conf) {
+    conf.set(READ_COLUMN_IDS_CONF_STR, "");
+  }
+
+  private ColumnProjectionUtils() {
+    // prevent instantiation
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
new file mode 100644
index 0000000..eab2356
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
@@ -0,0 +1,32 @@
+/**
+ * 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.rcfile;
+
+import java.io.IOException;
+
+/**
+ * Used to call back lazy decompression process.
+ *
+ * @see org.apache.tajo.storage.rcfile.BytesRefWritable
+ */
+public interface LazyDecompressionCallback {
+
+  byte[] decompress() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
new file mode 100644
index 0000000..bb6af22
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
@@ -0,0 +1,113 @@
+/**
+ * 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.rcfile;
+
+import java.io.ByteArrayInputStream;
+
+/**
+ * A thread-not-safe version of ByteArrayInputStream, which removes all
+ * synchronized modifiers.
+ */
+public class NonSyncByteArrayInputStream extends ByteArrayInputStream {
+  public NonSyncByteArrayInputStream() {
+    super(new byte[] {});
+  }
+
+  public NonSyncByteArrayInputStream(byte[] bs) {
+    super(bs);
+  }
+
+  public NonSyncByteArrayInputStream(byte[] buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public void reset(byte[] input, int start, int length) {
+    buf = input;
+    count = start + length;
+    mark = start;
+    pos = start;
+  }
+
+  public int getPosition() {
+    return pos;
+  }
+
+  public int getLength() {
+    return count;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public int read() {
+    return (pos < count) ? (buf[pos++] & 0xff) : -1;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public int read(byte b[], int off, int len) {
+    if (b == null) {
+      throw new NullPointerException();
+    } else if (off < 0 || len < 0 || len > b.length - off) {
+      throw new IndexOutOfBoundsException();
+    }
+    if (pos >= count) {
+      return -1;
+    }
+    if (pos + len > count) {
+      len = count - pos;
+    }
+    if (len <= 0) {
+      return 0;
+    }
+    System.arraycopy(buf, pos, b, off, len);
+    pos += len;
+    return len;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long skip(long n) {
+    if (pos + n > count) {
+      n = count - pos;
+    }
+    if (n < 0) {
+      return 0;
+    }
+    pos += n;
+    return n;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public int available() {
+    return count - pos;
+  }
+
+  public void seek(int pos) {
+    this.pos = pos;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
new file mode 100644
index 0000000..53a3dca
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
@@ -0,0 +1,144 @@
+/**
+ * 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.rcfile;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A thread-not-safe version of ByteArrayOutputStream, which removes all
+ * synchronized modifiers.
+ */
+public class NonSyncByteArrayOutputStream extends ByteArrayOutputStream {
+  public NonSyncByteArrayOutputStream(int size) {
+    super(size);
+  }
+
+  public NonSyncByteArrayOutputStream() {
+    super(64 * 1024);
+  }
+
+  public byte[] getData() {
+    return buf;
+  }
+
+  public int getLength() {
+    return count;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void reset() {
+    count = 0;
+  }
+
+  public void write(DataInput in, int length) throws IOException {
+    enLargeBuffer(length);
+    in.readFully(buf, count, length);
+    count += length;
+  }
+
+  private byte[] vLongBytes = new byte[9];
+
+  public int writeVLongToByteArray(byte[] bytes, int offset, long l) {
+    if (l >= -112 && l <= 127) {
+      bytes[offset] = (byte) l;
+      return 1;
+    }
+
+    int len = -112;
+    if (l < 0) {
+      l ^= -1L; // take one's complement'
+      len = -120;
+    }
+
+    long tmp = l;
+    while (tmp != 0) {
+      tmp = tmp >> 8;
+      len--;
+    }
+
+    bytes[offset++] = (byte) len;
+    len = (len < -120) ? -(len + 120) : -(len + 112);
+
+    for (int idx = len; idx != 0; idx--) {
+      int shiftbits = (idx - 1) * 8;
+      bytes[offset++] = (byte) ((l & (0xFFL << shiftbits)) >> shiftbits);
+    }
+    return 1 + len;
+  }
+
+  public int writeVLong(long l) {
+    int len = writeVLongToByteArray(vLongBytes, 0, l);
+    write(vLongBytes, 0, len);
+    return len;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void write(int b) {
+    enLargeBuffer(1);
+    buf[count] = (byte) b;
+    count += 1;
+  }
+
+  private int enLargeBuffer(int increment) {
+    int temp = count + increment;
+    int newLen = temp;
+    if (temp > buf.length) {
+      if ((buf.length << 1) > temp) {
+        newLen = buf.length << 1;
+      }
+      byte newbuf[] = new byte[newLen];
+      System.arraycopy(buf, 0, newbuf, 0, count);
+      buf = newbuf;
+    }
+    return newLen;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void write(byte b[], int off, int len) {
+    if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length)
+        || ((off + len) < 0)) {
+      throw new IndexOutOfBoundsException();
+    } else if (len == 0) {
+      return;
+    }
+    enLargeBuffer(len);
+    System.arraycopy(b, off, buf, count, len);
+    count += len;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void writeTo(OutputStream out) throws IOException {
+    out.write(buf, 0, count);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
new file mode 100644
index 0000000..46745ab
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
@@ -0,0 +1,507 @@
+/**
+ * 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.rcfile;
+
+import org.apache.hadoop.fs.Seekable;
+
+import java.io.*;
+
+/**
+ * A thread-not-safe version of Hadoop's DataInputBuffer, which removes all
+ * synchronized modifiers.
+ */
+public class NonSyncDataInputBuffer extends FilterInputStream implements
+    DataInput, Seekable {
+
+  private final NonSyncByteArrayInputStream buffer;
+
+  byte[] buff = new byte[16];
+
+  /** Constructs a new empty buffer. */
+  public NonSyncDataInputBuffer() {
+    this(new NonSyncByteArrayInputStream());
+  }
+
+  private NonSyncDataInputBuffer(NonSyncByteArrayInputStream buffer) {
+    super(buffer);
+    this.buffer = buffer;
+  }
+
+  /** Resets the data that the buffer reads. */
+  public void reset(byte[] input, int length) {
+    buffer.reset(input, 0, length);
+  }
+
+  /** Resets the data that the buffer reads. */
+  public void reset(byte[] input, int start, int length) {
+    buffer.reset(input, start, length);
+  }
+
+  /** Returns the current position in the input. */
+  public int getPosition() {
+    return buffer.getPosition();
+  }
+
+  /** Returns the length of the input. */
+  public int getLength() {
+    return buffer.getLength();
+  }
+
+  /**
+   * Reads bytes from the source stream into the byte array <code>buffer</code>.
+   * The number of bytes actually read is returned.
+   *
+   * @param buffer
+   *          the buffer to read bytes into
+   * @return the number of bytes actually read or -1 if end of stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  @Override
+  public final int read(byte[] buffer) throws IOException {
+    return in.read(buffer, 0, buffer.length);
+  }
+
+  /**
+   * Read at most <code>length</code> bytes from this DataInputStream and stores
+   * them in byte array <code>buffer</code> starting at <code>offset</code>.
+   * Answer the number of bytes actually read or -1 if no bytes were read and
+   * end of stream was encountered.
+   *
+   * @param buffer
+   *          the byte array in which to store the read bytes.
+   * @param offset
+   *          the offset in <code>buffer</code> to store the read bytes.
+   * @param length
+   *          the maximum number of bytes to store in <code>buffer</code>.
+   * @return the number of bytes actually read or -1 if end of stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  @Deprecated
+  @Override
+  public final int read(byte[] buffer, int offset, int length)
+      throws IOException {
+    return in.read(buffer, offset, length);
+  }
+
+  /**
+   * Reads a boolean from this stream.
+   *
+   * @return the next boolean value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final boolean readBoolean() throws IOException {
+    int temp = in.read();
+    if (temp < 0) {
+      throw new EOFException();
+    }
+    return temp != 0;
+  }
+
+  /**
+   * Reads an 8-bit byte value from this stream.
+   *
+   * @return the next byte value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final byte readByte() throws IOException {
+    int temp = in.read();
+    if (temp < 0) {
+      throw new EOFException();
+    }
+    return (byte) temp;
+  }
+
+  /**
+   * Reads a 16-bit character value from this stream.
+   *
+   * @return the next <code>char</code> value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  private int readToBuff(int count) throws IOException {
+    int offset = 0;
+
+    while (offset < count) {
+      int bytesRead = in.read(buff, offset, count - offset);
+      if (bytesRead == -1) {
+        return bytesRead;
+      }
+      offset += bytesRead;
+    }
+    return offset;
+  }
+
+  public final char readChar() throws IOException {
+    if (readToBuff(2) < 0) {
+      throw new EOFException();
+    }
+    return (char) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
+
+  }
+
+  /**
+   * Reads a 64-bit <code>double</code> value from this stream.
+   *
+   * @return the next <code>double</code> value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final double readDouble() throws IOException {
+    return Double.longBitsToDouble(readLong());
+  }
+
+  /**
+   * Reads a 32-bit <code>float</code> value from this stream.
+   *
+   * @return the next <code>float</code> value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final float readFloat() throws IOException {
+    return Float.intBitsToFloat(readInt());
+  }
+
+  /**
+   * Reads bytes from this stream into the byte array <code>buffer</code>. This
+   * method will block until <code>buffer.length</code> number of bytes have
+   * been read.
+   *
+   * @param buffer
+   *          to read bytes into
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final void readFully(byte[] buffer) throws IOException {
+    readFully(buffer, 0, buffer.length);
+  }
+
+  /**
+   * Reads bytes from this stream and stores them in the byte array
+   * <code>buffer</code> starting at the position <code>offset</code>. This
+   * method blocks until <code>count</code> bytes have been read.
+   *
+   * @param buffer
+   *          the byte array into which the data is read
+   * @param offset
+   *          the offset the operation start at
+   * @param length
+   *          the maximum number of bytes to read
+   *
+   * @throws java.io.IOException
+   *           if a problem occurs while reading from this stream
+   * @throws java.io.EOFException
+   *           if reaches the end of the stream before enough bytes have been
+   *           read
+   */
+  public final void readFully(byte[] buffer, int offset, int length)
+      throws IOException {
+    if (length < 0) {
+      throw new IndexOutOfBoundsException();
+    }
+    if (length == 0) {
+      return;
+    }
+    if (in == null || buffer == null) {
+      throw new NullPointerException("Null Pointer to underlying input stream");
+    }
+
+    if (offset < 0 || offset > buffer.length - length) {
+      throw new IndexOutOfBoundsException();
+    }
+    while (length > 0) {
+      int result = in.read(buffer, offset, length);
+      if (result < 0) {
+        throw new EOFException();
+      }
+      offset += result;
+      length -= result;
+    }
+  }
+
+  /**
+   * Reads a 32-bit integer value from this stream.
+   *
+   * @return the next <code>int</code> value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final int readInt() throws IOException {
+    if (readToBuff(4) < 0) {
+      throw new EOFException();
+    }
+    return ((buff[0] & 0xff) << 24) | ((buff[1] & 0xff) << 16)
+        | ((buff[2] & 0xff) << 8) | (buff[3] & 0xff);
+  }
+
+  /**
+   * Answers a <code>String</code> representing the next line of text available
+   * in this BufferedReader. A line is represented by 0 or more characters
+   * followed by <code>'\n'</code>, <code>'\r'</code>, <code>"\n\r"</code> or
+   * end of stream. The <code>String</code> does not include the newline
+   * sequence.
+   *
+   * @return the contents of the line or null if no characters were read before
+   *         end of stream.
+   *
+   * @throws java.io.IOException
+   *           If the DataInputStream is already closed or some other IO error
+   *           occurs.
+   *
+   * @deprecated Use BufferedReader
+   */
+  @Deprecated
+  public final String readLine() throws IOException {
+    StringBuilder line = new StringBuilder(80); // Typical line length
+    boolean foundTerminator = false;
+    while (true) {
+      int nextByte = in.read();
+      switch (nextByte) {
+        case -1:
+          if (line.length() == 0 && !foundTerminator) {
+            return null;
+          }
+          return line.toString();
+        case (byte) '\r':
+          if (foundTerminator) {
+            ((PushbackInputStream) in).unread(nextByte);
+            return line.toString();
+          }
+          foundTerminator = true;
+        /* Have to be able to peek ahead one byte */
+          if (!(in.getClass() == PushbackInputStream.class)) {
+            in = new PushbackInputStream(in);
+          }
+          break;
+        case (byte) '\n':
+          return line.toString();
+        default:
+          if (foundTerminator) {
+            ((PushbackInputStream) in).unread(nextByte);
+            return line.toString();
+          }
+          line.append((char) nextByte);
+      }
+    }
+  }
+
+  /**
+   * Reads a 64-bit <code>long</code> value from this stream.
+   *
+   * @return the next <code>long</code> value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final long readLong() throws IOException {
+    if (readToBuff(8) < 0) {
+      throw new EOFException();
+    }
+    int i1 = ((buff[0] & 0xff) << 24) | ((buff[1] & 0xff) << 16)
+        | ((buff[2] & 0xff) << 8) | (buff[3] & 0xff);
+    int i2 = ((buff[4] & 0xff) << 24) | ((buff[5] & 0xff) << 16)
+        | ((buff[6] & 0xff) << 8) | (buff[7] & 0xff);
+
+    return ((i1 & 0xffffffffL) << 32) | (i2 & 0xffffffffL);
+  }
+
+  /**
+   * Reads a 16-bit <code>short</code> value from this stream.
+   *
+   * @return the next <code>short</code> value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final short readShort() throws IOException {
+    if (readToBuff(2) < 0) {
+      throw new EOFException();
+    }
+    return (short) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
+  }
+
+  /**
+   * Reads an unsigned 8-bit <code>byte</code> value from this stream and
+   * returns it as an int.
+   *
+   * @return the next unsigned byte value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final int readUnsignedByte() throws IOException {
+    int temp = in.read();
+    if (temp < 0) {
+      throw new EOFException();
+    }
+    return temp;
+  }
+
+  /**
+   * Reads a 16-bit unsigned <code>short</code> value from this stream and
+   * returns it as an int.
+   *
+   * @return the next unsigned <code>short</code> value from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final int readUnsignedShort() throws IOException {
+    if (readToBuff(2) < 0) {
+      throw new EOFException();
+    }
+    return (char) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
+  }
+
+  /**
+   * Reads a UTF format String from this Stream.
+   *
+   * @return the next UTF String from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public final String readUTF() throws IOException {
+    return decodeUTF(readUnsignedShort());
+  }
+
+  String decodeUTF(int utfSize) throws IOException {
+    return decodeUTF(utfSize, this);
+  }
+
+  private static String decodeUTF(int utfSize, DataInput in) throws IOException {
+    byte[] buf = new byte[utfSize];
+    char[] out = new char[utfSize];
+    in.readFully(buf, 0, utfSize);
+
+    return convertUTF8WithBuf(buf, out, 0, utfSize);
+  }
+
+  /**
+   * Reads a UTF format String from the DataInput Stream <code>in</code>.
+   *
+   * @param in
+   *          the input stream to read from
+   * @return the next UTF String from the source stream.
+   *
+   * @throws java.io.IOException
+   *           If a problem occurs reading from this DataInputStream.
+   *
+   */
+  public static final String readUTF(DataInput in) throws IOException {
+    return decodeUTF(in.readUnsignedShort(), in);
+  }
+
+  /**
+   * Skips <code>count</code> number of bytes in this stream. Subsequent
+   * <code>read()</code>'s will not return these bytes unless
+   * <code>reset()</code> is used.
+   *
+   * @param count
+   *          the number of bytes to skip.
+   * @return the number of bytes actually skipped.
+   *
+   * @throws java.io.IOException
+   *           If the stream is already closed or another IOException occurs.
+   */
+  public final int skipBytes(int count) throws IOException {
+    int skipped = 0;
+    long skip;
+    while (skipped < count && (skip = in.skip(count - skipped)) != 0) {
+      skipped += skip;
+    }
+    if (skipped < 0) {
+      throw new EOFException();
+    }
+    return skipped;
+  }
+
+  public static String convertUTF8WithBuf(byte[] buf, char[] out, int offset,
+                                          int utfSize) throws UTFDataFormatException {
+    int count = 0, s = 0, a;
+    while (count < utfSize) {
+      if ((out[s] = (char) buf[offset + count++]) < '\u0080') {
+        s++;
+      } else if (((a = out[s]) & 0xe0) == 0xc0) {
+        if (count >= utfSize) {
+          throw new UTFDataFormatException();
+        }
+        int b = buf[count++];
+        if ((b & 0xC0) != 0x80) {
+          throw new UTFDataFormatException();
+        }
+        out[s++] = (char) (((a & 0x1F) << 6) | (b & 0x3F));
+      } else if ((a & 0xf0) == 0xe0) {
+        if (count + 1 >= utfSize) {
+          throw new UTFDataFormatException();
+        }
+        int b = buf[count++];
+        int c = buf[count++];
+        if (((b & 0xC0) != 0x80) || ((c & 0xC0) != 0x80)) {
+          throw new UTFDataFormatException();
+        }
+        out[s++] = (char) (((a & 0x0F) << 12) | ((b & 0x3F) << 6) | (c & 0x3F));
+      } else {
+        throw new UTFDataFormatException();
+      }
+    }
+    return new String(out, 0, s);
+  }
+
+  @Override
+  public void seek(long pos) throws IOException {
+    buffer.seek((int)pos);
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return buffer.getPosition();
+  }
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
new file mode 100644
index 0000000..3944f38
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
@@ -0,0 +1,91 @@
+/**
+ * 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.rcfile;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+/**
+ * A thread-not-safe version of Hadoop's DataOutputBuffer, which removes all
+ * synchronized modifiers.
+ */
+public class NonSyncDataOutputBuffer extends DataOutputStream {
+
+  private final NonSyncByteArrayOutputStream buffer;
+
+  /** Constructs a new empty buffer. */
+  public NonSyncDataOutputBuffer() {
+    this(new NonSyncByteArrayOutputStream());
+  }
+
+  private NonSyncDataOutputBuffer(NonSyncByteArrayOutputStream buffer) {
+    super(buffer);
+    this.buffer = buffer;
+  }
+
+  /**
+   * Returns the current contents of the buffer. Data is only valid to
+   * {@link #getLength()}.
+   */
+  public byte[] getData() {
+    return buffer.getData();
+  }
+
+  /** Returns the length of the valid data currently in the buffer. */
+  public int getLength() {
+    return buffer.getLength();
+  }
+
+  /** Resets the buffer to empty. */
+  public NonSyncDataOutputBuffer reset() {
+    written = 0;
+    buffer.reset();
+    return this;
+  }
+
+  /** Writes bytes from a DataInput directly into the buffer. */
+  public void write(DataInput in, int length) throws IOException {
+    buffer.write(in, length);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    buffer.write(b);
+    incCount(1);
+  }
+
+  @Override
+  public void write(byte b[], int off, int len) throws IOException {
+    buffer.write(b, off, len);
+    incCount(len);
+  }
+
+  public void writeTo(DataOutputStream out) throws IOException {
+    buffer.writeTo(out);
+  }
+
+  private void incCount(int value) {
+    if (written + value < 0) {
+      written = Integer.MAX_VALUE;
+    } else {
+      written += value;
+    }
+  }
+}


[24/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
deleted file mode 100644
index d143e58..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java
+++ /dev/null
@@ -1,445 +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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.*;
-import org.apache.hadoop.hbase.filter.*;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.ColumnStats;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.storage.Scanner;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.BytesUtils;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class HBaseScanner implements Scanner {
-  private static final Log LOG = LogFactory.getLog(HBaseScanner.class);
-  private static final int DEFAULT_FETCH_SIZE = 1000;
-  private static final int MAX_LIST_SIZE = 100;
-
-  protected boolean inited = false;
-  private TajoConf conf;
-  private Schema schema;
-  private TableMeta meta;
-  private HBaseFragment fragment;
-  private Scan scan;
-  private HTableInterface htable;
-  private Configuration hbaseConf;
-  private Column[] targets;
-  private TableStats tableStats;
-  private ResultScanner scanner;
-  private AtomicBoolean finished = new AtomicBoolean(false);
-  private float progress = 0.0f;
-  private int scanFetchSize;
-  private Result[] scanResults;
-  private int scanResultIndex = -1;
-  private Column[] schemaColumns;
-
-  private ColumnMapping columnMapping;
-  private int[] targetIndexes;
-
-  private int numRows = 0;
-
-  private byte[][][] mappingColumnFamilies;
-  private boolean[] isRowKeyMappings;
-  private boolean[] isBinaryColumns;
-  private boolean[] isColumnKeys;
-  private boolean[] isColumnValues;
-
-  private int[] rowKeyFieldIndexes;
-  private char rowKeyDelimiter;
-
-  public HBaseScanner (Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
-    this.conf = (TajoConf)conf;
-    this.schema = schema;
-    this.meta = meta;
-    this.fragment = (HBaseFragment)fragment;
-    this.tableStats = new TableStats();
-  }
-
-  @Override
-  public void init() throws IOException {
-    inited = true;
-    schemaColumns = schema.toArray();
-    if (fragment != null) {
-      tableStats.setNumBytes(0);
-      tableStats.setNumBlocks(1);
-    }
-    if (schema != null) {
-      for(Column eachColumn: schema.getColumns()) {
-        ColumnStats columnStats = new ColumnStats(eachColumn);
-        tableStats.addColumnStat(columnStats);
-      }
-    }
-
-    scanFetchSize = Integer.parseInt(meta.getOption(HBaseStorageConstants.META_FETCH_ROWNUM_KEY, "" + DEFAULT_FETCH_SIZE));
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-
-    columnMapping = new ColumnMapping(schema, meta);
-    targetIndexes = new int[targets.length];
-    int index = 0;
-    for (Column eachTargetColumn: targets) {
-      targetIndexes[index++] = schema.getColumnId(eachTargetColumn.getQualifiedName());
-    }
-
-    mappingColumnFamilies = columnMapping.getMappingColumns();
-    isRowKeyMappings = columnMapping.getIsRowKeyMappings();
-    isBinaryColumns = columnMapping.getIsBinaryColumns();
-    isColumnKeys = columnMapping.getIsColumnKeys();
-    isColumnValues = columnMapping.getIsColumnValues();
-
-    rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
-    rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
-
-    hbaseConf = HBaseStorageManager.getHBaseConfiguration(conf, meta);
-
-    initScanner();
-  }
-
-  private void initScanner() throws IOException {
-    scan = new Scan();
-    scan.setBatch(scanFetchSize);
-    scan.setCacheBlocks(false);
-    scan.setCaching(scanFetchSize);
-
-    FilterList filters = null;
-    if (targetIndexes == null || targetIndexes.length == 0) {
-      filters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
-      filters.addFilter(new FirstKeyOnlyFilter());
-      filters.addFilter(new KeyOnlyFilter());
-    } else {
-      boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
-      for (int eachIndex : targetIndexes) {
-        if (isRowKeyMappings[eachIndex]) {
-          continue;
-        }
-        byte[][] mappingColumn = columnMapping.getMappingColumns()[eachIndex];
-        if (mappingColumn[1] == null) {
-          scan.addFamily(mappingColumn[0]);
-        } else {
-          scan.addColumn(mappingColumn[0], mappingColumn[1]);
-        }
-      }
-    }
-
-    scan.setStartRow(fragment.getStartRow());
-    if (fragment.isLast() && fragment.getStopRow() != null &&
-        fragment.getStopRow().length > 0) {
-      // last and stopRow is not empty
-      if (filters == null) {
-        filters = new FilterList();
-      }
-      filters.addFilter(new InclusiveStopFilter(fragment.getStopRow()));
-    } else {
-      scan.setStopRow(fragment.getStopRow());
-    }
-
-    if (filters != null) {
-      scan.setFilter(filters);
-    }
-
-    if (htable == null) {
-      HConnection hconn = ((HBaseStorageManager)StorageManager.getStorageManager(conf, StoreType.HBASE))
-          .getConnection(hbaseConf);
-      htable = hconn.getTable(fragment.getHbaseTableName());
-    }
-    scanner = htable.getScanner(scan);
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    if (finished.get()) {
-      return null;
-    }
-
-    if (scanResults == null || scanResultIndex >= scanResults.length) {
-      scanResults = scanner.next(scanFetchSize);
-      if (scanResults == null || scanResults.length == 0) {
-        finished.set(true);
-        progress = 1.0f;
-        return null;
-      }
-      scanResultIndex = 0;
-    }
-
-    Result result = scanResults[scanResultIndex++];
-    Tuple resultTuple = new VTuple(schema.size());
-    for (int i = 0; i < targetIndexes.length; i++) {
-      resultTuple.put(targetIndexes[i], getDatum(result, targetIndexes[i]));
-    }
-    numRows++;
-    return resultTuple;
-  }
-
-  private Datum getDatum(Result result, int fieldId) throws IOException {
-    byte[] value = null;
-    if (isRowKeyMappings[fieldId]) {
-      value = result.getRow();
-      if (!isBinaryColumns[fieldId] && rowKeyFieldIndexes[fieldId] >= 0) {
-        int rowKeyFieldIndex = rowKeyFieldIndexes[fieldId];
-
-        byte[][] rowKeyFields = BytesUtils.splitPreserveAllTokens(value, rowKeyDelimiter);
-
-        if (rowKeyFields.length < rowKeyFieldIndex) {
-          return NullDatum.get();
-        } else {
-          value = rowKeyFields[rowKeyFieldIndex];
-        }
-      }
-    } else {
-      if (isColumnKeys[fieldId]) {
-        NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
-        if (cfMap != null) {
-          Set<byte[]> keySet = cfMap.keySet();
-          if (keySet.size() == 1) {
-            try {
-              return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], keySet.iterator().next());
-            } catch (Exception e) {
-              LOG.error(e.getMessage(), e);
-              throw new RuntimeException(e.getMessage(), e);
-            }
-          } else {
-            StringBuilder sb = new StringBuilder();
-            sb.append("[");
-            int count = 0;
-            for (byte[] eachKey : keySet) {
-              if (count > 0) {
-                sb.append(", ");
-              }
-              Datum datum = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], eachKey);
-              sb.append("\"").append(datum.asChars()).append("\"");
-              count++;
-              if (count > MAX_LIST_SIZE) {
-                break;
-              }
-            }
-            sb.append("]");
-            return new TextDatum(sb.toString());
-          }
-        }
-      } else if (isColumnValues[fieldId]) {
-        NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
-        if (cfMap != null) {
-          Collection<byte[]> valueList = cfMap.values();
-          if (valueList.size() == 1) {
-            try {
-              return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], valueList.iterator().next());
-            } catch (Exception e) {
-              LOG.error(e.getMessage(), e);
-              throw new RuntimeException(e.getMessage(), e);
-            }
-          } else {
-            StringBuilder sb = new StringBuilder();
-            sb.append("[");
-            int count = 0;
-            for (byte[] eachValue : valueList) {
-              if (count > 0) {
-                sb.append(", ");
-              }
-              Datum datum = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], eachValue);
-              sb.append("\"").append(datum.asChars()).append("\"");
-              count++;
-              if (count > MAX_LIST_SIZE) {
-                break;
-              }
-            }
-            sb.append("]");
-            return new TextDatum(sb.toString());
-          }
-        }
-      } else {
-        if (mappingColumnFamilies[fieldId][1] == null) {
-          NavigableMap<byte[], byte[]> cfMap = result.getFamilyMap(mappingColumnFamilies[fieldId][0]);
-          if (cfMap != null && !cfMap.isEmpty()) {
-            int count = 0;
-            String delim = "";
-
-            if (cfMap.size() == 0) {
-              return NullDatum.get();
-            } else if (cfMap.size() == 1) {
-              // If a column family is mapped without column name like "cf1:" and the number of cells is one,
-              // return value is flat format not json format.
-              NavigableMap.Entry<byte[], byte[]> entry = cfMap.entrySet().iterator().next();
-              byte[] entryKey = entry.getKey();
-              byte[] entryValue = entry.getValue();
-              if (entryKey == null || entryKey.length == 0) {
-                try {
-                  if (isBinaryColumns[fieldId]) {
-                    return HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue);
-                  } else {
-                    return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue);
-                  }
-                } catch (Exception e) {
-                  LOG.error(e.getMessage(), e);
-                  throw new RuntimeException(e.getMessage(), e);
-                }
-              }
-            }
-            StringBuilder sb = new StringBuilder();
-            sb.append("{");
-            for (NavigableMap.Entry<byte[], byte[]> entry : cfMap.entrySet()) {
-              byte[] entryKey = entry.getKey();
-              byte[] entryValue = entry.getValue();
-
-              String keyText = new String(entryKey);
-              String valueText = null;
-              if (entryValue != null) {
-                try {
-                  if (isBinaryColumns[fieldId]) {
-                    valueText = HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue).asChars();
-                  } else {
-                    valueText = HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], entryValue).asChars();
-                  }
-                } catch (Exception e) {
-                  LOG.error(e.getMessage(), e);
-                  throw new RuntimeException(e.getMessage(), e);
-                }
-              }
-              sb.append(delim).append("\"").append(keyText).append("\":\"").append(valueText).append("\"");
-              delim = ", ";
-              count++;
-              if (count > MAX_LIST_SIZE) {
-                break;
-              }
-            } //end of for
-            sb.append("}");
-            return new TextDatum(sb.toString());
-          } else {
-            value = null;
-          }
-        } else {
-          value = result.getValue(mappingColumnFamilies[fieldId][0], mappingColumnFamilies[fieldId][1]);
-        }
-      }
-    }
-
-    if (value == null) {
-      return NullDatum.get();
-    } else {
-      try {
-        if (isBinaryColumns[fieldId]) {
-          return HBaseBinarySerializerDeserializer.deserialize(schemaColumns[fieldId], value);
-        } else {
-          return HBaseTextSerializerDeserializer.deserialize(schemaColumns[fieldId], value);
-        }
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
-        throw new RuntimeException(e.getMessage(), e);
-      }
-    }
-  }
-
-  @Override
-  public void reset() throws IOException {
-    progress = 0.0f;
-    scanResultIndex = -1;
-    scanResults = null;
-    finished.set(false);
-    tableStats = new TableStats();
-
-    if (scanner != null) {
-      scanner.close();
-      scanner = null;
-    }
-
-    initScanner();
-  }
-
-  @Override
-  public void close() throws IOException {
-    progress = 1.0f;
-    finished.set(true);
-    if (scanner != null) {
-      try {
-        scanner.close();
-        scanner = null;
-      } catch (Exception e) {
-        LOG.warn("Error while closing hbase scanner: " + e.getMessage(), e);
-      }
-    }
-    if (htable != null) {
-      htable.close();
-      htable = null;
-    }
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return true;
-  }
-
-  @Override
-  public void setTarget(Column[] targets) {
-    if (inited) {
-      throw new IllegalStateException("Should be called before init()");
-    }
-    this.targets = targets;
-  }
-
-  @Override
-  public boolean isSelectable() {
-    return false;
-  }
-
-  @Override
-  public void setSearchCondition(Object expr) {
-    // TODO implements adding column filter to scanner.
-  }
-
-  @Override
-  public boolean isSplittable() {
-    return true;
-  }
-
-  @Override
-  public float getProgress() {
-    return progress;
-  }
-
-  @Override
-  public TableStats getInputStats() {
-    tableStats.setNumRows(numRows);
-    return tableStats;
-  }
-
-  @Override
-  public Schema getSchema() {
-    return schema;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
deleted file mode 100644
index 2c525a1..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageConstants.java
+++ /dev/null
@@ -1,33 +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;
-
-public interface HBaseStorageConstants {
-  public static final String KEY_COLUMN_MAPPING = "key";
-  public static final String VALUE_COLUMN_MAPPING = "value";
-  public static final String META_FETCH_ROWNUM_KEY = "fetch.rownum";
-  public static final String META_TABLE_KEY = "table";
-  public static final String META_COLUMNS_KEY = "columns";
-  public static final String META_SPLIT_ROW_KEYS_KEY = "hbase.split.rowkeys";
-  public static final String META_SPLIT_ROW_KEYS_FILE_KEY = "hbase.split.rowkeys.file";
-  public static final String META_ZK_QUORUM_KEY = "hbase.zookeeper.quorum";
-  public static final String META_ROWKEY_DELIMITER = "hbase.rowkey.delimiter";
-
-  public static final String INSERT_PUT_MODE = "tajo.hbase.insert.put.mode";
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
deleted file mode 100644
index b47b98c..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseStorageManager.java
+++ /dev/null
@@ -1,1126 +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 com.google.common.collect.Sets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.client.*;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.expr.*;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.plan.rewrite.RewriteRule;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.*;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.*;
-
-/**
- * StorageManager for HBase table.
- */
-public class HBaseStorageManager extends StorageManager {
-  private static final Log LOG = LogFactory.getLog(HBaseStorageManager.class);
-
-  private Map<HConnectionKey, HConnection> connMap = new HashMap<HConnectionKey, HConnection>();
-
-  public HBaseStorageManager (StoreType storeType) {
-    super(storeType);
-  }
-
-  @Override
-  public void storageInit() throws IOException {
-  }
-
-  @Override
-  public void closeStorageManager() {
-    synchronized (connMap) {
-      for (HConnection eachConn: connMap.values()) {
-        try {
-          eachConn.close();
-        } catch (Exception e) {
-          LOG.error(e.getMessage(), e);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
-    createTable(tableDesc.getMeta(), tableDesc.getSchema(), tableDesc.isExternal(), ifNotExists);
-    TableStats stats = new TableStats();
-    stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
-    tableDesc.setStats(stats);
-  }
-
-  private void createTable(TableMeta tableMeta, Schema schema,
-                           boolean isExternal, boolean ifNotExists) throws IOException {
-    String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
-    if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
-      throw new IOException("HBase mapped table is required a '" +
-          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
-    }
-    TableName hTableName = TableName.valueOf(hbaseTableName);
-
-    String mappedColumns = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
-    if (mappedColumns != null && mappedColumns.split(",").length > schema.size()) {
-      throw new IOException("Columns property has more entry than Tajo table columns");
-    }
-
-    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
-    int numRowKeys = 0;
-    boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
-    for (int i = 0; i < isRowKeyMappings.length; i++) {
-      if (isRowKeyMappings[i]) {
-        numRowKeys++;
-      }
-    }
-    if (numRowKeys > 1) {
-      for (int i = 0; i < isRowKeyMappings.length; i++) {
-        if (isRowKeyMappings[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
-          throw new IOException("Key field type should be TEXT type.");
-        }
-      }
-    }
-
-    for (int i = 0; i < isRowKeyMappings.length; i++) {
-      if (columnMapping.getIsColumnKeys()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
-        throw new IOException("Column key field('<cfname>:key:') type should be TEXT type.");
-      }
-      if (columnMapping.getIsColumnValues()[i] && schema.getColumn(i).getDataType().getType() != Type.TEXT) {
-        throw new IOException("Column value field(('<cfname>:value:') type should be TEXT type.");
-      }
-    }
-
-    Configuration hConf = getHBaseConfiguration(conf, tableMeta);
-    HBaseAdmin hAdmin =  new HBaseAdmin(hConf);
-
-    try {
-      if (isExternal) {
-        // If tajo table is external table, only check validation.
-        if (mappedColumns == null || mappedColumns.isEmpty()) {
-          throw new IOException("HBase mapped table is required a '" +
-              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
-        }
-        if (!hAdmin.tableExists(hTableName)) {
-          throw new IOException("HBase table [" + hbaseTableName + "] not exists. " +
-              "External table should be a existed table.");
-        }
-        HTableDescriptor hTableDescriptor = hAdmin.getTableDescriptor(hTableName);
-        Set<String> tableColumnFamilies = new HashSet<String>();
-        for (HColumnDescriptor eachColumn : hTableDescriptor.getColumnFamilies()) {
-          tableColumnFamilies.add(eachColumn.getNameAsString());
-        }
-
-        Collection<String> mappingColumnFamilies =columnMapping.getColumnFamilyNames();
-        if (mappingColumnFamilies.isEmpty()) {
-          throw new IOException("HBase mapped table is required a '" +
-              HBaseStorageConstants.META_COLUMNS_KEY + "' attribute.");
-        }
-
-        for (String eachMappingColumnFamily : mappingColumnFamilies) {
-          if (!tableColumnFamilies.contains(eachMappingColumnFamily)) {
-            throw new IOException("There is no " + eachMappingColumnFamily + " column family in " + hbaseTableName);
-          }
-        }
-      } else {
-        if (hAdmin.tableExists(hbaseTableName)) {
-          if (ifNotExists) {
-            return;
-          } else {
-            throw new IOException("HBase table [" + hbaseTableName + "] already exists.");
-          }
-        }
-        // Creating hbase table
-        HTableDescriptor hTableDescriptor = parseHTableDescriptor(tableMeta, schema);
-
-        byte[][] splitKeys = getSplitKeys(conf, schema, tableMeta);
-        if (splitKeys == null) {
-          hAdmin.createTable(hTableDescriptor);
-        } else {
-          hAdmin.createTable(hTableDescriptor, splitKeys);
-        }
-      }
-    } finally {
-      hAdmin.close();
-    }
-  }
-
-  /**
-   * Returns initial region split keys.
-   *
-   * @param conf
-   * @param schema
-   * @param meta
-   * @return
-   * @throws IOException
-   */
-  private byte[][] getSplitKeys(TajoConf conf, Schema schema, TableMeta meta) throws IOException {
-    String splitRowKeys = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_KEY, "");
-    String splitRowKeysFile = meta.getOption(HBaseStorageConstants.META_SPLIT_ROW_KEYS_FILE_KEY, "");
-
-    if ((splitRowKeys == null || splitRowKeys.isEmpty()) &&
-        (splitRowKeysFile == null || splitRowKeysFile.isEmpty())) {
-      return null;
-    }
-
-    ColumnMapping columnMapping = new ColumnMapping(schema, meta);
-    boolean[] isBinaryColumns = columnMapping.getIsBinaryColumns();
-    boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
-
-    boolean rowkeyBinary = false;
-    int numRowKeys = 0;
-    Column rowKeyColumn = null;
-    for (int i = 0; i < isBinaryColumns.length; i++) {
-      if (isBinaryColumns[i] && isRowKeys[i]) {
-        rowkeyBinary = true;
-      }
-      if (isRowKeys[i]) {
-        numRowKeys++;
-        rowKeyColumn = schema.getColumn(i);
-      }
-    }
-
-    if (rowkeyBinary && numRowKeys > 1) {
-      throw new IOException("If rowkey is mapped to multi column and a rowkey is binary, " +
-          "Multiple region for creation is not support.");
-    }
-
-    if (splitRowKeys != null && !splitRowKeys.isEmpty()) {
-      String[] splitKeyTokens = splitRowKeys.split(",");
-      byte[][] splitKeys = new byte[splitKeyTokens.length][];
-      for (int i = 0; i < splitKeyTokens.length; i++) {
-        if (numRowKeys == 1 && rowkeyBinary) {
-          splitKeys[i] = HBaseBinarySerializerDeserializer.serialize(rowKeyColumn, new TextDatum(splitKeyTokens[i]));
-        } else {
-          splitKeys[i] = HBaseTextSerializerDeserializer.serialize(rowKeyColumn, new TextDatum(splitKeyTokens[i]));
-        }
-      }
-      return splitKeys;
-    }
-
-    if (splitRowKeysFile != null && !splitRowKeysFile.isEmpty()) {
-      // If there is many split keys, Tajo allows to define in the file.
-      Path path = new Path(splitRowKeysFile);
-      FileSystem fs = path.getFileSystem(conf);
-      if (!fs.exists(path)) {
-        throw new IOException("hbase.split.rowkeys.file=" + path.toString() + " not exists.");
-      }
-
-      SortedSet<String> splitKeySet = new TreeSet<String>();
-      BufferedReader reader = null;
-      try {
-        reader = new BufferedReader(new InputStreamReader(fs.open(path)));
-        String line = null;
-        while ( (line = reader.readLine()) != null ) {
-          if (line.isEmpty()) {
-            continue;
-          }
-          splitKeySet.add(line);
-        }
-      } finally {
-        if (reader != null) {
-          reader.close();
-        }
-      }
-
-      if (splitKeySet.isEmpty()) {
-        return null;
-      }
-
-      byte[][] splitKeys = new byte[splitKeySet.size()][];
-      int index = 0;
-      for (String eachKey: splitKeySet) {
-        if (numRowKeys == 1 && rowkeyBinary) {
-          splitKeys[index++] = HBaseBinarySerializerDeserializer.serialize(rowKeyColumn, new TextDatum(eachKey));
-        } else {
-          splitKeys[index++] = HBaseTextSerializerDeserializer.serialize(rowKeyColumn, new TextDatum(eachKey));
-        }
-      }
-
-      return splitKeys;
-    }
-
-    return null;
-  }
-
-  /**
-   * Creates Configuration instance and sets with hbase connection options.
-   *
-   * @param conf
-   * @param tableMeta
-   * @return
-   * @throws IOException
-   */
-  public static Configuration getHBaseConfiguration(Configuration conf, TableMeta tableMeta) throws IOException {
-    String zkQuorum = tableMeta.getOption(HBaseStorageConstants.META_ZK_QUORUM_KEY, "");
-    if (zkQuorum == null || zkQuorum.trim().isEmpty()) {
-      throw new IOException("HBase mapped table is required a '" +
-          HBaseStorageConstants.META_ZK_QUORUM_KEY + "' attribute.");
-    }
-
-    Configuration hbaseConf = (conf == null) ? HBaseConfiguration.create() : HBaseConfiguration.create(conf);
-    hbaseConf.set(HConstants.ZOOKEEPER_QUORUM, zkQuorum);
-
-    for (Map.Entry<String, String> eachOption: tableMeta.getOptions().getAllKeyValus().entrySet()) {
-      String key = eachOption.getKey();
-      if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) {
-        hbaseConf.set(key, eachOption.getValue());
-      }
-    }
-    return hbaseConf;
-  }
-
-  /**
-   * Creates HTableDescription using table meta data.
-   *
-   * @param tableMeta
-   * @param schema
-   * @return
-   * @throws IOException
-   */
-  public static HTableDescriptor parseHTableDescriptor(TableMeta tableMeta, Schema schema) throws IOException {
-    String hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY, "");
-    if (hbaseTableName == null || hbaseTableName.trim().isEmpty()) {
-      throw new IOException("HBase mapped table is required a '" +
-          HBaseStorageConstants.META_TABLE_KEY + "' attribute.");
-    }
-    TableName hTableName = TableName.valueOf(hbaseTableName);
-
-    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
-
-    HTableDescriptor hTableDescriptor = new HTableDescriptor(hTableName);
-
-    Collection<String> columnFamilies = columnMapping.getColumnFamilyNames();
-    //If 'columns' attribute is empty, Tajo table columns are mapped to all HBase table column.
-    if (columnFamilies.isEmpty()) {
-      for (Column eachColumn: schema.getColumns()) {
-        columnFamilies.add(eachColumn.getSimpleName());
-      }
-    }
-
-    for (String eachColumnFamily: columnFamilies) {
-      hTableDescriptor.addFamily(new HColumnDescriptor(eachColumnFamily));
-    }
-
-    return hTableDescriptor;
-  }
-
-  @Override
-  public void purgeTable(TableDesc tableDesc) throws IOException {
-    HBaseAdmin hAdmin =  new HBaseAdmin(getHBaseConfiguration(conf, tableDesc.getMeta()));
-
-    try {
-      HTableDescriptor hTableDesc = parseHTableDescriptor(tableDesc.getMeta(), tableDesc.getSchema());
-      LOG.info("Deleting hbase table: " + new String(hTableDesc.getName()));
-      hAdmin.disableTable(hTableDesc.getName());
-      hAdmin.deleteTable(hTableDesc.getName());
-    } finally {
-      hAdmin.close();
-    }
-  }
-
-  /**
-   * Returns columns which are mapped to the rowkey of the hbase table.
-   *
-   * @param tableDesc
-   * @return
-   * @throws IOException
-   */
-  private Column[] getIndexableColumns(TableDesc tableDesc) throws IOException {
-    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
-    boolean[] isRowKeyMappings = columnMapping.getIsRowKeyMappings();
-    int[] rowKeyIndexes = columnMapping.getRowKeyFieldIndexes();
-
-    Column indexColumn = null;
-    for (int i = 0; i < isRowKeyMappings.length; i++) {
-      if (isRowKeyMappings[i]) {
-        if (columnMapping.getNumRowKeys() == 1 ||
-            rowKeyIndexes[i] == 0) {
-          indexColumn = tableDesc.getSchema().getColumn(i);
-        }
-      }
-    }
-    return new Column[]{indexColumn};
-  }
-
-  @Override
-  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc, ScanNode scanNode) throws IOException {
-    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
-
-    List<IndexPredication> indexPredications = getIndexPredications(columnMapping, tableDesc, scanNode);
-    Configuration hconf = getHBaseConfiguration(conf, tableDesc.getMeta());
-    HTable htable = null;
-    HBaseAdmin hAdmin = null;
-
-    try {
-      htable = new HTable(hconf, tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY));
-
-      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
-      if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
-        HRegionLocation regLoc = htable.getRegionLocation(HConstants.EMPTY_BYTE_ARRAY, false);
-        if (null == regLoc) {
-          throw new IOException("Expecting at least one region.");
-        }
-        List<Fragment> fragments = new ArrayList<Fragment>(1);
-        Fragment fragment = new HBaseFragment(fragmentId, htable.getName().getNameAsString(),
-            HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, regLoc.getHostname());
-        fragments.add(fragment);
-        return fragments;
-      }
-
-      List<byte[]> startRows;
-      List<byte[]> stopRows;
-
-      if (indexPredications != null && !indexPredications.isEmpty()) {
-        // indexPredications is Disjunctive set
-        startRows = new ArrayList<byte[]>();
-        stopRows = new ArrayList<byte[]>();
-        for (IndexPredication indexPredication: indexPredications) {
-          byte[] startRow;
-          byte[] stopRow;
-          if (indexPredication.getStartValue() != null) {
-            startRow = serialize(columnMapping, indexPredication, indexPredication.getStartValue());
-          } else {
-            startRow = HConstants.EMPTY_START_ROW;
-          }
-          if (indexPredication.getStopValue() != null) {
-            stopRow = serialize(columnMapping, indexPredication, indexPredication.getStopValue());
-          } else {
-            stopRow = HConstants.EMPTY_END_ROW;
-          }
-          startRows.add(startRow);
-          stopRows.add(stopRow);
-        }
-      } else {
-        startRows = TUtil.newList(HConstants.EMPTY_START_ROW);
-        stopRows = TUtil.newList(HConstants.EMPTY_END_ROW);
-      }
-
-      hAdmin =  new HBaseAdmin(hconf);
-      Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
-
-      // region startkey -> HBaseFragment
-      Map<byte[], HBaseFragment> fragmentMap = new HashMap<byte[], HBaseFragment>();
-      for (int i = 0; i < keys.getFirst().length; i++) {
-        HRegionLocation location = htable.getRegionLocation(keys.getFirst()[i], false);
-
-        byte[] regionStartKey = keys.getFirst()[i];
-        byte[] regionStopKey = keys.getSecond()[i];
-
-        int startRowsSize = startRows.size();
-        for (int j = 0; j < startRowsSize; j++) {
-          byte[] startRow = startRows.get(j);
-          byte[] stopRow = stopRows.get(j);
-          // determine if the given start an stop key fall into the region
-          if ((startRow.length == 0 || regionStopKey.length == 0 || Bytes.compareTo(startRow, regionStopKey) < 0)
-              && (stopRow.length == 0 || Bytes.compareTo(stopRow, regionStartKey) > 0)) {
-            byte[] fragmentStart = (startRow.length == 0 || Bytes.compareTo(regionStartKey, startRow) >= 0) ?
-                regionStartKey : startRow;
-
-            byte[] fragmentStop = (stopRow.length == 0 || Bytes.compareTo(regionStopKey, stopRow) <= 0) &&
-                regionStopKey.length > 0 ? regionStopKey : stopRow;
-
-            String regionName = location.getRegionInfo().getRegionNameAsString();
-
-            ServerLoad serverLoad = serverLoadMap.get(location.getServerName());
-            if (serverLoad == null) {
-              serverLoad = hAdmin.getClusterStatus().getLoad(location.getServerName());
-              serverLoadMap.put(location.getServerName(), serverLoad);
-            }
-
-            if (fragmentMap.containsKey(regionStartKey)) {
-              HBaseFragment prevFragment = fragmentMap.get(regionStartKey);
-              if (Bytes.compareTo(fragmentStart, prevFragment.getStartRow()) < 0) {
-                prevFragment.setStartRow(fragmentStart);
-              }
-              if (Bytes.compareTo(fragmentStop, prevFragment.getStopRow()) > 0) {
-                prevFragment.setStopRow(fragmentStop);
-              }
-            } else {
-              HBaseFragment fragment = new HBaseFragment(fragmentId, htable.getName().getNameAsString(),
-                  fragmentStart, fragmentStop, location.getHostname());
-
-              // get region size
-              boolean foundLength = false;
-              for (Map.Entry<byte[], RegionLoad> entry : serverLoad.getRegionsLoad().entrySet()) {
-                if (regionName.equals(Bytes.toString(entry.getKey()))) {
-                  RegionLoad regionLoad = entry.getValue();
-                  long storeFileSize = (regionLoad.getStorefileSizeMB() + regionLoad.getMemStoreSizeMB()) * 1024L * 1024L;
-                  fragment.setLength(storeFileSize);
-                  foundLength = true;
-                  break;
-                }
-              }
-
-              if (!foundLength) {
-                fragment.setLength(TajoConstants.UNKNOWN_LENGTH);
-              }
-
-              fragmentMap.put(regionStartKey, fragment);
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("getFragments: fragment -> " + i + " -> " + fragment);
-              }
-            }
-          }
-        }
-      }
-
-      List<HBaseFragment> fragments = new ArrayList<HBaseFragment>(fragmentMap.values());
-      Collections.sort(fragments);
-      if (!fragments.isEmpty()) {
-        fragments.get(fragments.size() - 1).setLast(true);
-      }
-      return (ArrayList<Fragment>) (ArrayList) fragments;
-    } finally {
-      if (htable != null) {
-        htable.close();
-      }
-      if (hAdmin != null) {
-        hAdmin.close();
-      }
-    }
-  }
-
-  private byte[] serialize(ColumnMapping columnMapping,
-                           IndexPredication indexPredication, Datum datum) throws IOException {
-    if (columnMapping.getIsBinaryColumns()[indexPredication.getColumnId()]) {
-      return HBaseBinarySerializerDeserializer.serialize(indexPredication.getColumn(), datum);
-    } else {
-      return HBaseTextSerializerDeserializer.serialize(indexPredication.getColumn(), datum);
-    }
-  }
-
-  @Override
-  public Appender getAppender(OverridableConf queryContext,
-                              QueryUnitAttemptId taskAttemptId, TableMeta meta, Schema schema, Path workDir)
-      throws IOException {
-    if ("true".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
-      return new HBasePutAppender(conf, taskAttemptId, schema, meta, workDir);
-    } else {
-      return super.getAppender(queryContext, taskAttemptId, meta, schema, workDir);
-    }
-  }
-
-  @Override
-  public List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments)
-      throws IOException {
-    Configuration hconf = getHBaseConfiguration(conf, tableDesc.getMeta());
-    HTable htable = null;
-    HBaseAdmin hAdmin = null;
-    try {
-      htable = new HTable(hconf, tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY));
-
-      org.apache.hadoop.hbase.util.Pair<byte[][], byte[][]> keys = htable.getStartEndKeys();
-      if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
-        return new ArrayList<Fragment>(1);
-      }
-      hAdmin =  new HBaseAdmin(hconf);
-      Map<ServerName, ServerLoad> serverLoadMap = new HashMap<ServerName, ServerLoad>();
-
-      List<Fragment> fragments = new ArrayList<Fragment>(keys.getFirst().length);
-
-      int start = currentPage * numFragments;
-      if (start >= keys.getFirst().length) {
-        return new ArrayList<Fragment>(1);
-      }
-      int end = (currentPage + 1) * numFragments;
-      if (end > keys.getFirst().length) {
-        end = keys.getFirst().length;
-      }
-      for (int i = start; i < end; i++) {
-        HRegionLocation location = htable.getRegionLocation(keys.getFirst()[i], false);
-
-        String regionName = location.getRegionInfo().getRegionNameAsString();
-        ServerLoad serverLoad = serverLoadMap.get(location.getServerName());
-        if (serverLoad == null) {
-          serverLoad = hAdmin.getClusterStatus().getLoad(location.getServerName());
-          serverLoadMap.put(location.getServerName(), serverLoad);
-        }
-
-        HBaseFragment fragment = new HBaseFragment(tableDesc.getName(), htable.getName().getNameAsString(),
-            location.getRegionInfo().getStartKey(), location.getRegionInfo().getEndKey(), location.getHostname());
-
-        // get region size
-        boolean foundLength = false;
-        for (Map.Entry<byte[], RegionLoad> entry : serverLoad.getRegionsLoad().entrySet()) {
-          if (regionName.equals(Bytes.toString(entry.getKey()))) {
-            RegionLoad regionLoad = entry.getValue();
-            long storeLength = (regionLoad.getStorefileSizeMB() + regionLoad.getMemStoreSizeMB()) * 1024L * 1024L;
-            if (storeLength == 0) {
-              // If store size is smaller than 1 MB, storeLength is zero
-              storeLength = 1 * 1024 * 1024;  //default 1MB
-            }
-            fragment.setLength(storeLength);
-            foundLength = true;
-            break;
-          }
-        }
-
-        if (!foundLength) {
-          fragment.setLength(TajoConstants.UNKNOWN_LENGTH);
-        }
-
-        fragments.add(fragment);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("getFragments: fragment -> " + i + " -> " + fragment);
-        }
-      }
-
-      if (!fragments.isEmpty()) {
-        ((HBaseFragment) fragments.get(fragments.size() - 1)).setLast(true);
-      }
-      return fragments;
-    } finally {
-      if (htable != null) {
-        htable.close();
-      }
-      if (hAdmin != null) {
-        hAdmin.close();
-      }
-    }
-  }
-
-  public HConnection getConnection(Configuration hbaseConf) throws IOException {
-    synchronized(connMap) {
-      HConnectionKey key = new HConnectionKey(hbaseConf);
-      HConnection conn = connMap.get(key);
-      if (conn == null) {
-        conn = HConnectionManager.createConnection(hbaseConf);
-        connMap.put(key, conn);
-      }
-
-      return conn;
-    }
-  }
-
-  static class HConnectionKey {
-    final static String[] CONNECTION_PROPERTIES = new String[] {
-        HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT,
-        HConstants.ZOOKEEPER_CLIENT_PORT,
-        HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
-        HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-        HConstants.HBASE_RPC_TIMEOUT_KEY,
-        HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
-        HConstants.HBASE_META_SCANNER_CACHING,
-        HConstants.HBASE_CLIENT_INSTANCE_ID,
-        HConstants.RPC_CODEC_CONF_KEY };
-
-    private Map<String, String> properties;
-    private String username;
-
-    HConnectionKey(Configuration conf) {
-      Map<String, String> m = new HashMap<String, String>();
-      if (conf != null) {
-        for (String property : CONNECTION_PROPERTIES) {
-          String value = conf.get(property);
-          if (value != null) {
-            m.put(property, value);
-          }
-        }
-      }
-      this.properties = Collections.unmodifiableMap(m);
-
-      try {
-        UserProvider provider = UserProvider.instantiate(conf);
-        User currentUser = provider.getCurrent();
-        if (currentUser != null) {
-          username = currentUser.getName();
-        }
-      } catch (IOException ioe) {
-        LOG.warn("Error obtaining current user, skipping username in HConnectionKey", ioe);
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 31;
-      int result = 1;
-      if (username != null) {
-        result = username.hashCode();
-      }
-      for (String property : CONNECTION_PROPERTIES) {
-        String value = properties.get(property);
-        if (value != null) {
-          result = prime * result + value.hashCode();
-        }
-      }
-
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj)
-        return true;
-      if (obj == null)
-        return false;
-      if (getClass() != obj.getClass())
-        return false;
-      HConnectionKey that = (HConnectionKey) obj;
-      if (this.username != null && !this.username.equals(that.username)) {
-        return false;
-      } else if (this.username == null && that.username != null) {
-        return false;
-      }
-      if (this.properties == null) {
-        if (that.properties != null) {
-          return false;
-        }
-      } else {
-        if (that.properties == null) {
-          return false;
-        }
-        for (String property : CONNECTION_PROPERTIES) {
-          String thisValue = this.properties.get(property);
-          String thatValue = that.properties.get(property);
-          //noinspection StringEquality
-          if (thisValue == thatValue) {
-            continue;
-          }
-          if (thisValue == null || !thisValue.equals(thatValue)) {
-            return false;
-          }
-        }
-      }
-      return true;
-    }
-
-    @Override
-    public String toString() {
-      return "HConnectionKey{" +
-          "properties=" + properties +
-          ", username='" + username + '\'' +
-          '}';
-    }
-  }
-
-  public List<IndexPredication> getIndexPredications(ColumnMapping columnMapping,
-                                                     TableDesc tableDesc, ScanNode scanNode) throws IOException {
-    List<IndexPredication> indexPredications = new ArrayList<IndexPredication>();
-    Column[] indexableColumns = getIndexableColumns(tableDesc);
-    if (indexableColumns != null && indexableColumns.length == 1) {
-      // Currently supports only single index column.
-      List<Set<EvalNode>> indexablePredicateList = findIndexablePredicateSet(scanNode, indexableColumns);
-      for (Set<EvalNode> eachEvalSet: indexablePredicateList) {
-        Pair<Datum, Datum> indexPredicationValues = getIndexablePredicateValue(columnMapping, eachEvalSet);
-        if (indexPredicationValues != null) {
-          IndexPredication indexPredication = new IndexPredication();
-          indexPredication.setColumn(indexableColumns[0]);
-          indexPredication.setColumnId(tableDesc.getLogicalSchema().getColumnId(indexableColumns[0].getQualifiedName()));
-          indexPredication.setStartValue(indexPredicationValues.getFirst());
-          indexPredication.setStopValue(indexPredicationValues.getSecond());
-
-          indexPredications.add(indexPredication);
-        }
-      }
-    }
-    return indexPredications;
-  }
-
-  public List<Set<EvalNode>> findIndexablePredicateSet(ScanNode scanNode, Column[] indexableColumns) throws IOException {
-    List<Set<EvalNode>> indexablePredicateList = new ArrayList<Set<EvalNode>>();
-
-    // if a query statement has a search condition, try to find indexable predicates
-    if (indexableColumns != null && scanNode.getQual() != null) {
-      EvalNode[] disjunctiveForms = AlgebraicUtil.toDisjunctiveNormalFormArray(scanNode.getQual());
-
-      // add qualifier to schema for qual
-      for (Column column : indexableColumns) {
-        for (EvalNode disjunctiveExpr : disjunctiveForms) {
-          EvalNode[] conjunctiveForms = AlgebraicUtil.toConjunctiveNormalFormArray(disjunctiveExpr);
-          Set<EvalNode> indexablePredicateSet = Sets.newHashSet();
-          for (EvalNode conjunctiveExpr : conjunctiveForms) {
-            if (checkIfIndexablePredicateOnTargetColumn(conjunctiveExpr, column)) {
-              indexablePredicateSet.add(conjunctiveExpr);
-            }
-          }
-          if (!indexablePredicateSet.isEmpty()) {
-            indexablePredicateList.add(indexablePredicateSet);
-          }
-        }
-      }
-    }
-
-    return indexablePredicateList;
-  }
-
-  private boolean checkIfIndexablePredicateOnTargetColumn(EvalNode evalNode, Column targetColumn) {
-    if (checkIfIndexablePredicate(evalNode) || checkIfConjunctiveButOneVariable(evalNode)) {
-      Set<Column> variables = EvalTreeUtil.findUniqueColumns(evalNode);
-      // if it contains only single variable matched to a target column
-      return variables.size() == 1 && variables.contains(targetColumn);
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   *
-   * @param evalNode The expression to be checked
-   * @return true if an conjunctive expression, consisting of indexable expressions
-   */
-  private boolean checkIfConjunctiveButOneVariable(EvalNode evalNode) {
-    if (evalNode.getType() == EvalType.AND) {
-      BinaryEval orEval = (BinaryEval) evalNode;
-      boolean indexable =
-          checkIfIndexablePredicate(orEval.getLeftExpr()) &&
-              checkIfIndexablePredicate(orEval.getRightExpr());
-
-      boolean sameVariable =
-          EvalTreeUtil.findUniqueColumns(orEval.getLeftExpr())
-              .equals(EvalTreeUtil.findUniqueColumns(orEval.getRightExpr()));
-
-      return indexable && sameVariable;
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * Check if an expression consists of one variable and one constant and
-   * the expression is a comparison operator.
-   *
-   * @param evalNode The expression to be checked
-   * @return true if an expression consists of one variable and one constant
-   * and the expression is a comparison operator. Other, false.
-   */
-  private boolean checkIfIndexablePredicate(EvalNode evalNode) {
-    return AlgebraicUtil.containSingleVar(evalNode) && isIndexableOperator(evalNode);
-  }
-
-  public static boolean isIndexableOperator(EvalNode expr) {
-    return expr.getType() == EvalType.EQUAL ||
-        expr.getType() == EvalType.LEQ ||
-        expr.getType() == EvalType.LTH ||
-        expr.getType() == EvalType.GEQ ||
-        expr.getType() == EvalType.GTH ||
-        expr.getType() == EvalType.BETWEEN;
-  }
-
-  public Pair<Datum, Datum> getIndexablePredicateValue(ColumnMapping columnMapping,
-                                                       Set<EvalNode> evalNodes) {
-    Datum startDatum = null;
-    Datum endDatum = null;
-    for (EvalNode evalNode: evalNodes) {
-      if (evalNode instanceof BinaryEval) {
-        BinaryEval binaryEval = (BinaryEval) evalNode;
-        EvalNode left = binaryEval.getLeftExpr();
-        EvalNode right = binaryEval.getRightExpr();
-
-        Datum constValue = null;
-        if (left.getType() == EvalType.CONST) {
-          constValue = ((ConstEval) left).getValue();
-        } else if (right.getType() == EvalType.CONST) {
-          constValue = ((ConstEval) right).getValue();
-        }
-
-        if (constValue != null) {
-          if (evalNode.getType() == EvalType.EQUAL ||
-              evalNode.getType() == EvalType.GEQ ||
-              evalNode.getType() == EvalType.GTH) {
-            if (startDatum != null) {
-              if (constValue.compareTo(startDatum) > 0) {
-                startDatum = constValue;
-              }
-            } else {
-              startDatum = constValue;
-            }
-          }
-
-          if (evalNode.getType() == EvalType.EQUAL ||
-              evalNode.getType() == EvalType.LEQ ||
-              evalNode.getType() == EvalType.LTH) {
-            if (endDatum != null) {
-              if (constValue.compareTo(endDatum) < 0) {
-                endDatum = constValue;
-              }
-            } else {
-              endDatum = constValue;
-            }
-          }
-        }
-      } else if (evalNode instanceof BetweenPredicateEval) {
-        BetweenPredicateEval betweenEval = (BetweenPredicateEval) evalNode;
-        if (betweenEval.getBegin().getType() == EvalType.CONST && betweenEval.getEnd().getType() == EvalType.CONST) {
-          Datum value = ((ConstEval) betweenEval.getBegin()).getValue();
-          if (startDatum != null) {
-            if (value.compareTo(startDatum) > 0) {
-              startDatum = value;
-            }
-          } else {
-            startDatum = value;
-          }
-
-          value = ((ConstEval) betweenEval.getEnd()).getValue();
-          if (endDatum != null) {
-            if (value.compareTo(endDatum) < 0) {
-              endDatum = value;
-            }
-          } else {
-            endDatum = value;
-          }
-        }
-      }
-    }
-
-    if (endDatum != null && columnMapping != null && columnMapping.getNumRowKeys() > 1) {
-      endDatum = new TextDatum(endDatum.asChars() +
-          new String(new char[]{columnMapping.getRowKeyDelimiter(), Character.MAX_VALUE}));
-    }
-    if (startDatum != null || endDatum != null) {
-      return new Pair<Datum, Datum>(startDatum, endDatum);
-    } else {
-      return null;
-    }
-  }
-
-  @Override
-  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
-                               LogicalPlan plan, Schema schema,
-                               TableDesc tableDesc) throws IOException {
-    if (tableDesc == null) {
-      throw new IOException("TableDesc is null while calling loadIncrementalHFiles: " + finalEbId);
-    }
-    Path stagingDir = new Path(queryContext.get(QueryVars.STAGING_DIR));
-    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-
-    Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
-    hbaseConf.set("hbase.loadincremental.threads.max", "2");
-
-    JobContextImpl jobContext = new JobContextImpl(hbaseConf,
-        new JobID(finalEbId.getQueryId().toString(), finalEbId.getId()));
-
-    FileOutputCommitter committer = new FileOutputCommitter(stagingResultDir, jobContext);
-    Path jobAttemptPath = committer.getJobAttemptPath(jobContext);
-    FileSystem fs = jobAttemptPath.getFileSystem(queryContext.getConf());
-    if (!fs.exists(jobAttemptPath) || fs.listStatus(jobAttemptPath) == null) {
-      LOG.warn("No query attempt file in " + jobAttemptPath);
-      return stagingResultDir;
-    }
-    committer.commitJob(jobContext);
-
-    if (tableDesc.getName() == null && tableDesc.getPath() != null) {
-
-      // insert into location
-      return super.commitOutputData(queryContext, finalEbId, plan, schema, tableDesc, false);
-    } else {
-      // insert into table
-      String tableName = tableDesc.getMeta().getOption(HBaseStorageConstants.META_TABLE_KEY);
-
-      HTable htable = new HTable(hbaseConf, tableName);
-      try {
-        LoadIncrementalHFiles loadIncrementalHFiles = null;
-        try {
-          loadIncrementalHFiles = new LoadIncrementalHFiles(hbaseConf);
-        } catch (Exception e) {
-          LOG.error(e.getMessage(), e);
-          throw new IOException(e.getMessage(), e);
-        }
-        loadIncrementalHFiles.doBulkLoad(stagingResultDir, htable);
-
-        return stagingResultDir;
-      } finally {
-        htable.close();
-      }
-    }
-  }
-
-  @Override
-  public TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
-                                          Schema inputSchema, SortSpec[] sortSpecs, TupleRange dataRange)
-      throws IOException {
-    try {
-      int[] sortKeyIndexes = new int[sortSpecs.length];
-      for (int i = 0; i < sortSpecs.length; i++) {
-        sortKeyIndexes[i] = inputSchema.getColumnId(sortSpecs[i].getSortKey().getQualifiedName());
-      }
-
-      ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
-      Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(queryContext.getConf(), tableDesc.getMeta());
-
-      HTable htable = new HTable(hbaseConf, columnMapping.getHbaseTableName());
-      try {
-        byte[][] endKeys = htable.getEndKeys();
-        if (endKeys.length == 1) {
-          return new TupleRange[]{dataRange};
-        }
-        List<TupleRange> tupleRanges = new ArrayList<TupleRange>(endKeys.length);
-
-        TupleComparator comparator = new BaseTupleComparator(inputSchema, sortSpecs);
-        Tuple previousTuple = dataRange.getStart();
-
-        for (byte[] eachEndKey : endKeys) {
-          Tuple endTuple = new VTuple(sortSpecs.length);
-          byte[][] rowKeyFields;
-          if (sortSpecs.length > 1) {
-            byte[][] splitValues = BytesUtils.splitPreserveAllTokens(eachEndKey, columnMapping.getRowKeyDelimiter());
-            if (splitValues.length == sortSpecs.length) {
-              rowKeyFields = splitValues;
-            } else {
-              rowKeyFields = new byte[sortSpecs.length][];
-              for (int j = 0; j < sortSpecs.length; j++) {
-                if (j < splitValues.length) {
-                  rowKeyFields[j] = splitValues[j];
-                } else {
-                  rowKeyFields[j] = null;
-                }
-              }
-            }
-
-          } else {
-            rowKeyFields = new byte[1][];
-            rowKeyFields[0] = eachEndKey;
-          }
-
-          for (int i = 0; i < sortSpecs.length; i++) {
-            if (columnMapping.getIsBinaryColumns()[sortKeyIndexes[i]]) {
-              endTuple.put(i,
-                  HBaseBinarySerializerDeserializer.deserialize(inputSchema.getColumn(sortKeyIndexes[i]),
-                      rowKeyFields[i]));
-            } else {
-              endTuple.put(i,
-                  HBaseTextSerializerDeserializer.deserialize(inputSchema.getColumn(sortKeyIndexes[i]),
-                      rowKeyFields[i]));
-            }
-          }
-          tupleRanges.add(new TupleRange(sortSpecs, previousTuple, endTuple));
-          previousTuple = endTuple;
-        }
-
-        // Last region endkey is empty. Tajo ignores empty key, so endkey is replaced with max data value.
-        if (comparator.compare(dataRange.getEnd(), tupleRanges.get(tupleRanges.size() - 1).getStart()) >= 0) {
-          tupleRanges.get(tupleRanges.size() - 1).setEnd(dataRange.getEnd());
-        } else {
-          tupleRanges.remove(tupleRanges.size() - 1);
-        }
-        return tupleRanges.toArray(new TupleRange[]{});
-      } finally {
-        htable.close();
-      }
-    } catch (Throwable t) {
-      LOG.error(t.getMessage(), t);
-      throw new IOException(t.getMessage(), t);
-    }
-  }
-
-  public List<RewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
-    if ("false".equalsIgnoreCase(queryContext.get(HBaseStorageConstants.INSERT_PUT_MODE, "false"))) {
-      List<RewriteRule> rules = new ArrayList<RewriteRule>();
-      rules.add(new AddSortForInsertRewriter(tableDesc, getIndexColumns(tableDesc)));
-      return rules;
-    } else {
-      return null;
-    }
-  }
-
-  private Column[] getIndexColumns(TableDesc tableDesc) throws IOException {
-    List<Column> indexColumns = new ArrayList<Column>();
-
-    ColumnMapping columnMapping = new ColumnMapping(tableDesc.getSchema(), tableDesc.getMeta());
-
-    boolean[] isRowKeys = columnMapping.getIsRowKeyMappings();
-    for (int i = 0; i < isRowKeys.length; i++) {
-      if (isRowKeys[i]) {
-        indexColumns.add(tableDesc.getSchema().getColumn(i));
-      }
-    }
-
-    return indexColumns.toArray(new Column[]{});
-  }
-
-  @Override
-  public StorageProperty getStorageProperty() {
-    StorageProperty storageProperty = new StorageProperty();
-    storageProperty.setSortedInsert(true);
-    storageProperty.setSupportsInsertInto(true);
-    return storageProperty;
-  }
-
-  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
-    if (node.getType() == NodeType.CREATE_TABLE) {
-      CreateTableNode cNode = (CreateTableNode)node;
-      if (!cNode.isExternal()) {
-        TableMeta tableMeta = new TableMeta(cNode.getStorageType(), cNode.getOptions());
-        createTable(tableMeta, cNode.getTableSchema(), cNode.isExternal(), cNode.isIfNotExists());
-      }
-    }
-  }
-
-  @Override
-  public void rollbackOutputCommit(LogicalNode node) throws IOException {
-    if (node.getType() == NodeType.CREATE_TABLE) {
-      CreateTableNode cNode = (CreateTableNode)node;
-      if (cNode.isExternal()) {
-        return;
-      }
-      TableMeta tableMeta = new TableMeta(cNode.getStorageType(), cNode.getOptions());
-      HBaseAdmin hAdmin =  new HBaseAdmin(getHBaseConfiguration(conf, tableMeta));
-
-      try {
-        HTableDescriptor hTableDesc = parseHTableDescriptor(tableMeta, cNode.getTableSchema());
-        LOG.info("Delete table cause query failed:" + hTableDesc.getName());
-        hAdmin.disableTable(hTableDesc.getName());
-        hAdmin.deleteTable(hTableDesc.getName());
-      } finally {
-        hAdmin.close();
-      }
-    }
-  }
-
-  @Override
-  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException  {
-    if (tableDesc != null) {
-      Schema tableSchema = tableDesc.getSchema();
-      if (tableSchema.size() != outSchema.size()) {
-        throw new IOException("The number of table columns is different from SELECT columns");
-      }
-
-      for (int i = 0; i < tableSchema.size(); i++) {
-        if (!tableSchema.getColumn(i).getDataType().equals(outSchema.getColumn(i).getDataType())) {
-          throw new IOException(outSchema.getColumn(i).getQualifiedName() +
-              "(" + outSchema.getColumn(i).getDataType().getType() + ")" +
-              " is different column type with " + tableSchema.getColumn(i).getSimpleName() +
-              "(" + tableSchema.getColumn(i).getDataType().getType() + ")");
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
deleted file mode 100644
index a0ad492..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
+++ /dev/null
@@ -1,71 +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.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.util.NumberUtil;
-
-import java.io.IOException;
-
-public class HBaseTextSerializerDeserializer {
-  public static Datum deserialize(Column col, byte[] bytes) throws IOException {
-    Datum datum;
-    switch (col.getDataType().getType()) {
-      case INT1:
-      case INT2:
-        datum = bytes == null  || bytes.length == 0 ? NullDatum.get() :
-            DatumFactory.createInt2((short)NumberUtil.parseInt(bytes, 0, bytes.length));
-        break;
-      case INT4:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
-            DatumFactory.createInt4(NumberUtil.parseInt(bytes, 0, bytes.length));
-        break;
-      case INT8:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
-            DatumFactory.createInt8(new String(bytes, 0, bytes.length));
-        break;
-      case FLOAT4:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
-            DatumFactory.createFloat4(new String(bytes, 0, bytes.length));
-        break;
-      case FLOAT8:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
-            DatumFactory.createFloat8(NumberUtil.parseDouble(bytes, 0, bytes.length));
-        break;
-      case TEXT:
-        datum = bytes == null ? NullDatum.get() : DatumFactory.createText(bytes);
-        break;
-      default:
-        datum = NullDatum.get();
-        break;
-    }
-    return datum;
-  }
-
-  public static byte[] serialize(Column col, Datum datum) throws IOException {
-    if (datum == null || datum instanceof NullDatum) {
-      return null;
-    }
-
-    return datum.asChars().getBytes();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
deleted file mode 100644
index b9425f9..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
+++ /dev/null
@@ -1,167 +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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.util.Bytes;
-
-import java.io.IOException;
-import java.util.TreeSet;
-
-public class HFileAppender extends AbstractHBaseAppender {
-  private static final Log LOG = LogFactory.getLog(HFileAppender.class);
-
-  private RecordWriter<ImmutableBytesWritable, Cell> writer;
-  private TaskAttemptContext writerContext;
-  private Path workingFilePath;
-  private FileOutputCommitter committer;
-
-  public HFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
-                       Schema schema, TableMeta meta, Path stagingDir) {
-    super(conf, taskAttemptId, schema, meta, stagingDir);
-  }
-
-  @Override
-  public void init() throws IOException {
-    super.init();
-
-    Configuration taskConf = new Configuration();
-    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-    taskConf.set(FileOutputFormat.OUTDIR, stagingResultDir.toString());
-
-    ExecutionBlockId ebId = taskAttemptId.getQueryUnitId().getExecutionBlockId();
-    writerContext = new TaskAttemptContextImpl(taskConf,
-        new TaskAttemptID(ebId.getQueryId().toString(), ebId.getId(), TaskType.MAP,
-            taskAttemptId.getQueryUnitId().getId(), taskAttemptId.getId()));
-
-    HFileOutputFormat2 hFileOutputFormat2 = new HFileOutputFormat2();
-    try {
-      writer = hFileOutputFormat2.getRecordWriter(writerContext);
-
-      committer = new FileOutputCommitter(FileOutputFormat.getOutputPath(writerContext), writerContext);
-      workingFilePath = committer.getWorkPath();
-    } catch (InterruptedException e) {
-      throw new IOException(e.getMessage(), e);
-    }
-
-    LOG.info("Created hbase file writer: " + workingFilePath);
-  }
-
-  long totalNumBytes = 0;
-  ImmutableBytesWritable keyWritable = new ImmutableBytesWritable();
-  boolean first = true;
-  TreeSet<KeyValue> kvSet = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
-
-
-  @Override
-  public void addTuple(Tuple tuple) throws IOException {
-    Datum datum;
-
-    byte[] rowkey = getRowKeyBytes(tuple);
-
-    if (!first && !Bytes.equals(keyWritable.get(), 0, keyWritable.getLength(), rowkey, 0, rowkey.length)) {
-      try {
-        for (KeyValue kv : kvSet) {
-          writer.write(keyWritable, kv);
-          totalNumBytes += keyWritable.getLength() + kv.getLength();
-        }
-        kvSet.clear();
-        // Statistical section
-        if (enabledStats) {
-          stats.incrementRow();
-        }
-      } catch (InterruptedException e) {
-        LOG.error(e.getMessage(), e);
-      }
-    }
-
-    first = false;
-
-    keyWritable.set(rowkey);
-
-    readKeyValues(tuple, rowkey);
-    if (keyValues != null) {
-      for (KeyValue eachKeyVal: keyValues) {
-        kvSet.add(eachKeyVal);
-      }
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-  }
-
-  @Override
-  public long getEstimatedOutputSize() throws IOException {
-    // StoreTableExec uses this value as rolling file length
-    // Not rolling
-    return 0;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (!kvSet.isEmpty()) {
-      try {
-        for (KeyValue kv : kvSet) {
-          writer.write(keyWritable, kv);
-          totalNumBytes += keyWritable.getLength() + keyWritable.getLength();
-        }
-        kvSet.clear();
-        // Statistical section
-        if (enabledStats) {
-          stats.incrementRow();
-        }
-      } catch (InterruptedException e) {
-        LOG.error(e.getMessage(), e);
-      }
-    }
-
-    if (enabledStats) {
-      stats.setNumBytes(totalNumBytes);
-    }
-    if (writer != null) {
-      try {
-        writer.close(writerContext);
-        committer.commitTask(writerContext);
-      } catch (InterruptedException e) {
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
deleted file mode 100644
index 3a58e50..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
+++ /dev/null
@@ -1,61 +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.datum.Datum;
-
-public class IndexPredication {
-  private Column column;
-  private int columnId;
-  private Datum startValue;
-  private Datum stopValue;
-
-  public Column getColumn() {
-    return column;
-  }
-
-  public void setColumn(Column column) {
-    this.column = column;
-  }
-
-  public int getColumnId() {
-    return columnId;
-  }
-
-  public void setColumnId(int columnId) {
-    this.columnId = columnId;
-  }
-
-  public Datum getStartValue() {
-    return startValue;
-  }
-
-  public void setStartValue(Datum startValue) {
-    this.startValue = startValue;
-  }
-
-  public Datum getStopValue() {
-    return stopValue;
-  }
-
-  public void setStopValue(Datum stopValue) {
-    this.stopValue = stopValue;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
deleted file mode 100644
index 4577703..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
+++ /dev/null
@@ -1,40 +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;
-
-public class RowKeyMapping {
-  private boolean isBinary;
-  private int keyFieldIndex;
-
-  public boolean isBinary() {
-    return isBinary;
-  }
-
-  public void setBinary(boolean isBinary) {
-    this.isBinary = isBinary;
-  }
-
-  public int getKeyFieldIndex() {
-    return keyFieldIndex;
-  }
-
-  public void setKeyFieldIndex(int keyFieldIndex) {
-    this.keyFieldIndex = keyFieldIndex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java b/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
deleted file mode 100644
index ccba3be..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
+++ /dev/null
@@ -1,33 +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.index;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.TupleComparator;
-
-import java.io.IOException;
-
-public interface IndexMethod {
-  IndexWriter getIndexWriter(final Path fileName, int level, Schema keySchema,
-      TupleComparator comparator) throws IOException;
-  IndexReader getIndexReader(final Path fileName, Schema keySchema,
-      TupleComparator comparator) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
deleted file mode 100644
index 7baf7aa..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
+++ /dev/null
@@ -1,35 +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.index;
-
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-
-public interface IndexReader {
-  
-  /**
-   * Find the offset corresponding to key which is equal to a given key.
-   * 
-   * @param key
-   * @return
-   * @throws IOException 
-   */
-  public long find(Tuple key) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
deleted file mode 100644
index 04738f8..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
+++ /dev/null
@@ -1,33 +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.index;
-
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-
-public abstract class IndexWriter {
-  
-  public abstract void write(Tuple key, long offset) throws IOException;
-  
-  public abstract void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
deleted file mode 100644
index 688bbc7..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package org.apache.tajo.storage.index;
-
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-
-public interface OrderIndexReader extends IndexReader {
-  /**
-   * Find the offset corresponding to key which is equal to or greater than 
-   * a given key.
-   * 
-   * @param key to find
-   * @return
-   * @throws IOException 
-   */
-  public long find(Tuple key, boolean nextKey) throws IOException;
-  
-  /**
-   * Return the next offset from the latest find or next offset
-   * @return
-   * @throws IOException
-   */
-  public long next() throws IOException;
-}


[27/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/LineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/LineReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/LineReader.java
deleted file mode 100644
index 66c610a..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/LineReader.java
+++ /dev/null
@@ -1,559 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-
-/**
- * A class that provides a line reader from an input stream.
- * Depending on the constructor used, lines will either be terminated by:
- * <ul>
- * <li>one of the following: '\n' (LF) , '\r' (CR),
- * or '\r\n' (CR+LF).</li>
- * <li><em>or</em>, a custom byte sequence delimiter</li>
- * </ul>
- * In both cases, EOF also terminates an otherwise unterminated
- * line.
- */
-
-public class LineReader implements Closeable {
-  private static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
-  private int bufferSize = DEFAULT_BUFFER_SIZE;
-  private InputStream in;
-  private byte[] buffer;
-  // the number of bytes of real data in the buffer
-  private int bufferLength = 0;
-  // the current position in the buffer
-  private int bufferPosn = 0;
-
-  private static final byte CR = '\r';
-  private static final byte LF = '\n';
-
-  // The line delimiter
-  private final byte[] recordDelimiterBytes;
-
-  /**
-   * Create a line reader that reads from the given stream using the
-   * default buffer-size (64k).
-   *
-   * @param in The input stream
-   * @throws IOException
-   */
-  public LineReader(InputStream in) {
-    this(in, DEFAULT_BUFFER_SIZE);
-  }
-
-  /**
-   * Create a line reader that reads from the given stream using the
-   * given buffer-size.
-   *
-   * @param in         The input stream
-   * @param bufferSize Size of the read buffer
-   * @throws IOException
-   */
-  public LineReader(InputStream in, int bufferSize) {
-    this.in = in;
-    this.bufferSize = bufferSize;
-    this.buffer = new byte[this.bufferSize];
-    this.recordDelimiterBytes = null;
-  }
-
-  /**
-   * Create a line reader that reads from the given stream using the
-   * <code>io.file.buffer.size</code> specified in the given
-   * <code>Configuration</code>.
-   *
-   * @param in   input stream
-   * @param conf configuration
-   * @throws IOException
-   */
-  public LineReader(InputStream in, Configuration conf) throws IOException {
-    this(in, conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE));
-  }
-
-  /**
-   * Create a line reader that reads from the given stream using the
-   * default buffer-size, and using a custom delimiter of array of
-   * bytes.
-   *
-   * @param in                   The input stream
-   * @param recordDelimiterBytes The delimiter
-   */
-  public LineReader(InputStream in, byte[] recordDelimiterBytes) {
-    this.in = in;
-    this.bufferSize = DEFAULT_BUFFER_SIZE;
-    this.buffer = new byte[this.bufferSize];
-    this.recordDelimiterBytes = recordDelimiterBytes;
-  }
-
-  /**
-   * Create a line reader that reads from the given stream using the
-   * given buffer-size, and using a custom delimiter of array of
-   * bytes.
-   *
-   * @param in                   The input stream
-   * @param bufferSize           Size of the read buffer
-   * @param recordDelimiterBytes The delimiter
-   * @throws IOException
-   */
-  public LineReader(InputStream in, int bufferSize,
-                    byte[] recordDelimiterBytes) {
-    this.in = in;
-    this.bufferSize = bufferSize;
-    this.buffer = new byte[this.bufferSize];
-    this.recordDelimiterBytes = recordDelimiterBytes;
-  }
-
-  /**
-   * Create a line reader that reads from the given stream using the
-   * <code>io.file.buffer.size</code> specified in the given
-   * <code>Configuration</code>, and using a custom delimiter of array of
-   * bytes.
-   *
-   * @param in                   input stream
-   * @param conf                 configuration
-   * @param recordDelimiterBytes The delimiter
-   * @throws IOException
-   */
-  public LineReader(InputStream in, Configuration conf,
-                    byte[] recordDelimiterBytes) throws IOException {
-    this.in = in;
-    this.bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE);
-    this.buffer = new byte[this.bufferSize];
-    this.recordDelimiterBytes = recordDelimiterBytes;
-  }
-
-
-  /**
-   * Close the underlying stream.
-   *
-   * @throws IOException
-   */
-  public void close() throws IOException {
-    in.close();
-  }
-
-  public void reset() {
-    bufferLength = 0;
-    bufferPosn = 0;
-
-  }
-
-  /**
-   * Read one line from the InputStream into the given Text.
-   *
-   * @param str               the object to store the given line (without newline)
-   * @param maxLineLength     the maximum number of bytes to store into str;
-   *                          the rest of the line is silently discarded.
-   * @param maxBytesToConsume the maximum number of bytes to consume
-   *                          in this call.  This is only a hint, because if the line cross
-   *                          this threshold, we allow it to happen.  It can overshoot
-   *                          potentially by as much as one buffer length.
-   * @return the number of bytes read including the (longest) newline
-   *         found.
-   * @throws IOException if the underlying stream throws
-   */
-  public int readLine(Text str, int maxLineLength,
-                      int maxBytesToConsume) throws IOException {
-    if (this.recordDelimiterBytes != null) {
-      return readCustomLine(str, maxLineLength, maxBytesToConsume);
-    } else {
-      return readDefaultLine(str, maxLineLength, maxBytesToConsume);
-    }
-  }
-
-  protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter)
-      throws IOException {
-    return in.read(buffer);
-  }
-  /**
-   * Read a line terminated by one of CR, LF, or CRLF.
-   */
-  private int readDefaultLine(Text str, int maxLineLength, int maxBytesToConsume)
-      throws IOException {
-    /* We're reading data from in, but the head of the stream may be
-     * already buffered in buffer, so we have several cases:
-     * 1. No newline characters are in the buffer, so we need to copy
-     *    everything and read another buffer from the stream.
-     * 2. An unambiguously terminated line is in buffer, so we just
-     *    copy to str.
-     * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
-     *    in CR.  In this case we copy everything up to CR to str, but
-     *    we also need to see what follows CR: if it's LF, then we
-     *    need consume LF as well, so next call to readLine will read
-     *    from after that.
-     * We use a flag prevCharCR to signal if previous character was CR
-     * and, if it happens to be at the end of the buffer, delay
-     * consuming it until we have a chance to look at the char that
-     * follows.
-     */
-    str.clear();
-    int txtLength = 0; //tracks str.getLength(), as an optimization
-    int newlineLength = 0; //length of terminating newline
-    boolean prevCharCR = false; //true of prev char was CR
-    long bytesConsumed = 0;
-    do {
-      int startPosn = bufferPosn; //starting from where we left off the last time
-      if (bufferPosn >= bufferLength) {
-        startPosn = bufferPosn = 0;
-        if (prevCharCR) {
-          ++bytesConsumed; //account for CR from previous read
-        }
-        bufferLength = fillBuffer(in, buffer, prevCharCR);
-        if (bufferLength <= 0) {
-          break; // EOF
-        }
-      }
-      for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
-        if (buffer[bufferPosn] == LF) {
-          newlineLength = (prevCharCR) ? 2 : 1;
-          ++bufferPosn; // at next invocation proceed from following byte
-          break;
-        }
-        if (prevCharCR) { //CR + notLF, we are at notLF
-          newlineLength = 1;
-          break;
-        }
-        prevCharCR = (buffer[bufferPosn] == CR);
-      }
-      int readLength = bufferPosn - startPosn;
-      if (prevCharCR && newlineLength == 0) {
-        --readLength; //CR at the end of the buffer
-      }
-      bytesConsumed += readLength;
-      int appendLength = readLength - newlineLength;
-      if (appendLength > maxLineLength - txtLength) {
-        appendLength = maxLineLength - txtLength;
-      }
-      if (appendLength > 0) {
-        str.append(buffer, startPosn, appendLength);
-        txtLength += appendLength;
-      }
-    } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
-
-    if (bytesConsumed > (long) Integer.MAX_VALUE) {
-      throw new IOException("Too many bytes before newline: " + bytesConsumed);
-    }
-    return (int) bytesConsumed;
-  }
-
-  /**
-   * Read a line terminated by one of CR, LF, or CRLF.
-   */
-  public int readDefaultLine(NonSyncByteArrayOutputStream str, ArrayList<Integer> offsets, int maxLineLength
-      , int maxBytesToConsume)
-      throws IOException {
-    /* We're reading data from in, but the head of the stream may be
-     * already buffered in buffer, so we have several cases:
-     * 1. No newline characters are in the buffer, so we need to copy
-     *    everything and read another buffer from the stream.
-     * 2. An unambiguously terminated line is in buffer, so we just
-     *    copy to str.
-     * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
-     *    in CR.  In this case we copy everything up to CR to str, but
-     *    we also need to see what follows CR: if it's LF, then we
-     *    need consume LF as well, so next call to readLine will read
-     *    from after that.
-     * We use a flag prevCharCR to signal if previous character was CR
-     * and, if it happens to be at the end of the buffer, delay
-     * consuming it until we have a chance to look at the char that
-     * follows.
-     */
-
-    int txtLength = 0; //tracks str.getLength(), as an optimization
-    int newlineLength = 0; //length of terminating newline
-    boolean prevCharCR = false; //true of prev char was CR
-    long bytesConsumed = 0;
-    do {
-      int startPosn = bufferPosn; //starting from where we left off the last time
-      if (bufferPosn >= bufferLength) {
-        startPosn = bufferPosn = 0;
-        if (prevCharCR) {
-          ++bytesConsumed; //account for CR from previous read
-        }
-        bufferLength = fillBuffer(in, buffer, prevCharCR);
-        if (bufferLength <= 0) {
-          break; // EOF
-        }
-      }
-      for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
-        if (buffer[bufferPosn] == LF) {
-          newlineLength = (prevCharCR) ? 2 : 1;
-          ++bufferPosn; // at next invocation proceed from following byte
-          break;
-        }
-        if (prevCharCR) { //CR + notLF, we are at notLF
-          newlineLength = 1;
-          break;
-        }
-        prevCharCR = (buffer[bufferPosn] == CR);
-      }
-      int readLength = bufferPosn - startPosn;
-      if (prevCharCR && newlineLength == 0) {
-        --readLength; //CR at the end of the buffer
-      }
-      bytesConsumed += readLength;
-      int appendLength = readLength - newlineLength;
-      if (appendLength > maxLineLength - txtLength) {
-        appendLength = maxLineLength - txtLength;
-      }
-      if (appendLength > 0) {
-        str.write(buffer, startPosn, appendLength);
-        txtLength += appendLength;
-      }
-    } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
-
-    if (bytesConsumed > (long) Integer.MAX_VALUE) {
-      throw new IOException("Too many bytes before newline: " + bytesConsumed);
-    }
-
-    if (bytesConsumed > 0) offsets.add(txtLength);
-    return (int) bytesConsumed;
-  }
-
-  /**
-   * Read a line terminated by one of CR, LF, or CRLF.
-   */
-
-/*  int validIdx = 0;
-  public int readDefaultLines(NonSyncByteArrayOutputStream str, ArrayList<Integer> offsets, ArrayList<Long> foffsets,
-                             long pos, int maxLineLength, int maxBytesToConsume)
-      throws IOException {
-    *//* We're reading data from in, but the head of the stream may be
-     * already buffered in buffer, so we have several cases:
-     * 1. No newline characters are in the buffer, so we need to copy
-     *    everything and read another buffer from the stream.
-     * 2. An unambiguously terminated line is in buffer, so we just
-     *    copy to str.
-     * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
-     *    in CR.  In this case we copy everything up to CR to str, but
-     *    we also need to see what follows CR: if it's LF, then we
-     *    need consume LF as well, so next call to readLine will read
-     *    from after that.
-     * We use a flag prevCharCR to signal if previous character was CR
-     * and, if it happens to be at the end of the buffer, delay
-     * consuming it until we have a chance to look at the char that
-     * follows.
-     *//*
-    //str.clear();
-    str.reset();
-    offsets.clear();
-    foffsets.clear();
-
-    validIdx = 0;
-    long bufferBytesConsumed = 0;
-
-    int txtLength = 0; //tracks str.getLength(), as an optimization
-    int newlineLength = 0; //length of terminating newline
-    boolean prevCharCR = false; //true of prev char was CR
-    long bytesConsumed = 0;
-    do {
-
-      int startPosn = bufferPosn; //starting from where we left off the last time
-      if (bufferPosn >= bufferLength) {
-        startPosn = bufferPosn = 0;
-        if (prevCharCR) {
-          ++bytesConsumed; //account for CR from previous read
-        }
-        bufferLength = in.read(buffer);
-        if (bufferLength <= 0) {
-          break; // EOF
-        }
-      }
-      for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
-        if (buffer[bufferPosn] == LF) {
-          newlineLength = (prevCharCR) ? 2 : 1;
-          ++bufferPosn; // at next invocation proceed from following byte
-          break;
-        }
-        if (prevCharCR) { //CR + notLF, we are at notLF
-          newlineLength = 1;
-          break;
-        }
-        prevCharCR = (buffer[bufferPosn] == CR);
-      }
-      int readLength = bufferPosn - startPosn;
-      if (prevCharCR && newlineLength == 0) {
-        --readLength; //CR at the end of the buffer
-      }
-      bytesConsumed += readLength;
-      int appendLength = readLength - newlineLength;
-      if (appendLength > maxLineLength - txtLength) {
-        appendLength = maxLineLength - txtLength;
-      }
-
-      if (appendLength > 0) {
-        str.write(buffer, startPosn, appendLength);
-        //System.out.println(startPosn + "," + appendLength);
-        //str.append(buffer, startPosn, appendLength);
-        txtLength += appendLength;
-      }
-
-      if(newlineLength > 0){
-        validIdx++;
-
-        if (bytesConsumed > (long)Integer.MAX_VALUE) {
-          throw new IOException("Too many bytes before newline: " + bytesConsumed);
-        }
-        offsets.add(txtLength);
-        foffsets.add(pos);
-        pos+= bytesConsumed;
-        bufferBytesConsumed += bytesConsumed;
-
-        txtLength = 0;
-        newlineLength = 0;
-        prevCharCR = false; //true of prev char was CR
-        bytesConsumed = 0;
-      } else {
-        bufferBytesConsumed += bytesConsumed;
-        bytesConsumed = 0;
-      }
-    } while ((bufferBytesConsumed < 256 * 1024));
-
-    return (int)bufferBytesConsumed;
-  }*/
-
-  /**
-   * Read a line terminated by a custom delimiter.
-   */
-  private int readCustomLine(Text str, int maxLineLength, int maxBytesToConsume)
-      throws IOException {
-   /* We're reading data from inputStream, but the head of the stream may be
-    *  already captured in the previous buffer, so we have several cases:
-    *
-    * 1. The buffer tail does not contain any character sequence which
-    *    matches with the head of delimiter. We count it as a
-    *    ambiguous byte count = 0
-    *
-    * 2. The buffer tail contains a X number of characters,
-    *    that forms a sequence, which matches with the
-    *    head of delimiter. We count ambiguous byte count = X
-    *
-    *    // ***  eg: A segment of input file is as follows
-    *
-    *    " record 1792: I found this bug very interesting and
-    *     I have completely read about it. record 1793: This bug
-    *     can be solved easily record 1794: This ."
-    *
-    *    delimiter = "record";
-    *
-    *    supposing:- String at the end of buffer =
-    *    "I found this bug very interesting and I have completely re"
-    *    There for next buffer = "ad about it. record 179       ...."
-    *
-    *     The matching characters in the input
-    *     buffer tail and delimiter head = "re"
-    *     Therefore, ambiguous byte count = 2 ****   //
-    *
-    *     2.1 If the following bytes are the remaining characters of
-    *         the delimiter, then we have to capture only up to the starting
-    *         position of delimiter. That means, we need not include the
-    *         ambiguous characters in str.
-    *
-    *     2.2 If the following bytes are not the remaining characters of
-    *         the delimiter ( as mentioned in the example ),
-    *         then we have to include the ambiguous characters in str.
-    */
-    str.clear();
-    int txtLength = 0; // tracks str.getLength(), as an optimization
-    long bytesConsumed = 0;
-    int delPosn = 0;
-    int ambiguousByteCount = 0; // To capture the ambiguous characters count
-    do {
-      int startPosn = bufferPosn; // Start from previous end position
-      if (bufferPosn >= bufferLength) {
-        startPosn = bufferPosn = 0;
-        bufferLength = fillBuffer(in, buffer, ambiguousByteCount > 0);
-        if (bufferLength <= 0) {
-          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
-          break; // EOF
-        }
-      }
-      for (; bufferPosn < bufferLength; ++bufferPosn) {
-        if (buffer[bufferPosn] == recordDelimiterBytes[delPosn]) {
-          delPosn++;
-          if (delPosn >= recordDelimiterBytes.length) {
-            bufferPosn++;
-            break;
-          }
-        } else if (delPosn != 0) {
-          bufferPosn--;
-          delPosn = 0;
-        }
-      }
-      int readLength = bufferPosn - startPosn;
-      bytesConsumed += readLength;
-      int appendLength = readLength - delPosn;
-      if (appendLength > maxLineLength - txtLength) {
-        appendLength = maxLineLength - txtLength;
-      }
-      if (appendLength > 0) {
-        if (ambiguousByteCount > 0) {
-          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
-          //appending the ambiguous characters (refer case 2.2)
-          bytesConsumed += ambiguousByteCount;
-          ambiguousByteCount = 0;
-        }
-        str.append(buffer, startPosn, appendLength);
-        txtLength += appendLength;
-      }
-      if (bufferPosn >= bufferLength) {
-        if (delPosn > 0 && delPosn < recordDelimiterBytes.length) {
-          ambiguousByteCount = delPosn;
-          bytesConsumed -= ambiguousByteCount; //to be consumed in next
-        }
-      }
-    } while (delPosn < recordDelimiterBytes.length
-        && bytesConsumed < maxBytesToConsume);
-    if (bytesConsumed > (long) Integer.MAX_VALUE) {
-      throw new IOException("Too many bytes before delimiter: " + bytesConsumed);
-    }
-    return (int) bytesConsumed;
-  }
-
-  /**
-   * Read from the InputStream into the given Text.
-   *
-   * @param str           the object to store the given line
-   * @param maxLineLength the maximum number of bytes to store into str.
-   * @return the number of bytes read including the newline
-   * @throws IOException if the underlying stream throws
-   */
-  public int readLine(Text str, int maxLineLength) throws IOException {
-    return readLine(str, maxLineLength, Integer.MAX_VALUE);
-  }
-
-  /**
-   * Read from the InputStream into the given Text.
-   *
-   * @param str the object to store the given line
-   * @return the number of bytes read including the newline
-   * @throws IOException if the underlying stream throws
-   */
-  public int readLine(Text str) throws IOException {
-    return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/MemoryUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/MemoryUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/MemoryUtil.java
deleted file mode 100644
index f19b61f..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/MemoryUtil.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.datum.*;
-import org.apache.tajo.util.ClassSize;
-
-public class MemoryUtil {
-
-  /** Overhead for an NullDatum */
-  public static final long NULL_DATUM;
-
-  /** Overhead for an BoolDatum */
-  public static final long BOOL_DATUM;
-
-  /** Overhead for an CharDatum */
-  public static final long CHAR_DATUM;
-
-  /** Overhead for an BitDatum */
-  public static final long BIT_DATUM;
-
-  /** Overhead for an Int2Datum */
-  public static final long INT2_DATUM;
-
-  /** Overhead for an Int4Datum */
-  public static final long INT4_DATUM;
-
-  /** Overhead for an Int8Datum */
-  public static final long INT8_DATUM;
-
-  /** Overhead for an Float4Datum */
-  public static final long FLOAT4_DATUM;
-
-  /** Overhead for an Float8Datum */
-  public static final long FLOAT8_DATUM;
-
-  /** Overhead for an TextDatum */
-  public static final long TEXT_DATUM;
-
-  /** Overhead for an BlobDatum */
-  public static final long BLOB_DATUM;
-
-  /** Overhead for an DateDatum */
-  public static final long DATE_DATUM;
-
-  /** Overhead for an TimeDatum */
-  public static final long TIME_DATUM;
-
-  /** Overhead for an TimestampDatum */
-  public static final long TIMESTAMP_DATUM;
-
-  static {
-    NULL_DATUM = ClassSize.estimateBase(NullDatum.class, false);
-
-    CHAR_DATUM = ClassSize.estimateBase(CharDatum.class, false);
-
-    BOOL_DATUM = ClassSize.estimateBase(BooleanDatum.class, false);
-
-    BIT_DATUM = ClassSize.estimateBase(BitDatum.class, false);
-
-    INT2_DATUM = ClassSize.estimateBase(Int2Datum.class, false);
-
-    INT4_DATUM = ClassSize.estimateBase(Int4Datum.class, false);
-
-    INT8_DATUM = ClassSize.estimateBase(Int8Datum.class, false);
-
-    FLOAT4_DATUM = ClassSize.estimateBase(Float4Datum.class, false);
-
-    FLOAT8_DATUM = ClassSize.estimateBase(Float8Datum.class, false);
-
-    TEXT_DATUM = ClassSize.estimateBase(TextDatum.class, false);
-
-    BLOB_DATUM = ClassSize.estimateBase(BlobDatum.class, false);
-
-    DATE_DATUM = ClassSize.estimateBase(DateDatum.class, false);
-
-    TIME_DATUM = ClassSize.estimateBase(TimeDatum.class, false);
-
-    TIMESTAMP_DATUM = ClassSize.estimateBase(TimestampDatum.class, false);
-  }
-
-  public static long calculateMemorySize(Tuple tuple) {
-    long total = ClassSize.OBJECT;
-    for (Datum datum : tuple.getValues()) {
-      switch (datum.type()) {
-
-      case NULL_TYPE:
-        total += NULL_DATUM;
-        break;
-
-      case BOOLEAN:
-        total += BOOL_DATUM;
-        break;
-
-      case BIT:
-        total += BIT_DATUM;
-        break;
-
-      case CHAR:
-        total += CHAR_DATUM + datum.size();
-        break;
-
-      case INT1:
-      case INT2:
-        total += INT2_DATUM;
-        break;
-
-      case INT4:
-        total += INT4_DATUM;
-        break;
-
-      case INT8:
-        total += INT8_DATUM;
-        break;
-
-      case FLOAT4:
-        total += FLOAT4_DATUM;
-        break;
-
-      case FLOAT8:
-        total += FLOAT4_DATUM;
-        break;
-
-      case TEXT:
-        total += TEXT_DATUM + datum.size();
-        break;
-
-      case DATE:
-        total += DATE_DATUM;
-        break;
-
-      case TIME:
-        total += TIME_DATUM;
-        break;
-
-      case TIMESTAMP:
-        total += TIMESTAMP_DATUM;
-        break;
-
-      default:
-        break;
-      }
-    }
-
-    return total;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
deleted file mode 100644
index 66b3667..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.ColumnStats;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-public class MergeScanner implements Scanner {
-  private Configuration conf;
-  private TableMeta meta;
-  private Schema schema;
-  private List<Fragment> fragments;
-  private Iterator<Fragment> iterator;
-  private Fragment currentFragment;
-  private Scanner currentScanner;
-  private Tuple tuple;
-  private boolean projectable = false;
-  private boolean selectable = false;
-  private Schema target;
-  private float progress;
-  protected TableStats tableStats;
-
-  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<Fragment> rawFragmentList)
-      throws IOException {
-    this(conf, schema, meta, rawFragmentList, schema);
-  }
-
-  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<Fragment> rawFragmentList,
-                      Schema target)
-      throws IOException {
-    this.conf = conf;
-    this.schema = schema;
-    this.meta = meta;
-    this.target = target;
-
-    this.fragments = new ArrayList<Fragment>();
-
-    long numBytes = 0;
-    for (Fragment eachFileFragment: rawFragmentList) {
-      long fragmentLength = StorageManager.getFragmentLength((TajoConf)conf, eachFileFragment);
-      if (fragmentLength > 0) {
-        numBytes += fragmentLength;
-        fragments.add(eachFileFragment);
-      }
-    }
-
-    // it should keep the input order. Otherwise, it causes wrong result of sort queries.
-    this.reset();
-
-    if (currentScanner != null) {
-      this.projectable = currentScanner.isProjectable();
-      this.selectable = currentScanner.isSelectable();
-    }
-
-    tableStats = new TableStats();
-
-    tableStats.setNumBytes(numBytes);
-    tableStats.setNumBlocks(fragments.size());
-
-    for(Column eachColumn: schema.getColumns()) {
-      ColumnStats columnStats = new ColumnStats(eachColumn);
-      tableStats.addColumnStat(columnStats);
-    }
-  }
-
-  @Override
-  public void init() throws IOException {
-    progress = 0.0f;
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    if (currentScanner != null)
-      tuple = currentScanner.next();
-
-    if (tuple != null) {
-      return tuple;
-    } else {
-      if (currentScanner != null) {
-        currentScanner.close();
-        TableStats scannerTableStsts = currentScanner.getInputStats();
-        if (scannerTableStsts != null) {
-          tableStats.setReadBytes(tableStats.getReadBytes() + scannerTableStsts.getReadBytes());
-          tableStats.setNumRows(tableStats.getNumRows() + scannerTableStsts.getNumRows());
-        }
-      }
-      currentScanner = getNextScanner();
-      if (currentScanner != null) {
-        tuple = currentScanner.next();
-      }
-    }
-    return tuple;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    this.iterator = fragments.iterator();
-    if (currentScanner != null) {
-      currentScanner.close();
-    }
-    this.currentScanner = getNextScanner();
-  }
-
-  private Scanner getNextScanner() throws IOException {
-    if (iterator.hasNext()) {
-      currentFragment = iterator.next();
-      currentScanner = StorageManager.getStorageManager((TajoConf)conf, meta.getStoreType()).getScanner(meta, schema,
-          currentFragment, target);
-      currentScanner.init();
-      return currentScanner;
-    } else {
-      return null;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if(currentScanner != null) {
-      currentScanner.close();
-      currentScanner = null;
-    }
-    iterator = null;
-    progress = 1.0f;
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return projectable;
-  }
-
-  @Override
-  public void setTarget(Column[] targets) {
-    this.target = new Schema(targets);
-  }
-
-  @Override
-  public boolean isSelectable() {
-    return selectable;
-  }
-
-  @Override
-  public void setSearchCondition(Object expr) {
-  }
-
-  @Override
-  public Schema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public boolean isSplittable(){
-    return false;
-  }
-
-  @Override
-  public float getProgress() {
-    if (currentScanner != null && iterator != null && tableStats.getNumBytes() > 0) {
-      TableStats scannerTableStsts = currentScanner.getInputStats();
-      long currentScannerReadBytes = 0;
-      if (scannerTableStsts != null) {
-        currentScannerReadBytes = scannerTableStsts.getReadBytes();
-      }
-
-      return (float)(tableStats.getReadBytes() + currentScannerReadBytes) / (float)tableStats.getNumBytes();
-    } else {
-      return progress;
-    }
-  }
-
-  @Override
-  public TableStats getInputStats() {
-    return tableStats;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/NullScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/NullScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/NullScanner.java
deleted file mode 100644
index 19d333e..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/NullScanner.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package org.apache.tajo.storage; /**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-
-public class NullScanner extends FileScanner {
-  public NullScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) {
-    super(conf, schema, meta, fragment);
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    progress = 1.0f;
-
-    return null;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    progress = 0.0f;
-  }
-
-  @Override
-  public void close() throws IOException {
-    progress = 0.0f;
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return false;
-  }
-
-  @Override
-  public boolean isSelectable() {
-    return true;
-  }
-
-  @Override
-  public boolean isSplittable() {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java b/tajo-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
deleted file mode 100644
index 94d13ee..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.fs.Path;
-
-import java.util.Comparator;
-
-public class NumericPathComparator implements Comparator<Path> {
-
-  @Override
-  public int compare(Path p1, Path p2) {
-    int num1 = Integer.parseInt(p1.getName());
-    int num2 = Integer.parseInt(p2.getName());
-
-    return num1 - num2;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java
deleted file mode 100644
index 5a3f32d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java
+++ /dev/null
@@ -1,774 +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.protobuf.Message;
-import io.netty.buffer.ByteBuf;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.ProtobufDatumFactory;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.unit.StorageUnit;
-import org.apache.tajo.util.BitArray;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-
-public class RawFile {
-  private static final Log LOG = LogFactory.getLog(RawFile.class);
-
-  public static class RawFileScanner extends FileScanner implements SeekableScanner {
-    private FileChannel channel;
-    private DataType[] columnTypes;
-
-    private ByteBuffer buffer;
-    private ByteBuf buf;
-    private Tuple tuple;
-
-    private int headerSize = 0; // Header size of a tuple
-    private BitArray nullFlags;
-    private static final int RECORD_SIZE = 4;
-    private boolean eos = false;
-    private long startOffset;
-    private long endOffset;
-    private FileInputStream fis;
-    private long recordCount;
-    private long totalReadBytes;
-    private long filePosition;
-    private boolean forceFillBuffer;
-
-    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
-      super(conf, schema, meta, fragment);
-    }
-
-    public void init() throws IOException {
-      File file;
-      try {
-        if (fragment.getPath().toUri().getScheme() != null) {
-          file = new File(fragment.getPath().toUri());
-        } else {
-          file = new File(fragment.getPath().toString());
-        }
-      } catch (IllegalArgumentException iae) {
-        throw new IOException(iae);
-      }
-      fis = new FileInputStream(file);
-      channel = fis.getChannel();
-      filePosition = startOffset = fragment.getStartKey();
-      endOffset = fragment.getStartKey() + fragment.getLength();
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RawFileScanner open:" + fragment + "," + channel.position() + ", file size :" + channel.size()
-            + ", fragment length :" + fragment.getLength());
-      }
-
-      buf = BufferPool.directBuffer(64 * StorageUnit.KB);
-      buffer = buf.nioBuffer(0, buf.capacity());
-
-      columnTypes = new DataType[schema.size()];
-      for (int i = 0; i < schema.size(); i++) {
-        columnTypes[i] = schema.getColumn(i).getDataType();
-      }
-
-      tuple = new VTuple(columnTypes.length);
-      nullFlags = new BitArray(schema.size());
-      headerSize = RECORD_SIZE + 2 + nullFlags.bytesLength(); // The middle 2 bytes is for NullFlagSize
-
-      // initial set position
-      if (fragment.getStartKey() > 0) {
-        channel.position(fragment.getStartKey());
-      }
-
-      forceFillBuffer = true;
-      super.init();
-    }
-
-    @Override
-    public long getNextOffset() throws IOException {
-      return filePosition - (forceFillBuffer ? 0 : buffer.remaining());
-    }
-
-    @Override
-    public void seek(long offset) throws IOException {
-      eos = false;
-      filePosition = channel.position();
-
-      // do not fill the buffer if the offset is already included in the buffer.
-      if(!forceFillBuffer && filePosition > offset && offset > filePosition - buffer.limit()){
-        buffer.position((int)(offset - (filePosition - buffer.limit())));
-      } else {
-        if(offset < startOffset || offset > startOffset + fragment.getLength()){
-          throw new IndexOutOfBoundsException(String.format("range(%d, %d), offset: %d",
-              startOffset, startOffset + fragment.getLength(), offset));
-        }
-        channel.position(offset);
-        filePosition = offset;
-        buffer.clear();
-        forceFillBuffer = true;
-        fillBuffer();
-      }
-    }
-
-    private boolean fillBuffer() throws IOException {
-      if(!forceFillBuffer) buffer.compact();
-
-      int bytesRead = channel.read(buffer);
-      forceFillBuffer = false;
-      if (bytesRead == -1) {
-        eos = true;
-        return false;
-      } else {
-        buffer.flip(); //The limit is set to the current filePosition and then the filePosition is set to zero
-        filePosition += bytesRead;
-        totalReadBytes += bytesRead;
-        return true;
-      }
-    }
-
-    /**
-     * Decode a ZigZag-encoded 32-bit value.  ZigZag encodes signed integers
-     * into values that can be efficiently encoded with varint.  (Otherwise,
-     * negative values must be sign-extended to 64 bits to be varint encoded,
-     * thus always taking 10 bytes on the wire.)
-     *
-     * @param n An unsigned 32-bit integer, stored in a signed int because
-     *          Java has no explicit unsigned support.
-     * @return A signed 32-bit integer.
-     */
-    public static int decodeZigZag32(final int n) {
-      return (n >>> 1) ^ -(n & 1);
-    }
-
-    /**
-     * Decode a ZigZag-encoded 64-bit value.  ZigZag encodes signed integers
-     * into values that can be efficiently encoded with varint.  (Otherwise,
-     * negative values must be sign-extended to 64 bits to be varint encoded,
-     * thus always taking 10 bytes on the wire.)
-     *
-     * @param n An unsigned 64-bit integer, stored in a signed int because
-     *          Java has no explicit unsigned support.
-     * @return A signed 64-bit integer.
-     */
-    public static long decodeZigZag64(final long n) {
-      return (n >>> 1) ^ -(n & 1);
-    }
-
-
-    /**
-     * Read a raw Varint from the stream.  If larger than 32 bits, discard the
-     * upper bits.
-     */
-    public int readRawVarint32() throws IOException {
-      byte tmp = buffer.get();
-      if (tmp >= 0) {
-        return tmp;
-      }
-      int result = tmp & 0x7f;
-      if ((tmp = buffer.get()) >= 0) {
-        result |= tmp << 7;
-      } else {
-        result |= (tmp & 0x7f) << 7;
-        if ((tmp = buffer.get()) >= 0) {
-          result |= tmp << 14;
-        } else {
-          result |= (tmp & 0x7f) << 14;
-          if ((tmp = buffer.get()) >= 0) {
-            result |= tmp << 21;
-          } else {
-            result |= (tmp & 0x7f) << 21;
-            result |= (tmp = buffer.get()) << 28;
-            if (tmp < 0) {
-              // Discard upper 32 bits.
-              for (int i = 0; i < 5; i++) {
-                if (buffer.get() >= 0) {
-                  return result;
-                }
-              }
-              throw new IOException("Invalid Variable int32");
-            }
-          }
-        }
-      }
-      return result;
-    }
-
-    /** Read a raw Varint from the stream. */
-    public long readRawVarint64() throws IOException {
-      int shift = 0;
-      long result = 0;
-      while (shift < 64) {
-        final byte b = buffer.get();
-        result |= (long)(b & 0x7F) << shift;
-        if ((b & 0x80) == 0) {
-          return result;
-        }
-        shift += 7;
-      }
-      throw new IOException("Invalid Variable int64");
-    }
-
-    @Override
-    public Tuple next() throws IOException {
-      if(eos) return null;
-
-      if (forceFillBuffer || buffer.remaining() < headerSize) {
-        if (!fillBuffer()) {
-          return null;
-        }
-      }
-
-      // backup the buffer state
-      int bufferLimit = buffer.limit();
-      int recordSize = buffer.getInt();
-      int nullFlagSize = buffer.getShort();
-
-      buffer.limit(buffer.position() + nullFlagSize);
-      nullFlags.fromByteBuffer(buffer);
-      // restore the start of record contents
-      buffer.limit(bufferLimit);
-      if (buffer.remaining() < (recordSize - headerSize)) {
-
-        //if the buffer reaches the writable size, the buffer increase the record size
-        reSizeBuffer(recordSize);
-
-        if (!fillBuffer()) {
-          return null;
-        }
-      }
-
-      for (int i = 0; i < columnTypes.length; i++) {
-        // check if the i'th column is null
-        if (nullFlags.get(i)) {
-          tuple.put(i, DatumFactory.createNullDatum());
-          continue;
-        }
-
-        switch (columnTypes[i].getType()) {
-          case BOOLEAN :
-            tuple.put(i, DatumFactory.createBool(buffer.get()));
-            break;
-
-          case BIT :
-            tuple.put(i, DatumFactory.createBit(buffer.get()));
-            break;
-
-          case CHAR :
-            int realLen = readRawVarint32();
-            byte[] buf = new byte[realLen];
-            buffer.get(buf);
-            tuple.put(i, DatumFactory.createChar(buf));
-            break;
-
-          case INT2 :
-            tuple.put(i, DatumFactory.createInt2(buffer.getShort()));
-            break;
-
-          case INT4 :
-            tuple.put(i, DatumFactory.createInt4(decodeZigZag32(readRawVarint32())));
-            break;
-
-          case INT8 :
-            tuple.put(i, DatumFactory.createInt8(decodeZigZag64(readRawVarint64())));
-            break;
-
-          case FLOAT4 :
-            tuple.put(i, DatumFactory.createFloat4(buffer.getFloat()));
-            break;
-
-          case FLOAT8 :
-            tuple.put(i, DatumFactory.createFloat8(buffer.getDouble()));
-            break;
-
-          case TEXT : {
-            int len = readRawVarint32();
-            byte [] strBytes = new byte[len];
-            buffer.get(strBytes);
-            tuple.put(i, DatumFactory.createText(strBytes));
-            break;
-          }
-
-          case BLOB : {
-            int len = readRawVarint32();
-            byte [] rawBytes = new byte[len];
-            buffer.get(rawBytes);
-            tuple.put(i, DatumFactory.createBlob(rawBytes));
-            break;
-          }
-
-          case PROTOBUF: {
-            int len = readRawVarint32();
-            byte [] rawBytes = new byte[len];
-            buffer.get(rawBytes);
-
-            ProtobufDatumFactory factory = ProtobufDatumFactory.get(columnTypes[i]);
-            Message.Builder builder = factory.newBuilder();
-            builder.mergeFrom(rawBytes);
-            tuple.put(i, factory.createDatum(builder.build()));
-            break;
-          }
-
-          case INET4 :
-            byte [] ipv4Bytes = new byte[4];
-            buffer.get(ipv4Bytes);
-            tuple.put(i, DatumFactory.createInet4(ipv4Bytes));
-            break;
-
-          case DATE: {
-            int val = buffer.getInt();
-            if (val < Integer.MIN_VALUE + 1) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            } else {
-              tuple.put(i, DatumFactory.createFromInt4(columnTypes[i], val));
-            }
-            break;
-          }
-          case TIME:
-          case TIMESTAMP: {
-            long val = buffer.getLong();
-            if (val < Long.MIN_VALUE + 1) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            } else {
-              tuple.put(i, DatumFactory.createFromInt8(columnTypes[i], val));
-            }
-            break;
-          }
-          case NULL_TYPE:
-            tuple.put(i, NullDatum.get());
-            break;
-
-          default:
-        }
-      }
-
-      recordCount++;
-
-      if(filePosition - buffer.remaining() >= endOffset){
-        eos = true;
-      }
-      return new VTuple(tuple);
-    }
-
-    private void reSizeBuffer(int writableBytes){
-      if (buffer.capacity() - buffer.remaining()  <  writableBytes) {
-        buf.setIndex(buffer.position(), buffer.limit());
-        buf.markReaderIndex();
-        buf.discardSomeReadBytes();
-        buf.ensureWritable(writableBytes);
-        buffer = buf.nioBuffer(0, buf.capacity());
-        buffer.limit(buf.writerIndex());
-      }
-    }
-
-    @Override
-    public void reset() throws IOException {
-      // reset the buffer
-      buffer.clear();
-      forceFillBuffer = true;
-      filePosition = fragment.getStartKey();
-      channel.position(filePosition);
-      eos = false;
-    }
-
-    @Override
-    public void close() throws IOException {
-      if(buf != null){
-        buffer.clear();
-        buffer = null;
-
-        buf.release();
-        buf = null;
-      }
-
-      IOUtils.cleanup(LOG, channel, fis);
-    }
-
-    @Override
-    public boolean isProjectable() {
-      return false;
-    }
-
-    @Override
-    public boolean isSelectable() {
-      return false;
-    }
-
-    @Override
-    public boolean isSplittable(){
-      return false;
-    }
-
-    @Override
-    public TableStats getInputStats() {
-      if(tableStats != null){
-        tableStats.setNumRows(recordCount);
-        tableStats.setReadBytes(totalReadBytes); // actual read bytes (scan + rescan * n)
-        tableStats.setNumBytes(fragment.getLength());
-      }
-      return tableStats;
-    }
-
-    @Override
-    public float getProgress() {
-      if(eos) {
-        return 1.0f;
-      }
-
-      if (filePosition - startOffset == 0) {
-        return 0.0f;
-      } else {
-        return Math.min(1.0f, ((float) filePosition / endOffset));
-      }
-    }
-  }
-
-  public static class RawFileAppender extends FileAppender {
-    private FileChannel channel;
-    private RandomAccessFile randomAccessFile;
-    private DataType[] columnTypes;
-
-    private ByteBuffer buffer;
-    private ByteBuf buf;
-    private BitArray nullFlags;
-    private int headerSize = 0;
-    private static final int RECORD_SIZE = 4;
-    private long pos;
-
-    private TableStatistics stats;
-
-    public RawFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
-                           Schema schema, TableMeta meta, Path workDir) throws IOException {
-      super(conf, taskAttemptId, schema, meta, workDir);
-    }
-
-    public void init() throws IOException {
-      File file;
-      try {
-        if (path.toUri().getScheme() != null) {
-          file = new File(path.toUri());
-        } else {
-          file = new File(path.toString());
-        }
-      } catch (IllegalArgumentException iae) {
-        throw new IOException(iae);
-      }
-
-      randomAccessFile = new RandomAccessFile(file, "rw");
-      channel = randomAccessFile.getChannel();
-      pos = 0;
-
-      columnTypes = new DataType[schema.size()];
-      for (int i = 0; i < schema.size(); i++) {
-        columnTypes[i] = schema.getColumn(i).getDataType();
-      }
-
-      buf = BufferPool.directBuffer(64 * StorageUnit.KB);
-      buffer = buf.nioBuffer(0, buf.capacity());
-
-      // comput the number of bytes, representing the null flags
-
-      nullFlags = new BitArray(schema.size());
-      headerSize = RECORD_SIZE + 2 + nullFlags.bytesLength();
-
-      if (enabledStats) {
-        this.stats = new TableStatistics(this.schema);
-      }
-
-      super.init();
-    }
-
-    @Override
-    public long getOffset() throws IOException {
-      return pos;
-    }
-
-    private void flushBuffer() throws IOException {
-      buffer.flip();
-      channel.write(buffer);
-      buffer.clear();
-    }
-
-    private boolean flushBufferAndReplace(int recordOffset, int sizeToBeWritten)
-        throws IOException {
-
-      // if the buffer reaches the limit,
-      // write the bytes from 0 to the previous record.
-      if (buffer.remaining() < sizeToBeWritten) {
-
-        int limit = buffer.position();
-        buffer.limit(recordOffset);
-        buffer.flip();
-        channel.write(buffer);
-        buffer.position(recordOffset);
-        buffer.limit(limit);
-        buffer.compact();
-
-        return true;
-      } else {
-        return false;
-      }
-    }
-
-    /**
-     * Encode a ZigZag-encoded 32-bit value.  ZigZag encodes signed integers
-     * into values that can be efficiently encoded with varint.  (Otherwise,
-     * negative values must be sign-extended to 64 bits to be varint encoded,
-     * thus always taking 10 bytes on the wire.)
-     *
-     * @param n A signed 32-bit integer.
-     * @return An unsigned 32-bit integer, stored in a signed int because
-     *         Java has no explicit unsigned support.
-     */
-    public static int encodeZigZag32(final int n) {
-      // Note:  the right-shift must be arithmetic
-      return (n << 1) ^ (n >> 31);
-    }
-
-    /**
-     * Encode a ZigZag-encoded 64-bit value.  ZigZag encodes signed integers
-     * into values that can be efficiently encoded with varint.  (Otherwise,
-     * negative values must be sign-extended to 64 bits to be varint encoded,
-     * thus always taking 10 bytes on the wire.)
-     *
-     * @param n A signed 64-bit integer.
-     * @return An unsigned 64-bit integer, stored in a signed int because
-     *         Java has no explicit unsigned support.
-     */
-    public static long encodeZigZag64(final long n) {
-      // Note:  the right-shift must be arithmetic
-      return (n << 1) ^ (n >> 63);
-    }
-
-    /**
-     * Encode and write a varint.  {@code value} is treated as
-     * unsigned, so it won't be sign-extended if negative.
-     */
-    public void writeRawVarint32(int value) throws IOException {
-      while (true) {
-        if ((value & ~0x7F) == 0) {
-          buffer.put((byte) value);
-          return;
-        } else {
-          buffer.put((byte) ((value & 0x7F) | 0x80));
-          value >>>= 7;
-        }
-      }
-    }
-
-    /**
-     * Compute the number of bytes that would be needed to encode a varint.
-     * {@code value} is treated as unsigned, so it won't be sign-extended if
-     * negative.
-     */
-    public static int computeRawVarint32Size(final int value) {
-      if ((value & (0xffffffff <<  7)) == 0) return 1;
-      if ((value & (0xffffffff << 14)) == 0) return 2;
-      if ((value & (0xffffffff << 21)) == 0) return 3;
-      if ((value & (0xffffffff << 28)) == 0) return 4;
-      return 5;
-    }
-
-    /** Encode and write a varint. */
-    public void writeRawVarint64(long value) throws IOException {
-      while (true) {
-        if ((value & ~0x7FL) == 0) {
-          buffer.put((byte) value);
-          return;
-        } else {
-          buffer.put((byte) ((value & 0x7F) | 0x80));
-          value >>>= 7;
-        }
-      }
-    }
-
-    @Override
-    public void addTuple(Tuple t) throws IOException {
-
-      if (buffer.remaining() < headerSize) {
-        flushBuffer();
-      }
-
-      // skip the row header
-      int recordOffset = buffer.position();
-      buffer.position(recordOffset + headerSize);
-      // reset the null flags
-      nullFlags.clear();
-      for (int i = 0; i < schema.size(); i++) {
-        if (enabledStats) {
-          stats.analyzeField(i, t.get(i));
-        }
-
-        if (t.isNull(i)) {
-          nullFlags.set(i);
-          continue;
-        }
-
-        // 8 is the maximum bytes size of all types
-        if (flushBufferAndReplace(recordOffset, 8)) {
-          recordOffset = 0;
-        }
-
-        switch(columnTypes[i].getType()) {
-          case NULL_TYPE:
-            nullFlags.set(i);
-            continue;
-
-          case BOOLEAN:
-          case BIT:
-            buffer.put(t.getByte(i));
-            break;
-
-          case INT2 :
-            buffer.putShort(t.getInt2(i));
-            break;
-
-          case INT4 :
-            writeRawVarint32(encodeZigZag32(t.getInt4(i)));
-            break;
-
-          case INT8 :
-            writeRawVarint64(encodeZigZag64(t.getInt8(i)));
-            break;
-
-          case FLOAT4 :
-            buffer.putFloat(t.getFloat4(i));
-            break;
-
-          case FLOAT8 :
-            buffer.putDouble(t.getFloat8(i));
-            break;
-
-          case CHAR:
-          case TEXT: {
-            byte [] strBytes = t.getBytes(i);
-            if (flushBufferAndReplace(recordOffset, strBytes.length + computeRawVarint32Size(strBytes.length))) {
-              recordOffset = 0;
-            }
-            writeRawVarint32(strBytes.length);
-            buffer.put(strBytes);
-            break;
-          }
-
-        case DATE:
-          buffer.putInt(t.getInt4(i));
-          break;
-
-        case TIME:
-        case TIMESTAMP:
-          buffer.putLong(t.getInt8(i));
-          break;
-
-          case BLOB : {
-            byte [] rawBytes = t.getBytes(i);
-            if (flushBufferAndReplace(recordOffset, rawBytes.length + computeRawVarint32Size(rawBytes.length))) {
-              recordOffset = 0;
-            }
-            writeRawVarint32(rawBytes.length);
-            buffer.put(rawBytes);
-            break;
-          }
-
-          case PROTOBUF: {
-            byte [] rawBytes = t.getBytes(i);
-            if (flushBufferAndReplace(recordOffset, rawBytes.length + computeRawVarint32Size(rawBytes.length))) {
-              recordOffset = 0;
-            }
-            writeRawVarint32(rawBytes.length);
-            buffer.put(rawBytes);
-            break;
-          }
-
-          case INET4 :
-            buffer.put(t.getBytes(i));
-            break;
-
-          default:
-            throw new IOException("Cannot support data type: " + columnTypes[i].getType());
-        }
-      }
-
-      // write a record header
-      int bufferPos = buffer.position();
-      buffer.position(recordOffset);
-      buffer.putInt(bufferPos - recordOffset);
-      byte [] flags = nullFlags.toArray();
-      buffer.putShort((short) flags.length);
-      buffer.put(flags);
-
-      pos += bufferPos - recordOffset;
-      buffer.position(bufferPos);
-
-      if (enabledStats) {
-        stats.incrementRow();
-      }
-    }
-
-    @Override
-    public void flush() throws IOException {
-      if(buffer != null){
-        flushBuffer();
-      }
-    }
-
-    @Override
-    public void close() throws IOException {
-      flush();
-      if (enabledStats) {
-        stats.setNumBytes(getOffset());
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RawFileAppender written: " + getOffset() + " bytes, path: " + path);
-      }
-
-      if(buf != null){
-        buffer.clear();
-        buffer = null;
-
-        buf.release();
-        buf = null;
-      }
-
-      IOUtils.cleanup(LOG, channel, randomAccessFile);
-    }
-
-    @Override
-    public TableStats getStats() {
-      if (enabledStats) {
-        stats.setNumBytes(pos);
-        return stats.getTableStat();
-      } else {
-        return null;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java
deleted file mode 100644
index efb23eb..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java
+++ /dev/null
@@ -1,498 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.BitArray;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-
-public class RowFile {
-  public static final Log LOG = LogFactory.getLog(RowFile.class);
-
-  private static final int SYNC_ESCAPE = -1;
-  private static final int SYNC_HASH_SIZE = 16;
-  private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE;
-  private final static int DEFAULT_BUFFER_SIZE = 65535;
-  public static int SYNC_INTERVAL;
-
-  public static class RowFileScanner extends FileScanner {
-    private FileSystem fs;
-    private FSDataInputStream in;
-    private Tuple tuple;
-
-    private byte[] sync = new byte[SYNC_HASH_SIZE];
-    private byte[] checkSync = new byte[SYNC_HASH_SIZE];
-    private long start, end;
-
-    private ByteBuffer buffer;
-    private final int tupleHeaderSize;
-    private BitArray nullFlags;
-    private long bufferStartPos;
-
-    public RowFileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment)
-        throws IOException {
-      super(conf, schema, meta, fragment);
-
-      SYNC_INTERVAL = conf.getInt(ConfVars.ROWFILE_SYNC_INTERVAL.varname,
-          ConfVars.ROWFILE_SYNC_INTERVAL.defaultIntVal) * SYNC_SIZE;
-
-      nullFlags = new BitArray(schema.size());
-      tupleHeaderSize = nullFlags.bytesLength() + (2 * Short.SIZE / 8);
-      this.start = this.fragment.getStartKey();
-      this.end = this.start + this.fragment.getLength();
-    }
-
-    public void init() throws IOException {
-      // set default page size.
-      fs = fragment.getPath().getFileSystem(conf);
-      in = fs.open(fragment.getPath());
-      buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE * schema.size());
-      buffer.flip();
-
-      readHeader();
-
-      // find the correct position from the start
-      if (this.start > in.getPos()) {
-        long realStart = start > SYNC_SIZE ? (start-SYNC_SIZE) : 0;
-        in.seek(realStart);
-      }
-      bufferStartPos = in.getPos();
-      fillBuffer();
-
-      if (start != 0) {
-        // TODO: improve
-        boolean syncFound = false;
-        while (!syncFound) {
-          if (buffer.remaining() < SYNC_SIZE) {
-            fillBuffer();
-          }
-          buffer.mark();
-          syncFound = checkSync();
-          if (!syncFound) {
-            buffer.reset();
-            buffer.get(); // proceed one byte
-          }
-        }
-        bufferStartPos += buffer.position();
-        buffer.compact();
-        buffer.flip();
-      }
-
-      super.init();
-    }
-
-    private void readHeader() throws IOException {
-      SYNC_INTERVAL = in.readInt();
-      StorageUtil.readFully(in, this.sync, 0, SYNC_HASH_SIZE);
-    }
-
-    /**
-     * Find the sync from the front of the buffer
-     *
-     * @return return true if it succeeds to find the sync.
-     * @throws IOException
-     */
-    private boolean checkSync() throws IOException {
-      buffer.getInt();                           // escape
-      buffer.get(checkSync, 0, SYNC_HASH_SIZE);  // sync
-      return Arrays.equals(checkSync, sync);
-    }
-
-    private int fillBuffer() throws IOException {
-      bufferStartPos += buffer.position();
-      buffer.compact();
-      int remain = buffer.remaining();
-      int read = in.read(buffer);
-      if (read == -1) {
-        buffer.flip();
-        return read;
-      } else {
-        int totalRead = read;
-        if (remain > totalRead) {
-          read = in.read(buffer);
-          totalRead += read > 0 ? read : 0;
-        }
-        buffer.flip();
-        return totalRead;
-      }
-    }
-
-    @Override
-    public Tuple next() throws IOException {
-      while (buffer.remaining() < SYNC_SIZE) {
-        if (fillBuffer() < 0) {
-          return null;
-        }
-      }
-
-      buffer.mark();
-      if (!checkSync()) {
-        buffer.reset();
-      } else {
-        if (bufferStartPos + buffer.position() > end) {
-          return null;
-        }
-      }
-
-      while (buffer.remaining() < tupleHeaderSize) {
-        if (fillBuffer() < 0) {
-          return null;
-        }
-      }
-
-      int i;
-      tuple = new VTuple(schema.size());
-
-      int nullFlagSize = buffer.getShort();
-      byte[] nullFlagBytes = new byte[nullFlagSize];
-      buffer.get(nullFlagBytes, 0, nullFlagSize);
-      nullFlags = new BitArray(nullFlagBytes);
-      int tupleSize = buffer.getShort();
-
-      while (buffer.remaining() < (tupleSize)) {
-        if (fillBuffer() < 0) {
-          return null;
-        }
-      }
-
-      Datum datum;
-      Column col;
-      for (i = 0; i < schema.size(); i++) {
-        if (!nullFlags.get(i)) {
-          col = schema.getColumn(i);
-          switch (col.getDataType().getType()) {
-            case BOOLEAN :
-              datum = DatumFactory.createBool(buffer.get());
-              tuple.put(i, datum);
-              break;
-
-            case BIT:
-              datum = DatumFactory.createBit(buffer.get());
-              tuple.put(i, datum );
-              break;
-
-            case CHAR :
-              int realLen = buffer.getInt();
-              byte[] buf = new byte[col.getDataType().getLength()];
-              buffer.get(buf);
-              byte[] charBuf = Arrays.copyOf(buf, realLen);
-              tuple.put(i, DatumFactory.createChar(charBuf));
-              break;
-
-            case INT2 :
-              datum = DatumFactory.createInt2(buffer.getShort());
-              tuple.put(i, datum );
-              break;
-
-            case INT4 :
-              datum = DatumFactory.createInt4(buffer.getInt());
-              tuple.put(i, datum );
-              break;
-
-            case INT8 :
-              datum = DatumFactory.createInt8(buffer.getLong());
-              tuple.put(i, datum );
-              break;
-
-            case FLOAT4 :
-              datum = DatumFactory.createFloat4(buffer.getFloat());
-              tuple.put(i, datum);
-              break;
-
-            case FLOAT8 :
-              datum = DatumFactory.createFloat8(buffer.getDouble());
-              tuple.put(i, datum);
-              break;
-
-            case TEXT:
-              short bytelen = buffer.getShort();
-              byte[] strbytes = new byte[bytelen];
-              buffer.get(strbytes, 0, bytelen);
-              datum = DatumFactory.createText(strbytes);
-              tuple.put(i, datum);
-              break;
-
-            case BLOB:
-              short bytesLen = buffer.getShort();
-              byte [] bytesBuf = new byte[bytesLen];
-              buffer.get(bytesBuf);
-              datum = DatumFactory.createBlob(bytesBuf);
-              tuple.put(i, datum);
-              break;
-
-            case INET4 :
-              byte[] ipv4 = new byte[4];
-              buffer.get(ipv4, 0, 4);
-              datum = DatumFactory.createInet4(ipv4);
-              tuple.put(i, datum);
-              break;
-
-            default:
-              break;
-          }
-        } else {
-          tuple.put(i, DatumFactory.createNullDatum());
-        }
-      }
-      return tuple;
-    }
-
-    @Override
-    public void reset() throws IOException {
-      init();
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (in != null) {
-        in.close();
-      }
-    }
-
-    @Override
-    public boolean isProjectable() {
-      return false;
-    }
-
-    @Override
-    public boolean isSelectable() {
-      return false;
-    }
-
-    @Override
-    public boolean isSplittable(){
-      return true;
-    }
-  }
-
-  public static class RowFileAppender extends FileAppender {
-    private FSDataOutputStream out;
-    private long lastSyncPos;
-    private FileSystem fs;
-    private byte[] sync;
-    private ByteBuffer buffer;
-
-    private BitArray nullFlags;
-    // statistics
-    private TableStatistics stats;
-
-    public RowFileAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
-                           final Schema schema, final TableMeta meta, final Path workDir)
-        throws IOException {
-      super(conf, taskAttemptId, schema, meta, workDir);
-    }
-
-    public void init() throws IOException {
-      SYNC_INTERVAL = conf.getInt(ConfVars.ROWFILE_SYNC_INTERVAL.varname,
-          ConfVars.ROWFILE_SYNC_INTERVAL.defaultIntVal);
-      fs = path.getFileSystem(conf);
-
-      if (!fs.exists(path.getParent())) {
-        throw new FileNotFoundException(path.toString());
-      }
-
-      if (fs.exists(path)) {
-        throw new AlreadyExistsStorageException(path);
-      }
-
-      sync = new byte[SYNC_HASH_SIZE];
-      lastSyncPos = 0;
-
-      out = fs.create(path);
-
-      MessageDigest md;
-      try {
-        md = MessageDigest.getInstance("MD5");
-        md.update((path.toString()+System.currentTimeMillis()).getBytes());
-        sync = md.digest();
-      } catch (NoSuchAlgorithmException e) {
-        LOG.error(e);
-      }
-
-      writeHeader();
-
-      buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
-
-      nullFlags = new BitArray(schema.size());
-
-      if (enabledStats) {
-        this.stats = new TableStatistics(this.schema);
-      }
-    }
-
-    private void writeHeader() throws IOException {
-      out.writeInt(SYNC_INTERVAL);
-      out.write(sync);
-      out.flush();
-      lastSyncPos = out.getPos();
-    }
-
-    @Override
-    public void addTuple(Tuple t) throws IOException {
-      checkAndWriteSync();
-      Column col;
-
-      buffer.clear();
-      nullFlags.clear();
-
-      for (int i = 0; i < schema.size(); i++) {
-        if (enabledStats) {
-          stats.analyzeField(i, t.get(i));
-        }
-
-        if (t.isNull(i)) {
-          nullFlags.set(i);
-        } else {
-          col = schema.getColumn(i);
-          switch (col.getDataType().getType()) {
-            case BOOLEAN:
-              buffer.put(t.get(i).asByte());
-              break;
-            case BIT:
-              buffer.put(t.get(i).asByte());
-              break;
-            case CHAR:
-              byte[] src = t.get(i).asByteArray();
-              byte[] dst = Arrays.copyOf(src, col.getDataType().getLength());
-              buffer.putInt(src.length);
-              buffer.put(dst);
-              break;
-            case TEXT:
-              byte [] strbytes = t.get(i).asByteArray();
-              buffer.putShort((short)strbytes.length);
-              buffer.put(strbytes, 0, strbytes.length);
-              break;
-            case INT2:
-              buffer.putShort(t.get(i).asInt2());
-              break;
-            case INT4:
-              buffer.putInt(t.get(i).asInt4());
-              break;
-            case INT8:
-              buffer.putLong(t.get(i).asInt8());
-              break;
-            case FLOAT4:
-              buffer.putFloat(t.get(i).asFloat4());
-              break;
-            case FLOAT8:
-              buffer.putDouble(t.get(i).asFloat8());
-              break;
-            case BLOB:
-              byte [] bytes = t.get(i).asByteArray();
-              buffer.putShort((short)bytes.length);
-              buffer.put(bytes);
-              break;
-            case INET4:
-              buffer.put(t.get(i).asByteArray());
-              break;
-            case INET6:
-              buffer.put(t.get(i).asByteArray());
-              break;
-            case NULL_TYPE:
-              nullFlags.set(i);
-              break;
-            default:
-              break;
-          }
-        }
-      }
-
-      byte[] bytes = nullFlags.toArray();
-      out.writeShort(bytes.length);
-      out.write(bytes);
-
-      bytes = buffer.array();
-      int dataLen = buffer.position();
-      out.writeShort(dataLen);
-      out.write(bytes, 0, dataLen);
-
-      // Statistical section
-      if (enabledStats) {
-        stats.incrementRow();
-      }
-    }
-
-    @Override
-    public long getOffset() throws IOException {
-      return out.getPos();
-    }
-
-    @Override
-    public void flush() throws IOException {
-      out.flush();
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (out != null) {
-        if (enabledStats) {
-          stats.setNumBytes(out.getPos());
-        }
-        sync();
-        out.flush();
-        out.close();
-      }
-    }
-
-    private void sync() throws IOException {
-      if (lastSyncPos != out.getPos()) {
-        out.writeInt(SYNC_ESCAPE);
-        out.write(sync);
-        lastSyncPos = out.getPos();
-      }
-    }
-
-    private void checkAndWriteSync() throws IOException {
-      if (out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
-        sync();
-      }
-    }
-
-    @Override
-    public TableStats getStats() {
-      if (enabledStats) {
-        return stats.getTableStat();
-      } else {
-        return null;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
deleted file mode 100644
index 24b6280..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
+++ /dev/null
@@ -1,377 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.IntervalDatum;
-import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.exception.UnsupportedException;
-import org.apache.tajo.storage.exception.UnknownDataTypeException;
-import org.apache.tajo.tuple.offheap.RowWriter;
-import org.apache.tajo.util.BitArray;
-
-import java.nio.ByteBuffer;
-
-public class RowStoreUtil {
-  public static int[] getTargetIds(Schema inSchema, Schema outSchema) {
-    int[] targetIds = new int[outSchema.size()];
-    int i = 0;
-    for (Column target : outSchema.getColumns()) {
-      targetIds[i] = inSchema.getColumnId(target.getQualifiedName());
-      i++;
-    }
-
-    return targetIds;
-  }
-
-  public static Tuple project(Tuple in, Tuple out, int[] targetIds) {
-    out.clear();
-    for (int idx = 0; idx < targetIds.length; idx++) {
-      out.put(idx, in.get(targetIds[idx]));
-    }
-    return out;
-  }
-
-  public static RowStoreEncoder createEncoder(Schema schema) {
-    return new RowStoreEncoder(schema);
-  }
-
-  public static RowStoreDecoder createDecoder(Schema schema) {
-    return new RowStoreDecoder(schema);
-  }
-
-  public static class RowStoreDecoder {
-
-    private Schema schema;
-    private BitArray nullFlags;
-    private int headerSize;
-
-    private RowStoreDecoder(Schema schema) {
-      this.schema = schema;
-      nullFlags = new BitArray(schema.size());
-      headerSize = nullFlags.bytesLength();
-    }
-
-
-    public Tuple toTuple(byte [] bytes) {
-      nullFlags.clear();
-      ByteBuffer bb = ByteBuffer.wrap(bytes);
-      Tuple tuple = new VTuple(schema.size());
-      Column col;
-      TajoDataTypes.DataType type;
-
-      bb.limit(headerSize);
-      nullFlags.fromByteBuffer(bb);
-      bb.limit(bytes.length);
-
-      for (int i =0; i < schema.size(); i++) {
-        if (nullFlags.get(i)) {
-          tuple.put(i, DatumFactory.createNullDatum());
-          continue;
-        }
-
-        col = schema.getColumn(i);
-        type = col.getDataType();
-        switch (type.getType()) {
-          case BOOLEAN: tuple.put(i, DatumFactory.createBool(bb.get())); break;
-          case BIT:
-            byte b = bb.get();
-            tuple.put(i, DatumFactory.createBit(b));
-            break;
-
-          case CHAR:
-            byte c = bb.get();
-            tuple.put(i, DatumFactory.createChar(c));
-            break;
-
-          case INT2:
-            short s = bb.getShort();
-            tuple.put(i, DatumFactory.createInt2(s));
-            break;
-
-          case INT4:
-          case DATE:
-            int i_ = bb.getInt();
-            tuple.put(i, DatumFactory.createFromInt4(type, i_));
-            break;
-
-          case INT8:
-          case TIME:
-          case TIMESTAMP:
-            long l = bb.getLong();
-            tuple.put(i, DatumFactory.createFromInt8(type, l));
-            break;
-
-        case INTERVAL:
-            int month  = bb.getInt();
-            long milliseconds  = bb.getLong();
-            tuple.put(i, new IntervalDatum(month, milliseconds));
-            break;
-
-          case FLOAT4:
-            float f = bb.getFloat();
-            tuple.put(i, DatumFactory.createFloat4(f));
-            break;
-
-          case FLOAT8:
-            double d = bb.getDouble();
-            tuple.put(i, DatumFactory.createFloat8(d));
-            break;
-
-          case TEXT:
-            byte [] _string = new byte[bb.getInt()];
-            bb.get(_string);
-            tuple.put(i, DatumFactory.createText(_string));
-            break;
-
-          case BLOB:
-            byte [] _bytes = new byte[bb.getInt()];
-            bb.get(_bytes);
-            tuple.put(i, DatumFactory.createBlob(_bytes));
-            break;
-
-          case INET4:
-            byte [] _ipv4 = new byte[4];
-            bb.get(_ipv4);
-            tuple.put(i, DatumFactory.createInet4(_ipv4));
-            break;
-          case INET6:
-            // TODO - to be implemented
-            throw new UnsupportedException(type.getType().name());
-          default:
-            throw new RuntimeException(new UnknownDataTypeException(type.getType().name()));
-        }
-      }
-      return tuple;
-    }
-
-    public Schema getSchema() {
-      return schema;
-    }
-  }
-
-  public static class RowStoreEncoder {
-    private Schema schema;
-    private BitArray nullFlags;
-    private int headerSize;
-
-    private RowStoreEncoder(Schema schema) {
-      this.schema = schema;
-      nullFlags = new BitArray(schema.size());
-      headerSize = nullFlags.bytesLength();
-    }
-
-    public byte[] toBytes(Tuple tuple) {
-      nullFlags.clear();
-      int size = estimateTupleDataSize(tuple);
-      ByteBuffer bb = ByteBuffer.allocate(size + headerSize);
-      bb.position(headerSize);
-      Column col;
-      for (int i = 0; i < schema.size(); i++) {
-        if (tuple.isNull(i)) {
-          nullFlags.set(i);
-          continue;
-        }
-
-        col = schema.getColumn(i);
-        switch (col.getDataType().getType()) {
-        case NULL_TYPE:
-          nullFlags.set(i);
-          break;
-        case BOOLEAN:
-          bb.put(tuple.get(i).asByte());
-          break;
-        case BIT:
-          bb.put(tuple.get(i).asByte());
-          break;
-        case CHAR:
-          bb.put(tuple.get(i).asByte());
-          break;
-        case INT2:
-          bb.putShort(tuple.get(i).asInt2());
-          break;
-        case INT4:
-          bb.putInt(tuple.get(i).asInt4());
-          break;
-        case INT8:
-          bb.putLong(tuple.get(i).asInt8());
-          break;
-        case FLOAT4:
-          bb.putFloat(tuple.get(i).asFloat4());
-          break;
-        case FLOAT8:
-          bb.putDouble(tuple.get(i).asFloat8());
-          break;
-        case TEXT:
-          byte[] _string = tuple.get(i).asByteArray();
-          bb.putInt(_string.length);
-          bb.put(_string);
-          break;
-        case DATE:
-          bb.putInt(tuple.get(i).asInt4());
-          break;
-        case TIME:
-        case TIMESTAMP:
-          bb.putLong(tuple.get(i).asInt8());
-          break;
-        case INTERVAL:
-          IntervalDatum interval = (IntervalDatum) tuple.get(i);
-          bb.putInt(interval.getMonths());
-          bb.putLong(interval.getMilliSeconds());
-          break;
-        case BLOB:
-          byte[] bytes = tuple.get(i).asByteArray();
-          bb.putInt(bytes.length);
-          bb.put(bytes);
-          break;
-        case INET4:
-          byte[] ipBytes = tuple.get(i).asByteArray();
-          bb.put(ipBytes);
-          break;
-        case INET6:
-          bb.put(tuple.get(i).asByteArray());
-          break;
-        default:
-          throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
-        }
-      }
-
-      byte[] flags = nullFlags.toArray();
-      int finalPosition = bb.position();
-      bb.position(0);
-      bb.put(flags);
-
-      bb.position(finalPosition);
-      bb.flip();
-      byte[] buf = new byte[bb.limit()];
-      bb.get(buf);
-      return buf;
-    }
-
-    // Note that, NULL values are treated separately
-    private int estimateTupleDataSize(Tuple tuple) {
-      int size = 0;
-      Column col;
-
-      for (int i = 0; i < schema.size(); i++) {
-        if (tuple.isNull(i)) {
-          continue;
-        }
-
-        col = schema.getColumn(i);
-        switch (col.getDataType().getType()) {
-        case BOOLEAN:
-        case BIT:
-        case CHAR:
-          size += 1;
-          break;
-        case INT2:
-          size += 2;
-          break;
-        case DATE:
-        case INT4:
-        case FLOAT4:
-          size += 4;
-          break;
-        case TIME:
-        case TIMESTAMP:
-        case INT8:
-        case FLOAT8:
-          size += 8;
-          break;
-        case INTERVAL:
-          size += 12;
-          break;
-        case TEXT:
-        case BLOB:
-          size += (4 + tuple.get(i).asByteArray().length);
-          break;
-        case INET4:
-        case INET6:
-          size += tuple.get(i).asByteArray().length;
-          break;
-        default:
-          throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
-        }
-      }
-
-      size += 100; // optimistic reservation
-
-      return size;
-    }
-
-    public Schema getSchema() {
-      return schema;
-    }
-  }
-
-  public static void convert(Tuple tuple, RowWriter writer) {
-    writer.startRow();
-
-    for (int i = 0; i < writer.dataTypes().length; i++) {
-      if (tuple.isNull(i)) {
-        writer.skipField();
-        continue;
-      }
-      switch (writer.dataTypes()[i].getType()) {
-      case BOOLEAN:
-        writer.putBool(tuple.getBool(i));
-        break;
-      case INT1:
-      case INT2:
-        writer.putInt2(tuple.getInt2(i));
-        break;
-      case INT4:
-      case DATE:
-      case INET4:
-        writer.putInt4(tuple.getInt4(i));
-        break;
-      case INT8:
-      case TIMESTAMP:
-      case TIME:
-        writer.putInt8(tuple.getInt8(i));
-        break;
-      case FLOAT4:
-        writer.putFloat4(tuple.getFloat4(i));
-        break;
-      case FLOAT8:
-        writer.putFloat8(tuple.getFloat8(i));
-        break;
-      case TEXT:
-        writer.putText(tuple.getBytes(i));
-        break;
-      case INTERVAL:
-        writer.putInterval((IntervalDatum) tuple.getInterval(i));
-        break;
-      case PROTOBUF:
-        writer.putProtoDatum((ProtobufDatum) tuple.getProtobufDatum(i));
-        break;
-      case NULL_TYPE:
-        writer.skipField();
-        break;
-      default:
-        throw new UnsupportedException("Unknown data type: " + writer.dataTypes()[i]);
-      }
-    }
-    writer.endRow();
-  }
-}


[03/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
new file mode 100644
index 0000000..9c167a0
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
@@ -0,0 +1,146 @@
+/**
+ * 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.thirdparty.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import parquet.filter.UnboundRecordFilter;
+import parquet.hadoop.Footer;
+import parquet.hadoop.ParquetFileReader;
+import parquet.hadoop.api.InitContext;
+import parquet.hadoop.api.ReadSupport;
+import parquet.hadoop.api.ReadSupport.ReadContext;
+import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.metadata.GlobalMetaData;
+import parquet.schema.MessageType;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Read records from a Parquet file.
+ */
+public class ParquetReader<T> implements Closeable {
+
+  private ReadSupport<T> readSupport;
+  private UnboundRecordFilter filter;
+  private Configuration conf;
+  private ReadContext readContext;
+  private Iterator<Footer> footersIterator;
+  private InternalParquetRecordReader<T> reader;
+  private GlobalMetaData globalMetaData;
+
+  /**
+   * @param file the file to read
+   * @param readSupport to materialize records
+   * @throws java.io.IOException
+   */
+  public ParquetReader(Path file, ReadSupport<T> readSupport) throws IOException {
+    this(file, readSupport, null);
+  }
+
+  /**
+   * @param conf the configuration
+   * @param file the file to read
+   * @param readSupport to materialize records
+   * @throws java.io.IOException
+   */
+  public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport) throws IOException {
+    this(conf, file, readSupport, null);
+  }
+
+  /**
+   * @param file the file to read
+   * @param readSupport to materialize records
+   * @param filter the filter to use to filter records
+   * @throws java.io.IOException
+   */
+  public ParquetReader(Path file, ReadSupport<T> readSupport, UnboundRecordFilter filter) throws IOException {
+    this(new Configuration(), file, readSupport, filter);
+  }
+
+  /**
+   * @param conf the configuration
+   * @param file the file to read
+   * @param readSupport to materialize records
+   * @param filter the filter to use to filter records
+   * @throws java.io.IOException
+   */
+  public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport, UnboundRecordFilter filter) throws IOException {
+    this.readSupport = readSupport;
+    this.filter = filter;
+    this.conf = conf;
+
+    FileSystem fs = file.getFileSystem(conf);
+    List<FileStatus> statuses = Arrays.asList(fs.listStatus(file));
+    List<Footer> footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(conf, statuses);
+    this.footersIterator = footers.iterator();
+    globalMetaData = ParquetFileWriter.getGlobalMetaData(footers);
+
+    List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
+    for (Footer footer : footers) {
+      blocks.addAll(footer.getParquetMetadata().getBlocks());
+    }
+
+    MessageType schema = globalMetaData.getSchema();
+    Map<String, Set<String>> extraMetadata = globalMetaData.getKeyValueMetaData();
+    readContext = readSupport.init(new InitContext(conf, extraMetadata, schema));
+  }
+
+  /**
+   * @return the next record or null if finished
+   * @throws java.io.IOException
+   */
+  public T read() throws IOException {
+    try {
+      if (reader != null && reader.nextKeyValue()) {
+        return reader.getCurrentValue();
+      } else {
+        initReader();
+        return reader == null ? null : read();
+      }
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private void initReader() throws IOException {
+    if (reader != null) {
+      reader.close();
+      reader = null;
+    }
+    if (footersIterator.hasNext()) {
+      Footer footer = footersIterator.next();
+      reader = new InternalParquetRecordReader<T>(readSupport, filter);
+      reader.initialize(
+          readContext.getRequestedSchema(), globalMetaData.getSchema(), footer.getParquetMetadata().getFileMetaData().getKeyValueMetaData(),
+          readContext.getReadSupportMetadata(), footer.getFile(), footer.getParquetMetadata().getBlocks(), conf);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (reader != null) {
+      reader.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
new file mode 100644
index 0000000..7527437
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
@@ -0,0 +1,224 @@
+/**
+ * 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.thirdparty.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import parquet.column.ParquetProperties;
+import parquet.hadoop.api.WriteSupport;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.schema.MessageType;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public class ParquetWriter<T> implements Closeable {
+
+  public static final int DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024;
+  public static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024;
+  public static final CompressionCodecName DEFAULT_COMPRESSION_CODEC_NAME =
+      CompressionCodecName.UNCOMPRESSED;
+  public static final boolean DEFAULT_IS_DICTIONARY_ENABLED = true;
+  public static final boolean DEFAULT_IS_VALIDATING_ENABLED = false;
+  public static final ParquetProperties.WriterVersion DEFAULT_WRITER_VERSION =
+      ParquetProperties.WriterVersion.PARQUET_1_0;
+
+  private final InternalParquetRecordWriter<T> writer;
+
+  /**
+   * Create a new ParquetWriter.
+   * (with dictionary encoding enabled and validation off)
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold
+   * @throws java.io.IOException
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, boolean, boolean)
+   */
+  public ParquetWriter(Path file, WriteSupport<T> writeSupport, CompressionCodecName compressionCodecName, int blockSize, int pageSize) throws IOException {
+    this(file, writeSupport, compressionCodecName, blockSize, pageSize,
+        DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED);
+  }
+
+  /**
+   * Create a new ParquetWriter.
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold (both data and dictionary)
+   * @param enableDictionary to turn dictionary encoding on
+   * @param validating to turn on validation using the schema
+   * @throws java.io.IOException
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, int, boolean, boolean)
+   */
+  public ParquetWriter(
+      Path file,
+      WriteSupport<T> writeSupport,
+      CompressionCodecName compressionCodecName,
+      int blockSize,
+      int pageSize,
+      boolean enableDictionary,
+      boolean validating) throws IOException {
+    this(file, writeSupport, compressionCodecName, blockSize, pageSize, pageSize, enableDictionary, validating);
+  }
+
+  /**
+   * Create a new ParquetWriter.
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold
+   * @param dictionaryPageSize the page size threshold for the dictionary pages
+   * @param enableDictionary to turn dictionary encoding on
+   * @param validating to turn on validation using the schema
+   * @throws java.io.IOException
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion)
+   */
+  public ParquetWriter(
+      Path file,
+      WriteSupport<T> writeSupport,
+      CompressionCodecName compressionCodecName,
+      int blockSize,
+      int pageSize,
+      int dictionaryPageSize,
+      boolean enableDictionary,
+      boolean validating) throws IOException {
+    this(file, writeSupport, compressionCodecName, blockSize, pageSize,
+        dictionaryPageSize, enableDictionary, validating,
+        DEFAULT_WRITER_VERSION);
+  }
+
+  /**
+   * Create a new ParquetWriter.
+   *
+   * Directly instantiates a Hadoop {@link org.apache.hadoop.conf.Configuration} which reads
+   * configuration from the classpath.
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold
+   * @param dictionaryPageSize the page size threshold for the dictionary pages
+   * @param enableDictionary to turn dictionary encoding on
+   * @param validating to turn on validation using the schema
+   * @param writerVersion version of parquetWriter from {@link parquet.column.ParquetProperties.WriterVersion}
+   * @throws java.io.IOException
+   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, parquet.hadoop.metadata.CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion, org.apache.hadoop.conf.Configuration)
+   */
+  public ParquetWriter(
+      Path file,
+      WriteSupport<T> writeSupport,
+      CompressionCodecName compressionCodecName,
+      int blockSize,
+      int pageSize,
+      int dictionaryPageSize,
+      boolean enableDictionary,
+      boolean validating,
+      ParquetProperties.WriterVersion writerVersion) throws IOException {
+    this(file, writeSupport, compressionCodecName, blockSize, pageSize, dictionaryPageSize, enableDictionary, validating, writerVersion, new Configuration());
+  }
+
+  /**
+   * Create a new ParquetWriter.
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold
+   * @param dictionaryPageSize the page size threshold for the dictionary pages
+   * @param enableDictionary to turn dictionary encoding on
+   * @param validating to turn on validation using the schema
+   * @param writerVersion version of parquetWriter from {@link parquet.column.ParquetProperties.WriterVersion}
+   * @param conf Hadoop configuration to use while accessing the filesystem
+   * @throws java.io.IOException
+   */
+  public ParquetWriter(
+      Path file,
+      WriteSupport<T> writeSupport,
+      CompressionCodecName compressionCodecName,
+      int blockSize,
+      int pageSize,
+      int dictionaryPageSize,
+      boolean enableDictionary,
+      boolean validating,
+      ParquetProperties.WriterVersion writerVersion,
+      Configuration conf) throws IOException {
+
+    WriteSupport.WriteContext writeContext = writeSupport.init(conf);
+    MessageType schema = writeContext.getSchema();
+
+    ParquetFileWriter fileWriter = new ParquetFileWriter(conf, schema, file);
+    fileWriter.start();
+
+    CodecFactory codecFactory = new CodecFactory(conf);
+    CodecFactory.BytesCompressor compressor =	codecFactory.getCompressor(compressionCodecName, 0);
+    this.writer = new InternalParquetRecordWriter<T>(
+        fileWriter,
+        writeSupport,
+        schema,
+        writeContext.getExtraMetaData(),
+        blockSize,
+        pageSize,
+        compressor,
+        dictionaryPageSize,
+        enableDictionary,
+        validating,
+        writerVersion);
+  }
+
+  /**
+   * Create a new ParquetWriter.  The default block size is 50 MB.The default
+   * page size is 1 MB.  Default compression is no compression. Dictionary encoding is disabled.
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @throws java.io.IOException
+   */
+  public ParquetWriter(Path file, WriteSupport<T> writeSupport) throws IOException {
+    this(file, writeSupport, DEFAULT_COMPRESSION_CODEC_NAME, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
+  }
+
+  public void write(T object) throws IOException {
+    try {
+      writer.write(object);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  public long getEstimatedWrittenSize() throws IOException {
+    return this.writer.getEstimatedWrittenSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      writer.close();
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/proto/StorageFragmentProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/proto/StorageFragmentProtos.proto b/tajo-storage/tajo-storage-hdfs/src/main/proto/StorageFragmentProtos.proto
new file mode 100644
index 0000000..ce9aab6
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/proto/StorageFragmentProtos.proto
@@ -0,0 +1,34 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.tajo.storage";
+option java_outer_classname = "StorageFragmentProtos";
+option optimize_for = SPEED;
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
+
+import "CatalogProtos.proto";
+
+message FileFragmentProto {
+  required string id = 1;
+  required string path = 2;
+  required int64 startOffset = 3;
+  required int64 length = 4;
+  repeated string hosts = 5;
+  repeated int32 diskIds = 6;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServer.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServer.java
new file mode 100644
index 0000000..cf8a54e
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServer.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.ChannelGroupFuture;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.Executors;
+
+public class HttpFileServer {
+  private final static Log LOG = LogFactory.getLog(HttpFileServer.class);
+
+  private final InetSocketAddress addr;
+  private InetSocketAddress bindAddr;
+  private ServerBootstrap bootstrap = null;
+  private ChannelFactory factory = null;
+  private ChannelGroup channelGroup = null;
+
+  public HttpFileServer(final InetSocketAddress addr) {
+    this.addr = addr;
+    this.factory = new NioServerSocketChannelFactory(
+        Executors.newCachedThreadPool(), Executors.newCachedThreadPool(),
+        2);
+
+    // Configure the server.
+    this.bootstrap = new ServerBootstrap(factory);
+    // Set up the event pipeline factory.
+    this.bootstrap.setPipelineFactory(new HttpFileServerPipelineFactory());
+    this.channelGroup = new DefaultChannelGroup();
+  }
+
+  public HttpFileServer(String bindaddr) {
+    this(NetUtils.createSocketAddr(bindaddr));
+  }
+
+  public void start() {
+    // Bind and start to accept incoming connections.
+    Channel channel = bootstrap.bind(addr);
+    channelGroup.add(channel);    
+    this.bindAddr = (InetSocketAddress) channel.getLocalAddress();
+    LOG.info("HttpFileServer starts up ("
+        + this.bindAddr.getAddress().getHostAddress() + ":" + this.bindAddr.getPort()
+        + ")");
+  }
+  
+  public InetSocketAddress getBindAddress() {
+    return this.bindAddr;
+  }
+
+  public void stop() {
+    ChannelGroupFuture future = channelGroup.close();
+    future.awaitUninterruptibly();
+    factory.releaseExternalResources();
+
+    LOG.info("HttpFileServer shutdown ("
+        + this.bindAddr.getAddress().getHostAddress() + ":"
+        + this.bindAddr.getPort() + ")");
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerHandler.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerHandler.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerHandler.java
new file mode 100644
index 0000000..6c77317
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerHandler.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedFile;
+import org.jboss.netty.util.CharsetUtil;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.RandomAccessFile;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+/**
+ * this is an implementation copied from HttpStaticFileServerHandler.java of netty 3.6
+ */
+public class HttpFileServerHandler extends SimpleChannelUpstreamHandler {
+
+  @Override
+  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
+    HttpRequest request = (HttpRequest) e.getMessage();
+    if (request.getMethod() != GET) {
+      sendError(ctx, METHOD_NOT_ALLOWED);
+      return;
+    }
+
+    final String path = sanitizeUri(request.getUri());
+    if (path == null) {
+      sendError(ctx, FORBIDDEN);
+      return;
+    }
+
+    File file = new File(path);
+    if (file.isHidden() || !file.exists()) {
+      sendError(ctx, NOT_FOUND);
+      return;
+    }
+    if (!file.isFile()) {
+      sendError(ctx, FORBIDDEN);
+      return;
+    }
+
+    RandomAccessFile raf;
+    try {
+      raf = new RandomAccessFile(file, "r");
+    } catch (FileNotFoundException fnfe) {
+      sendError(ctx, NOT_FOUND);
+      return;
+    }
+    long fileLength = raf.length();
+
+    HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+    setContentLength(response, fileLength);
+    setContentTypeHeader(response);
+
+    Channel ch = e.getChannel();
+
+    // Write the initial line and the header.
+    ch.write(response);
+
+    // Write the content.
+    ChannelFuture writeFuture;
+    if (ch.getPipeline().get(SslHandler.class) != null) {
+      // Cannot use zero-copy with HTTPS.
+      writeFuture = ch.write(new ChunkedFile(raf, 0, fileLength, 8192));
+    } else {
+      // No encryption - use zero-copy.
+      final FileRegion region =
+          new DefaultFileRegion(raf.getChannel(), 0, fileLength);
+      writeFuture = ch.write(region);
+      writeFuture.addListener(new ChannelFutureProgressListener() {
+        public void operationComplete(ChannelFuture future) {
+          region.releaseExternalResources();
+        }
+
+        public void operationProgressed(
+            ChannelFuture future, long amount, long current, long total) {
+          System.out.printf("%s: %d / %d (+%d)%n", path, current, total, amount);
+        }
+      });
+    }
+
+    // Decide whether to close the connection or not.
+    if (!isKeepAlive(request)) {
+      // Close the connection when the whole content is written out.
+      writeFuture.addListener(ChannelFutureListener.CLOSE);
+    }
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+      throws Exception {
+    Channel ch = e.getChannel();
+    Throwable cause = e.getCause();
+    if (cause instanceof TooLongFrameException) {
+      sendError(ctx, BAD_REQUEST);
+      return;
+    }
+
+    cause.printStackTrace();
+    if (ch.isConnected()) {
+      sendError(ctx, INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  private static String sanitizeUri(String uri) {
+    // Decode the path.
+    try {
+      uri = URLDecoder.decode(uri, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      try {
+        uri = URLDecoder.decode(uri, "ISO-8859-1");
+      } catch (UnsupportedEncodingException e1) {
+        throw new Error();
+      }
+    }
+
+    // Convert file separators.
+    uri = uri.replace('/', File.separatorChar);
+
+    // Simplistic dumb security check.
+    // You will have to do something serious in the production environment.
+    if (uri.contains(File.separator + '.') ||
+        uri.contains('.' + File.separator) ||
+        uri.startsWith(".") || uri.endsWith(".")) {
+      return null;
+    }
+
+    return uri;
+  }
+
+  private static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
+    HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+    response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+    response.setContent(ChannelBuffers.copiedBuffer(
+        "Failure: " + status.toString() + "\r\n",
+        CharsetUtil.UTF_8));
+
+    // Close the connection as soon as the error message is sent.
+    ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+  }
+
+  /**
+   * Sets the content type header for the HTTP Response
+   *
+   * @param response
+   *            HTTP response
+   */
+  private static void setContentTypeHeader(HttpResponse response) {
+    response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
new file mode 100644
index 0000000..cecf93b
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+
+import static org.jboss.netty.channel.Channels.pipeline;
+
+// Uncomment the following lines if you want HTTPS
+//import javax.net.ssl.SSLEngine;
+//import org.jboss.netty.example.securechat.SecureChatSslContextFactory;
+//import org.jboss.netty.handler.ssl.SslHandler;
+
+//this class is copied from HttpStaticFileServerPipelineFactory.java of netty 3.6
+public class HttpFileServerPipelineFactory implements ChannelPipelineFactory {
+  public ChannelPipeline getPipeline() throws Exception {
+    // Create a default pipeline implementation.
+    ChannelPipeline pipeline = pipeline();
+
+    // Uncomment the following lines if you want HTTPS
+    //SSLEngine engine = SecureChatSslContextFactory.getServerContext().createSSLEngine();
+    //engine.setUseClientMode(false);
+    //pipeline.addLast("ssl", new SslHandler(engine));
+
+    pipeline.addLast("decoder", new HttpRequestDecoder());
+    pipeline.addLast("aggregator", new HttpChunkAggregator(65536));
+    pipeline.addLast("encoder", new HttpResponseEncoder());
+    pipeline.addLast("chunkedWriter", new ChunkedWriteHandler());
+
+    pipeline.addLast("handler", new HttpFileServerHandler());
+    return pipeline;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..3c78d6b
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.compress.*;
+import org.apache.hadoop.io.compress.zlib.ZlibFactory;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.sequencefile.SequenceFileScanner;
+import org.apache.tajo.storage.text.DelimitedTextFile;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class TestCompressionStorages {
+  private TajoConf conf;
+  private static String TEST_PATH = "target/test-data/TestCompressionStorages";
+
+  private StoreType storeType;
+  private Path testDir;
+  private FileSystem fs;
+
+  public TestCompressionStorages(StoreType type) throws IOException {
+    this.storeType = type;
+    conf = new TajoConf();
+
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return Arrays.asList(new Object[][]{
+        {StoreType.CSV},
+        {StoreType.RCFILE},
+        {StoreType.SEQUENCEFILE},
+        {StoreType.TEXTFILE}
+    });
+  }
+
+  @Test
+  public void testDeflateCodecCompressionData() throws IOException {
+    storageCompressionTest(storeType, DeflateCodec.class);
+  }
+
+  @Test
+  public void testGzipCodecCompressionData() throws IOException {
+    if (storeType == StoreType.RCFILE) {
+      if( ZlibFactory.isNativeZlibLoaded(conf)) {
+        storageCompressionTest(storeType, GzipCodec.class);
+      }
+    } else if (storeType == StoreType.SEQUENCEFILE) {
+      if( ZlibFactory.isNativeZlibLoaded(conf)) {
+        storageCompressionTest(storeType, GzipCodec.class);
+      }
+    } else {
+      storageCompressionTest(storeType, GzipCodec.class);
+    }
+  }
+
+  @Test
+  public void testSnappyCodecCompressionData() throws IOException {
+    if (SnappyCodec.isNativeCodeLoaded()) {
+      storageCompressionTest(storeType, SnappyCodec.class);
+    }
+  }
+
+  @Test
+  public void testLz4CodecCompressionData() throws IOException {
+    if(NativeCodeLoader.isNativeCodeLoaded() && Lz4Codec.isNativeCodeLoaded())
+    storageCompressionTest(storeType, Lz4Codec.class);
+  }
+
+  private void storageCompressionTest(StoreType storeType, Class<? extends CompressionCodec> codec) throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.FLOAT4);
+    schema.addColumn("name", Type.TEXT);
+
+    TableMeta meta = CatalogUtil.newTableMeta(storeType);
+    meta.putOption("compression.codec", codec.getCanonicalName());
+    meta.putOption("compression.type", SequenceFile.CompressionType.BLOCK.name());
+    meta.putOption("rcfile.serde", TextSerializerDeserializer.class.getName());
+    meta.putOption("sequencefile.serde", TextSerializerDeserializer.class.getName());
+
+    String fileName = "Compression_" + codec.getSimpleName();
+    Path tablePath = new Path(testDir, fileName);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.enableStats();
+
+    appender.init();
+
+    String extension = "";
+    if (appender instanceof CSVFile.CSVAppender) {
+      extension = ((CSVFile.CSVAppender) appender).getExtension();
+    } else if (appender instanceof DelimitedTextFile.DelimitedTextFileAppender) {
+      extension = ((DelimitedTextFile.DelimitedTextFileAppender) appender).getExtension();
+    }
+
+    int tupleNum = 100000;
+    VTuple vTuple;
+
+    for (int i = 0; i < tupleNum; i++) {
+      vTuple = new VTuple(3);
+      vTuple.put(0, DatumFactory.createInt4(i + 1));
+      vTuple.put(1, DatumFactory.createFloat4((float) i));
+      vTuple.put(2, DatumFactory.createText(String.valueOf(i)));
+      appender.addTuple(vTuple);
+    }
+    appender.close();
+
+    TableStats stat = appender.getStats();
+    assertEquals(tupleNum, stat.getNumRows().longValue());
+    tablePath = tablePath.suffix(extension);
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment[] tablets = new FileFragment[1];
+    tablets[0] = new FileFragment(fileName, tablePath, 0, fileLen);
+
+    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+
+    if (StoreType.CSV == storeType) {
+      if (SplittableCompressionCodec.class.isAssignableFrom(codec)) {
+        assertTrue(scanner.isSplittable());
+      } else {
+        assertFalse(scanner.isSplittable());
+      }
+    }
+    scanner.init();
+
+    if (storeType == StoreType.SEQUENCEFILE) {
+      assertTrue(scanner instanceof SequenceFileScanner);
+      Writable key = ((SequenceFileScanner) scanner).getKey();
+      assertEquals(key.getClass().getCanonicalName(), LongWritable.class.getCanonicalName());
+    }
+
+    int tupleCnt = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      tupleCnt++;
+    }
+    scanner.close();
+    assertEquals(tupleNum, tupleCnt);
+    assertNotSame(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
+    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..19a39a2
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileStorageManager.java
@@ -0,0 +1,203 @@
+/**
+ * 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.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.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.File;
+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";
+  StorageManager sm = null;
+  private Path testDir;
+  private FileSystem fs;
+
+	@Before
+	public void setUp() throws Exception {
+		conf = new TajoConf();
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
+	}
+
+	@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(StoreType.CSV);
+		
+		Tuple[] tuples = new Tuple[4];
+		for(int i=0; i < tuples.length; i++) {
+		  tuples[i] = new VTuple(3);
+		  tuples[i].put(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());
+		Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, path);
+    appender.init();
+		for(Tuple t : tuples) {
+		  appender.addTuple(t);
+		}
+		appender.close();
+
+		Scanner scanner = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).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();
+
+    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));
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(new TajoConf(conf), tablePath);
+
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age",Type.INT4);
+      schema.addColumn("name",Type.TEXT);
+      TableMeta meta = CatalogUtil.newTableMeta(StoreType.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();
+
+      File dir = new File(testDataPath);
+      dir.delete();
+    }
+  }
+
+  @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();
+
+    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));
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(new TajoConf(conf), tablePath);
+
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age", Type.INT4);
+      schema.addColumn("name", Type.TEXT);
+      TableMeta meta = CatalogUtil.newTableMeta(StoreType.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();
+
+      File dir = new File(testDataPath);
+      dir.delete();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..088fda9
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3.S3FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.s3.InMemoryFileSystemStore;
+import org.apache.tajo.storage.s3.SmallBlockS3FileSystem;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class TestFileSystems {
+
+  protected byte[] data = null;
+
+  private static String TEST_PATH = "target/test-data/TestFileSystem";
+  private TajoConf conf = null;
+  private FileStorageManager sm = null;
+  private FileSystem fs = null;
+  Path testDir;
+
+  public TestFileSystems(FileSystem fs) throws IOException {
+    conf = new TajoConf();
+
+    if(fs instanceof S3FileSystem){
+      conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "10");
+      fs.initialize(URI.create(fs.getScheme() + ":///"), conf);
+    }
+    this.fs = fs;
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    testDir = getTestDir(this.fs, TEST_PATH);
+  }
+
+  public Path getTestDir(FileSystem fs, String dir) throws IOException {
+    Path path = new Path(dir);
+    if(fs.exists(path))
+      fs.delete(path, true);
+
+    fs.mkdirs(path);
+
+    return fs.makeQualified(path);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return Arrays.asList(new Object[][] {
+        {new SmallBlockS3FileSystem(new InMemoryFileSystemStore())},
+    });
+  }
+
+  @Test
+  public void testBlockSplit() 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(StoreType.CSV);
+
+    Tuple[] tuples = new Tuple[4];
+    for (int i = 0; i < tuples.length; i++) {
+      tuples[i] = new VTuple(3);
+      tuples[i]
+          .put(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());
+
+    Appender appender = sm.getAppender(meta, schema, path);
+    appender.init();
+    for (Tuple t : tuples) {
+      appender.addTuple(t);
+    }
+    appender.close();
+    FileStatus fileStatus = fs.getFileStatus(path);
+
+    List<Fragment> splits = sm.getSplits("table", meta, schema, path);
+    int splitSize = (int) Math.ceil(fileStatus.getLen() / (double) fileStatus.getBlockSize());
+    assertEquals(splitSize, splits.size());
+
+    for (Fragment fragment : splits) {
+      assertTrue(fragment.getLength() <= fileStatus.getBlockSize());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..a0daa7d
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.util.TUtil;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class TestMergeScanner {
+  private TajoConf conf;
+  StorageManager sm;
+  private static String TEST_PATH = "target/test-data/TestMergeScanner";
+
+  private static String TEST_MULTIPLE_FILES_AVRO_SCHEMA =
+      "{\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"namespace\": \"org.apache.tajo\",\n" +
+      "  \"name\": \"testMultipleFiles\",\n" +
+      "  \"fields\": [\n" +
+      "    { \"name\": \"id\", \"type\": \"int\" },\n" +
+      "    { \"name\": \"file\", \"type\": \"string\" },\n" +
+      "    { \"name\": \"name\", \"type\": \"string\" },\n" +
+      "    { \"name\": \"age\", \"type\": \"long\" }\n" +
+      "  ]\n" +
+      "}\n";
+
+  private Path testDir;
+  private StoreType storeType;
+  private FileSystem fs;
+
+  public TestMergeScanner(StoreType storeType) {
+    this.storeType = storeType;
+  }
+
+  @Parameters
+  public static Collection<Object[]> generateParameters() {
+    return Arrays.asList(new Object[][] {
+        {StoreType.CSV},
+        {StoreType.RAW},
+        {StoreType.RCFILE},
+        {StoreType.PARQUET},
+        {StoreType.SEQUENCEFILE},
+        {StoreType.AVRO},
+        // RowFile requires Byte-buffer read support, so we omitted RowFile.
+        //{StoreType.ROWFILE},
+    });
+  }
+
+  @Before
+  public void setup() throws Exception {
+    conf = new TajoConf();
+    conf.setVar(ConfVars.ROOT_DIR, TEST_PATH);
+    conf.setStrings("tajo.storage.projectable-scanner", "rcfile", "parquet", "avro");
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+    sm = StorageManager.getFileStorageManager(conf, testDir);
+  }
+
+  @Test
+  public void testMultipleFiles() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("file", Type.TEXT);
+    schema.addColumn("name", Type.TEXT);
+    schema.addColumn("age", Type.INT8);
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
+    if (storeType == StoreType.AVRO) {
+      meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL,
+                     TEST_MULTIPLE_FILES_AVRO_SCHEMA);
+    }
+
+    Path table1Path = new Path(testDir, storeType + "_1.data");
+    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(null, null, meta, schema, table1Path);
+    appender1.enableStats();
+    appender1.init();
+    int tupleNum = 10000;
+    VTuple vTuple;
+
+    for(int i = 0; i < tupleNum; i++) {
+      vTuple = new VTuple(4);
+      vTuple.put(0, DatumFactory.createInt4(i + 1));
+      vTuple.put(1, DatumFactory.createText("hyunsik"));
+      vTuple.put(2, DatumFactory.createText("jihoon"));
+      vTuple.put(3, DatumFactory.createInt8(25l));
+      appender1.addTuple(vTuple);
+    }
+    appender1.close();
+
+    TableStats stat1 = appender1.getStats();
+    if (stat1 != null) {
+      assertEquals(tupleNum, stat1.getNumRows().longValue());
+    }
+
+    Path table2Path = new Path(testDir, storeType + "_2.data");
+    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(null, null, meta, schema, table2Path);
+    appender2.enableStats();
+    appender2.init();
+
+    for(int i = 0; i < tupleNum; i++) {
+      vTuple = new VTuple(4);
+      vTuple.put(0, DatumFactory.createInt4(i + 1));
+      vTuple.put(1, DatumFactory.createText("hyunsik"));
+      vTuple.put(2, DatumFactory.createText("jihoon"));
+      vTuple.put(3, DatumFactory.createInt8(25l));
+      appender2.addTuple(vTuple);
+    }
+    appender2.close();
+
+    TableStats stat2 = appender2.getStats();
+    if (stat2 != null) {
+      assertEquals(tupleNum, stat2.getNumRows().longValue());
+    }
+
+
+    FileStatus status1 = fs.getFileStatus(table1Path);
+    FileStatus status2 = fs.getFileStatus(table2Path);
+    Fragment[] fragment = new Fragment[2];
+    fragment[0] = new FileFragment("tablet1", table1Path, 0, status1.getLen());
+    fragment[1] = new FileFragment("tablet1", table2Path, 0, status2.getLen());
+
+    Schema targetSchema = new Schema();
+    targetSchema.addColumn(schema.getColumn(0));
+    targetSchema.addColumn(schema.getColumn(2));
+
+    Scanner scanner = new MergeScanner(conf, schema, meta, TUtil.newList(fragment), targetSchema);
+    assertEquals(isProjectableStorage(meta.getStoreType()), scanner.isProjectable());
+
+    scanner.init();
+    int totalCounts = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      totalCounts++;
+      if (isProjectableStorage(meta.getStoreType())) {
+        assertNotNull(tuple.get(0));
+        assertNull(tuple.get(1));
+        assertNotNull(tuple.get(2));
+        assertNull(tuple.get(3));
+      }
+    }
+    scanner.close();
+
+    assertEquals(tupleNum * 2, totalCounts);
+	}
+
+  private static boolean isProjectableStorage(StoreType type) {
+    switch (type) {
+      case RCFILE:
+      case PARQUET:
+      case SEQUENCEFILE:
+      case CSV:
+      case AVRO:
+        return true;
+      default:
+        return false;
+    }
+  }
+}


[05/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
new file mode 100644
index 0000000..d88223b
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
@@ -0,0 +1,1807 @@
+/**
+ * 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.rcfile;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang.StringUtils;
+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.FileSystem;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.SequenceFile.Metadata;
+import org.apache.hadoop.io.compress.*;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.Closeable;
+import java.io.*;
+import java.rmi.server.UID;
+import java.security.MessageDigest;
+import java.util.Arrays;
+
+/**
+ * <code>RCFile</code>s, short of Record Columnar File, are flat files
+ * consisting of binary key/value pairs, which shares much similarity with
+ * <code>SequenceFile</code>.
+ * <p/>
+ * RCFile stores columns of a table in a record columnar way. It first
+ * partitions rows horizontally into row splits. and then it vertically
+ * partitions each row split in a columnar way. RCFile first stores the meta
+ * data of a row split, as the key part of a record, and all the data of a row
+ * split as the value part. When writing, RCFile.Writer first holds records'
+ * value bytes in memory, and determines a row split if the raw bytes size of
+ * buffered records overflow a given parameter<tt>Writer.columnsBufferSize</tt>,
+ * which can be set like: <code>conf.setInt(COLUMNS_BUFFER_SIZE_CONF_STR,
+ * 4 * 1024 * 1024)</code> .
+ * <p>
+ * <code>RCFile</code> provides {@link java.io.Writer}, {@link java.io.Reader} and classes for
+ * writing, reading respectively.
+ * </p>
+ * <p/>
+ * <p>
+ * RCFile stores columns of a table in a record columnar way. It first
+ * partitions rows horizontally into row splits. and then it vertically
+ * partitions each row split in a columnar way. RCFile first stores the meta
+ * data of a row split, as the key part of a record, and all the data of a row
+ * split as the value part.
+ * </p>
+ * <p/>
+ * <p>
+ * RCFile compresses values in a more fine-grained manner then record level
+ * compression. However, It currently does not support compress the key part
+ * yet. The actual compression algorithm used to compress key and/or values can
+ * be specified by using the appropriate {@link org.apache.hadoop.io.compress.CompressionCodec}.
+ * </p>
+ * <p/>
+ * <p>
+ * The {@link java.io.Reader} is used to read and explain the bytes of RCFile.
+ * </p>
+ * <p/>
+ * <h4 id="Formats">RCFile Formats</h4>
+ * <p/>
+ * <p/>
+ * <h5 id="Header">RC Header</h5>
+ * <ul>
+ * <li>version - 3 bytes of magic header <b>RCF</b>, followed by 1 byte of
+ * actual version number (e.g. RCF1)</li>
+ * <li>compression - A boolean which specifies if compression is turned on for
+ * keys/values in this file.</li>
+ * <li>compression codec - <code>CompressionCodec</code> class which is used
+ * for compression of keys and/or values (if compression is enabled).</li>
+ * <li>metadata - {@link org.apache.hadoop.io.SequenceFile.Metadata} for this file.</li>
+ * <li>sync - A sync marker to denote end of the header.</li>
+ * </ul>
+ * <p/>
+ * <h5>RCFile Format</h5>
+ * <ul>
+ * <li><a href="#Header">Header</a></li>
+ * <li>Record
+ * <li>Key part
+ * <ul>
+ * <li>Record length in bytes</li>
+ * <li>Key length in bytes</li>
+ * <li>Number_of_rows_in_this_record(vint)</li>
+ * <li>Column_1_ondisk_length(vint)</li>
+ * <li>Column_1_row_1_value_plain_length</li>
+ * <li>Column_1_row_2_value_plain_length</li>
+ * <li>...</li>
+ * <li>Column_2_ondisk_length(vint)</li>
+ * <li>Column_2_row_1_value_plain_length</li>
+ * <li>Column_2_row_2_value_plain_length</li>
+ * <li>...</li>
+ * </ul>
+ * </li>
+ * </li>
+ * <li>Value part
+ * <ul>
+ * <li>Compressed or plain data of [column_1_row_1_value,
+ * column_1_row_2_value,....]</li>
+ * <li>Compressed or plain data of [column_2_row_1_value,
+ * column_2_row_2_value,....]</li>
+ * </ul>
+ * </li>
+ * </ul>
+ * <p>
+ * <pre>
+ * {@code
+ * The following is a pseudo-BNF grammar for RCFile. Comments are prefixed
+ * with dashes:
+ *
+ * rcfile ::=
+ *   <file-header>
+ *   <rcfile-rowgroup>+
+ *
+ * file-header ::=
+ *   <file-version-header>
+ *   <file-key-class-name>              (only exists if version is seq6)
+ *   <file-value-class-name>            (only exists if version is seq6)
+ *   <file-is-compressed>
+ *   <file-is-block-compressed>         (only exists if version is seq6)
+ *   [<file-compression-codec-class>]
+ *   <file-header-metadata>
+ *   <file-sync-field>
+ *
+ * -- The normative RCFile implementation included with Hive is actually
+ * -- based on a modified version of Hadoop's SequenceFile code. Some
+ * -- things which should have been modified were not, including the code
+ * -- that writes out the file version header. Consequently, RCFile and
+ * -- SequenceFile originally shared the same version header.  A newer
+ * -- release has created a unique version string.
+ *
+ * file-version-header ::= Byte[4] {'S', 'E', 'Q', 6}
+ *                     |   Byte[4] {'R', 'C', 'F', 1}
+ *
+ * -- The name of the Java class responsible for reading the key buffer
+ * -- component of the rowgroup.
+ *
+ * file-key-class-name ::=
+ *   Text {"org.apache.hadoop.hive.ql.io.RCFile$KeyBuffer"}
+ *
+ * -- The name of the Java class responsible for reading the value buffer
+ * -- component of the rowgroup.
+ *
+ * file-value-class-name ::=
+ *   Text {"org.apache.hadoop.hive.ql.io.RCFile$ValueBuffer"}
+ *
+ * -- Boolean variable indicating whether or not the file uses compression
+ * -- for the key and column buffer sections.
+ *
+ * file-is-compressed ::= Byte[1]
+ *
+ * -- A boolean field indicating whether or not the file is block compressed.
+ * -- This field is *always* false. According to comments in the original
+ * -- RCFile implementation this field was retained for backwards
+ * -- compatability with the SequenceFile format.
+ *
+ * file-is-block-compressed ::= Byte[1] {false}
+ *
+ * -- The Java class name of the compression codec iff <file-is-compressed>
+ * -- is true. The named class must implement
+ * -- org.apache.hadoop.io.compress.CompressionCodec.
+ * -- The expected value is org.apache.hadoop.io.compress.GzipCodec.
+ *
+ * file-compression-codec-class ::= Text
+ *
+ * -- A collection of key-value pairs defining metadata values for the
+ * -- file. The Map is serialized using standard JDK serialization, i.e.
+ * -- an Int corresponding to the number of key-value pairs, followed by
+ * -- Text key and value pairs. The following metadata properties are
+ * -- mandatory for all RCFiles:
+ * --
+ * -- hive.io.rcfile.column.number: the number of columns in the RCFile
+ *
+ * file-header-metadata ::= Map<Text, Text>
+ *
+ * -- A 16 byte marker that is generated by the writer. This marker appears
+ * -- at regular intervals at the beginning of rowgroup-headers, and is
+ * -- intended to enable readers to skip over corrupted rowgroups.
+ *
+ * file-sync-hash ::= Byte[16]
+ *
+ * -- Each row group is split into three sections: a header, a set of
+ * -- key buffers, and a set of column buffers. The header section includes
+ * -- an optional sync hash, information about the size of the row group, and
+ * -- the total number of rows in the row group. Each key buffer
+ * -- consists of run-length encoding data which is used to decode
+ * -- the length and offsets of individual fields in the corresponding column
+ * -- buffer.
+ *
+ * rcfile-rowgroup ::=
+ *   <rowgroup-header>
+ *   <rowgroup-key-data>
+ *   <rowgroup-column-buffers>
+ *
+ * rowgroup-header ::=
+ *   [<rowgroup-sync-marker>, <rowgroup-sync-hash>]
+ *   <rowgroup-record-length>
+ *   <rowgroup-key-length>
+ *   <rowgroup-compressed-key-length>
+ *
+ * -- rowgroup-key-data is compressed if the column data is compressed.
+ * rowgroup-key-data ::=
+ *   <rowgroup-num-rows>
+ *   <rowgroup-key-buffers>
+ *
+ * -- An integer (always -1) signaling the beginning of a sync-hash
+ * -- field.
+ *
+ * rowgroup-sync-marker ::= Int
+ *
+ * -- A 16 byte sync field. This must match the <file-sync-hash> value read
+ * -- in the file header.
+ *
+ * rowgroup-sync-hash ::= Byte[16]
+ *
+ * -- The record-length is the sum of the number of bytes used to store
+ * -- the key and column parts, i.e. it is the total length of the current
+ * -- rowgroup.
+ *
+ * rowgroup-record-length ::= Int
+ *
+ * -- Total length in bytes of the rowgroup's key sections.
+ *
+ * rowgroup-key-length ::= Int
+ *
+ * -- Total compressed length in bytes of the rowgroup's key sections.
+ *
+ * rowgroup-compressed-key-length ::= Int
+ *
+ * -- Number of rows in the current rowgroup.
+ *
+ * rowgroup-num-rows ::= VInt
+ *
+ * -- One or more column key buffers corresponding to each column
+ * -- in the RCFile.
+ *
+ * rowgroup-key-buffers ::= <rowgroup-key-buffer>+
+ *
+ * -- Data in each column buffer is stored using a run-length
+ * -- encoding scheme that is intended to reduce the cost of
+ * -- repeated column field values. This mechanism is described
+ * -- in more detail in the following entries.
+ *
+ * rowgroup-key-buffer ::=
+ *   <column-buffer-length>
+ *   <column-buffer-uncompressed-length>
+ *   <column-key-buffer-length>
+ *   <column-key-buffer>
+ *
+ * -- The serialized length on disk of the corresponding column buffer.
+ *
+ * column-buffer-length ::= VInt
+ *
+ * -- The uncompressed length of the corresponding column buffer. This
+ * -- is equivalent to column-buffer-length if the RCFile is not compressed.
+ *
+ * column-buffer-uncompressed-length ::= VInt
+ *
+ * -- The length in bytes of the current column key buffer
+ *
+ * column-key-buffer-length ::= VInt
+ *
+ * -- The column-key-buffer contains a sequence of serialized VInt values
+ * -- corresponding to the byte lengths of the serialized column fields
+ * -- in the corresponding rowgroup-column-buffer. For example, consider
+ * -- an integer column that contains the consecutive values 1, 2, 3, 44.
+ * -- The RCFile format stores these values as strings in the column buffer,
+ * -- e.g. "12344". The length of each column field is recorded in
+ * -- the column-key-buffer as a sequence of VInts: 1,1,1,2. However,
+ * -- if the same length occurs repeatedly, then we replace repeated
+ * -- run lengths with the complement (i.e. negative) of the number of
+ * -- repetitions, so 1,1,1,2 becomes 1,~2,2.
+ *
+ * column-key-buffer ::= Byte[column-key-buffer-length]
+ *
+ * rowgroup-column-buffers ::= <rowgroup-value-buffer>+
+ *
+ * -- RCFile stores all column data as strings regardless of the
+ * -- underlying column type. The strings are neither length-prefixed or
+ * -- null-terminated, and decoding them into individual fields requires
+ * -- the use of the run-length information contained in the corresponding
+ * -- column-key-buffer.
+ *
+ * rowgroup-column-buffer ::= Byte[column-buffer-length]
+ *
+ * Byte ::= An eight-bit byte
+ *
+ * VInt ::= Variable length integer. The high-order bit of each byte
+ * indicates whether more bytes remain to be read. The low-order seven
+ * bits are appended as increasingly more significant bits in the
+ * resulting integer value.
+ *
+ * Int ::= A four-byte integer in big-endian format.
+ *
+ * Text ::= VInt, Chars (Length prefixed UTF-8 characters)
+ * }
+ * </pre>
+ * </p>
+ */
+public class RCFile {
+
+  private static final Log LOG = LogFactory.getLog(RCFile.class);
+
+  public static final String RECORD_INTERVAL_CONF_STR = "hive.io.rcfile.record.interval";
+  public static final String COLUMN_NUMBER_METADATA_STR = "hive.io.rcfile.column.number";
+
+  // All of the versions should be place in this list.
+  private static final int ORIGINAL_VERSION = 0;  // version with SEQ
+  private static final int NEW_MAGIC_VERSION = 1; // version with RCF
+
+  private static final int CURRENT_VERSION = NEW_MAGIC_VERSION;
+
+  // The first version of RCFile used the sequence file header.
+  private static final byte[] ORIGINAL_MAGIC = new byte[]{
+      (byte) 'S', (byte) 'E', (byte) 'Q'};
+  // the version that was included with the original magic, which is mapped
+  // into ORIGINAL_VERSION
+  private static final byte ORIGINAL_MAGIC_VERSION_WITH_METADATA = 6;
+
+  private static final byte[] ORIGINAL_MAGIC_VERSION = new byte[]{
+      (byte) 'S', (byte) 'E', (byte) 'Q', ORIGINAL_MAGIC_VERSION_WITH_METADATA
+  };
+
+  // The 'magic' bytes at the beginning of the RCFile
+  private static final byte[] MAGIC = new byte[]{
+      (byte) 'R', (byte) 'C', (byte) 'F'};
+
+  private static final int SYNC_ESCAPE = -1; // "length" of sync entries
+  private static final int SYNC_HASH_SIZE = 16; // number of bytes in hash
+  private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE; // escape + hash
+
+  /**
+   * The number of bytes between sync points.
+   */
+  public static final int SYNC_INTERVAL = 100 * SYNC_SIZE;
+  public static final String NULL = "rcfile.null";
+  public static final String SERDE = "rcfile.serde";
+
+  /**
+   * KeyBuffer is the key of each record in RCFile. Its on-disk layout is as
+   * below:
+   * <p/>
+   * <ul>
+   * <li>record length in bytes,it is the sum of bytes used to store the key
+   * part and the value part.</li>
+   * <li>Key length in bytes, it is how many bytes used by the key part.</li>
+   * <li>number_of_rows_in_this_record(vint),</li>
+   * <li>column_1_ondisk_length(vint),</li>
+   * <li>column_1_row_1_value_plain_length,</li>
+   * <li>column_1_row_2_value_plain_length,</li>
+   * <li>....</li>
+   * <li>column_2_ondisk_length(vint),</li>
+   * <li>column_2_row_1_value_plain_length,</li>
+   * <li>column_2_row_2_value_plain_length,</li>
+   * <li>.... .</li>
+   * <li>{the end of the key part}</li>
+   * </ul>
+   */
+  public static class KeyBuffer {
+    // each column's length in the value
+    private int[] eachColumnValueLen = null;
+    private int[] eachColumnUncompressedValueLen = null;
+    // stores each cell's length of a column in one DataOutputBuffer element
+    private NonSyncByteArrayOutputStream[] allCellValLenBuffer = null;
+    // how many rows in this split
+    private int numberRows = 0;
+    // how many columns
+    private int columnNumber = 0;
+
+    KeyBuffer(int columnNum) {
+      columnNumber = columnNum;
+      eachColumnValueLen = new int[columnNumber];
+      eachColumnUncompressedValueLen = new int[columnNumber];
+      allCellValLenBuffer = new NonSyncByteArrayOutputStream[columnNumber];
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      eachColumnValueLen = new int[columnNumber];
+      eachColumnUncompressedValueLen = new int[columnNumber];
+      allCellValLenBuffer = new NonSyncByteArrayOutputStream[columnNumber];
+
+      numberRows = WritableUtils.readVInt(in);
+      for (int i = 0; i < columnNumber; i++) {
+        eachColumnValueLen[i] = WritableUtils.readVInt(in);
+        eachColumnUncompressedValueLen[i] = WritableUtils.readVInt(in);
+        int bufLen = WritableUtils.readVInt(in);
+        if (allCellValLenBuffer[i] == null) {
+          allCellValLenBuffer[i] = new NonSyncByteArrayOutputStream();
+        } else {
+          allCellValLenBuffer[i].reset();
+        }
+        allCellValLenBuffer[i].write(in, bufLen);
+      }
+    }
+
+    /**
+     * @return the numberRows
+     */
+    public int getNumberRows() {
+      return numberRows;
+    }
+  }
+
+  /**
+   * ValueBuffer is the value of each record in RCFile. Its on-disk layout is as
+   * below:
+   * <ul>
+   * <li>Compressed or plain data of [column_1_row_1_value,
+   * column_1_row_2_value,....]</li>
+   * <li>Compressed or plain data of [column_2_row_1_value,
+   * column_2_row_2_value,....]</li>
+   * </ul>
+   */
+  public static class ValueBuffer implements Closeable{
+
+    // used to load columns' value into memory
+    private NonSyncByteArrayOutputStream[] loadedColumnsValueBuffer = null;
+
+    boolean inited = false;
+
+    // used for readFields
+    KeyBuffer keyBuffer;
+    private int columnNumber = 0;
+
+    // set true for columns that needed to skip loading into memory.
+    boolean[] skippedColIDs = null;
+
+    CompressionCodec codec;
+    Decompressor decompressor = null;
+    NonSyncDataInputBuffer decompressBuffer = new NonSyncDataInputBuffer();
+    private long readBytes = 0;
+
+
+    public ValueBuffer(KeyBuffer currentKey, int columnNumber,
+                       int[] targets, CompressionCodec codec, boolean[] skippedIDs)
+        throws IOException {
+      keyBuffer = currentKey;
+      this.columnNumber = columnNumber;
+      this.skippedColIDs = skippedIDs;
+      this.codec = codec;
+      loadedColumnsValueBuffer = new NonSyncByteArrayOutputStream[targets.length];
+      if (codec != null) {
+        decompressor = org.apache.tajo.storage.compress.CodecPool.getDecompressor(codec);
+      }
+
+      for (int i = 0; i < targets.length; i++) {
+        loadedColumnsValueBuffer[i] = new NonSyncByteArrayOutputStream();
+      }
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      int addIndex = 0;
+      int skipTotal = 0;
+
+
+      for (int i = 0; i < columnNumber; i++) {
+        int vaRowsLen = keyBuffer.eachColumnValueLen[i];
+        // skip this column
+        if (skippedColIDs[i]) {
+          skipTotal += vaRowsLen;
+          continue;
+        }
+
+        if (skipTotal != 0) {
+          StorageUtil.skipFully(in, skipTotal);
+          skipTotal = 0;
+        }
+
+        NonSyncByteArrayOutputStream valBuf;
+        if (codec != null) {
+          // load into compressed buf first
+
+          byte[] compressedBytes = new byte[vaRowsLen];
+          in.readFully(compressedBytes, 0, vaRowsLen);
+
+          decompressBuffer.reset(compressedBytes, vaRowsLen);
+          if(decompressor != null) decompressor.reset();
+
+          DataInputStream is;
+          if (codec instanceof SplittableCompressionCodec) {
+            SplitCompressionInputStream deflatFilter = ((SplittableCompressionCodec) codec).createInputStream(
+                decompressBuffer, decompressor, 0, vaRowsLen, SplittableCompressionCodec.READ_MODE.BYBLOCK);
+            is = new DataInputStream(deflatFilter);
+          } else {
+            CompressionInputStream deflatFilter = codec.createInputStream(decompressBuffer, decompressor);
+            is = new DataInputStream(deflatFilter);
+          }
+
+          valBuf = loadedColumnsValueBuffer[addIndex];
+          valBuf.reset();
+          valBuf.write(is, keyBuffer.eachColumnUncompressedValueLen[i]);
+          is.close();
+          decompressBuffer.close();
+        } else {
+          valBuf = loadedColumnsValueBuffer[addIndex];
+          valBuf.reset();
+          valBuf.write(in, vaRowsLen);
+        }
+        readBytes += keyBuffer.eachColumnUncompressedValueLen[i];
+        addIndex++;
+      }
+
+      if (skipTotal != 0) {
+        StorageUtil.skipFully(in, skipTotal);
+      }
+    }
+
+    public long getReadBytes() {
+      return readBytes;
+    }
+
+    public void clearColumnBuffer() throws IOException {
+      decompressBuffer.reset();
+      readBytes = 0;
+    }
+
+    @Override
+    public void close() {
+      for (NonSyncByteArrayOutputStream element : loadedColumnsValueBuffer) {
+        IOUtils.closeStream(element);
+      }
+      if (codec != null) {
+        IOUtils.closeStream(decompressBuffer);
+        if (decompressor != null) {
+          // Make sure we only return decompressor once.
+          org.apache.tajo.storage.compress.CodecPool.returnDecompressor(decompressor);
+          decompressor = null;
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a metadata object with alternating key-value pairs.
+   * Eg. metadata(key1, value1, key2, value2)
+   */
+  public static Metadata createMetadata(Text... values) {
+    if (values.length % 2 != 0) {
+      throw new IllegalArgumentException("Must have a matched set of " +
+          "key-value pairs. " + values.length +
+          " strings supplied.");
+    }
+    Metadata result = new Metadata();
+    for (int i = 0; i < values.length; i += 2) {
+      result.set(values[i], values[i + 1]);
+    }
+    return result;
+  }
+
+  /**
+   * Write KeyBuffer/ValueBuffer pairs to a RCFile. RCFile's format is
+   * compatible with SequenceFile's.
+   */
+  public static class RCFileAppender extends FileAppender {
+    FSDataOutputStream out;
+
+    CompressionCodec codec = null;
+    Metadata metadata = null;
+    FileSystem fs = null;
+    TableStatistics stats = null;
+    int columnNumber = 0;
+
+    // how many records the writer buffers before it writes to disk
+    private int RECORD_INTERVAL = Integer.MAX_VALUE;
+    // the max size of memory for buffering records before writes them out
+    private int COLUMNS_BUFFER_SIZE = 16 * 1024 * 1024; // 16M
+    // the conf string for COLUMNS_BUFFER_SIZE
+    public static final String COLUMNS_BUFFER_SIZE_CONF_STR = "hive.io.rcfile.record.buffer.size";
+
+    // how many records already buffered
+    private int bufferedRecords = 0;
+    private ColumnBuffer[] columnBuffers = null;
+    boolean useNewMagic = true;
+    private byte[] nullChars;
+    private SerializerDeserializer serde;
+    private boolean isShuffle;
+
+    // Insert a globally unique 16-byte value every few entries, so that one
+    // can seek into the middle of a file and then synchronize with record
+    // starts and ends by scanning for this value.
+    long lastSyncPos; // position of last sync
+    byte[] sync; // 16 random bytes
+
+    {
+      try {
+        MessageDigest digester = MessageDigest.getInstance("MD5");
+        long time = System.currentTimeMillis();
+        digester.update((new UID() + "@" + time).getBytes());
+        sync = digester.digest();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    /*
+     * used for buffering appends before flush them out
+     */
+    class ColumnBuffer {
+      // used for buffer a column's values
+      NonSyncByteArrayOutputStream columnValBuffer;
+      // used to store each value's length
+      NonSyncByteArrayOutputStream valLenBuffer;
+
+      /*
+       * use a run-length encoding. We only record run length if a same
+       * 'prevValueLen' occurs more than one time. And we negative the run
+       * length to distinguish a runLength and a normal value length. For
+       * example, if the values' lengths are 1,1,1,2, we record 1, ~2,2. And for
+       * value lengths 1,2,3 we record 1,2,3.
+       */
+      int columnValueLength = 0;
+      int uncompressedColumnValueLength = 0;
+      int columnKeyLength = 0;
+      int runLength = 0;
+      int prevValueLength = -1;
+
+      ColumnBuffer() throws IOException {
+        columnValBuffer = new NonSyncByteArrayOutputStream();
+        valLenBuffer = new NonSyncByteArrayOutputStream();
+      }
+
+      public int append(Column column, Datum datum) throws IOException {
+        int currentLen = serde.serialize(column, datum, columnValBuffer, nullChars);
+        columnValueLength += currentLen;
+        uncompressedColumnValueLength += currentLen;
+
+        if (prevValueLength < 0) {
+          startNewGroup(currentLen);
+          return currentLen;
+        }
+
+        if (currentLen != prevValueLength) {
+          flushGroup();
+          startNewGroup(currentLen);
+        } else {
+          runLength++;
+        }
+        return currentLen;
+      }
+
+      private void startNewGroup(int currentLen) {
+        prevValueLength = currentLen;
+        runLength = 0;
+      }
+
+      public void clear() {
+        valLenBuffer.reset();
+        columnValBuffer.reset();
+        prevValueLength = -1;
+        runLength = 0;
+        columnValueLength = 0;
+        columnKeyLength = 0;
+        uncompressedColumnValueLength = 0;
+      }
+
+      public int flushGroup() {
+        int len = 0;
+        if (prevValueLength >= 0) {
+          len += valLenBuffer.writeVLong(prevValueLength);
+          if (runLength > 0) {
+            len += valLenBuffer.writeVLong(~runLength);
+          }
+          columnKeyLength += len;
+          runLength = -1;
+          prevValueLength = -1;
+        }
+        return len;
+      }
+
+      public int UnFlushedGroupSize() {
+        int len = 0;
+        if (prevValueLength >= 0) {
+          len += WritableUtils.getVIntSize(prevValueLength);
+          if (runLength > 0) {
+            len += WritableUtils.getVIntSize(~runLength);
+          }
+        }
+        return len;
+      }
+    }
+
+    public long getLength() throws IOException {
+      return out.getPos();
+    }
+
+    public RCFileAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
+                          final Schema schema, final TableMeta meta, final Path workDir) throws IOException {
+      super(conf, taskAttemptId, schema, meta, workDir);
+
+      RECORD_INTERVAL = conf.getInt(RECORD_INTERVAL_CONF_STR, RECORD_INTERVAL);
+      COLUMNS_BUFFER_SIZE = conf.getInt(COLUMNS_BUFFER_SIZE_CONF_STR, COLUMNS_BUFFER_SIZE);
+      columnNumber = schema.size();
+    }
+
+    public void init() throws IOException {
+      fs = path.getFileSystem(conf);
+
+      if (!fs.exists(path.getParent())) {
+        throw new FileNotFoundException(path.toString());
+      }
+
+      //determine the intermediate file type
+      String store = conf.get(TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.varname,
+          TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.defaultVal);
+      if (enabledStats && CatalogProtos.StoreType.RCFILE == CatalogProtos.StoreType.valueOf(store.toUpperCase())) {
+        isShuffle = true;
+      } else {
+        isShuffle = false;
+      }
+
+      if (this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
+        String codecClassname = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
+        try {
+          Class<? extends CompressionCodec> codecClass = conf.getClassByName(
+              codecClassname).asSubclass(CompressionCodec.class);
+          codec = ReflectionUtils.newInstance(codecClass, conf);
+        } catch (ClassNotFoundException cnfe) {
+          throw new IllegalArgumentException(
+              "Unknown codec: " + codecClassname, cnfe);
+        }
+      }
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.RCFILE_NULL,
+          NullDatum.DEFAULT_TEXT));
+      if (StringUtils.isEmpty(nullCharacters)) {
+        nullChars = NullDatum.get().asTextBytes();
+      } else {
+        nullChars = nullCharacters.getBytes();
+      }
+
+      if (metadata == null) {
+        metadata = new Metadata();
+      }
+
+      metadata.set(new Text(COLUMN_NUMBER_METADATA_STR), new Text("" + columnNumber));
+
+      String serdeClass = this.meta.getOption(StorageConstants.RCFILE_SERDE,
+          BinarySerializerDeserializer.class.getName());
+      try {
+        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new IOException(e);
+      }
+      metadata.set(new Text(StorageConstants.RCFILE_SERDE), new Text(serdeClass));
+
+      columnBuffers = new ColumnBuffer[columnNumber];
+      for (int i = 0; i < columnNumber; i++) {
+        columnBuffers[i] = new ColumnBuffer();
+      }
+
+      init(conf, fs.create(path, true, 4096, (short) 3, fs.getDefaultBlockSize(), null), codec, metadata);
+      initializeFileHeader();
+      writeFileHeader();
+      finalizeFileHeader();
+
+      if (enabledStats) {
+        this.stats = new TableStatistics(this.schema);
+      }
+      super.init();
+    }
+
+    /**
+     * Write the initial part of file header.
+     */
+    void initializeFileHeader() throws IOException {
+      if (useNewMagic) {
+        out.write(MAGIC);
+        out.write(CURRENT_VERSION);
+      } else {
+        out.write(ORIGINAL_MAGIC_VERSION);
+      }
+    }
+
+    /**
+     * Write the final part of file header.
+     */
+    void finalizeFileHeader() throws IOException {
+      out.write(sync); // write the sync bytes
+      out.flush(); // flush header
+    }
+
+    boolean isCompressed() {
+      return codec != null;
+    }
+
+    /**
+     * Write and flush the file header.
+     */
+    void writeFileHeader() throws IOException {
+      if (useNewMagic) {
+        out.writeBoolean(isCompressed());
+      } else {
+        Text.writeString(out, "org.apache.hadoop.hive.ql.io.RCFile$KeyBuffer");
+        Text.writeString(out, "org.apache.hadoop.hive.ql.io.RCFile$ValueBuffer");
+        out.writeBoolean(isCompressed());
+        out.writeBoolean(false);
+      }
+
+      if (isCompressed()) {
+        Text.writeString(out, (codec.getClass()).getName());
+      }
+      metadata.write(out);
+    }
+
+    void init(Configuration conf, FSDataOutputStream out,
+              CompressionCodec codec, Metadata metadata) throws IOException {
+      this.out = out;
+      this.codec = codec;
+      this.metadata = metadata;
+      this.useNewMagic = conf.getBoolean(TajoConf.ConfVars.HIVEUSEEXPLICITRCFILEHEADER.varname, true);
+    }
+
+    /**
+     * create a sync point.
+     */
+    public void sync() throws IOException {
+      if (sync != null && lastSyncPos != out.getPos()) {
+        out.writeInt(SYNC_ESCAPE); // mark the start of the sync
+        out.write(sync); // write sync
+        lastSyncPos = out.getPos(); // update lastSyncPos
+      }
+    }
+
+    private void checkAndWriteSync() throws IOException {
+      if (sync != null && out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
+        sync();
+      }
+    }
+
+    private int columnBufferSize = 0;
+
+    @Override
+    public long getOffset() throws IOException {
+      return out.getPos();
+    }
+
+    @Override
+    public void flush() throws IOException {
+      flushRecords();
+      out.flush();
+    }
+
+    @Override
+    public void addTuple(Tuple t) throws IOException {
+      append(t);
+      // Statistical section
+
+      if (enabledStats) {
+        stats.incrementRow();
+      }
+    }
+
+    /**
+     * Append a row of values. Currently it only can accept <
+     * {@link org.apache.tajo.storage.Tuple}. If its <code>size()</code> is less than the
+     * column number in the file, zero bytes are appended for the empty columns.
+     * If its size() is greater then the column number in the file, the exceeded
+     * columns' bytes are ignored.
+     *
+     * @param tuple a Tuple with the list of serialized columns
+     * @throws java.io.IOException
+     */
+    public void append(Tuple tuple) throws IOException {
+      int size = schema.size();
+
+      for (int i = 0; i < size; i++) {
+        Datum datum = tuple.get(i);
+        int length = columnBuffers[i].append(schema.getColumn(i), datum);
+        columnBufferSize += length;
+        if (isShuffle) {
+          // it is to calculate min/max values, and it is only used for the intermediate file.
+          stats.analyzeField(i, datum);
+        }
+      }
+
+      if (size < columnNumber) {
+        for (int i = size; i < columnNumber; i++) {
+          columnBuffers[i].append(schema.getColumn(i), NullDatum.get());
+          if (isShuffle) {
+            stats.analyzeField(i, NullDatum.get());
+          }
+        }
+      }
+
+      bufferedRecords++;
+      //TODO compression rate base flush
+      if ((columnBufferSize > COLUMNS_BUFFER_SIZE)
+          || (bufferedRecords >= RECORD_INTERVAL)) {
+        flushRecords();
+      }
+    }
+
+    /**
+     * get number of bytes to store the keyBuffer.
+     *
+     * @return number of bytes used to store this KeyBuffer on disk
+     * @throws java.io.IOException
+     */
+    public int getKeyBufferSize() throws IOException {
+      int ret = 0;
+      ret += WritableUtils.getVIntSize(bufferedRecords);
+      for (int i = 0; i < columnBuffers.length; i++) {
+        ColumnBuffer currentBuf = columnBuffers[i];
+        ret += WritableUtils.getVIntSize(currentBuf.columnValueLength);
+        ret += WritableUtils.getVIntSize(currentBuf.uncompressedColumnValueLength);
+        ret += WritableUtils.getVIntSize(currentBuf.columnKeyLength);
+        ret += currentBuf.columnKeyLength;
+      }
+
+      return ret;
+    }
+
+    /**
+     * get number of bytes to store the key part.
+     *
+     * @return number of bytes used to store this Key part on disk
+     * @throws java.io.IOException
+     */
+    public int getKeyPartSize() throws IOException {
+      int ret = 12; //12 bytes |record count, key length, compressed key length|
+
+      ret += WritableUtils.getVIntSize(bufferedRecords);
+      for (int i = 0; i < columnBuffers.length; i++) {
+        ColumnBuffer currentBuf = columnBuffers[i];
+        ret += WritableUtils.getVIntSize(currentBuf.columnValueLength);
+        ret += WritableUtils.getVIntSize(currentBuf.uncompressedColumnValueLength);
+        ret += WritableUtils.getVIntSize(currentBuf.columnKeyLength);
+        ret += currentBuf.columnKeyLength;
+        ret += currentBuf.UnFlushedGroupSize();
+      }
+
+      return ret;
+    }
+
+    private void WriteKeyBuffer(DataOutputStream out) throws IOException {
+      WritableUtils.writeVLong(out, bufferedRecords);
+      for (int i = 0; i < columnBuffers.length; i++) {
+        ColumnBuffer currentBuf = columnBuffers[i];
+        WritableUtils.writeVLong(out, currentBuf.columnValueLength);
+        WritableUtils.writeVLong(out, currentBuf.uncompressedColumnValueLength);
+        WritableUtils.writeVLong(out, currentBuf.columnKeyLength);
+        currentBuf.valLenBuffer.writeTo(out);
+      }
+    }
+
+    private void flushRecords() throws IOException {
+
+      Compressor compressor = null;
+      NonSyncByteArrayOutputStream valueBuffer = null;
+      CompressionOutputStream deflateFilter = null;
+      DataOutputStream deflateOut = null;
+      boolean isCompressed = isCompressed();
+
+      int valueLength = 0;
+      if (isCompressed) {
+        compressor = org.apache.tajo.storage.compress.CodecPool.getCompressor(codec);
+        if (compressor != null) compressor.reset();  //builtin gzip is null
+
+        valueBuffer = new NonSyncByteArrayOutputStream();
+        deflateFilter = codec.createOutputStream(valueBuffer, compressor);
+        deflateOut = new DataOutputStream(deflateFilter);
+      }
+
+      try {
+        for (int columnIndex = 0; columnIndex < columnNumber; columnIndex++) {
+          ColumnBuffer currentBuf = columnBuffers[columnIndex];
+          currentBuf.flushGroup();
+
+          NonSyncByteArrayOutputStream columnValue = currentBuf.columnValBuffer;
+          int colLen;
+          int plainLen = columnValue.getLength();
+          if (isCompressed) {
+            deflateFilter.resetState();
+            deflateOut.write(columnValue.getData(), 0, columnValue.getLength());
+            deflateOut.flush();
+            deflateFilter.finish();
+            columnValue.close();
+            // find how much compressed data was added for this column
+            colLen = valueBuffer.getLength() - valueLength;
+            currentBuf.columnValueLength = colLen;
+          } else {
+            colLen = plainLen;
+          }
+          valueLength += colLen;
+        }
+      } catch (IOException e) {
+        IOUtils.cleanup(LOG, deflateOut, out);
+        throw e;
+      }
+
+      if (compressor != null) {
+        org.apache.tajo.storage.compress.CodecPool.returnCompressor(compressor);
+      }
+
+      int keyLength = getKeyBufferSize();
+      if (keyLength < 0) {
+        throw new IOException("negative length keys not allowed: " + keyLength);
+      }
+      // Write the key out
+      writeKey(keyLength + valueLength, keyLength);
+      // write the value out
+      if (isCompressed) {
+        try {
+          out.write(valueBuffer.getData(), 0, valueBuffer.getLength());
+        } finally {
+          IOUtils.cleanup(LOG, valueBuffer);
+        }
+      } else {
+        for (int columnIndex = 0; columnIndex < columnNumber; ++columnIndex) {
+          columnBuffers[columnIndex].columnValBuffer.writeTo(out);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Column#" + columnIndex + " : Plain Total Column Value Length: "
+                + columnBuffers[columnIndex].uncompressedColumnValueLength
+                + ",  Compr Total Column Value Length: " + columnBuffers[columnIndex].columnValueLength);
+          }
+        }
+      }
+      // clear the columnBuffers
+      clearColumnBuffers();
+
+      bufferedRecords = 0;
+      columnBufferSize = 0;
+    }
+
+    private void writeKey(int recordLen, int keyLength) throws IOException {
+      checkAndWriteSync(); // sync
+      out.writeInt(recordLen); // total record length
+      out.writeInt(keyLength); // key portion length
+
+      if (this.isCompressed()) {
+        Compressor compressor = org.apache.tajo.storage.compress.CodecPool.getCompressor(codec);
+        if (compressor != null) compressor.reset();  //builtin gzip is null
+
+        NonSyncByteArrayOutputStream compressionBuffer = new NonSyncByteArrayOutputStream();
+        CompressionOutputStream deflateFilter = codec.createOutputStream(compressionBuffer, compressor);
+        DataOutputStream deflateOut = new DataOutputStream(deflateFilter);
+
+        //compress key and write key out
+        compressionBuffer.reset();
+        deflateFilter.resetState();
+        WriteKeyBuffer(deflateOut);
+        deflateOut.flush();
+        deflateFilter.finish();
+        int compressedKeyLen = compressionBuffer.getLength();
+        out.writeInt(compressedKeyLen);
+        compressionBuffer.writeTo(out);
+        compressionBuffer.reset();
+        deflateOut.close();
+        org.apache.tajo.storage.compress.CodecPool.returnCompressor(compressor);
+      } else {
+        out.writeInt(keyLength);
+        WriteKeyBuffer(out);
+      }
+    }
+
+    private void clearColumnBuffers() throws IOException {
+      for (int i = 0; i < columnNumber; i++) {
+        columnBuffers[i].clear();
+      }
+    }
+
+    @Override
+    public TableStats getStats() {
+      if (enabledStats) {
+        return stats.getTableStat();
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (bufferedRecords > 0) {
+        flushRecords();
+      }
+      clearColumnBuffers();
+
+      if (out != null) {
+        // Statistical section
+        if (enabledStats) {
+          stats.setNumBytes(getOffset());
+        }
+        // Close the underlying stream if we own it...
+        out.flush();
+        IOUtils.cleanup(LOG, out);
+        out = null;
+      }
+    }
+  }
+
+  /**
+   * Read KeyBuffer/ValueBuffer pairs from a RCFile.
+   */
+  public static class RCFileScanner extends FileScanner {
+    private static class SelectedColumn {
+      public int colIndex;
+      public int rowReadIndex;
+      public int runLength;
+      public int prvLength;
+      public boolean isNulled;
+    }
+
+    private FSDataInputStream in;
+
+    private byte version;
+
+    private CompressionCodec codec = null;
+    private Metadata metadata = null;
+
+    private byte[] sync;
+    private byte[] syncCheck;
+    private boolean syncSeen;
+    private long lastSeenSyncPos = 0;
+
+    private long headerEnd;
+    private long start, end;
+    private final long startOffset, endOffset;
+    private int[] targetColumnIndexes;
+
+    private int currentKeyLength;
+    private int currentRecordLength;
+
+    private ValueBuffer currentValue;
+
+    private int readRowsIndexInBuffer = 0;
+
+    private int recordsNumInValBuffer = 0;
+
+    private int columnNumber = 0;
+
+    private boolean more = true;
+
+    private int passedRowsNum = 0;
+
+    private boolean decompress = false;
+
+    private Decompressor keyDecompressor;
+
+    private long readBytes = 0;
+
+    //Current state of each selected column - e.g. current run length, etc.
+    // The size of the array is equal to the number of selected columns
+    private SelectedColumn[] selectedColumns;
+
+    // column value lengths for each of the selected columns
+    private NonSyncDataInputBuffer[] colValLenBufferReadIn;
+
+    private LongWritable rowId;
+    private byte[] nullChars;
+    private SerializerDeserializer serde;
+
+    public RCFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
+                         final Fragment fragment) throws IOException {
+      super(conf, schema, meta, fragment);
+      conf.setInt("io.file.buffer.size", 4096); //TODO remove
+
+      startOffset = this.fragment.getStartKey();
+      endOffset = startOffset + this.fragment.getLength();
+      start = 0;
+    }
+
+    @Override
+    public void init() throws IOException {
+      sync = new byte[SYNC_HASH_SIZE];
+      syncCheck = new byte[SYNC_HASH_SIZE];
+
+      more = startOffset < endOffset;
+      rowId = new LongWritable();
+      readBytes = 0;
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.RCFILE_NULL,
+          NullDatum.DEFAULT_TEXT));
+      if (StringUtils.isEmpty(nullCharacters)) {
+        nullChars = NullDatum.get().asTextBytes();
+      } else {
+        nullChars = nullCharacters.getBytes();
+      }
+
+      // projection
+      if (targets == null) {
+        targets = schema.toArray();
+      }
+
+      targetColumnIndexes = new int[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
+      }
+      Arrays.sort(targetColumnIndexes);
+
+      FileSystem fs = fragment.getPath().getFileSystem(conf);
+      end = fs.getFileStatus(fragment.getPath()).getLen();
+      in = openFile(fs, fragment.getPath(), 4096);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RCFile open:" + fragment.getPath() + "," + start + "," + (endOffset - startOffset) +
+            "," + fs.getFileStatus(fragment.getPath()).getLen());
+      }
+      //init RCFILE Header
+      boolean succeed = false;
+      try {
+        if (start > 0) {
+          seek(0);
+          initHeader();
+        } else {
+          initHeader();
+        }
+        succeed = true;
+      } finally {
+        if (!succeed) {
+          if (in != null) {
+            try {
+              in.close();
+            } catch (IOException e) {
+              if (LOG != null && LOG.isDebugEnabled()) {
+                LOG.debug("Exception in closing " + in, e);
+              }
+            }
+          }
+        }
+      }
+
+      columnNumber = Integer.parseInt(metadata.get(new Text(COLUMN_NUMBER_METADATA_STR)).toString());
+      selectedColumns = new SelectedColumn[targetColumnIndexes.length];
+      colValLenBufferReadIn = new NonSyncDataInputBuffer[targetColumnIndexes.length];
+      boolean[] skippedColIDs = new boolean[columnNumber];
+      Arrays.fill(skippedColIDs, true);
+      super.init();
+
+      for (int i = 0; i < targetColumnIndexes.length; i++) {
+        int tid = targetColumnIndexes[i];
+        if (tid < columnNumber) {
+          skippedColIDs[tid] = false;
+
+          SelectedColumn col = new SelectedColumn();
+          col.colIndex = tid;
+          col.runLength = 0;
+          col.prvLength = -1;
+          col.rowReadIndex = 0;
+          selectedColumns[i] = col;
+          colValLenBufferReadIn[i] = new NonSyncDataInputBuffer();
+        }
+      }
+
+      currentKey = createKeyBuffer();
+      currentValue = new ValueBuffer(null, columnNumber, targetColumnIndexes, codec, skippedColIDs);
+
+      if (startOffset > getPosition()) {    // TODO use sync cache
+        sync(startOffset); // sync to start
+      }
+    }
+
+    /**
+     * Return the metadata (Text to Text map) that was written into the
+     * file.
+     */
+    public Metadata getMetadata() {
+      return metadata;
+    }
+
+    /**
+     * Return the metadata value associated with the given key.
+     *
+     * @param key the metadata key to retrieve
+     */
+    public Text getMetadataValueOf(Text key) {
+      return metadata.get(key);
+    }
+
+    /**
+     * Override this method to specialize the type of
+     * {@link org.apache.hadoop.fs.FSDataInputStream} returned.
+     */
+    protected FSDataInputStream openFile(FileSystem fs, Path file, int bufferSize) throws IOException {
+      return fs.open(file, bufferSize);
+    }
+
+    private void initHeader() throws IOException {
+      byte[] magic = new byte[MAGIC.length];
+      in.readFully(magic);
+
+      if (Arrays.equals(magic, ORIGINAL_MAGIC)) {
+        byte vers = in.readByte();
+        if (vers != ORIGINAL_MAGIC_VERSION_WITH_METADATA) {
+          throw new IOException(fragment.getPath() + " is a version " + vers +
+              " SequenceFile instead of an RCFile.");
+        }
+        version = ORIGINAL_VERSION;
+      } else {
+        if (!Arrays.equals(magic, MAGIC)) {
+          throw new IOException(fragment.getPath() + " not a RCFile and has magic of " +
+              new String(magic));
+        }
+
+        // Set 'version'
+        version = in.readByte();
+        if (version > CURRENT_VERSION) {
+          throw new VersionMismatchException((byte) CURRENT_VERSION, version);
+        }
+      }
+
+      if (version == ORIGINAL_VERSION) {
+        try {
+          Class<?> keyCls = conf.getClassByName(Text.readString(in));
+          Class<?> valCls = conf.getClassByName(Text.readString(in));
+          if (!keyCls.equals(KeyBuffer.class)
+              || !valCls.equals(ValueBuffer.class)) {
+            throw new IOException(fragment.getPath() + " not a RCFile");
+          }
+        } catch (ClassNotFoundException e) {
+          throw new IOException(fragment.getPath() + " not a RCFile", e);
+        }
+      }
+
+      decompress = in.readBoolean(); // is compressed?
+
+      if (version == ORIGINAL_VERSION) {
+        // is block-compressed? it should be always false.
+        boolean blkCompressed = in.readBoolean();
+        if (blkCompressed) {
+          throw new IOException(fragment.getPath() + " not a RCFile.");
+        }
+      }
+
+      // setup the compression codec
+      if (decompress) {
+        String codecClassname = Text.readString(in);
+        try {
+          Class<? extends CompressionCodec> codecClass = conf.getClassByName(
+              codecClassname).asSubclass(CompressionCodec.class);
+          codec = ReflectionUtils.newInstance(codecClass, conf);
+        } catch (ClassNotFoundException cnfe) {
+          throw new IllegalArgumentException(
+              "Unknown codec: " + codecClassname, cnfe);
+        }
+
+        keyDecompressor = org.apache.tajo.storage.compress.CodecPool.getDecompressor(codec);
+      }
+
+      metadata = new Metadata();
+      metadata.readFields(in);
+
+      Text text = metadata.get(new Text(StorageConstants.RCFILE_SERDE));
+
+      try {
+        String serdeClass;
+        if(text != null && !text.toString().isEmpty()){
+          serdeClass = text.toString();
+        } else{
+          serdeClass = this.meta.getOption(StorageConstants.RCFILE_SERDE, BinarySerializerDeserializer.class.getName());
+        }
+        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new IOException(e);
+      }
+
+      in.readFully(sync); // read sync bytes
+      headerEnd = in.getPos();
+      lastSeenSyncPos = headerEnd; //initial sync position
+      readBytes += headerEnd;
+    }
+
+    /**
+     * Return the current byte position in the input file.
+     */
+    public long getPosition() throws IOException {
+      return in.getPos();
+    }
+
+    /**
+     * Set the current byte position in the input file.
+     * <p/>
+     * <p/>
+     * The position passed must be a position returned by
+     * {@link org.apache.tajo.storage.rcfile.RCFile.RCFileAppender#getLength()} when writing this file. To seek to an
+     * arbitrary position, use {@link org.apache.tajo.storage.rcfile.RCFile.RCFileScanner#sync(long)}. In another
+     * words, the current seek can only seek to the end of the file. For other
+     * positions, use {@link org.apache.tajo.storage.rcfile.RCFile.RCFileScanner#sync(long)}.
+     */
+    public void seek(long position) throws IOException {
+      in.seek(position);
+    }
+
+    /**
+     * Resets the values which determine if there are more rows in the buffer
+     * <p/>
+     * This can be used after one calls seek or sync, if one called next before that.
+     * Otherwise, the seek or sync will have no effect, it will continue to get rows from the
+     * buffer built up from the call to next.
+     */
+    public void resetBuffer() {
+      readRowsIndexInBuffer = 0;
+      recordsNumInValBuffer = 0;
+    }
+
+    /**
+     * Seek to the next sync mark past a given position.
+     */
+    public void sync(long position) throws IOException {
+      if (position + SYNC_SIZE >= end) {
+        seek(end);
+        return;
+      }
+
+      //this is to handle syn(pos) where pos < headerEnd.
+      if (position < headerEnd) {
+        // seek directly to first record
+        in.seek(headerEnd);
+        // note the sync marker "seen" in the header
+        syncSeen = true;
+        return;
+      }
+
+      try {
+        seek(position + 4); // skip escape
+
+        int prefix = sync.length;
+        int n = conf.getInt("io.bytes.per.checksum", 512);
+        byte[] buffer = new byte[prefix + n];
+        n = (int) Math.min(n, end - in.getPos());
+        /* fill array with a pattern that will never match sync */
+        Arrays.fill(buffer, (byte) (~sync[0]));
+        while (n > 0 && (in.getPos() + n) <= end) {
+          position = in.getPos();
+          in.readFully(buffer, prefix, n);
+          readBytes += n;
+          /* the buffer has n+sync bytes */
+          for (int i = 0; i < n; i++) {
+            int j;
+            for (j = 0; j < sync.length && sync[j] == buffer[i + j]; j++) {
+              /* nothing */
+            }
+            if (j == sync.length) {
+              /* simplified from (position + (i - prefix) + sync.length) - SYNC_SIZE */
+              in.seek(position + i - SYNC_SIZE);
+              return;
+            }
+          }
+          /* move the last 16 bytes to the prefix area */
+          System.arraycopy(buffer, buffer.length - prefix, buffer, 0, prefix);
+          n = (int) Math.min(n, end - in.getPos());
+        }
+      } catch (ChecksumException e) { // checksum failure
+        handleChecksumException(e);
+      }
+    }
+
+    private void handleChecksumException(ChecksumException e) throws IOException {
+      if (conf.getBoolean("io.skip.checksum.errors", false)) {
+        LOG.warn("Bad checksum at " + getPosition() + ". Skipping entries.");
+        sync(getPosition() + conf.getInt("io.bytes.per.checksum", 512));
+      } else {
+        throw e;
+      }
+    }
+
+    private KeyBuffer createKeyBuffer() {
+      return new KeyBuffer(columnNumber);
+    }
+
+    /**
+     * Read and return the next record length, potentially skipping over a sync
+     * block.
+     *
+     * @return the length of the next record or -1 if there is no next record
+     * @throws java.io.IOException
+     */
+    private int readRecordLength() throws IOException {
+      if (in.getPos() >= end) {
+        return -1;
+      }
+      int length = in.readInt();
+      readBytes += 4;
+      if (sync != null && length == SYNC_ESCAPE) { // process
+        // a
+        // sync entry
+        lastSeenSyncPos = in.getPos() - 4; // minus SYNC_ESCAPE's length
+        in.readFully(syncCheck); // read syncCheck
+        readBytes += SYNC_HASH_SIZE;
+        if (!Arrays.equals(sync, syncCheck)) {
+          throw new IOException("File is corrupt!");
+        }
+        syncSeen = true;
+        if (in.getPos() >= end) {
+          return -1;
+        }
+        length = in.readInt(); // re-read length
+        readBytes += 4;
+      } else {
+        syncSeen = false;
+      }
+      return length;
+    }
+
+    private void seekToNextKeyBuffer() throws IOException {
+      if (!keyInit) {
+        return;
+      }
+      if (!currentValue.inited) {
+        IOUtils.skipFully(in, currentRecordLength - currentKeyLength);
+      }
+    }
+
+    private int compressedKeyLen = 0;
+    NonSyncDataInputBuffer keyDataIn = new NonSyncDataInputBuffer();
+    NonSyncDataInputBuffer keyDecompressBuffer = new NonSyncDataInputBuffer();
+
+    KeyBuffer currentKey = null;
+    boolean keyInit = false;
+
+    protected int nextKeyBuffer() throws IOException {
+      seekToNextKeyBuffer();
+      currentRecordLength = readRecordLength();
+      if (currentRecordLength == -1) {
+        keyInit = false;
+        return -1;
+      }
+      currentKeyLength = in.readInt();
+      compressedKeyLen = in.readInt();
+      readBytes += 8;
+      if (decompress) {
+
+        byte[] compressedBytes = new byte[compressedKeyLen];
+        in.readFully(compressedBytes, 0, compressedKeyLen);
+
+        if (keyDecompressor != null) keyDecompressor.reset();
+        keyDecompressBuffer.reset(compressedBytes, compressedKeyLen);
+
+        DataInputStream is;
+        if (codec instanceof SplittableCompressionCodec) {
+          SplitCompressionInputStream deflatFilter = ((SplittableCompressionCodec) codec).createInputStream(
+              keyDecompressBuffer, keyDecompressor, 0, compressedKeyLen, SplittableCompressionCodec.READ_MODE.BYBLOCK);
+
+          keyDecompressBuffer.seek(deflatFilter.getAdjustedStart());
+          is = new DataInputStream(deflatFilter);
+        } else {
+          CompressionInputStream deflatFilter = codec.createInputStream(keyDecompressBuffer, keyDecompressor);
+          is = new DataInputStream(deflatFilter);
+        }
+
+        byte[] deCompressedBytes = new byte[currentKeyLength];
+
+        is.readFully(deCompressedBytes, 0, currentKeyLength);
+        keyDataIn.reset(deCompressedBytes, currentKeyLength);
+        currentKey.readFields(keyDataIn);
+        is.close();
+      } else {
+        currentKey.readFields(in);
+      }
+      readBytes += currentKeyLength;
+      keyInit = true;
+      currentValue.inited = false;
+
+      readRowsIndexInBuffer = 0;
+      recordsNumInValBuffer = currentKey.numberRows;
+
+      for (int selIx = 0; selIx < selectedColumns.length; selIx++) {
+        SelectedColumn col = selectedColumns[selIx];
+        if (col == null) {
+          col = new SelectedColumn();
+          col.isNulled = true;
+          selectedColumns[selIx] = col;
+          continue;
+        }
+
+        int colIx = col.colIndex;
+        NonSyncByteArrayOutputStream buf = currentKey.allCellValLenBuffer[colIx];
+        colValLenBufferReadIn[selIx].reset(buf.getData(), buf.getLength());
+        col.rowReadIndex = 0;
+        col.runLength = 0;
+        col.prvLength = -1;
+        col.isNulled = buf.getLength() == 0;
+      }
+
+      return currentKeyLength;
+    }
+
+    protected void currentValueBuffer() throws IOException {
+      if (!keyInit) {
+        nextKeyBuffer();
+      }
+      currentValue.keyBuffer = currentKey;
+      currentValue.clearColumnBuffer();
+      currentValue.readFields(in);
+      currentValue.inited = true;
+      readBytes += currentValue.getReadBytes();
+
+      if (tableStats != null) {
+        tableStats.setReadBytes(readBytes);
+        tableStats.setNumRows(passedRowsNum);
+      }
+    }
+
+    private boolean rowFetched = false;
+
+    @Override
+    public Tuple next() throws IOException {
+      if (!more) {
+        return null;
+      }
+
+      more = nextBuffer(rowId);
+      long lastSeenSyncPos = lastSeenSyncPos();
+      if (lastSeenSyncPos >= endOffset) {
+        more = false;
+        return null;
+      }
+
+      if (!more) {
+        return null;
+      }
+
+      Tuple tuple = new VTuple(schema.size());
+      getCurrentRow(tuple);
+      return tuple;
+    }
+
+    @Override
+    public float getProgress() {
+      try {
+        if(!more) {
+          return 1.0f;
+        }
+        long filePos = getPosition();
+        if (startOffset == filePos) {
+          return 0.0f;
+        } else {
+          //if scanner read the header, filePos moved to zero
+          return Math.min(1.0f, (float)(Math.max(filePos - startOffset, 0)) / (float)(fragment.getLength()));
+        }
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+        return 0.0f;
+      }
+    }
+
+    /**
+     * Returns how many rows we fetched with nextBuffer(). It only means how many rows
+     * are read by nextBuffer(). The returned result may be smaller than actual number
+     * of rows passed by, because {@link #seek(long)} can change the underlying key buffer and
+     * value buffer.
+     *
+     * @return next row number
+     * @throws java.io.IOException
+     */
+    public boolean nextBuffer(LongWritable readRows) throws IOException {
+      if (readRowsIndexInBuffer < recordsNumInValBuffer) {
+        readRows.set(passedRowsNum);
+        readRowsIndexInBuffer++;
+        passedRowsNum++;
+        rowFetched = false;
+        return true;
+      } else {
+        keyInit = false;
+      }
+
+      int ret = -1;
+      try {
+        ret = nextKeyBuffer();
+      } catch (EOFException eof) {
+        eof.printStackTrace();
+      }
+      return (ret > 0) && nextBuffer(readRows);
+    }
+
+    /**
+     * get the current row used,make sure called {@link #next()}
+     * first.
+     *
+     * @throws java.io.IOException
+     */
+    public void getCurrentRow(Tuple tuple) throws IOException {
+      if (!keyInit || rowFetched) {
+        return;
+      }
+
+      if (!currentValue.inited) {
+        currentValueBuffer();
+      }
+
+      for (int j = 0; j < selectedColumns.length; ++j) {
+        SelectedColumn col = selectedColumns[j];
+        int i = col.colIndex;
+
+        if (col.isNulled) {
+          tuple.put(i, NullDatum.get());
+        } else {
+          colAdvanceRow(j, col);
+
+          Datum datum = serde.deserialize(schema.getColumn(i),
+              currentValue.loadedColumnsValueBuffer[j].getData(), col.rowReadIndex, col.prvLength, nullChars);
+          tuple.put(i, datum);
+          col.rowReadIndex += col.prvLength;
+        }
+      }
+      rowFetched = true;
+    }
+
+    /**
+     * Advance column state to the next now: update offsets, run lengths etc
+     *
+     * @param selCol - index among selectedColumns
+     * @param col    - column object to update the state of.  prvLength will be
+     *               set to the new read position
+     * @throws java.io.IOException
+     */
+    private void colAdvanceRow(int selCol, SelectedColumn col) throws IOException {
+      if (col.runLength > 0) {
+        --col.runLength;
+      } else {
+        int length = (int) WritableUtils.readVLong(colValLenBufferReadIn[selCol]);
+        if (length < 0) {
+          // we reach a runlength here, use the previous length and reset
+          // runlength
+          col.runLength = (~length) - 1;
+        } else {
+          col.prvLength = length;
+          col.runLength = 0;
+        }
+      }
+    }
+
+    /**
+     * Returns true if the previous call to next passed a sync mark.
+     */
+    public boolean syncSeen() {
+      return syncSeen;
+    }
+
+    /**
+     * Returns the last seen sync position.
+     */
+    public long lastSeenSyncPos() {
+      return lastSeenSyncPos;
+    }
+
+    /**
+     * Returns the name of the file.
+     */
+    @Override
+    public String toString() {
+      return fragment.getPath().toString();
+    }
+
+    @Override
+    public void reset() throws IOException {
+      seek(startOffset);
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return true;
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public boolean isSplittable() {
+      return true;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (tableStats != null) {
+        tableStats.setReadBytes(readBytes);  //Actual Processed Bytes. (decompressed bytes + header - seek)
+        tableStats.setNumRows(passedRowsNum);
+      }
+
+      IOUtils.cleanup(LOG, in, currentValue);
+      if (keyDecompressor != null) {
+        // Make sure we only return decompressor once.
+        org.apache.tajo.storage.compress.CodecPool.returnDecompressor(keyDecompressor);
+        keyDecompressor = null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
new file mode 100644
index 0000000..60f1b06
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
@@ -0,0 +1,43 @@
+/**
+ * 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.rcfile;
+
+import org.apache.hadoop.io.compress.CompressionInputStream;
+
+import java.io.InputStream;
+
+/**
+ *
+ * SchemaAwareCompressionInputStream adds the ability to inform the compression
+ * stream what column is being read.
+ *
+ */
+public abstract class SchemaAwareCompressionInputStream extends CompressionInputStream {
+
+  protected SchemaAwareCompressionInputStream(InputStream in) throws java.io.IOException {
+    super(in);
+  }
+
+  /**
+   * The column being read
+   *
+   * @param columnIndex the index of the column. Use -1 for non-column data
+   */
+  public abstract void setColumnIndex(int columnIndex);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
new file mode 100644
index 0000000..c0ce8b3
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
@@ -0,0 +1,44 @@
+/**
+ * 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.rcfile;
+
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+
+import java.io.OutputStream;
+
+/**
+ *
+ * SchemaAwareCompressionOutputStream adds the ability to inform the comression stream
+ * the current column being compressed.
+ *
+ */
+public abstract class SchemaAwareCompressionOutputStream extends CompressionOutputStream {
+
+  protected SchemaAwareCompressionOutputStream(OutputStream out) {
+    super(out);
+  }
+
+  /**
+   *
+   * The column being output
+   *
+   * @param columnIndex the index of the column. Use -1 for non-column data
+   */
+  public abstract void setColumnIndex(int columnIndex);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
new file mode 100644
index 0000000..14e0f26
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
@@ -0,0 +1,274 @@
+/**
+ * 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.sequencefile;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
+import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+import org.apache.tajo.util.BytesUtils;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class SequenceFileAppender extends FileAppender {
+  private static final Log LOG = LogFactory.getLog(SequenceFileScanner.class);
+
+  private SequenceFile.Writer writer;
+
+  private TableMeta meta;
+  private Schema schema;
+  private TableStatistics stats = null;
+
+  private int columnNum;
+  private FileSystem fs;
+  private char delimiter;
+  private byte[] nullChars;
+
+  private final static int BUFFER_SIZE = 128 * 1024;
+  private long pos = 0;
+
+  private CompressionCodecFactory codecFactory;
+  private CompressionCodec codec;
+
+  private NonSyncByteArrayOutputStream os;
+  private SerializerDeserializer serde;
+
+  long rowCount;
+  private boolean isShuffle;
+
+  private Writable EMPTY_KEY;
+
+  public SequenceFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                              Schema schema, TableMeta meta, Path workDir) throws IOException {
+    super(conf, taskAttemptId, schema, meta, workDir);
+    this.meta = meta;
+    this.schema = schema;
+  }
+
+  @Override
+  public void init() throws IOException {
+    os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
+
+    this.fs = path.getFileSystem(conf);
+
+    //determine the intermediate file type
+    String store = conf.get(TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.varname,
+        TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.defaultVal);
+    if (enabledStats && CatalogProtos.StoreType.SEQUENCEFILE == CatalogProtos.StoreType.valueOf(store.toUpperCase())) {
+      isShuffle = true;
+    } else {
+      isShuffle = false;
+    }
+
+    this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_DELIMITER,
+        StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
+    this.columnNum = schema.size();
+    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL,
+        NullDatum.DEFAULT_TEXT));
+    if (StringUtils.isEmpty(nullCharacters)) {
+      nullChars = NullDatum.get().asTextBytes();
+    } else {
+      nullChars = nullCharacters.getBytes();
+    }
+
+    if (!fs.exists(path.getParent())) {
+      throw new FileNotFoundException(path.toString());
+    }
+
+    if(this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
+      String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
+      codecFactory = new CompressionCodecFactory(conf);
+      codec = codecFactory.getCodecByClassName(codecName);
+    } else {
+      if (fs.exists(path)) {
+        throw new AlreadyExistsStorageException(path);
+      }
+    }
+
+    try {
+      String serdeClass = this.meta.getOption(StorageConstants.SEQUENCEFILE_SERDE,
+          TextSerializerDeserializer.class.getName());
+      serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new IOException(e);
+    }
+
+    Class<? extends Writable>  keyClass, valueClass;
+    if (serde instanceof BinarySerializerDeserializer) {
+      keyClass = BytesWritable.class;
+      EMPTY_KEY = new BytesWritable();
+      valueClass = BytesWritable.class;
+    } else {
+      keyClass = LongWritable.class;
+      EMPTY_KEY = new LongWritable();
+      valueClass = Text.class;
+    }
+
+    String type = this.meta.getOption(StorageConstants.COMPRESSION_TYPE, CompressionType.NONE.name());
+    if (type.equals(CompressionType.BLOCK.name())) {
+      writer = SequenceFile.createWriter(fs, conf, path, keyClass, valueClass, CompressionType.BLOCK, codec);
+    } else if (type.equals(CompressionType.RECORD.name())) {
+      writer = SequenceFile.createWriter(fs, conf, path, keyClass, valueClass, CompressionType.RECORD, codec);
+    } else {
+      writer = SequenceFile.createWriter(fs, conf, path, keyClass, valueClass, CompressionType.NONE, codec);
+    }
+
+    if (enabledStats) {
+      this.stats = new TableStatistics(this.schema);
+    }
+
+    super.init();
+  }
+
+  @Override
+  public void addTuple(Tuple tuple) throws IOException {
+    Datum datum;
+
+    if (serde instanceof BinarySerializerDeserializer) {
+      byte nullByte = 0;
+      int lasti = 0;
+      for (int i = 0; i < columnNum; i++) {
+        datum = tuple.get(i);
+
+        // set bit to 1 if a field is not null
+        if (null != datum) {
+          nullByte |= 1 << (i % 8);
+        }
+
+        // write the null byte every eight elements or
+        // if this is the last element and serialize the
+        // corresponding 8 struct fields at the same time
+        if (7 == i % 8 || i == columnNum - 1) {
+          os.write(nullByte);
+
+          for (int j = lasti; j <= i; j++) {
+            datum = tuple.get(j);
+
+            switch (schema.getColumn(j).getDataType().getType()) {
+              case TEXT:
+                BytesUtils.writeVLong(os, datum.asTextBytes().length);
+                break;
+              case PROTOBUF:
+                ProtobufDatum protobufDatum = (ProtobufDatum) datum;
+                BytesUtils.writeVLong(os, protobufDatum.asByteArray().length);
+                break;
+              case CHAR:
+              case INET4:
+              case BLOB:
+                BytesUtils.writeVLong(os, datum.asByteArray().length);
+                break;
+              default:
+            }
+
+            serde.serialize(schema.getColumn(j), datum, os, nullChars);
+
+            if (isShuffle) {
+              // it is to calculate min/max values, and it is only used for the intermediate file.
+              stats.analyzeField(j, datum);
+            }
+          }
+          lasti = i + 1;
+          nullByte = 0;
+        }
+      }
+
+      BytesWritable b = new BytesWritable();
+      b.set(os.getData(), 0, os.getLength());
+      writer.append(EMPTY_KEY, b);
+
+    } else {
+      for (int i = 0; i < columnNum; i++) {
+        datum = tuple.get(i);
+        serde.serialize(schema.getColumn(i), datum, os, nullChars);
+
+        if (columnNum -1 > i) {
+          os.write((byte) delimiter);
+        }
+
+        if (isShuffle) {
+          // it is to calculate min/max values, and it is only used for the intermediate file.
+          stats.analyzeField(i, datum);
+        }
+
+      }
+      writer.append(EMPTY_KEY, new Text(os.toByteArray()));
+    }
+
+    os.reset();
+    pos += writer.getLength();
+    rowCount++;
+
+    if (enabledStats) {
+      stats.incrementRow();
+    }
+  }
+
+  @Override
+  public long getOffset() throws IOException {
+    return pos;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    os.flush();
+    writer.close();
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Statistical section
+    if (enabledStats) {
+      stats.setNumBytes(getOffset());
+    }
+
+    os.close();
+    writer.close();
+  }
+
+  @Override
+  public TableStats getStats() {
+    if (enabledStats) {
+      return stats.getTableStat();
+    } else {
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
new file mode 100644
index 0000000..74563ff
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
@@ -0,0 +1,336 @@
+/**
+ * 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.sequencefile;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang.StringUtils;
+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.io.*;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.BytesUtils;
+
+import java.io.IOException;
+
+public class SequenceFileScanner extends FileScanner {
+  private static final Log LOG = LogFactory.getLog(SequenceFileScanner.class);
+
+  private FileSystem fs;
+  private SequenceFile.Reader reader;
+  private SerializerDeserializer serde;
+  private byte[] nullChars;
+  private char delimiter;
+
+  private int currentIdx = 0;
+  private int[] projectionMap;
+
+  private boolean hasBinarySerDe = false;
+  private long totalBytes = 0L;
+
+  private long start, end;
+  private boolean  more = true;
+
+  /**
+   * Whether a field is null or not. Because length is 0 does not means the
+   * field is null. In particular, a 0-length string is not null.
+   */
+  private boolean[] fieldIsNull;
+
+  /**
+   * The start positions and lengths of fields. Only valid when the data is parsed.
+   */
+  private int[] fieldStart;
+  private int[] fieldLength;
+
+  private int elementOffset, elementSize;
+
+  private Writable EMPTY_KEY;
+
+  public SequenceFileScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
+    super(conf, schema, meta, fragment);
+  }
+
+  @Override
+  public void init() throws IOException {
+    // FileFragment information
+    if(fs == null) {
+      fs = FileScanner.getFileSystem((TajoConf)conf, fragment.getPath());
+    }
+
+    reader = new SequenceFile.Reader(fs, fragment.getPath(), conf);
+
+    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL,
+        NullDatum.DEFAULT_TEXT));
+    if (StringUtils.isEmpty(nullCharacters)) {
+      nullChars = NullDatum.get().asTextBytes();
+    } else {
+      nullChars = nullCharacters.getBytes();
+    }
+
+    String delim  = meta.getOption(StorageConstants.SEQUENCEFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+    this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0);
+
+    this.start = fragment.getStartKey();
+    this.end = start + fragment.getLength();
+
+    if (fragment.getStartKey() > reader.getPosition())
+      reader.sync(this.start);
+
+    more = start < end;
+
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+
+
+    fieldIsNull = new boolean[schema.getColumns().size()];
+    fieldStart = new int[schema.getColumns().size()];
+    fieldLength = new int[schema.getColumns().size()];
+
+    prepareProjection(targets);
+
+    try {
+      String serdeClass = this.meta.getOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
+      serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
+
+      if (serde instanceof BinarySerializerDeserializer) {
+        hasBinarySerDe = true;
+      }
+
+      Class<? extends Writable> keyClass = (Class<? extends Writable>)Class.forName(reader.getKeyClassName());
+      EMPTY_KEY = keyClass.newInstance();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new IOException(e);
+    }
+    super.init();
+  }
+
+  public Writable getKey() {
+    return EMPTY_KEY;
+  }
+
+  private void prepareProjection(Column [] targets) {
+    projectionMap = new int[targets.length];
+
+    int tid;
+    for (int i = 0; i < targets.length; i++) {
+      tid = schema.getColumnId(targets[i].getQualifiedName());
+      projectionMap[i] = tid;
+    }
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (!more) return null;
+
+    long pos = reader.getPosition();
+    boolean remaining = reader.next(EMPTY_KEY);
+
+    if (pos >= end && reader.syncSeen()) {
+      more = false;
+    } else {
+      more = remaining;
+    }
+
+    if (more) {
+      Tuple tuple = null;
+      byte[][] cells;
+
+      if (hasBinarySerDe) {
+        BytesWritable bytesWritable = new BytesWritable();
+        reader.getCurrentValue(bytesWritable);
+        tuple = makeTuple(bytesWritable);
+        totalBytes += (long)bytesWritable.getBytes().length;
+      } else {
+        Text text = new Text();
+        reader.getCurrentValue(text);
+        cells = BytesUtils.splitPreserveAllTokens(text.getBytes(), delimiter, projectionMap);
+        totalBytes += (long)text.getBytes().length;
+        tuple = new LazyTuple(schema, cells, 0, nullChars, serde);
+      }
+      currentIdx++;
+      return tuple;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * In hive, LazyBinarySerDe is serialized as follows: start A B A B A B end bytes[] ->
+   * |-----|---------|--- ... ---|-----|---------|
+   *
+   * Section A is one null-byte, corresponding to eight struct fields in Section
+   * B. Each bit indicates whether the corresponding field is null (0) or not null
+   * (1). Each field is a LazyBinaryObject.
+   *
+   * Following B, there is another section A and B. This pattern repeats until the
+   * all struct fields are serialized.
+   *
+   * So, tajo must make a tuple after parsing hive style BinarySerDe.
+   */
+  private Tuple makeTuple(BytesWritable value) throws IOException{
+    Tuple tuple = new VTuple(schema.getColumns().size());
+
+    int start = 0;
+    int length = value.getLength();
+
+    /**
+     * Please note that one null byte is followed by eight fields, then more
+     * null byte and fields.
+     */
+    int structByteEnd = start + length;
+    byte[] bytes = value.getBytes();
+
+    byte nullByte = bytes[start];
+    int lastFieldByteEnd = start + 1;
+
+    // Go through all bytes in the byte[]
+    for (int i = 0; i < schema.getColumns().size(); i++) {
+      fieldIsNull[i] = true;
+      if ((nullByte & (1 << (i % 8))) != 0) {
+        fieldIsNull[i] = false;
+        parse(schema.getColumn(i), bytes, lastFieldByteEnd);
+
+        fieldStart[i] = lastFieldByteEnd + elementOffset;
+        fieldLength[i] = elementSize;
+        lastFieldByteEnd = fieldStart[i] + fieldLength[i];
+
+        for (int j = 0; j < projectionMap.length; j++) {
+          if (projectionMap[j] == i) {
+            Datum datum = serde.deserialize(schema.getColumn(i), bytes, fieldStart[i], fieldLength[i], nullChars);
+            tuple.put(i, datum);
+          }
+        }
+      }
+
+      // next byte is a null byte if there are more bytes to go
+      if (7 == (i % 8)) {
+        if (lastFieldByteEnd < structByteEnd) {
+          nullByte = bytes[lastFieldByteEnd];
+          lastFieldByteEnd++;
+        } else {
+          // otherwise all null afterwards
+          nullByte = 0;
+          lastFieldByteEnd++;
+        }
+      }
+    }
+
+    return tuple;
+  }
+
+  /**
+   * Check a particular field and set its size and offset in bytes based on the
+   * field type and the bytes arrays.
+   *
+   * For void, boolean, byte, short, int, long, float and double, there is no
+   * offset and the size is fixed. For string, the first four bytes are used to store the size.
+   * So the offset is 4 and the size is computed by concating the first four bytes together.
+   * The first four bytes are defined with respect to the offset in the bytes arrays.
+   *
+   * @param col
+   *          catalog column information
+   * @param bytes
+   *          bytes arrays store the table row
+   * @param offset
+   *          offset of this field
+   */
+  private void parse(Column col, byte[] bytes, int offset) throws
+      IOException {
+    switch (col.getDataType().getType()) {
+      case BOOLEAN:
+      case BIT:
+        elementOffset = 0;
+        elementSize = 1;
+        break;
+      case INT2:
+        elementOffset = 0;
+        elementSize = 2;
+        break;
+      case INT4:
+      case INT8:
+        elementOffset = 0;
+        elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
+        break;
+      case FLOAT4:
+        elementOffset = 0;
+        elementSize = 4;
+        break;
+      case FLOAT8:
+        elementOffset = 0;
+        elementSize = 8;
+        break;
+      case BLOB:
+      case PROTOBUF:
+      case INET4:
+      case CHAR:
+      case TEXT:
+        elementOffset = 1;
+        elementSize = bytes[offset];
+        break;
+      default:
+        elementOffset = 0;
+        elementSize = 0;
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    if (reader != null) {
+      reader.sync(0);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (reader != null)
+      reader.close();
+
+    if (tableStats != null) {
+      tableStats.setReadBytes(totalBytes);
+      tableStats.setNumRows(currentIdx);
+    }
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return true;
+  }
+
+  @Override
+  public boolean isSplittable(){
+    return true;
+  }
+}


[15/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java b/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java
deleted file mode 100644
index c43ba38..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestOffHeapRowBlock.java
+++ /dev/null
@@ -1,577 +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.tuple.offheap;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.RowStoreUtil;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.unit.StorageUnit;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.ProtoUtil;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-
-import static org.apache.tajo.common.TajoDataTypes.Type;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestOffHeapRowBlock {
-  private static final Log LOG = LogFactory.getLog(TestOffHeapRowBlock.class);
-  public static String UNICODE_FIELD_PREFIX = "abc_가나다_";
-  public static Schema schema;
-
-  static {
-    schema = new Schema();
-    schema.addColumn("col0", Type.BOOLEAN);
-    schema.addColumn("col1", Type.INT2);
-    schema.addColumn("col2", Type.INT4);
-    schema.addColumn("col3", Type.INT8);
-    schema.addColumn("col4", Type.FLOAT4);
-    schema.addColumn("col5", Type.FLOAT8);
-    schema.addColumn("col6", Type.TEXT);
-    schema.addColumn("col7", Type.TIMESTAMP);
-    schema.addColumn("col8", Type.DATE);
-    schema.addColumn("col9", Type.TIME);
-    schema.addColumn("col10", Type.INTERVAL);
-    schema.addColumn("col11", Type.INET4);
-    schema.addColumn("col12",
-        CatalogUtil.newDataType(TajoDataTypes.Type.PROTOBUF, PrimitiveProtos.StringProto.class.getName()));
-  }
-
-  private void explainRowBlockAllocation(OffHeapRowBlock rowBlock, long startTime, long endTime) {
-    LOG.info(FileUtil.humanReadableByteCount(rowBlock.size(), true) + " bytes allocated "
-        + (endTime - startTime) + " msec");
-  }
-
-  @Test
-  public void testPutAndReadValidation() {
-    int rowNum = 1000;
-
-    long allocStart = System.currentTimeMillis();
-    OffHeapRowBlock rowBlock = new OffHeapRowBlock(schema, 1024);
-    long allocEnd = System.currentTimeMillis();
-    explainRowBlockAllocation(rowBlock, allocStart, allocEnd);
-
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-
-    ZeroCopyTuple tuple = new ZeroCopyTuple();
-    long writeStart = System.currentTimeMillis();
-    for (int i = 0; i < rowNum; i++) {
-      fillRow(i, rowBlock.getWriter());
-
-      reader.reset();
-      int j = 0;
-      while(reader.next(tuple)) {
-        validateTupleResult(j, tuple);
-
-        j++;
-      }
-    }
-    long writeEnd = System.currentTimeMillis();
-    LOG.info("writing and validating take " + (writeEnd - writeStart) + " msec");
-
-    long readStart = System.currentTimeMillis();
-    tuple = new ZeroCopyTuple();
-    int j = 0;
-    reader.reset();
-    while(reader.next(tuple)) {
-      validateTupleResult(j, tuple);
-      j++;
-    }
-    long readEnd = System.currentTimeMillis();
-    LOG.info("reading takes " + (readEnd - readStart) + " msec");
-
-    rowBlock.release();
-  }
-
-  @Test
-  public void testNullityValidation() {
-    int rowNum = 1000;
-
-    long allocStart = System.currentTimeMillis();
-    OffHeapRowBlock rowBlock = new OffHeapRowBlock(schema, 1024);
-    long allocEnd = System.currentTimeMillis();
-    explainRowBlockAllocation(rowBlock, allocStart, allocEnd);
-
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-    ZeroCopyTuple tuple = new ZeroCopyTuple();
-    long writeStart = System.currentTimeMillis();
-    for (int i = 0; i < rowNum; i++) {
-      fillRowBlockWithNull(i, rowBlock.getWriter());
-
-      reader.reset();
-      int j = 0;
-      while(reader.next(tuple)) {
-        validateNullity(j, tuple);
-
-        j++;
-      }
-    }
-    long writeEnd = System.currentTimeMillis();
-    LOG.info("writing and nullity validating take " + (writeEnd - writeStart) +" msec");
-
-    long readStart = System.currentTimeMillis();
-    tuple = new ZeroCopyTuple();
-    int j = 0;
-    reader.reset();
-    while(reader.next(tuple)) {
-      validateNullity(j, tuple);
-
-      j++;
-    }
-    long readEnd = System.currentTimeMillis();
-    LOG.info("reading takes " + (readEnd - readStart) + " msec");
-
-    rowBlock.release();
-  }
-
-  @Test
-  public void testEmptyRow() {
-    int rowNum = 1000;
-
-    long allocStart = System.currentTimeMillis();
-    OffHeapRowBlock rowBlock = new OffHeapRowBlock(schema, StorageUnit.MB * 10);
-    long allocEnd = System.currentTimeMillis();
-    explainRowBlockAllocation(rowBlock, allocStart, allocEnd);
-
-    long writeStart = System.currentTimeMillis();
-    for (int i = 0; i < rowNum; i++) {
-      rowBlock.getWriter().startRow();
-      // empty columns
-      rowBlock.getWriter().endRow();
-    }
-    long writeEnd = System.currentTimeMillis();
-    LOG.info("writing tooks " + (writeEnd - writeStart) + " msec");
-
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-
-    long readStart = System.currentTimeMillis();
-    ZeroCopyTuple tuple = new ZeroCopyTuple();
-    int j = 0;
-    reader.reset();
-    while(reader.next(tuple)) {
-      j++;
-    }
-    long readEnd = System.currentTimeMillis();
-    LOG.info("reading takes " + (readEnd - readStart) + " msec");
-    rowBlock.release();
-
-    assertEquals(rowNum, j);
-    assertEquals(rowNum, rowBlock.rows());
-  }
-
-  @Test
-  public void testSortBenchmark() {
-    int rowNum = 1000;
-
-    OffHeapRowBlock rowBlock = createRowBlock(rowNum);
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-
-    List<ZeroCopyTuple> unSafeTuples = Lists.newArrayList();
-
-    long readStart = System.currentTimeMillis();
-    ZeroCopyTuple tuple = new ZeroCopyTuple();
-    reader.reset();
-    while(reader.next(tuple)) {
-      unSafeTuples.add(tuple);
-      tuple = new ZeroCopyTuple();
-    }
-    long readEnd = System.currentTimeMillis();
-    LOG.info("reading takes " + (readEnd - readStart) + " msec");
-
-    SortSpec sortSpec = new SortSpec(new Column("col2", Type.INT4));
-    BaseTupleComparator comparator = new BaseTupleComparator(schema, new SortSpec[] {sortSpec});
-
-    long sortStart = System.currentTimeMillis();
-    Collections.sort(unSafeTuples, comparator);
-    long sortEnd = System.currentTimeMillis();
-    LOG.info("sorting took " + (sortEnd - sortStart) + " msec");
-    rowBlock.release();
-  }
-
-  @Test
-  public void testVTuplePutAndGetBenchmark() {
-    int rowNum = 1000;
-
-    List<VTuple> rowBlock = Lists.newArrayList();
-    long writeStart = System.currentTimeMillis();
-    VTuple tuple;
-    for (int i = 0; i < rowNum; i++) {
-      tuple = new VTuple(schema.size());
-      fillVTuple(i, tuple);
-      rowBlock.add(tuple);
-    }
-    long writeEnd = System.currentTimeMillis();
-    LOG.info("Writing takes " + (writeEnd - writeStart) + " msec");
-
-    long readStart = System.currentTimeMillis();
-    int j = 0;
-    for (VTuple t : rowBlock) {
-      validateTupleResult(j, t);
-      j++;
-    }
-    long readEnd = System.currentTimeMillis();
-    LOG.info("reading takes " + (readEnd - readStart) + " msec");
-
-    int count = 0;
-    for (int l = 0; l < rowBlock.size(); l++) {
-      for(int m = 0; m < schema.size(); m++ ) {
-        if (rowBlock.get(l).contains(m) && rowBlock.get(l).get(m).type() == Type.INT4) {
-          count ++;
-        }
-      }
-    }
-    // For preventing unnecessary code elimination optimization.
-    LOG.info("The number of INT4 values is " + count + ".");
-  }
-
-  @Test
-  public void testVTuplePutAndGetBenchmarkViaDirectRowEncoder() {
-    int rowNum = 1000;
-
-    OffHeapRowBlock rowBlock = new OffHeapRowBlock(schema, StorageUnit.MB * 100);
-
-    long writeStart = System.currentTimeMillis();
-    VTuple tuple = new VTuple(schema.size());
-    for (int i = 0; i < rowNum; i++) {
-      fillVTuple(i, tuple);
-
-      RowStoreUtil.convert(tuple, rowBlock.getWriter());
-    }
-    long writeEnd = System.currentTimeMillis();
-    LOG.info("Writing takes " + (writeEnd - writeStart) + " msec");
-
-    validateResults(rowBlock);
-    rowBlock.release();
-  }
-
-  @Test
-  public void testSerDerOfRowBlock() {
-    int rowNum = 1000;
-
-    OffHeapRowBlock rowBlock = createRowBlock(rowNum);
-
-    ByteBuffer bb = rowBlock.nioBuffer();
-    OffHeapRowBlock restoredRowBlock = new OffHeapRowBlock(schema, bb);
-    validateResults(restoredRowBlock);
-    rowBlock.release();
-  }
-
-  @Test
-  public void testSerDerOfZeroCopyTuple() {
-    int rowNum = 1000;
-
-    OffHeapRowBlock rowBlock = createRowBlock(rowNum);
-
-    ByteBuffer bb = rowBlock.nioBuffer();
-    OffHeapRowBlock restoredRowBlock = new OffHeapRowBlock(schema, bb);
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(restoredRowBlock);
-
-    long readStart = System.currentTimeMillis();
-    ZeroCopyTuple tuple = new ZeroCopyTuple();
-    ZeroCopyTuple copyTuple = new ZeroCopyTuple();
-    int j = 0;
-    reader.reset();
-    while(reader.next(tuple)) {
-      ByteBuffer copy = tuple.nioBuffer();
-      copyTuple.set(copy, SchemaUtil.toDataTypes(schema));
-
-      validateTupleResult(j, copyTuple);
-
-      j++;
-    }
-    long readEnd = System.currentTimeMillis();
-    LOG.info("reading takes " + (readEnd - readStart) + " msec");
-
-    rowBlock.release();
-  }
-
-  public static OffHeapRowBlock createRowBlock(int rowNum) {
-    long allocateStart = System.currentTimeMillis();
-    OffHeapRowBlock rowBlock = new OffHeapRowBlock(schema, StorageUnit.MB * 8);
-    long allocatedEnd = System.currentTimeMillis();
-    LOG.info(FileUtil.humanReadableByteCount(rowBlock.size(), true) + " bytes allocated "
-        + (allocatedEnd - allocateStart) + " msec");
-
-    long writeStart = System.currentTimeMillis();
-    for (int i = 0; i < rowNum; i++) {
-      fillRow(i, rowBlock.getWriter());
-    }
-    long writeEnd = System.currentTimeMillis();
-    LOG.info("writing takes " + (writeEnd - writeStart) + " msec");
-
-    return rowBlock;
-  }
-
-  public static OffHeapRowBlock createRowBlockWithNull(int rowNum) {
-    long allocateStart = System.currentTimeMillis();
-    OffHeapRowBlock rowBlock = new OffHeapRowBlock(schema, StorageUnit.MB * 8);
-    long allocatedEnd = System.currentTimeMillis();
-    LOG.info(FileUtil.humanReadableByteCount(rowBlock.size(), true) + " bytes allocated "
-        + (allocatedEnd - allocateStart) + " msec");
-
-    long writeStart = System.currentTimeMillis();
-    for (int i = 0; i < rowNum; i++) {
-      fillRowBlockWithNull(i, rowBlock.getWriter());
-    }
-    long writeEnd = System.currentTimeMillis();
-    LOG.info("writing and validating take " + (writeEnd - writeStart) + " msec");
-
-    return rowBlock;
-  }
-
-  public static void fillRow(int i, RowWriter builder) {
-    builder.startRow();
-    builder.putBool(i % 1 == 0 ? true : false); // 0
-    builder.putInt2((short) 1);                 // 1
-    builder.putInt4(i);                         // 2
-    builder.putInt8(i);                         // 3
-    builder.putFloat4(i);                       // 4
-    builder.putFloat8(i);                       // 5
-    builder.putText((UNICODE_FIELD_PREFIX + i).getBytes());  // 6
-    builder.putTimestamp(DatumFactory.createTimestamp("2014-04-16 08:48:00").asInt8() + i); // 7
-    builder.putDate(DatumFactory.createDate("2014-04-16").asInt4() + i); // 8
-    builder.putTime(DatumFactory.createTime("08:48:00").asInt8() + i); // 9
-    builder.putInterval(DatumFactory.createInterval((i + 1) + " hours")); // 10
-    builder.putInet4(DatumFactory.createInet4("192.168.0.1").asInt4() + i); // 11
-    builder.putProtoDatum(new ProtobufDatum(ProtoUtil.convertString(i + ""))); // 12
-    builder.endRow();
-  }
-
-  public static void fillRowBlockWithNull(int i, RowWriter writer) {
-    writer.startRow();
-
-    if (i == 0) {
-      writer.skipField();
-    } else {
-      writer.putBool(i % 1 == 0 ? true : false); // 0
-    }
-    if (i % 1 == 0) {
-      writer.skipField();
-    } else {
-      writer.putInt2((short) 1);                 // 1
-    }
-
-    if (i % 2 == 0) {
-      writer.skipField();
-    } else {
-      writer.putInt4(i);                         // 2
-    }
-
-    if (i % 3 == 0) {
-      writer.skipField();
-    } else {
-      writer.putInt8(i);                         // 3
-    }
-
-    if (i % 4 == 0) {
-      writer.skipField();
-    } else {
-      writer.putFloat4(i);                       // 4
-    }
-
-    if (i % 5 == 0) {
-      writer.skipField();
-    } else {
-      writer.putFloat8(i);                       // 5
-    }
-
-    if (i % 6 == 0) {
-      writer.skipField();
-    } else {
-      writer.putText((UNICODE_FIELD_PREFIX + i).getBytes());  // 6
-    }
-
-    if (i % 7 == 0) {
-      writer.skipField();
-    } else {
-      writer.putTimestamp(DatumFactory.createTimestamp("2014-04-16 08:48:00").asInt8() + i); // 7
-    }
-
-    if (i % 8 == 0) {
-      writer.skipField();
-    } else {
-      writer.putDate(DatumFactory.createDate("2014-04-16").asInt4() + i); // 8
-    }
-
-    if (i % 9 == 0) {
-      writer.skipField();
-    } else {
-      writer.putTime(DatumFactory.createTime("08:48:00").asInt8() + i); // 9
-    }
-
-    if (i % 10 == 0) {
-      writer.skipField();
-    } else {
-      writer.putInterval(DatumFactory.createInterval((i + 1) + " hours")); // 10
-    }
-
-    if (i % 11 == 0) {
-      writer.skipField();
-    } else {
-      writer.putInet4(DatumFactory.createInet4("192.168.0.1").asInt4() + i); // 11
-    }
-
-    if (i % 12 == 0) {
-      writer.skipField();
-    } else {
-      writer.putProtoDatum(new ProtobufDatum(ProtoUtil.convertString(i + ""))); // 12
-    }
-
-    writer.endRow();
-  }
-
-  public static void fillVTuple(int i, VTuple tuple) {
-    tuple.put(0, DatumFactory.createBool(i % 1 == 0));
-    tuple.put(1, DatumFactory.createInt2((short) 1));
-    tuple.put(2, DatumFactory.createInt4(i));
-    tuple.put(3, DatumFactory.createInt8(i));
-    tuple.put(4, DatumFactory.createFloat4(i));
-    tuple.put(5, DatumFactory.createFloat8(i));
-    tuple.put(6, DatumFactory.createText((UNICODE_FIELD_PREFIX + i).getBytes()));
-    tuple.put(7, DatumFactory.createTimestamp(DatumFactory.createTimestamp("2014-04-16 08:48:00").asInt8() + i)); // 7
-    tuple.put(8, DatumFactory.createDate(DatumFactory.createDate("2014-04-16").asInt4() + i)); // 8
-    tuple.put(9, DatumFactory.createTime(DatumFactory.createTime("08:48:00").asInt8() + i)); // 9
-    tuple.put(10, DatumFactory.createInterval((i + 1) + " hours")); // 10
-    tuple.put(11, DatumFactory.createInet4(DatumFactory.createInet4("192.168.0.1").asInt4() + i)); // 11
-    tuple.put(12, new ProtobufDatum(ProtoUtil.convertString(i + ""))); // 12;
-  }
-
-  public static void validateResults(OffHeapRowBlock rowBlock) {
-    long readStart = System.currentTimeMillis();
-    ZeroCopyTuple tuple = new ZeroCopyTuple();
-    int j = 0;
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-    reader.reset();
-    while(reader.next(tuple)) {
-      validateTupleResult(j, tuple);
-      j++;
-    }
-    long readEnd = System.currentTimeMillis();
-    LOG.info("Reading takes " + (readEnd - readStart) + " msec");
-  }
-
-  public static void validateTupleResult(int j, Tuple t) {
-    assertTrue((j % 1 == 0) == t.getBool(0));
-    assertTrue(1 == t.getInt2(1));
-    assertEquals(j, t.getInt4(2));
-    assertEquals(j, t.getInt8(3));
-    assertTrue(j == t.getFloat4(4));
-    assertTrue(j == t.getFloat8(5));
-    assertEquals(new String(UNICODE_FIELD_PREFIX + j), t.getText(6));
-    assertEquals(DatumFactory.createTimestamp("2014-04-16 08:48:00").asInt8() + (long) j, t.getInt8(7));
-    assertEquals(DatumFactory.createDate("2014-04-16").asInt4() + j, t.getInt4(8));
-    assertEquals(DatumFactory.createTime("08:48:00").asInt8() + j, t.getInt8(9));
-    assertEquals(DatumFactory.createInterval((j + 1) + " hours"), t.getInterval(10));
-    assertEquals(DatumFactory.createInet4("192.168.0.1").asInt4() + j, t.getInt4(11));
-    assertEquals(new ProtobufDatum(ProtoUtil.convertString(j + "")), t.getProtobufDatum(12));
-  }
-
-  public static void validateNullity(int j, Tuple tuple) {
-    if (j == 0) {
-      tuple.isNull(0);
-    } else {
-      assertTrue((j % 1 == 0) == tuple.getBool(0));
-    }
-
-    if (j % 1 == 0) {
-      tuple.isNull(1);
-    } else {
-      assertTrue(1 == tuple.getInt2(1));
-    }
-
-    if (j % 2 == 0) {
-      tuple.isNull(2);
-    } else {
-      assertEquals(j, tuple.getInt4(2));
-    }
-
-    if (j % 3 == 0) {
-      tuple.isNull(3);
-    } else {
-      assertEquals(j, tuple.getInt8(3));
-    }
-
-    if (j % 4 == 0) {
-      tuple.isNull(4);
-    } else {
-      assertTrue(j == tuple.getFloat4(4));
-    }
-
-    if (j % 5 == 0) {
-      tuple.isNull(5);
-    } else {
-      assertTrue(j == tuple.getFloat8(5));
-    }
-
-    if (j % 6 == 0) {
-      tuple.isNull(6);
-    } else {
-      assertEquals(new String(UNICODE_FIELD_PREFIX + j), tuple.getText(6));
-    }
-
-    if (j % 7 == 0) {
-      tuple.isNull(7);
-    } else {
-      assertEquals(DatumFactory.createTimestamp("2014-04-16 08:48:00").asInt8() + (long) j, tuple.getInt8(7));
-    }
-
-    if (j % 8 == 0) {
-      tuple.isNull(8);
-    } else {
-      assertEquals(DatumFactory.createDate("2014-04-16").asInt4() + j, tuple.getInt4(8));
-    }
-
-    if (j % 9 == 0) {
-      tuple.isNull(9);
-    } else {
-      assertEquals(DatumFactory.createTime("08:48:00").asInt8() + j, tuple.getInt8(9));
-    }
-
-    if (j % 10 == 0) {
-      tuple.isNull(10);
-    } else {
-      assertEquals(DatumFactory.createInterval((j + 1) + " hours"), tuple.getInterval(10));
-    }
-
-    if (j % 11 == 0) {
-      tuple.isNull(11);
-    } else {
-      assertEquals(DatumFactory.createInet4("192.168.0.1").asInt4() + j, tuple.getInt4(11));
-    }
-
-    if (j % 12 == 0) {
-      tuple.isNull(12);
-    } else {
-      assertEquals(new ProtobufDatum(ProtoUtil.convertString(j + "")), tuple.getProtobufDatum(12));
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java b/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
deleted file mode 100644
index 1eb9c17..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestResizableSpec.java
+++ /dev/null
@@ -1,59 +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.tuple.offheap;
-
-import org.apache.tajo.unit.StorageUnit;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class TestResizableSpec {
-
-  @Test
-  public void testResizableLimit() {
-    ResizableLimitSpec limit = new ResizableLimitSpec(10 * StorageUnit.MB, 1000 * StorageUnit.MB, 0.1f, 1.0f);
-
-    long expectedMaxSize = (long) (1000 * StorageUnit.MB + (1000 * StorageUnit.MB * 0.1f));
-
-    assertTrue(limit.limit() == 1000 * StorageUnit.MB + (1000 * StorageUnit.MB * 0.1f));
-
-    assertEquals(20971520, limit.increasedSize(10 * StorageUnit.MB));
-
-    assertEquals(expectedMaxSize, limit.increasedSize(1600 * StorageUnit.MB));
-
-    assertEquals(0.98f, limit.remainRatio(980 * StorageUnit.MB), 0.1);
-
-    assertFalse(limit.canIncrease(limit.limit()));
-  }
-
-  @Test
-  public void testFixedLimit() {
-    FixedSizeLimitSpec limit = new FixedSizeLimitSpec(100 * StorageUnit.MB, 0.0f);
-
-    assertEquals(limit.limit(), 100 * StorageUnit.MB);
-
-    assertEquals(100 * StorageUnit.MB, limit.increasedSize(1000));
-
-    assertEquals(100 * StorageUnit.MB, limit.increasedSize(1600 * StorageUnit.MB));
-
-    assertTrue(0.98f == limit.remainRatio(98 * StorageUnit.MB));
-
-    assertFalse(limit.canIncrease(limit.limit()));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/resources/storage-default.xml b/tajo-storage/src/test/resources/storage-default.xml
deleted file mode 100644
index 790d5a8..0000000
--- a/tajo-storage/src/test/resources/storage-default.xml
+++ /dev/null
@@ -1,154 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<!--
-  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.
-  -->
-
-<configuration>
-  <property>
-    <name>fs.s3.impl</name>
-    <value>org.apache.tajo.storage.s3.SmallBlockS3FileSystem</value>
-  </property>
-
-  <!--- Registered Scanner Handler -->
-  <property>
-    <name>tajo.storage.scanner-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
-  </property>
-
-  <!--- Fragment Class Configurations -->
-  <property>
-    <name>tajo.storage.fragment.textfile.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.csv.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.raw.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.rcfile.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.row.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.parquet.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.sequencefile.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.avro.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-
-  <!--- Scanner Handler -->
-  <property>
-    <name>tajo.storage.scanner-handler.textfile.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.csv.class</name>
-    <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.raw.class</name>
-    <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.rcfile.class</name>
-    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.rowfile.class</name>
-    <value>org.apache.tajo.storage.RowFile$RowFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.parquet.class</name>
-    <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.sequencefile.class</name>
-    <value>org.apache.tajo.storage.sequencefile.SequenceFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.avro.class</name>
-    <value>org.apache.tajo.storage.avro.AvroScanner</value>
-  </property>
-
-  <!--- Appender Handler -->
-  <property>
-    <name>tajo.storage.appender-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.textfile.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.csv.class</name>
-    <value>org.apache.tajo.storage.CSVFile$CSVAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.raw.class</name>
-    <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.rcfile.class</name>
-    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.rowfile.class</name>
-    <value>org.apache.tajo.storage.RowFile$RowFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.parquet.class</name>
-    <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.sequencefile.class</name>
-    <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.avro.class</name>
-    <value>org.apache.tajo.storage.avro.AvroAppender</value>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/resources/testVariousTypes.avsc
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/resources/testVariousTypes.avsc b/tajo-storage/src/test/resources/testVariousTypes.avsc
deleted file mode 100644
index 611b97f..0000000
--- a/tajo-storage/src/test/resources/testVariousTypes.avsc
+++ /dev/null
@@ -1,21 +0,0 @@
-{
-  "type": "record",
-  "namespace": "org.apache.tajo",
-  "name": "testVariousTypes",
-  "fields": [
-    { "name": "col1", "type": "boolean" },
-    { "name": "col2", "type": "int" },
-    { "name": "col3", "type": "string" },
-    { "name": "col4", "type": "int" },
-    { "name": "col5", "type": "int" },
-    { "name": "col6", "type": "long" },
-    { "name": "col7", "type": "float" },
-    { "name": "col8", "type": "double" },
-    { "name": "col9", "type": "string" },
-    { "name": "col10", "type": "bytes" },
-    { "name": "col11", "type": "bytes" },
-    { "name": "col12", "type": "null" },
-    { "name": "col13", "type": "bytes" }
-  ]
-}
-

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..c600b4b
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/pom.xml
@@ -0,0 +1,337 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.9.1-SNAPSHOT</version>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>tajo-storage-common</artifactId>
+  <packaging>jar</packaging>
+  <name>Tajo Storage Common</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/IndexProtos.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-pmd-plugin</artifactId>
+        <version>2.7.1</version>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-plan</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>slf4j-api</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jersey-json</artifactId>
+          <groupId>com.sun.jersey</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-server-tests</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-hs</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-buffer</artifactId>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Appender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Appender.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Appender.java
new file mode 100644
index 0000000..c5e96ac
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Appender.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.catalog.statistics.TableStats;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public interface Appender extends Closeable {
+
+  void init() throws IOException;
+
+  void addTuple(Tuple t) throws IOException;
+  
+  void flush() throws IOException;
+
+  long getEstimatedOutputSize() throws IOException;
+  
+  void close() throws IOException;
+
+  void enableStats();
+  
+  TableStats getStats();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
new file mode 100644
index 0000000..b829f60
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BaseTupleComparator.java
@@ -0,0 +1,206 @@
+/**
+ * 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.Objects;
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.datum.Datum;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.TupleComparatorSpecProto;
+import static org.apache.tajo.index.IndexProtos.TupleComparatorProto;
+
+/**
+ * The Comparator class for Tuples
+ * 
+ * @see Tuple
+ */
+public class BaseTupleComparator extends TupleComparator implements ProtoObject<TupleComparatorProto> {
+  private final Schema schema;
+  private final SortSpec [] sortSpecs;
+  private final int[] sortKeyIds;
+  private final boolean[] asc;
+  @SuppressWarnings("unused")
+  private final boolean[] nullFirsts;  
+
+  private Datum left;
+  private Datum right;
+  private int compVal;
+
+  /**
+   * @param schema The schema of input tuples
+   * @param sortKeys The description of sort keys
+   */
+  public BaseTupleComparator(Schema schema, SortSpec[] sortKeys) {
+    Preconditions.checkArgument(sortKeys.length > 0, 
+        "At least one sort key must be specified.");
+
+    this.schema = schema;
+    this.sortSpecs = sortKeys;
+    this.sortKeyIds = new int[sortKeys.length];
+    this.asc = new boolean[sortKeys.length];
+    this.nullFirsts = new boolean[sortKeys.length];
+    for (int i = 0; i < sortKeys.length; i++) {
+      if (sortKeys[i].getSortKey().hasQualifier()) {
+        this.sortKeyIds[i] = schema.getColumnId(sortKeys[i].getSortKey().getQualifiedName());
+      } else {
+        this.sortKeyIds[i] = schema.getColumnIdByName(sortKeys[i].getSortKey().getSimpleName());
+      }
+          
+      this.asc[i] = sortKeys[i].isAscending();
+      this.nullFirsts[i]= sortKeys[i].isNullFirst();
+    }
+  }
+
+  public BaseTupleComparator(TupleComparatorProto proto) {
+    this.schema = new Schema(proto.getSchema());
+
+    this.sortSpecs = new SortSpec[proto.getSortSpecsCount()];
+    for (int i = 0; i < proto.getSortSpecsCount(); i++) {
+      sortSpecs[i] = new SortSpec(proto.getSortSpecs(i));
+    }
+
+    this.sortKeyIds = new int[proto.getCompSpecsCount()];
+    this.asc = new boolean[proto.getCompSpecsCount()];
+    this.nullFirsts = new boolean[proto.getCompSpecsCount()];
+
+    for (int i = 0; i < proto.getCompSpecsCount(); i++) {
+      TupleComparatorSpecProto sortSepcProto = proto.getCompSpecs(i);
+      sortKeyIds[i] = sortSepcProto.getColumnId();
+      asc[i] = sortSepcProto.getAscending();
+      nullFirsts[i] = sortSepcProto.getNullFirst();
+    }
+  }
+
+  public Schema getSchema() {
+    return schema;
+  }
+
+  public SortSpec [] getSortSpecs() {
+    return sortSpecs;
+  }
+
+  public int [] getSortKeyIds() {
+    return sortKeyIds;
+  }
+
+  @Override
+  public boolean isAscendingFirstKey() {
+    return this.asc[0];
+  }
+
+  @Override
+  public int compare(Tuple tuple1, Tuple tuple2) {
+    for (int i = 0; i < sortKeyIds.length; i++) {
+      left = tuple1.get(sortKeyIds[i]);
+      right = tuple2.get(sortKeyIds[i]);
+
+      if (left.isNull() || right.isNull()) {
+        if (!left.equals(right)) {
+          if (left.isNull()) {
+            compVal = 1;
+          } else if (right.isNull()) {
+            compVal = -1;
+          }
+          if (nullFirsts[i]) {
+            if (compVal != 0) {
+              compVal *= -1;
+            }
+          }
+        } else {
+          compVal = 0;
+        }
+      } else {
+        if (asc[i]) {
+          compVal = left.compareTo(right);
+        } else {
+          compVal = right.compareTo(left);
+        }
+      }
+
+      if (compVal < 0 || compVal > 0) {
+        return compVal;
+      }
+    }
+    return 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(sortKeyIds);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof BaseTupleComparator) {
+      BaseTupleComparator other = (BaseTupleComparator) obj;
+      if (sortKeyIds.length != other.sortKeyIds.length) {
+        return false;
+      }
+
+      for (int i = 0; i < sortKeyIds.length; i++) {
+        if (sortKeyIds[i] != other.sortKeyIds[i] ||
+            asc[i] != other.asc[i] ||
+            nullFirsts[i] != other.nullFirsts[i]) {
+          return false;
+        }
+      }
+
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public TupleComparatorProto getProto() {
+    TupleComparatorProto.Builder builder = TupleComparatorProto.newBuilder();
+    builder.setSchema(schema.getProto());
+    for (int i = 0; i < sortSpecs.length; i++) {
+      builder.addSortSpecs(sortSpecs[i].getProto());
+    }
+
+    TupleComparatorSpecProto.Builder sortSpecBuilder;
+    for (int i = 0; i < sortKeyIds.length; i++) {
+      sortSpecBuilder = TupleComparatorSpecProto.newBuilder();
+      sortSpecBuilder.setColumnId(sortKeyIds[i]);
+      sortSpecBuilder.setAscending(asc[i]);
+      sortSpecBuilder.setNullFirst(nullFirsts[i]);
+      builder.addCompSpecs(sortSpecBuilder);
+    }
+
+    return builder.build();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+
+    String prefix = "";
+    for (int i = 0; i < sortKeyIds.length; i++) {
+      sb.append(prefix).append("SortKeyId=").append(sortKeyIds[i])
+        .append(",Asc=").append(asc[i])
+        .append(",NullFirst=").append(nullFirsts[i]);
+      prefix = " ,";
+    }
+    return sb.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
new file mode 100644
index 0000000..00112e7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
@@ -0,0 +1,258 @@
+/**
+ * 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.Preconditions;
+import com.google.protobuf.Message;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.util.Bytes;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+@Deprecated
+public class BinarySerializerDeserializer implements SerializerDeserializer {
+
+  static final byte[] INVALID_UTF__SINGLE_BYTE = {(byte) Integer.parseInt("10111111", 2)};
+
+  @Override
+  public int serialize(Column col, Datum datum, OutputStream out, byte[] nullCharacters)
+      throws IOException {
+    byte[] bytes;
+    int length = 0;
+    if (datum == null || datum instanceof NullDatum) {
+      return 0;
+    }
+
+    switch (col.getDataType().getType()) {
+      case BOOLEAN:
+      case BIT:
+      case CHAR:
+        bytes = datum.asByteArray();
+        length = bytes.length;
+        out.write(bytes, 0, length);
+        break;
+      case INT2:
+        length = writeShort(out, datum.asInt2());
+        break;
+      case INT4:
+        length = writeVLong(out, datum.asInt4());
+        break;
+      case INT8:
+        length = writeVLong(out, datum.asInt8());
+        break;
+      case FLOAT4:
+        length = writeFloat(out, datum.asFloat4());
+        break;
+      case FLOAT8:
+        length = writeDouble(out, datum.asFloat8());
+        break;
+      case TEXT: {
+        bytes = datum.asTextBytes();
+        length = datum.size();
+        if (length == 0) {
+          bytes = INVALID_UTF__SINGLE_BYTE;
+          length = INVALID_UTF__SINGLE_BYTE.length;
+        }
+        out.write(bytes, 0, bytes.length);
+        break;
+      }
+      case BLOB:
+      case INET4:
+      case INET6:
+        bytes = datum.asByteArray();
+        length = bytes.length;
+        out.write(bytes, 0, length);
+        break;
+      case PROTOBUF:
+        ProtobufDatum protobufDatum = (ProtobufDatum) datum;
+        bytes = protobufDatum.asByteArray();
+        length = bytes.length;
+        out.write(bytes, 0, length);
+        break;
+      case NULL_TYPE:
+        break;
+      default:
+        throw new IOException("Does not support type");
+    }
+    return length;
+  }
+
+  @Override
+  public Datum deserialize(Column col, byte[] bytes, int offset, int length, byte[] nullCharacters) throws IOException {
+    if (length == 0) return NullDatum.get();
+
+    Datum datum;
+    switch (col.getDataType().getType()) {
+      case BOOLEAN:
+        datum = DatumFactory.createBool(bytes[offset]);
+        break;
+      case BIT:
+        datum = DatumFactory.createBit(bytes[offset]);
+        break;
+      case CHAR: {
+        byte[] chars = new byte[length];
+        System.arraycopy(bytes, offset, chars, 0, length);
+        datum = DatumFactory.createChar(chars);
+        break;
+      }
+      case INT2:
+        datum = DatumFactory.createInt2(Bytes.toShort(bytes, offset, length));
+        break;
+      case INT4:
+        datum = DatumFactory.createInt4((int) Bytes.readVLong(bytes, offset));
+        break;
+      case INT8:
+        datum = DatumFactory.createInt8(Bytes.readVLong(bytes, offset));
+        break;
+      case FLOAT4:
+        datum = DatumFactory.createFloat4(toFloat(bytes, offset, length));
+        break;
+      case FLOAT8:
+        datum = DatumFactory.createFloat8(toDouble(bytes, offset, length));
+        break;
+      case TEXT: {
+        byte[] chars = new byte[length];
+        System.arraycopy(bytes, offset, chars, 0, length);
+
+        if (Bytes.equals(INVALID_UTF__SINGLE_BYTE, chars)) {
+          datum = DatumFactory.createText(new byte[0]);
+        } else {
+          datum = DatumFactory.createText(chars);
+        }
+        break;
+      }
+      case PROTOBUF: {
+        ProtobufDatumFactory factory = ProtobufDatumFactory.get(col.getDataType().getCode());
+        Message.Builder builder = factory.newBuilder();
+        builder.mergeFrom(bytes, offset, length);
+        datum = factory.createDatum(builder);
+        break;
+      }
+      case INET4:
+        datum = DatumFactory.createInet4(bytes, offset, length);
+        break;
+      case BLOB:
+        datum = DatumFactory.createBlob(bytes, offset, length);
+        break;
+      default:
+        datum = NullDatum.get();
+    }
+    return datum;
+  }
+
+  private byte[] shortBytes = new byte[2];
+
+  public int writeShort(OutputStream out, short val) throws IOException {
+    shortBytes[0] = (byte) (val >> 8);
+    shortBytes[1] = (byte) val;
+    out.write(shortBytes, 0, 2);
+    return 2;
+  }
+
+  public float toFloat(byte[] bytes, int offset, int length) {
+    Preconditions.checkArgument(length == 4);
+
+    int val = ((bytes[offset] & 0x000000FF) << 24) +
+        ((bytes[offset + 1] & 0x000000FF) << 16) +
+        ((bytes[offset + 2] & 0x000000FF) << 8) +
+        (bytes[offset + 3] & 0x000000FF);
+    return Float.intBitsToFloat(val);
+  }
+
+  private byte[] floatBytes = new byte[4];
+
+  public int writeFloat(OutputStream out, float f) throws IOException {
+    int val = Float.floatToIntBits(f);
+
+    floatBytes[0] = (byte) (val >> 24);
+    floatBytes[1] = (byte) (val >> 16);
+    floatBytes[2] = (byte) (val >> 8);
+    floatBytes[3] = (byte) val;
+    out.write(floatBytes, 0, 4);
+    return floatBytes.length;
+  }
+
+  public double toDouble(byte[] bytes, int offset, int length) {
+    Preconditions.checkArgument(length == 8);
+    long val = ((long) (bytes[offset] & 0x00000000000000FF) << 56) +
+        ((long) (bytes[offset + 1] & 0x00000000000000FF) << 48) +
+        ((long) (bytes[offset + 2] & 0x00000000000000FF) << 40) +
+        ((long) (bytes[offset + 3] & 0x00000000000000FF) << 32) +
+        ((long) (bytes[offset + 4] & 0x00000000000000FF) << 24) +
+        ((long) (bytes[offset + 5] & 0x00000000000000FF) << 16) +
+        ((long) (bytes[offset + 6] & 0x00000000000000FF) << 8) +
+        (long) (bytes[offset + 7] & 0x00000000000000FF);
+    return Double.longBitsToDouble(val);
+  }
+
+  private byte[] doubleBytes = new byte[8];
+
+  public int writeDouble(OutputStream out, double d) throws IOException {
+    long val = Double.doubleToLongBits(d);
+
+    doubleBytes[0] = (byte) (val >> 56);
+    doubleBytes[1] = (byte) (val >> 48);
+    doubleBytes[2] = (byte) (val >> 40);
+    doubleBytes[3] = (byte) (val >> 32);
+    doubleBytes[4] = (byte) (val >> 24);
+    doubleBytes[5] = (byte) (val >> 16);
+    doubleBytes[6] = (byte) (val >> 8);
+    doubleBytes[7] = (byte) val;
+    out.write(doubleBytes, 0, 8);
+    return doubleBytes.length;
+  }
+
+  private byte[] vLongBytes = new byte[9];
+
+  public static int writeVLongToByteArray(byte[] bytes, int offset, long l) {
+    if (l >= -112 && l <= 127) {
+      bytes[offset] = (byte) l;
+      return 1;
+    }
+
+    int len = -112;
+    if (l < 0) {
+      l ^= -1L; // take one's complement'
+      len = -120;
+    }
+
+    long tmp = l;
+    while (tmp != 0) {
+      tmp = tmp >> 8;
+      len--;
+    }
+
+    bytes[offset++] = (byte) len;
+    len = (len < -120) ? -(len + 120) : -(len + 112);
+
+    for (int idx = len; idx != 0; idx--) {
+      int shiftbits = (idx - 1) * 8;
+      bytes[offset++] = (byte) ((l & (0xFFL << shiftbits)) >> shiftbits);
+    }
+    return 1 + len;
+  }
+
+  public int writeVLong(OutputStream out, long l) throws IOException {
+    int len = writeVLongToByteArray(vLongBytes, 0, l);
+    out.write(vLongBytes, 0, len);
+    return len;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BufferPool.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BufferPool.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BufferPool.java
new file mode 100644
index 0000000..85c79fa
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BufferPool.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.util.internal.PlatformDependent;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/* this class is PooledBuffer holder */
+public class BufferPool {
+
+  private static final PooledByteBufAllocator allocator;
+
+  private BufferPool() {
+  }
+
+  static {
+    //TODO we need determine the default params
+    allocator = new PooledByteBufAllocator(PlatformDependent.directBufferPreferred());
+
+    /* if you are finding memory leak, please enable this line */
+    //ResourceLeakDetector.setLevel(ResourceLeakDetector.Level.ADVANCED);
+  }
+
+  public static long maxDirectMemory() {
+    return PlatformDependent.maxDirectMemory();
+  }
+
+
+  public synchronized static ByteBuf directBuffer(int size) {
+    return allocator.directBuffer(size);
+  }
+
+  /**
+   *
+   * @param size the initial capacity
+   * @param max the max capacity
+   * @return allocated ByteBuf from pool
+   */
+  public static ByteBuf directBuffer(int size, int max) {
+    return allocator.directBuffer(size, max);
+  }
+
+  @InterfaceStability.Unstable
+  public static void forceRelease(ByteBuf buf) {
+    buf.release(buf.refCnt());
+  }
+
+  /**
+   * the ByteBuf will increase to writable size
+   * @param buf
+   * @param minWritableBytes required minimum writable size
+   */
+  public static void ensureWritable(ByteBuf buf, int minWritableBytes) {
+    buf.ensureWritable(minWritableBytes);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
new file mode 100644
index 0000000..b1b6d65
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/ByteBufInputChannel.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.io.IOUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.ScatteringByteChannel;
+import java.nio.channels.spi.AbstractInterruptibleChannel;
+
+public class ByteBufInputChannel extends AbstractInterruptibleChannel implements ScatteringByteChannel {
+
+  ByteBufferReadable byteBufferReadable;
+  ReadableByteChannel channel;
+  InputStream inputStream;
+
+  public ByteBufInputChannel(InputStream inputStream) {
+    if (inputStream instanceof DFSInputStream && inputStream instanceof ByteBufferReadable) {
+      this.byteBufferReadable = (ByteBufferReadable) inputStream;
+    } else {
+      this.channel = Channels.newChannel(inputStream);
+    }
+
+    this.inputStream = inputStream;
+  }
+
+  @Override
+  public long read(ByteBuffer[] dsts, int offset, int length) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long read(ByteBuffer[] dsts) {
+    return read(dsts, 0, dsts.length);
+  }
+
+  @Override
+  public int read(ByteBuffer dst) throws IOException {
+    if (byteBufferReadable != null) {
+      return byteBufferReadable.read(dst);
+    } else {
+      return channel.read(dst);
+    }
+  }
+
+  @Override
+  protected void implCloseChannel() throws IOException {
+    IOUtils.cleanup(null, channel, inputStream);
+  }
+
+  public int available() throws IOException {
+    return inputStream.available();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DataLocation.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DataLocation.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DataLocation.java
new file mode 100644
index 0000000..8841a31
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DataLocation.java
@@ -0,0 +1,45 @@
+/*
+ * 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 DataLocation {
+  private String host;
+  private int volumeId;
+
+  public DataLocation(String host, int volumeId) {
+    this.host = host;
+    this.volumeId = volumeId;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public int getVolumeId() {
+    return volumeId;
+  }
+
+  @Override
+  public String toString() {
+    return "DataLocation{" +
+        "host=" + host +
+        ", volumeId=" + volumeId +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
new file mode 100644
index 0000000..2396349
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskDeviceInfo.java
@@ -0,0 +1,62 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+public class DiskDeviceInfo {
+	private int id;
+	private String name;
+	
+	private List<DiskMountInfo> mountInfos = new ArrayList<DiskMountInfo>();
+
+	public DiskDeviceInfo(int id) {
+		this.id = id;
+	}
+	
+	public int getId() {
+		return id;
+	}
+
+	public String getName() {
+		return name;
+	}
+
+	public void setName(String name) {
+		this.name = name;
+	}
+	
+	@Override
+	public String toString() {
+		return id + "," + name;
+	}
+
+	public void addMountPath(DiskMountInfo diskMountInfo) {
+		mountInfos.add(diskMountInfo);
+	}
+
+	public List<DiskMountInfo> getMountInfos() {
+		return mountInfos;
+	}
+
+	public void setMountInfos(List<DiskMountInfo> mountInfos) {
+		this.mountInfos = mountInfos;
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskInfo.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskInfo.java
new file mode 100644
index 0000000..22f18ba
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskInfo.java
@@ -0,0 +1,75 @@
+/**
+ * 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 DiskInfo {
+	private int id;
+	private String partitionName;
+	private String mountPath;
+	
+	private long capacity;
+	private long used;
+	
+	public DiskInfo(int id, String partitionName) {
+		this.id = id;
+		this.partitionName = partitionName;
+	}
+
+	public int getId() {
+		return id;
+	}
+
+	public void setId(int id) {
+		this.id = id;
+	}
+
+	public String getPartitionName() {
+		return partitionName;
+	}
+
+	public void setPartitionName(String partitionName) {
+		this.partitionName = partitionName;
+	}
+
+	public String getMountPath() {
+		return mountPath;
+	}
+
+	public void setMountPath(String mountPath) {
+		this.mountPath = mountPath;
+	}
+
+	public long getCapacity() {
+		return capacity;
+	}
+
+	public void setCapacity(long capacity) {
+		this.capacity = capacity;
+	}
+
+	public long getUsed() {
+		return used;
+	}
+
+	public void setUsed(long used) {
+		this.used = used;
+	}
+	
+	
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskMountInfo.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
new file mode 100644
index 0000000..aadb0e7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskMountInfo.java
@@ -0,0 +1,101 @@
+/**
+ * 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.Objects;
+
+public class DiskMountInfo implements Comparable<DiskMountInfo> {
+	private String mountPath;
+	
+	private long capacity;
+	private long used;
+	
+	private int deviceId;
+	
+	public DiskMountInfo(int deviceId, String mountPath) {
+		this.mountPath = mountPath;
+	}
+
+	public String getMountPath() {
+		return mountPath;
+	}
+
+	public void setMountPath(String mountPath) {
+		this.mountPath = mountPath;
+	}
+
+	public long getCapacity() {
+		return capacity;
+	}
+
+	public void setCapacity(long capacity) {
+		this.capacity = capacity;
+	}
+
+	public long getUsed() {
+		return used;
+	}
+
+	public void setUsed(long used) {
+		this.used = used;
+	}
+
+	public int getDeviceId() {
+		return deviceId;
+	}
+
+  @Override
+  public boolean equals(Object obj){
+    if (!(obj instanceof DiskMountInfo)) return false;
+
+    if (compareTo((DiskMountInfo) obj) == 0) return true;
+    else return false;
+  }
+
+  @Override
+  public int hashCode(){
+    return Objects.hashCode(mountPath);
+  }
+
+	@Override
+	public int compareTo(DiskMountInfo other) {
+		String path1 = mountPath;
+		String path2 = other.mountPath;
+		
+		int path1Depth = "/".equals(path1) ? 0 : path1.split("/", -1).length - 1 ;
+		int path2Depth = "/".equals(path2) ? 0 : path2.split("/", -1).length - 1 ;
+		
+		if(path1Depth > path2Depth) {
+			return -1;
+		} else if(path1Depth < path2Depth) {
+			return 1;
+		} else {
+			int path1Length = path1.length();
+			int path2Length = path2.length();
+			
+			if(path1Length < path2Length) {
+				return 1;
+			} else if(path1Length > path2Length) {
+				return -1;
+			} else {
+				return path1.compareTo(path2);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java
new file mode 100644
index 0000000..2d68870
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java
@@ -0,0 +1,207 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.common.Util;
+
+import java.io.*;
+import java.net.URI;
+import java.util.*;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+
+public class DiskUtil {
+
+  static String UNIX_DISK_DEVICE_PATH = "/proc/partitions";
+
+  public enum OSType {
+		OS_TYPE_UNIX, OS_TYPE_WINXP, OS_TYPE_SOLARIS, OS_TYPE_MAC
+	}
+
+	static private OSType getOSType() {
+		String osName = System.getProperty("os.name");
+		if (osName.contains("Windows")
+				&& (osName.contains("XP") || osName.contains("2003")
+						|| osName.contains("Vista")
+						|| osName.contains("Windows_7")
+						|| osName.contains("Windows 7") || osName
+							.contains("Windows7"))) {
+			return OSType.OS_TYPE_WINXP;
+		} else if (osName.contains("SunOS") || osName.contains("Solaris")) {
+			return OSType.OS_TYPE_SOLARIS;
+		} else if (osName.contains("Mac")) {
+			return OSType.OS_TYPE_MAC;
+		} else {
+			return OSType.OS_TYPE_UNIX;
+		}
+	}
+	
+	public static List<DiskDeviceInfo> getDiskDeviceInfos() throws IOException {
+		List<DiskDeviceInfo> deviceInfos;
+		
+		if(getOSType() == OSType.OS_TYPE_UNIX) {
+			deviceInfos = getUnixDiskDeviceInfos();
+			setDeviceMountInfo(deviceInfos);
+		} else {
+			deviceInfos = getDefaultDiskDeviceInfos();
+		}
+		
+		return deviceInfos;
+	}
+
+	private static List<DiskDeviceInfo> getUnixDiskDeviceInfos() {
+		List<DiskDeviceInfo> infos = new ArrayList<DiskDeviceInfo>();
+		
+		File file = new File(UNIX_DISK_DEVICE_PATH);
+		if(!file.exists()) {
+			System.out.println("No partition file:" + file.getAbsolutePath());
+			return getDefaultDiskDeviceInfos();
+		}
+		
+		BufferedReader reader = null;
+		try {
+			reader = new BufferedReader(new InputStreamReader(new FileInputStream(UNIX_DISK_DEVICE_PATH)));
+			String line = null;
+			
+			int count = 0;
+			Set<String> deviceNames = new TreeSet<String>();
+			while((line = reader.readLine()) != null) {
+				if(count > 0 && !line.trim().isEmpty()) {
+					String[] tokens = line.trim().split(" +");
+					if(tokens.length == 4) {
+						String deviceName = getDiskDeviceName(tokens[3]);
+						deviceNames.add(deviceName);
+					}
+				}
+				count++;
+			}
+			
+			int id = 0;
+			for(String eachDeviceName: deviceNames) {
+				DiskDeviceInfo diskDeviceInfo = new DiskDeviceInfo(id++);
+				diskDeviceInfo.setName(eachDeviceName);
+				
+				//TODO set addtional info
+				// /sys/block/sda/queue
+				infos.add(diskDeviceInfo);
+			}
+		} catch (Exception e) {
+			e.printStackTrace();
+		} finally {
+			if(reader != null) {
+				try {
+					reader.close();
+				} catch (IOException e) {
+				}
+			}
+		}
+		
+		return infos;
+	}
+	
+	private static String getDiskDeviceName(String partitionName) {
+		byte[] bytes = partitionName.getBytes();
+		
+		byte[] result = new byte[bytes.length];
+		int length = 0;
+		for(int i = 0; i < bytes.length; i++, length++) {
+			if(bytes[i] >= '0' && bytes[i] <= '9') {
+				break;
+			} else {
+				result[i] = bytes[i];
+			}
+		}
+		
+		return new String(result, 0, length);
+	}
+	
+	public static List<DiskDeviceInfo> getDefaultDiskDeviceInfos() {
+		DiskDeviceInfo diskDeviceInfo = new DiskDeviceInfo(0);
+		diskDeviceInfo.setName("default");
+		
+		List<DiskDeviceInfo> infos = new ArrayList<DiskDeviceInfo>();
+		
+		infos.add(diskDeviceInfo);
+		
+		return infos;
+	}
+	
+	
+	private static void setDeviceMountInfo(List<DiskDeviceInfo> deviceInfos) throws IOException {
+		Map<String, DiskDeviceInfo> deviceMap = new HashMap<String, DiskDeviceInfo>();
+		for(DiskDeviceInfo eachDevice: deviceInfos) {
+			deviceMap.put(eachDevice.getName(), eachDevice);
+		}
+		
+		BufferedReader mountOutput = null;
+		try {
+			Process mountProcess = Runtime.getRuntime().exec("mount");
+			mountOutput = new BufferedReader(new InputStreamReader(
+					mountProcess.getInputStream()));
+			while (true) {
+				String line = mountOutput.readLine();
+				if (line == null) {
+					break;
+				}
+
+				int indexStart = line.indexOf(" on /");
+				int indexEnd = line.indexOf(" ", indexStart + 4);
+
+				String deviceName = line.substring(0, indexStart).trim();
+				String[] deviceNameTokens = deviceName.split("/");
+				if(deviceNameTokens.length == 3) {
+					if("dev".equals(deviceNameTokens[1])) {
+						String realDeviceName = getDiskDeviceName(deviceNameTokens[2]);
+						String mountPath = new File(line.substring(indexStart + 4, indexEnd)).getAbsolutePath();
+						
+						DiskDeviceInfo diskDeviceInfo = deviceMap.get(realDeviceName);
+						if(diskDeviceInfo != null) {
+							diskDeviceInfo.addMountPath(new DiskMountInfo(diskDeviceInfo.getId(), mountPath));
+						}
+					}
+				}
+			}
+		} catch (IOException e) {
+			throw e;
+		} finally {
+			if (mountOutput != null) {
+				mountOutput.close();
+			}
+		}
+	}
+
+  public static int getDataNodeStorageSize(){
+    return getStorageDirs().size();
+  }
+
+  public static List<URI> getStorageDirs(){
+    Configuration conf = new HdfsConfiguration();
+    Collection<String> dirNames = conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
+    return Util.stringCollectionAsURIs(dirNames);
+  }
+
+	public static void main(String[] args) throws Exception {
+		System.out.println("/dev/sde1".split("/").length);
+		for(String eachToken: "/dev/sde1".split("/")) {
+			System.out.println(eachToken);
+		}
+ 	}
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
new file mode 100644
index 0000000..7df4584
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
@@ -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.
+ */
+
+package org.apache.tajo.storage;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+
+public interface FieldSerializerDeserializer {
+
+  public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) throws IOException;
+
+  public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java
new file mode 100644
index 0000000..8b7e2e0
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/FrameTuple.java
@@ -0,0 +1,225 @@
+/**
+ * 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.Preconditions;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.IntervalDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.exception.UnsupportedException;
+
+/**
+ * An instance of FrameTuple is an immutable tuple.
+ * It contains two tuples and pretends to be one instance of Tuple for
+ * join qual evaluatations.
+ */
+public class FrameTuple implements Tuple, Cloneable {
+  private int size;
+  private int leftSize;
+  
+  private Tuple left;
+  private Tuple right;
+  
+  public FrameTuple() {}
+  
+  public FrameTuple(Tuple left, Tuple right) {
+    set(left, right);
+  }
+  
+  public void set(Tuple left, Tuple right) {
+    this.size = left.size() + right.size();
+    this.left = left;
+    this.leftSize = left.size();
+    this.right = right;
+  }
+
+  @Override
+  public int size() {
+    return size;
+  }
+
+  @Override
+  public boolean contains(int fieldId) {
+    Preconditions.checkArgument(fieldId < size, 
+        "Out of field access: " + fieldId);
+    
+    if (fieldId < leftSize) {
+      return left.contains(fieldId);
+    } else {
+      return right.contains(fieldId - leftSize);
+    }
+  }
+
+  @Override
+  public boolean isNull(int fieldid) {
+    return get(fieldid).isNull();
+  }
+
+  @Override
+  public boolean isNotNull(int fieldid) {
+    return !isNull(fieldid);
+  }
+
+  @Override
+  public void clear() {
+    throw new UnsupportedException();
+  }
+
+  @Override
+  public void put(int fieldId, Datum value) {
+    throw new UnsupportedException();
+  }
+
+  @Override
+  public void put(int fieldId, Datum[] values) {
+    throw new UnsupportedException();
+  }
+
+  @Override
+  public void put(int fieldId, Tuple tuple) {
+    throw new UnsupportedException();
+  }
+
+  @Override
+  public void setOffset(long offset) {
+    throw new UnsupportedException();
+  }
+  
+  @Override
+  public long getOffset() {
+    throw new UnsupportedException();
+  }
+
+  @Override
+  public void put(Datum [] values) {
+    throw new UnsupportedException();
+  }
+
+  @Override
+  public Datum get(int fieldId) {
+    Preconditions.checkArgument(fieldId < size, 
+        "Out of field access: " + fieldId);
+    
+    if (fieldId < leftSize) {
+      return left.get(fieldId);
+    } else {
+      return right.get(fieldId - leftSize);
+    }
+  }
+
+  @Override
+  public boolean getBool(int fieldId) {
+    return get(fieldId).asBool();
+  }
+
+  @Override
+  public byte getByte(int fieldId) {
+    return get(fieldId).asByte();
+  }
+
+  @Override
+  public char getChar(int fieldId) {
+    return get(fieldId).asChar();
+  }
+
+  @Override
+  public byte [] getBytes(int fieldId) {
+    return get(fieldId).asByteArray();
+  }
+
+  @Override
+  public short getInt2(int fieldId) {
+    return get(fieldId).asInt2();
+  }
+
+  @Override
+  public int getInt4(int fieldId) {
+    return get(fieldId).asInt4();
+  }
+
+  @Override
+  public long getInt8(int fieldId) {
+    return get(fieldId).asInt8();
+  }
+
+  @Override
+  public float getFloat4(int fieldId) {
+    return get(fieldId).asFloat4();
+  }
+
+  @Override
+  public double getFloat8(int fieldId) {
+    return get(fieldId).asFloat8();
+  }
+
+  @Override
+  public String getText(int fieldId) {
+    return get(fieldId).asChars();
+  }
+
+  @Override
+  public ProtobufDatum getProtobufDatum(int fieldId) {
+    return (ProtobufDatum) get(fieldId);
+  }
+
+  @Override
+  public IntervalDatum getInterval(int fieldId) {
+    return (IntervalDatum) get(fieldId);
+  }
+
+  @Override
+  public char [] getUnicodeChars(int fieldId) {
+    return get(fieldId).asUnicodeChars();
+  }
+
+  @Override
+  public Tuple clone() throws CloneNotSupportedException {
+    FrameTuple frameTuple = (FrameTuple) super.clone();
+    frameTuple.set(this.left.clone(), this.right.clone());
+    return frameTuple;
+  }
+
+  @Override
+  public Datum[] getValues(){
+    throw new UnsupportedException();
+  }
+
+  public String toString() {
+    boolean first = true;
+    StringBuilder str = new StringBuilder();
+    str.append("(");
+    for(int i=0; i < size(); i++) {      
+      if(contains(i)) {
+        if(first) {
+          first = false;
+        } else {
+          str.append(", ");
+        }
+        str.append(i)
+        .append("=>")
+        .append(get(i));
+      }
+    }
+    str.append(")");
+    return str.toString();
+  }
+}


[25/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
deleted file mode 100644
index 72472fc..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java
+++ /dev/null
@@ -1,287 +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.avro;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.Message;
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.file.SeekableInput;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericFixed;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.mapred.FsInput;
-import org.apache.avro.util.Utf8;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.datum.*;
-import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * FileScanner for reading Avro files
- */
-public class AvroScanner extends FileScanner {
-  private Schema avroSchema;
-  private List<Schema.Field> avroFields;
-  private DataFileReader<GenericRecord> dataFileReader;
-  private int[] projectionMap;
-
-  /**
-   * Creates a new AvroScanner.
-   *
-   * @param conf
-   * @param schema
-   * @param meta
-   * @param fragment
-   */
-  public AvroScanner(Configuration conf,
-                     final org.apache.tajo.catalog.Schema schema,
-                     final TableMeta meta, final Fragment fragment) {
-    super(conf, schema, meta, fragment);
-  }
-
-  /**
-   * Initializes the AvroScanner.
-   */
-  @Override
-  public void init() throws IOException {
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-    prepareProjection(targets);
-
-    avroSchema = AvroUtil.getAvroSchema(meta, conf);
-    avroFields = avroSchema.getFields();
-
-    DatumReader<GenericRecord> datumReader =
-        new GenericDatumReader<GenericRecord>(avroSchema);
-    SeekableInput input = new FsInput(fragment.getPath(), conf);
-    dataFileReader = new DataFileReader<GenericRecord>(input, datumReader);
-    super.init();
-  }
-
-  private void prepareProjection(Column[] targets) {
-    projectionMap = new int[targets.length];
-    for (int i = 0; i < targets.length; ++i) {
-      projectionMap[i] = schema.getColumnId(targets[i].getQualifiedName());
-    }
-  }
-
-  private static String fromAvroString(Object value) {
-    if (value instanceof Utf8) {
-      Utf8 utf8 = (Utf8)value;
-      return utf8.toString();
-    }
-    return value.toString();
-  }
-
-  private static Schema getNonNull(Schema schema) {
-    if (!schema.getType().equals(Schema.Type.UNION)) {
-      return schema;
-    }
-    List<Schema> schemas = schema.getTypes();
-    if (schemas.size() != 2) {
-      return schema;
-    }
-    if (schemas.get(0).getType().equals(Schema.Type.NULL)) {
-      return schemas.get(1);
-    } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) {
-      return schemas.get(0);
-    } else {
-      return schema;
-    }
-  }
-
-  private Datum convertInt(Object value, TajoDataTypes.Type tajoType) {
-    int intValue = (Integer)value;
-    switch (tajoType) {
-      case BIT:
-        return DatumFactory.createBit((byte)(intValue & 0xff));
-      case INT2:
-        return DatumFactory.createInt2((short)intValue);
-      default:
-        return DatumFactory.createInt4(intValue);
-    }
-  }
-
-  private Datum convertBytes(Object value, TajoDataTypes.Type tajoType,
-                             DataType dataType) {
-    ByteBuffer buffer = (ByteBuffer)value;
-    byte[] bytes = new byte[buffer.capacity()];
-    buffer.get(bytes, 0, bytes.length);
-    switch (tajoType) {
-      case INET4:
-        return DatumFactory.createInet4(bytes);
-      case PROTOBUF:
-        try {
-          ProtobufDatumFactory factory =
-              ProtobufDatumFactory.get(dataType.getCode());
-          Message.Builder builder = factory.newBuilder();
-          builder.mergeFrom(bytes);
-          return factory.createDatum(builder);
-        } catch (InvalidProtocolBufferException e) {
-          throw new RuntimeException(e);
-        }
-      default:
-        return new BlobDatum(bytes);
-    }
-  }
-
-  private Datum convertString(Object value, TajoDataTypes.Type tajoType) {
-    switch (tajoType) {
-      case CHAR:
-        return DatumFactory.createChar(fromAvroString(value));
-      default:
-        return DatumFactory.createText(fromAvroString(value));
-    }
-  }
-
-  /**
-   * Reads the next Tuple from the Avro file.
-   *
-   * @return The next Tuple from the Avro file or null if end of file is
-   *         reached.
-   */
-  @Override
-  public Tuple next() throws IOException {
-    if (!dataFileReader.hasNext()) {
-      return null;
-    }
-
-    Tuple tuple = new VTuple(schema.size());
-    GenericRecord record = dataFileReader.next();
-    for (int i = 0; i < projectionMap.length; ++i) {
-      int columnIndex = projectionMap[i];
-      Object value = record.get(columnIndex);
-      if (value == null) {
-        tuple.put(columnIndex, NullDatum.get());
-        continue;
-      }
-
-      // Get Avro type.
-      Schema.Field avroField = avroFields.get(columnIndex);
-      Schema nonNullAvroSchema = getNonNull(avroField.schema());
-      Schema.Type avroType = nonNullAvroSchema.getType();
-
-      // Get Tajo type.
-      Column column = schema.getColumn(columnIndex);
-      DataType dataType = column.getDataType();
-      TajoDataTypes.Type tajoType = dataType.getType();
-      switch (avroType) {
-        case NULL:
-          tuple.put(columnIndex, NullDatum.get());
-          break;
-        case BOOLEAN:
-          tuple.put(columnIndex, DatumFactory.createBool((Boolean)value));
-          break;
-        case INT:
-          tuple.put(columnIndex, convertInt(value, tajoType));
-          break;
-        case LONG:
-          tuple.put(columnIndex, DatumFactory.createInt8((Long)value));
-          break;
-        case FLOAT:
-          tuple.put(columnIndex, DatumFactory.createFloat4((Float)value));
-          break;
-        case DOUBLE:
-          tuple.put(columnIndex, DatumFactory.createFloat8((Double)value));
-          break;
-        case BYTES:
-          tuple.put(columnIndex, convertBytes(value, tajoType, dataType));
-          break;
-        case STRING:
-          tuple.put(columnIndex, convertString(value, tajoType));
-          break;
-        case RECORD:
-          throw new RuntimeException("Avro RECORD not supported.");
-        case ENUM:
-          throw new RuntimeException("Avro ENUM not supported.");
-        case MAP:
-          throw new RuntimeException("Avro MAP not supported.");
-        case UNION:
-          throw new RuntimeException("Avro UNION not supported.");
-        case FIXED:
-          tuple.put(columnIndex, new BlobDatum(((GenericFixed)value).bytes()));
-          break;
-        default:
-          throw new RuntimeException("Unknown type.");
-      }
-    }
-    return tuple;
-  }
-
-  /**
-   * Resets the scanner
-   */
-  @Override
-  public void reset() throws IOException {
-  }
-
-  /**
-   * Closes the scanner.
-   */
-  @Override
-  public void close() throws IOException {
-    if (dataFileReader != null) {
-      dataFileReader.close();
-    }
-  }
-
-  /**
-   * Returns whether this scanner is projectable.
-   *
-   * @return true
-   */
-  @Override
-  public boolean isProjectable() {
-    return true;
-  }
-
-  /**
-   * Returns whether this scanner is selectable.
-   *
-   * @return false
-   */
-  @Override
-  public boolean isSelectable() {
-    return false;
-  }
-
-  /**
-   * Returns whether this scanner is splittable.
-   *
-   * @return false
-   */
-  @Override
-  public boolean isSplittable() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
deleted file mode 100644
index 0d14c3d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroUtil.java
+++ /dev/null
@@ -1,77 +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.avro;
-
-import org.apache.avro.Schema;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.StorageConstants;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-
-public class AvroUtil {
-  public static Schema getAvroSchema(TableMeta meta, Configuration conf)
-      throws IOException {
-
-    boolean isSchemaLiteral = meta.containsOption(StorageConstants.AVRO_SCHEMA_LITERAL);
-    boolean isSchemaUrl = meta.containsOption(StorageConstants.AVRO_SCHEMA_URL);
-    if (!isSchemaLiteral && !isSchemaUrl) {
-      throw new RuntimeException("No Avro schema for table.");
-    }
-    if (isSchemaLiteral) {
-      String schema = meta.getOption(StorageConstants.AVRO_SCHEMA_LITERAL);
-      return new Schema.Parser().parse(schema);
-    }
-
-    String schemaURL = meta.getOption(StorageConstants.AVRO_SCHEMA_URL);
-    if (schemaURL.toLowerCase().startsWith("http")) {
-      return getAvroSchemaFromHttp(schemaURL);
-    } else {
-      return getAvroSchemaFromFileSystem(schemaURL, conf);
-    }
-  }
-
-  public static Schema getAvroSchemaFromHttp(String schemaURL) throws IOException {
-    InputStream inputStream = new URL(schemaURL).openStream();
-
-    try {
-      return new Schema.Parser().parse(inputStream);
-    } finally {
-      IOUtils.closeStream(inputStream);
-    }
-  }
-
-  public static Schema getAvroSchemaFromFileSystem(String schemaURL, Configuration conf) throws IOException {
-    Path schemaPath = new Path(schemaURL);
-    FileSystem fs = schemaPath.getFileSystem(conf);
-    FSDataInputStream inputStream = fs.open(schemaPath);
-
-    try {
-      return new Schema.Parser().parse(inputStream);
-    } finally {
-      IOUtils.closeStream(inputStream);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/avro/package-info.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/package-info.java b/tajo-storage/src/main/java/org/apache/tajo/storage/avro/package-info.java
deleted file mode 100644
index 40d1545..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/package-info.java
+++ /dev/null
@@ -1,85 +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.
- */
-/**
- * <p>
- * Provides read and write support for Avro files. Avro schemas are
- * converted to Tajo schemas according to the following mapping of Avro
- * and Tajo types:
- * </p>
- *
- * <table>
- *   <tr>
- *     <th>Avro type</th>
- *     <th>Tajo type</th>
- *   </tr>
- *   <tr>
- *     <td>NULL</td>
- *     <td>NULL_TYPE</td>
- *   </tr>
- *   <tr>
- *     <td>BOOLEAN</td>
- *     <td>BOOLEAN</td>
- *   </tr>
- *   <tr>
- *     <td>INT</td>
- *     <td>INT4</td>
- *   </tr>
- *   <tr>
- *     <td>LONG</td>
- *     <td>INT8</td>
- *   </tr>
- *   <tr>
- *     <td>FLOAT</td>
- *     <td>FLOAT4</td>
- *   </tr>
- *   <tr>
- *     <td>DOUBLE</td>
- *     <td>FLOAT8</td>
- *   </tr>
- *   <tr>
- *     <td>BYTES</td>
- *     <td>BLOB</td>
- *   </tr>
- *   <tr>
- *     <td>STRING</td>
- *     <td>TEXT</td>
- *   </tr>
- *   <tr>
- *     <td>FIXED</td>
- *     <td>BLOB</td>
- *   </tr>
- *   <tr>
- *     <td>RECORD</td>
- *     <td>Not currently supported</td>
- *   </tr>
- *   <tr>
- *     <td>ENUM</td>
- *     <td>Not currently supported.</td>
- *   </tr>
- *   <tr>
- *     <td>MAP</td>
- *     <td>Not currently supported.</td>
- *   </tr>
- *   <tr>
- *     <td>UNION</td>
- *     <td>Not currently supported.</td>
- *   </tr>
- * </table>
- */
-
-package org.apache.tajo.storage.avro;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java b/tajo-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
deleted file mode 100644
index baeda8c..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
+++ /dev/null
@@ -1,185 +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.compress;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.compress.DoNotPool;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A global compressor/decompressor pool used to save and reuse (possibly
- * native) compression/decompression codecs.
- */
-public final class CodecPool {
-  private static final Log LOG = LogFactory.getLog(CodecPool.class);
-
-  /**
-   * A global compressor pool used to save the expensive
-   * construction/destruction of (possibly native) decompression codecs.
-   */
-  private static final Map<Class<Compressor>, List<Compressor>> COMPRESSOR_POOL =
-      new HashMap<Class<Compressor>, List<Compressor>>();
-
-  /**
-   * A global decompressor pool used to save the expensive
-   * construction/destruction of (possibly native) decompression codecs.
-   */
-  private static final Map<Class<Decompressor>, List<Decompressor>> DECOMPRESSOR_POOL =
-      new HashMap<Class<Decompressor>, List<Decompressor>>();
-
-  private static <T> T borrow(Map<Class<T>, List<T>> pool,
-      Class<? extends T> codecClass) {
-    T codec = null;
-
-    // Check if an appropriate codec is available
-    synchronized (pool) {
-      if (pool.containsKey(codecClass)) {
-        List<T> codecList = pool.get(codecClass);
-
-        if (codecList != null) {
-          synchronized (codecList) {
-            if (!codecList.isEmpty()) {
-              codec = codecList.remove(codecList.size() - 1);
-            }
-          }
-        }
-      }
-    }
-
-    return codec;
-  }
-
-  private static <T> void payback(Map<Class<T>, List<T>> pool, T codec) {
-    if (codec != null) {
-      Class<T> codecClass = (Class<T>) codec.getClass();
-      synchronized (pool) {
-        if (!pool.containsKey(codecClass)) {
-          pool.put(codecClass, new ArrayList<T>());
-        }
-
-        List<T> codecList = pool.get(codecClass);
-        synchronized (codecList) {
-          codecList.add(codec);
-        }
-      }
-    }
-  }
-
-  /**
-   * Get a {@link Compressor} for the given {@link CompressionCodec} from the
-   * pool or a new one.
-   *
-   * @param codec
-   *          the <code>CompressionCodec</code> for which to get the
-   *          <code>Compressor</code>
-   * @param conf the <code>Configuration</code> object which contains confs for creating or reinit the compressor
-   * @return <code>Compressor</code> for the given <code>CompressionCodec</code>
-   *         from the pool or a new one
-   */
-  public static Compressor getCompressor(CompressionCodec codec, Configuration conf) {
-    Compressor compressor = borrow(COMPRESSOR_POOL, codec.getCompressorType());
-    if (compressor == null) {
-      compressor = codec.createCompressor();
-      LOG.info("Got brand-new compressor ["+codec.getDefaultExtension()+"]");
-    } else {
-      compressor.reinit(conf);
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Got recycled compressor");
-      }
-    }
-    return compressor;
-  }
-
-  public static Compressor getCompressor(CompressionCodec codec) {
-    return getCompressor(codec, null);
-  }
-
-  /**
-   * Get a {@link Decompressor} for the given {@link CompressionCodec} from the
-   * pool or a new one.
-   *
-   * @param codec
-   *          the <code>CompressionCodec</code> for which to get the
-   *          <code>Decompressor</code>
-   * @return <code>Decompressor</code> for the given
-   *         <code>CompressionCodec</code> the pool or a new one
-   */
-  public static Decompressor getDecompressor(CompressionCodec codec) {
-    Decompressor decompressor = borrow(DECOMPRESSOR_POOL, codec
-        .getDecompressorType());
-    if (decompressor == null) {
-      decompressor = codec.createDecompressor();
-      LOG.info("Got brand-new decompressor ["+codec.getDefaultExtension()+"]");
-    } else {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Got recycled decompressor");
-      }
-    }
-    return decompressor;
-  }
-
-  /**
-   * Return the {@link Compressor} to the pool.
-   *
-   * @param compressor
-   *          the <code>Compressor</code> to be returned to the pool
-   */
-  public static void returnCompressor(Compressor compressor) {
-    if (compressor == null) {
-      return;
-    }
-    // if the compressor can't be reused, don't pool it.
-    if (compressor.getClass().isAnnotationPresent(DoNotPool.class)) {
-      return;
-    }
-    compressor.reset();
-    payback(COMPRESSOR_POOL, compressor);
-  }
-
-  /**
-   * Return the {@link Decompressor} to the pool.
-   *
-   * @param decompressor
-   *          the <code>Decompressor</code> to be returned to the pool
-   */
-  public static void returnDecompressor(Decompressor decompressor) {
-    if (decompressor == null) {
-      return;
-    }
-    // if the decompressor can't be reused, don't pool it.
-    if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
-      return;
-    }
-    decompressor.reset();
-    payback(DECOMPRESSOR_POOL, decompressor);
-  }
-
-  private CodecPool() {
-    // prevent instantiation
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java b/tajo-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
deleted file mode 100644
index bb035a8..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
+++ /dev/null
@@ -1,39 +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.exception;
-
-import org.apache.hadoop.fs.Path;
-
-import java.io.IOException;
-
-public class AlreadyExistsStorageException extends IOException {
-  private static final long serialVersionUID = 965518916144019032L;
-
-
-  public AlreadyExistsStorageException(String path) {
-    super("Error: "+path+" alreay exists");    
-  }
-  
-  public AlreadyExistsStorageException(Path path) {
-    this(path.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java b/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
deleted file mode 100644
index a67d1f7..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
+++ /dev/null
@@ -1,32 +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.exception;
-
-public class UnknownCodecException extends Exception {
-
-  private static final long serialVersionUID = 4287230843540404529L;
-
-  public UnknownCodecException() {
-
-  }
-
-  public UnknownCodecException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java b/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
deleted file mode 100644
index d18b5a0..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
+++ /dev/null
@@ -1,32 +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.exception;
-
-public class UnknownDataTypeException extends Exception {
-
-  private static final long serialVersionUID = -2630390595968966164L;
-
-  public UnknownDataTypeException() {
-
-  }
-
-  public UnknownDataTypeException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java b/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
deleted file mode 100644
index 8b197d6..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
+++ /dev/null
@@ -1,36 +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.exception;
-
-public class UnsupportedFileTypeException extends RuntimeException {
-	private static final long serialVersionUID = -8160289695849000342L;
-
-	public UnsupportedFileTypeException() {
-	}
-
-	/**
-	 * @param message
-	 */
-	public UnsupportedFileTypeException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java b/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
deleted file mode 100644
index 4a83dbf..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
+++ /dev/null
@@ -1,237 +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.fragment;
-
-import com.google.common.base.Objects;
-import com.google.gson.annotations.Expose;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.util.TUtil;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.FileFragmentProto;
-import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-
-public class FileFragment implements Fragment, Comparable<FileFragment>, Cloneable {
-  @Expose private String tableName; // required
-  @Expose private Path uri; // required
-  @Expose public Long startOffset; // required
-  @Expose public Long length; // required
-
-  private String[] hosts; // Datanode hostnames
-  @Expose private int[] diskIds;
-
-  public FileFragment(ByteString raw) throws InvalidProtocolBufferException {
-    FileFragmentProto.Builder builder = FileFragmentProto.newBuilder();
-    builder.mergeFrom(raw);
-    builder.build();
-    init(builder.build());
-  }
-
-  public FileFragment(String tableName, Path uri, BlockLocation blockLocation)
-      throws IOException {
-    this.set(tableName, uri, blockLocation.getOffset(), blockLocation.getLength(), blockLocation.getHosts(), null);
-  }
-
-  public FileFragment(String tableName, Path uri, long start, long length, String[] hosts, int[] diskIds) {
-    this.set(tableName, uri, start, length, hosts, diskIds);
-  }
-  // Non splittable
-  public FileFragment(String tableName, Path uri, long start, long length, String[] hosts) {
-    this.set(tableName, uri, start, length, hosts, null);
-  }
-
-  public FileFragment(String fragmentId, Path path, long start, long length) {
-    this.set(fragmentId, path, start, length, null, null);
-  }
-
-  public FileFragment(FileFragmentProto proto) {
-    init(proto);
-  }
-
-  private void init(FileFragmentProto proto) {
-    int[] diskIds = new int[proto.getDiskIdsList().size()];
-    int i = 0;
-    for(Integer eachValue: proto.getDiskIdsList()) {
-      diskIds[i++] = eachValue;
-    }
-    this.set(proto.getId(), new Path(proto.getPath()),
-        proto.getStartOffset(), proto.getLength(),
-        proto.getHostsList().toArray(new String[]{}),
-        diskIds);
-  }
-
-  private void set(String tableName, Path path, long start,
-      long length, String[] hosts, int[] diskIds) {
-    this.tableName = tableName;
-    this.uri = path;
-    this.startOffset = start;
-    this.length = length;
-    this.hosts = hosts;
-    this.diskIds = diskIds;
-  }
-
-
-  /**
-   * Get the list of hosts (hostname) hosting this block
-   */
-  public String[] getHosts() {
-    if (hosts == null) {
-      this.hosts = new String[0];
-    }
-    return hosts;
-  }
-
-  /**
-   * Get the list of Disk Ids
-   * Unknown disk is -1. Others 0 ~ N
-   */
-  public int[] getDiskIds() {
-    if (diskIds == null) {
-      this.diskIds = new int[getHosts().length];
-      Arrays.fill(this.diskIds, -1);
-    }
-    return diskIds;
-  }
-
-  public void setDiskIds(int[] diskIds){
-    this.diskIds = diskIds;
-  }
-
-  @Override
-  public String getTableName() {
-    return this.tableName;
-  }
-
-  public Path getPath() {
-    return this.uri;
-  }
-
-  public void setPath(Path path) {
-    this.uri = path;
-  }
-
-  public Long getStartKey() {
-    return this.startOffset;
-  }
-
-  @Override
-  public String getKey() {
-    return this.uri.toString();
-  }
-
-  @Override
-  public long getLength() {
-    return this.length;
-  }
-
-  @Override
-  public boolean isEmpty() {
-    return this.length <= 0;
-  }
-  /**
-   * 
-   * The offset range of tablets <b>MUST NOT</b> be overlapped.
-   * 
-   * @param t
-   * @return If the table paths are not same, return -1.
-   */
-  @Override
-  public int compareTo(FileFragment t) {
-    if (getPath().equals(t.getPath())) {
-      long diff = this.getStartKey() - t.getStartKey();
-      if (diff < 0) {
-        return -1;
-      } else if (diff > 0) {
-        return 1;
-      } else {
-        return 0;
-      }
-    } else {
-      return -1;
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof FileFragment) {
-      FileFragment t = (FileFragment) o;
-      if (getPath().equals(t.getPath())
-          && TUtil.checkEquals(t.getStartKey(), this.getStartKey())
-          && TUtil.checkEquals(t.getLength(), this.getLength())) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(tableName, uri, startOffset, length);
-  }
-  
-  public Object clone() throws CloneNotSupportedException {
-    FileFragment frag = (FileFragment) super.clone();
-    frag.tableName = tableName;
-    frag.uri = uri;
-    frag.diskIds = diskIds;
-    frag.hosts = hosts;
-
-    return frag;
-  }
-
-  @Override
-  public String toString() {
-    return "\"fragment\": {\"id\": \""+ tableName +"\", \"path\": "
-    		+getPath() + "\", \"start\": " + this.getStartKey() + ",\"length\": "
-        + getLength() + "}" ;
-  }
-
-  public FragmentProto getProto() {
-    FileFragmentProto.Builder builder = FileFragmentProto.newBuilder();
-    builder.setId(this.tableName);
-    builder.setStartOffset(this.startOffset);
-    builder.setLength(this.length);
-    builder.setPath(this.uri.toString());
-    if(diskIds != null) {
-      List<Integer> idList = new ArrayList<Integer>();
-      for(int eachId: diskIds) {
-        idList.add(eachId);
-      }
-      builder.addAllDiskIds(idList);
-    }
-
-    if(hosts != null) {
-      builder.addAllHosts(TUtil.newList(hosts));
-    }
-
-    FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder();
-    fragmentBuilder.setId(this.tableName);
-    fragmentBuilder.setStoreType(StoreType.CSV.name());
-    fragmentBuilder.setContents(builder.buildPartial().toByteString());
-    return fragmentBuilder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java b/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
deleted file mode 100644
index ac43197..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
+++ /dev/null
@@ -1,39 +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.fragment;
-
-import org.apache.tajo.common.ProtoObject;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-
-public interface Fragment extends ProtoObject<FragmentProto> {
-
-  public abstract String getTableName();
-
-  @Override
-  public abstract FragmentProto getProto();
-
-  public abstract long getLength();
-
-  public abstract String getKey();
-
-  public String[] getHosts();
-
-  public abstract boolean isEmpty();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java b/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
deleted file mode 100644
index 07720c7..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
+++ /dev/null
@@ -1,129 +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.fragment;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.protobuf.ByteString;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.annotation.ThreadSafe;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-
-@ThreadSafe
-public class FragmentConvertor {
-  /**
-   * Cache of fragment classes
-   */
-  protected static final Map<String, Class<? extends Fragment>> CACHED_FRAGMENT_CLASSES = Maps.newConcurrentMap();
-  /**
-   * Cache of constructors for each class.
-   */
-  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
-  /**
-   * default parameter for all constructors
-   */
-  private static final Class<?>[] DEFAULT_FRAGMENT_PARAMS = { ByteString.class };
-
-  public static Class<? extends Fragment> getFragmentClass(Configuration conf, String storeType)
-  throws IOException {
-    Class<? extends Fragment> fragmentClass = CACHED_FRAGMENT_CLASSES.get(storeType.toLowerCase());
-    if (fragmentClass == null) {
-      fragmentClass = conf.getClass(
-          String.format("tajo.storage.fragment.%s.class", storeType.toLowerCase()), null, Fragment.class);
-      CACHED_FRAGMENT_CLASSES.put(storeType.toLowerCase(), fragmentClass);
-    }
-
-    if (fragmentClass == null) {
-      throw new IOException("No such a fragment for " + storeType.toLowerCase());
-    }
-
-    return fragmentClass;
-  }
-
-  public static <T extends Fragment> T convert(Class<T> clazz, FragmentProto fragment) {
-    T result;
-    try {
-      Constructor<T> constructor = (Constructor<T>) CONSTRUCTOR_CACHE.get(clazz);
-      if (constructor == null) {
-        constructor = clazz.getDeclaredConstructor(DEFAULT_FRAGMENT_PARAMS);
-        constructor.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(clazz, constructor);
-      }
-      result = constructor.newInstance(new Object[]{fragment.getContents()});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    return result;
-  }
-
-  public static <T extends Fragment> T convert(Configuration conf, FragmentProto fragment)
-      throws IOException {
-    Class<T> fragmentClass = (Class<T>) getFragmentClass(conf, fragment.getStoreType().toLowerCase());
-    if (fragmentClass == null) {
-      throw new IOException("No such a fragment class for " + fragment.getStoreType());
-    }
-    return convert(fragmentClass, fragment);
-  }
-
-  public static <T extends Fragment> List<T> convert(Class<T> clazz, FragmentProto...fragments)
-      throws IOException {
-    List<T> list = Lists.newArrayList();
-    if (fragments == null) {
-      return list;
-    }
-    for (FragmentProto proto : fragments) {
-      list.add(convert(clazz, proto));
-    }
-    return list;
-  }
-
-  public static <T extends Fragment> List<T> convert(Configuration conf, FragmentProto...fragments) throws IOException {
-    List<T> list = Lists.newArrayList();
-    if (fragments == null) {
-      return list;
-    }
-    for (FragmentProto proto : fragments) {
-      list.add((T) convert(conf, proto));
-    }
-    return list;
-  }
-
-  public static List<FragmentProto> toFragmentProtoList(Fragment... fragments) {
-    List<FragmentProto> list = Lists.newArrayList();
-    if (fragments == null) {
-      return list;
-    }
-    for (Fragment fragment : fragments) {
-      list.add(fragment.getProto());
-    }
-    return list;
-  }
-
-  public static FragmentProto [] toFragmentProtoArray(Fragment... fragments) {
-    List<FragmentProto> list = toFragmentProtoList(fragments);
-    return list.toArray(new FragmentProto[list.size()]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
deleted file mode 100644
index 8615235..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
+++ /dev/null
@@ -1,223 +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.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.storage.Appender;
-import org.apache.tajo.storage.TableStatistics;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.util.TUtil;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * An abstract class for HBase appender.
- */
-public abstract class AbstractHBaseAppender implements Appender {
-  protected Configuration conf;
-  protected Schema schema;
-  protected TableMeta meta;
-  protected QueryUnitAttemptId taskAttemptId;
-  protected Path stagingDir;
-  protected boolean inited = false;
-
-  protected ColumnMapping columnMapping;
-  protected TableStatistics stats;
-  protected boolean enabledStats;
-
-  protected int columnNum;
-
-  protected byte[][][] mappingColumnFamilies;
-  protected boolean[] isBinaryColumns;
-  protected boolean[] isRowKeyMappings;
-  protected boolean[] isColumnKeys;
-  protected boolean[] isColumnValues;
-  protected int[] rowKeyFieldIndexes;
-  protected int[] rowkeyColumnIndexes;
-  protected char rowKeyDelimiter;
-
-  // the following four variables are used for '<cfname>:key:' or '<cfname>:value:' mapping
-  protected int[] columnKeyValueDataIndexes;
-  protected byte[][] columnKeyDatas;
-  protected byte[][] columnValueDatas;
-  protected byte[][] columnKeyCfNames;
-
-  protected KeyValue[] keyValues;
-
-  public AbstractHBaseAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
-                       Schema schema, TableMeta meta, Path stagingDir) {
-    this.conf = conf;
-    this.schema = schema;
-    this.meta = meta;
-    this.stagingDir = stagingDir;
-    this.taskAttemptId = taskAttemptId;
-  }
-
-  @Override
-  public void init() throws IOException {
-    if (inited) {
-      throw new IllegalStateException("FileAppender is already initialized.");
-    }
-    inited = true;
-    if (enabledStats) {
-      stats = new TableStatistics(this.schema);
-    }
-    columnMapping = new ColumnMapping(schema, meta);
-
-    mappingColumnFamilies = columnMapping.getMappingColumns();
-
-    isRowKeyMappings = columnMapping.getIsRowKeyMappings();
-    List<Integer> rowkeyColumnIndexList = new ArrayList<Integer>();
-    for (int i = 0; i < isRowKeyMappings.length; i++) {
-      if (isRowKeyMappings[i]) {
-        rowkeyColumnIndexList.add(i);
-      }
-    }
-    rowkeyColumnIndexes = TUtil.toArray(rowkeyColumnIndexList);
-
-    isBinaryColumns = columnMapping.getIsBinaryColumns();
-    isColumnKeys = columnMapping.getIsColumnKeys();
-    isColumnValues = columnMapping.getIsColumnValues();
-    rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
-    rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
-
-    this.columnNum = schema.size();
-
-    // In the case of '<cfname>:key:' or '<cfname>:value:' KeyValue object should be set with the qualifier and value
-    // which are mapped to the same column family.
-    columnKeyValueDataIndexes = new int[isColumnKeys.length];
-    int index = 0;
-    int numKeyValues = 0;
-    Map<String, Integer> cfNameIndexMap = new HashMap<String, Integer>();
-    for (int i = 0; i < isColumnKeys.length; i++) {
-      if (isRowKeyMappings[i]) {
-        continue;
-      }
-      if (isColumnKeys[i] || isColumnValues[i]) {
-        String cfName = new String(mappingColumnFamilies[i][0]);
-        if (!cfNameIndexMap.containsKey(cfName)) {
-          cfNameIndexMap.put(cfName, index);
-          columnKeyValueDataIndexes[i] = index;
-          index++;
-          numKeyValues++;
-        } else {
-          columnKeyValueDataIndexes[i] = cfNameIndexMap.get(cfName);
-        }
-      } else {
-        numKeyValues++;
-      }
-    }
-    columnKeyCfNames = new byte[cfNameIndexMap.size()][];
-    for (Map.Entry<String, Integer> entry: cfNameIndexMap.entrySet()) {
-      columnKeyCfNames[entry.getValue()] = entry.getKey().getBytes();
-    }
-    columnKeyDatas = new byte[cfNameIndexMap.size()][];
-    columnValueDatas = new byte[cfNameIndexMap.size()][];
-
-    keyValues = new KeyValue[numKeyValues];
-  }
-
-  private ByteArrayOutputStream bout = new ByteArrayOutputStream();
-
-  protected byte[] getRowKeyBytes(Tuple tuple) throws IOException {
-    Datum datum;
-    byte[] rowkey;
-    if (rowkeyColumnIndexes.length > 1) {
-      bout.reset();
-      for (int i = 0; i < rowkeyColumnIndexes.length; i++) {
-        datum = tuple.get(rowkeyColumnIndexes[i]);
-        if (isBinaryColumns[rowkeyColumnIndexes[i]]) {
-          rowkey = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(rowkeyColumnIndexes[i]), datum);
-        } else {
-          rowkey = HBaseTextSerializerDeserializer.serialize(schema.getColumn(rowkeyColumnIndexes[i]), datum);
-        }
-        bout.write(rowkey);
-        if (i < rowkeyColumnIndexes.length - 1) {
-          bout.write(rowKeyDelimiter);
-        }
-      }
-      rowkey = bout.toByteArray();
-    } else {
-      int index = rowkeyColumnIndexes[0];
-      datum = tuple.get(index);
-      if (isBinaryColumns[index]) {
-        rowkey = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(index), datum);
-      } else {
-        rowkey = HBaseTextSerializerDeserializer.serialize(schema.getColumn(index), datum);
-      }
-    }
-
-    return rowkey;
-  }
-
-  protected void readKeyValues(Tuple tuple, byte[] rowkey) throws IOException {
-    int keyValIndex = 0;
-    for (int i = 0; i < columnNum; i++) {
-      if (isRowKeyMappings[i]) {
-        continue;
-      }
-      Datum datum = tuple.get(i);
-      byte[] value;
-      if (isBinaryColumns[i]) {
-        value = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(i), datum);
-      } else {
-        value = HBaseTextSerializerDeserializer.serialize(schema.getColumn(i), datum);
-      }
-
-      if (isColumnKeys[i]) {
-        columnKeyDatas[columnKeyValueDataIndexes[i]] = value;
-      } else if (isColumnValues[i]) {
-        columnValueDatas[columnKeyValueDataIndexes[i]] = value;
-      } else {
-        keyValues[keyValIndex] = new KeyValue(rowkey, mappingColumnFamilies[i][0], mappingColumnFamilies[i][1], value);
-        keyValIndex++;
-      }
-    }
-
-    for (int i = 0; i < columnKeyDatas.length; i++) {
-      keyValues[keyValIndex++] = new KeyValue(rowkey, columnKeyCfNames[i], columnKeyDatas[i], columnValueDatas[i]);
-    }
-  }
-
-  @Override
-  public void enableStats() {
-    enabledStats = true;
-  }
-
-  @Override
-  public TableStats getStats() {
-    if (enabledStats) {
-      return stats.getTableStat();
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
deleted file mode 100644
index 8044494..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
+++ /dev/null
@@ -1,87 +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.catalog.Schema;
-import org.apache.tajo.catalog.SortSpec;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.PlanningException;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.plan.logical.SortNode.SortPurpose;
-import org.apache.tajo.plan.rewrite.RewriteRule;
-import org.apache.tajo.plan.util.PlannerUtil;
-
-public class AddSortForInsertRewriter implements RewriteRule {
-  private int[] sortColumnIndexes;
-  private Column[] sortColumns;
-  public AddSortForInsertRewriter(TableDesc tableDesc, Column[] sortColumns) {
-    this.sortColumns = sortColumns;
-    this.sortColumnIndexes = new int[sortColumns.length];
-
-    Schema tableSchema = tableDesc.getSchema();
-    for (int i = 0; i < sortColumns.length; i++) {
-      sortColumnIndexes[i] = tableSchema.getColumnId(sortColumns[i].getQualifiedName());
-    }
-  }
-
-  @Override
-  public String getName() {
-    return "AddSortForInsertRewriter";
-  }
-
-  @Override
-  public boolean isEligible(LogicalPlan plan) {
-    StoreType storeType = PlannerUtil.getStoreType(plan);
-    return storeType != null;
-  }
-
-  @Override
-  public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
-    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
-    UnaryNode insertNode = rootNode.getChild();
-    LogicalNode childNode = insertNode.getChild();
-
-    Schema sortSchema = childNode.getOutSchema();
-    SortNode sortNode = plan.createNode(SortNode.class);
-    sortNode.setSortPurpose(SortPurpose.STORAGE_SPECIFIED);
-    sortNode.setInSchema(sortSchema);
-    sortNode.setOutSchema(sortSchema);
-
-    SortSpec[] sortSpecs = new SortSpec[sortColumns.length];
-    int index = 0;
-
-    for (int i = 0; i < sortColumnIndexes.length; i++) {
-      Column sortColumn = sortSchema.getColumn(sortColumnIndexes[i]);
-      if (sortColumn == null) {
-        throw new PlanningException("Can't fine proper sort column:" + sortColumns[i]);
-      }
-      sortSpecs[index++] = new SortSpec(sortColumn, true, true);
-    }
-    sortNode.setSortSpecs(sortSpecs);
-
-    sortNode.setChild(insertNode.getChild());
-    insertNode.setChild(sortNode);
-    plan.getRootBlock().registerNode(sortNode);
-
-    return plan;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
deleted file mode 100644
index f80bd5e..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/ColumnMapping.java
+++ /dev/null
@@ -1,236 +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.hadoop.hbase.util.Bytes;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.util.BytesUtils;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-public class ColumnMapping {
-  private TableMeta tableMeta;
-  private Schema schema;
-  private char rowKeyDelimiter;
-
-  private String hbaseTableName;
-
-  private int[] rowKeyFieldIndexes;
-  private boolean[] isRowKeyMappings;
-  private boolean[] isBinaryColumns;
-  private boolean[] isColumnKeys;
-  private boolean[] isColumnValues;
-
-  // schema order -> 0: cf name, 1: column name -> name bytes
-  private byte[][][] mappingColumns;
-
-  private int numRowKeys;
-
-  public ColumnMapping(Schema schema, TableMeta tableMeta) throws IOException {
-    this.schema = schema;
-    this.tableMeta = tableMeta;
-
-    init();
-  }
-
-  private void init() throws IOException {
-    hbaseTableName = tableMeta.getOption(HBaseStorageConstants.META_TABLE_KEY);
-    String delim = tableMeta.getOption(HBaseStorageConstants.META_ROWKEY_DELIMITER, "").trim();
-    if (delim.length() > 0) {
-      rowKeyDelimiter = delim.charAt(0);
-    }
-    isRowKeyMappings = new boolean[schema.size()];
-    rowKeyFieldIndexes = new int[schema.size()];
-    isBinaryColumns = new boolean[schema.size()];
-    isColumnKeys = new boolean[schema.size()];
-    isColumnValues = new boolean[schema.size()];
-
-    mappingColumns = new byte[schema.size()][][];
-
-    for (int i = 0; i < schema.size(); i++) {
-      rowKeyFieldIndexes[i] = -1;
-    }
-
-    String columnMapping = tableMeta.getOption(HBaseStorageConstants.META_COLUMNS_KEY, "");
-    if (columnMapping == null || columnMapping.isEmpty()) {
-      throw new IOException("'columns' property is required.");
-    }
-
-    String[] columnMappingTokens = columnMapping.split(",");
-
-    if (columnMappingTokens.length != schema.getColumns().size()) {
-      throw new IOException("The number of mapped HBase columns is great than the number of Tajo table columns");
-    }
-
-    int index = 0;
-    for (String eachToken: columnMappingTokens) {
-      mappingColumns[index] = new byte[2][];
-
-      byte[][] mappingTokens = BytesUtils.splitPreserveAllTokens(eachToken.trim().getBytes(), ':');
-
-      if (mappingTokens.length == 3) {
-        if (mappingTokens[0].length == 0) {
-          // cfname
-          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
-              "or '<cfname>:value:' or '<cfname>:value:#b'");
-        }
-        //<cfname>:key: or <cfname>:value:
-        if (mappingTokens[2].length != 0) {
-          String binaryOption = new String(mappingTokens[2]);
-          if ("#b".equals(binaryOption)) {
-            isBinaryColumns[index] = true;
-          } else {
-            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:key:#b' " +
-                "or '<cfname>:value:' or '<cfname>:value:#b'");
-          }
-        }
-        mappingColumns[index][0] = mappingTokens[0];
-        String keyOrValue = new String(mappingTokens[1]);
-        if (HBaseStorageConstants.KEY_COLUMN_MAPPING.equalsIgnoreCase(keyOrValue)) {
-          isColumnKeys[index] = true;
-        } else if (HBaseStorageConstants.VALUE_COLUMN_MAPPING.equalsIgnoreCase(keyOrValue)) {
-          isColumnValues[index] = true;
-        } else {
-          throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
-        }
-      } else if (mappingTokens.length == 2) {
-        //<cfname>: or <cfname>:<qualifier> or :key
-        String cfName = new String(mappingTokens[0]);
-        String columnName = new String(mappingTokens[1]);
-        RowKeyMapping rowKeyMapping = getRowKeyMapping(cfName, columnName);
-        if (rowKeyMapping != null) {
-          isRowKeyMappings[index] = true;
-          numRowKeys++;
-          isBinaryColumns[index] = rowKeyMapping.isBinary();
-          if (!cfName.isEmpty()) {
-            if (rowKeyDelimiter == 0) {
-              throw new IOException("hbase.rowkey.delimiter is required.");
-            }
-            rowKeyFieldIndexes[index] = Integer.parseInt(cfName);
-          } else {
-            rowKeyFieldIndexes[index] = -1; //rowkey is mapped a single column.
-          }
-        } else {
-          if (cfName.isEmpty()) {
-            throw new IOException(eachToken + " 'column' attribute should be '<cfname>:key:' or '<cfname>:value:'");
-          }
-          if (cfName != null) {
-            mappingColumns[index][0] = Bytes.toBytes(cfName);
-          }
-
-          if (columnName != null && !columnName.isEmpty()) {
-            String[] columnNameTokens = columnName.split("#");
-            if (columnNameTokens[0].isEmpty()) {
-              mappingColumns[index][1] = null;
-            } else {
-              mappingColumns[index][1] = Bytes.toBytes(columnNameTokens[0]);
-            }
-            if (columnNameTokens.length == 2 && "b".equals(columnNameTokens[1])) {
-              isBinaryColumns[index] = true;
-            }
-          }
-        }
-      } else {
-        throw new IOException(eachToken + " 'column' attribute '[cfname]:[qualfier]:'");
-      }
-
-      index++;
-    } // for loop
-  }
-
-  public List<String> getColumnFamilyNames() {
-    List<String> cfNames = new ArrayList<String>();
-
-    for (byte[][] eachCfName: mappingColumns) {
-      if (eachCfName != null && eachCfName.length > 0 && eachCfName[0] != null) {
-        String cfName = new String(eachCfName[0]);
-        if (!cfNames.contains(cfName)) {
-          cfNames.add(cfName);
-        }
-      }
-    }
-
-    return cfNames;
-  }
-
-  private RowKeyMapping getRowKeyMapping(String cfName, String columnName) {
-    if (columnName == null || columnName.isEmpty()) {
-      return null;
-    }
-
-    String[] tokens = columnName.split("#");
-    if (!tokens[0].equalsIgnoreCase(HBaseStorageConstants.KEY_COLUMN_MAPPING)) {
-      return null;
-    }
-
-    RowKeyMapping rowKeyMapping = new RowKeyMapping();
-
-    if (tokens.length == 2 && "b".equals(tokens[1])) {
-      rowKeyMapping.setBinary(true);
-    }
-
-    if (cfName != null && !cfName.isEmpty()) {
-      rowKeyMapping.setKeyFieldIndex(Integer.parseInt(cfName));
-    }
-    return rowKeyMapping;
-  }
-
-  public char getRowKeyDelimiter() {
-    return rowKeyDelimiter;
-  }
-
-  public int[] getRowKeyFieldIndexes() {
-    return rowKeyFieldIndexes;
-  }
-
-  public boolean[] getIsRowKeyMappings() {
-    return isRowKeyMappings;
-  }
-
-  public byte[][][] getMappingColumns() {
-    return mappingColumns;
-  }
-
-  public Schema getSchema() {
-    return schema;
-  }
-
-  public boolean[] getIsBinaryColumns() {
-    return isBinaryColumns;
-  }
-
-  public String getHbaseTableName() {
-    return hbaseTableName;
-  }
-
-  public boolean[] getIsColumnKeys() {
-    return isColumnKeys;
-  }
-
-  public int getNumRowKeys() {
-    return numRowKeys;
-  }
-
-  public boolean[] getIsColumnValues() {
-    return isColumnValues;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
deleted file mode 100644
index c05c5bb..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseBinarySerializerDeserializer.java
+++ /dev/null
@@ -1,97 +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.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.util.Bytes;
-
-import java.io.IOException;
-
-public class HBaseBinarySerializerDeserializer {
-
-  public static Datum deserialize(Column col, byte[] bytes) throws IOException {
-    Datum datum;
-    switch (col.getDataType().getType()) {
-      case INT1:
-      case INT2:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt2(Bytes.toShort(bytes));
-        break;
-      case INT4:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt4(Bytes.toInt(bytes));
-        break;
-      case INT8:
-        if (bytes.length == 4) {
-          datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt8(Bytes.toInt(bytes));
-        } else {
-          datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createInt8(Bytes.toLong(bytes));
-        }
-        break;
-      case FLOAT4:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createFloat4(Bytes.toFloat(bytes));
-        break;
-      case FLOAT8:
-        datum = bytes == null || bytes.length == 0 ? NullDatum.get() : DatumFactory.createFloat8(Bytes.toDouble(bytes));
-        break;
-      case TEXT:
-        datum = bytes == null ? NullDatum.get() : DatumFactory.createText(bytes);
-        break;
-      default:
-        datum = NullDatum.get();
-        break;
-    }
-    return datum;
-  }
-
-  public static byte[] serialize(Column col, Datum datum) throws IOException {
-    if (datum == null || datum instanceof NullDatum) {
-      return null;
-    }
-
-    byte[] bytes;
-    switch (col.getDataType().getType()) {
-      case INT1:
-      case INT2:
-        bytes = Bytes.toBytes(datum.asInt2());
-        break;
-      case INT4:
-        bytes = Bytes.toBytes(datum.asInt4());
-        break;
-      case INT8:
-        bytes = Bytes.toBytes(datum.asInt8());
-        break;
-      case FLOAT4:
-        bytes = Bytes.toBytes(datum.asFloat4());
-        break;
-      case FLOAT8:
-        bytes = Bytes.toBytes(datum.asFloat8());
-        break;
-      case TEXT:
-        bytes = Bytes.toBytes(datum.asChars());
-        break;
-      default:
-        bytes = null;
-        break;
-    }
-
-    return bytes;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java
deleted file mode 100644
index 43ad7f3..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java
+++ /dev/null
@@ -1,198 +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 com.google.common.base.Objects;
-import com.google.gson.annotations.Expose;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.fragment.StorageFragmentProtos.HBaseFragmentProto;
-
-public class HBaseFragment implements Fragment, Comparable<HBaseFragment>, Cloneable {
-  @Expose
-  private String tableName;
-  @Expose
-  private String hbaseTableName;
-  @Expose
-  private byte[] startRow;
-  @Expose
-  private byte[] stopRow;
-  @Expose
-  private String regionLocation;
-  @Expose
-  private boolean last;
-  @Expose
-  private long length;
-
-  public HBaseFragment(String tableName, String hbaseTableName, byte[] startRow, byte[] stopRow, String regionLocation) {
-    this.tableName = tableName;
-    this.hbaseTableName = hbaseTableName;
-    this.startRow = startRow;
-    this.stopRow = stopRow;
-    this.regionLocation = regionLocation;
-    this.last = false;
-  }
-
-  public HBaseFragment(ByteString raw) throws InvalidProtocolBufferException {
-    HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder();
-    builder.mergeFrom(raw);
-    builder.build();
-    init(builder.build());
-  }
-
-  private void init(HBaseFragmentProto proto) {
-    this.tableName = proto.getTableName();
-    this.hbaseTableName = proto.getHbaseTableName();
-    this.startRow = proto.getStartRow().toByteArray();
-    this.stopRow = proto.getStopRow().toByteArray();
-    this.regionLocation = proto.getRegionLocation();
-    this.length = proto.getLength();
-    this.last = proto.getLast();
-  }
-
-  @Override
-  public int compareTo(HBaseFragment t) {
-    return Bytes.compareTo(startRow, t.startRow);
-  }
-
-  @Override
-  public String getTableName() {
-    return tableName;
-  }
-
-  @Override
-  public String getKey() {
-    return new String(startRow);
-  }
-
-  @Override
-  public boolean isEmpty() {
-    return startRow == null || stopRow == null;
-  }
-
-  @Override
-  public long getLength() {
-    return length;
-  }
-
-  public void setLength(long length) {
-    this.length = length;
-  }
-
-  @Override
-  public String[] getHosts() {
-    return new String[] {regionLocation};
-  }
-
-  public Object clone() throws CloneNotSupportedException {
-    HBaseFragment frag = (HBaseFragment) super.clone();
-    frag.tableName = tableName;
-    frag.hbaseTableName = hbaseTableName;
-    frag.startRow = startRow;
-    frag.stopRow = stopRow;
-    frag.regionLocation = regionLocation;
-    frag.last = last;
-    frag.length = length;
-    return frag;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof HBaseFragment) {
-      HBaseFragment t = (HBaseFragment) o;
-      if (tableName.equals(t.tableName)
-          && Bytes.equals(startRow, t.startRow)
-          && Bytes.equals(stopRow, t.stopRow)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(tableName, hbaseTableName, startRow, stopRow);
-  }
-
-  @Override
-  public String toString() {
-    return "\"fragment\": {\"tableName\": \""+ tableName + "\", hbaseTableName\": \"" + hbaseTableName + "\"" +
-        ", \"startRow\": \"" + new String(startRow) + "\"" +
-        ", \"stopRow\": \"" + new String(stopRow) + "\"" +
-        ", \"length\": \"" + length + "\"}" ;
-  }
-
-  @Override
-  public FragmentProto getProto() {
-    HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder();
-    builder.setTableName(tableName)
-        .setHbaseTableName(hbaseTableName)
-        .setStartRow(ByteString.copyFrom(startRow))
-        .setStopRow(ByteString.copyFrom(stopRow))
-        .setLast(last)
-        .setLength(length)
-        .setRegionLocation(regionLocation);
-
-    FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder();
-    fragmentBuilder.setId(this.tableName);
-    fragmentBuilder.setContents(builder.buildPartial().toByteString());
-    fragmentBuilder.setStoreType(StoreType.HBASE.name());
-    return fragmentBuilder.build();
-  }
-
-  public byte[] getStartRow() {
-    return startRow;
-  }
-
-  public byte[] getStopRow() {
-    return stopRow;
-  }
-
-  public String getRegionLocation() {
-    return regionLocation;
-  }
-
-  public boolean isLast() {
-    return last;
-  }
-
-  public void setLast(boolean last) {
-    this.last = last;
-  }
-
-  public String getHbaseTableName() {
-    return hbaseTableName;
-  }
-
-  public void setHbaseTableName(String hbaseTableName) {
-    this.hbaseTableName = hbaseTableName;
-  }
-
-  public void setStartRow(byte[] startRow) {
-    this.startRow = startRow;
-  }
-
-  public void setStopRow(byte[] stopRow) {
-    this.stopRow = stopRow;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
deleted file mode 100644
index 50f61a8..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/hbase/HBasePutAppender.java
+++ /dev/null
@@ -1,120 +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.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-
-public class HBasePutAppender extends AbstractHBaseAppender {
-  private HTableInterface htable;
-  private long totalNumBytes;
-
-  public HBasePutAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
-                          Schema schema, TableMeta meta, Path stagingDir) {
-    super(conf, taskAttemptId, schema, meta, stagingDir);
-  }
-
-  @Override
-  public void init() throws IOException {
-    super.init();
-
-    Configuration hbaseConf = HBaseStorageManager.getHBaseConfiguration(conf, meta);
-    HConnection hconn = ((HBaseStorageManager) StorageManager.getStorageManager((TajoConf)conf, StoreType.HBASE))
-        .getConnection(hbaseConf);
-    htable = hconn.getTable(columnMapping.getHbaseTableName());
-    htable.setAutoFlushTo(false);
-    htable.setWriteBufferSize(5 * 1024 * 1024);
-  }
-
-  @Override
-  public void addTuple(Tuple tuple) throws IOException {
-    byte[] rowkey = getRowKeyBytes(tuple);
-    totalNumBytes += rowkey.length;
-    Put put = new Put(rowkey);
-    readKeyValues(tuple, rowkey);
-
-    for (int i = 0; i < columnNum; i++) {
-      if (isRowKeyMappings[i]) {
-        continue;
-      }
-      Datum datum = tuple.get(i);
-      byte[] value;
-      if (isBinaryColumns[i]) {
-        value = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(i), datum);
-      } else {
-        value = HBaseTextSerializerDeserializer.serialize(schema.getColumn(i), datum);
-      }
-
-      if (isColumnKeys[i]) {
-        columnKeyDatas[columnKeyValueDataIndexes[i]] = value;
-      } else if (isColumnValues[i]) {
-        columnValueDatas[columnKeyValueDataIndexes[i]] = value;
-      } else {
-        put.add(mappingColumnFamilies[i][0], mappingColumnFamilies[i][1], value);
-        totalNumBytes += value.length;
-      }
-    }
-
-    for (int i = 0; i < columnKeyDatas.length; i++) {
-     put.add(columnKeyCfNames[i], columnKeyDatas[i], columnValueDatas[i]);
-      totalNumBytes += columnKeyDatas[i].length + columnValueDatas[i].length;
-    }
-
-    htable.put(put);
-
-    if (enabledStats) {
-      stats.incrementRow();
-      stats.setNumBytes(totalNumBytes);
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    htable.flushCommits();
-  }
-
-  @Override
-  public long getEstimatedOutputSize() throws IOException {
-    return 0;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (htable != null) {
-      htable.flushCommits();
-      htable.close();
-    }
-    if (enabledStats) {
-      stats.setNumBytes(totalNumBytes);
-    }
-  }
-}


[30/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
TAJO-1122: Refactor the tajo-storage project structure.

Closes #247


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

Branch: refs/heads/hbase_storage
Commit: dfd7f996d3c4a3e095dcd2ddc62fd7d6d793640d
Parents: 87c957e
Author: HyoungJun Kim <ba...@babokim-mbp.server.gruter.com>
Authored: Thu Nov 27 10:43:34 2014 +0900
Committer: HyoungJun Kim <ba...@babokim-mbp.server.gruter.com>
Committed: Thu Nov 27 10:43:34 2014 +0900

----------------------------------------------------------------------
 CHANGES                                         |    3 +
 .../src/main/proto/CatalogProtos.proto          |    9 -
 .../tajo-catalog-drivers/tajo-hcatalog/pom.xml  |    2 +-
 tajo-client/pom.xml                             |    6 +-
 .../org/apache/tajo/jdbc/TajoResultSet.java     |    5 +-
 .../main/java/org/apache/tajo/util/TUtil.java   |    9 +
 tajo-core/pom.xml                               |   13 +-
 .../engine/planner/PhysicalPlannerImpl.java     |   12 +-
 .../planner/physical/ColPartitionStoreExec.java |    3 +-
 .../planner/physical/PhysicalPlanUtil.java      |    6 +-
 .../physical/RangeShuffleFileWriteExec.java     |    4 +-
 .../engine/planner/physical/StoreTableExec.java |    4 +-
 .../org/apache/tajo/master/GlobalEngine.java    |    4 +-
 .../apache/tajo/master/LazyTaskScheduler.java   |    1 +
 .../java/org/apache/tajo/master/TajoMaster.java |    2 +-
 .../tajo/master/querymaster/Repartitioner.java  |   14 +-
 .../tajo/master/querymaster/SubQuery.java       |    2 +-
 .../master/rm/TajoWorkerResourceManager.java    |    1 +
 .../main/java/org/apache/tajo/worker/Task.java  |   10 +-
 .../org/apache/tajo/BackendTestingUtil.java     |    2 +-
 .../tajo/engine/planner/TestPlannerUtil.java    |    2 +-
 .../planner/global/TestBroadcastJoinPlan.java   |    9 +-
 .../planner/physical/TestBNLJoinExec.java       |    6 +-
 .../planner/physical/TestBSTIndexExec.java      |    5 +-
 .../planner/physical/TestExternalSortExec.java  |    5 +-
 .../physical/TestFullOuterHashJoinExec.java     |   15 +-
 .../physical/TestFullOuterMergeJoinExec.java    |   16 +-
 .../planner/physical/TestHashAntiJoinExec.java  |    7 +-
 .../planner/physical/TestHashJoinExec.java      |    7 +-
 .../planner/physical/TestHashSemiJoinExec.java  |    7 +-
 .../physical/TestLeftOuterHashJoinExec.java     |   13 +-
 .../physical/TestLeftOuterNLJoinExec.java       |   13 +-
 .../planner/physical/TestMergeJoinExec.java     |    7 +-
 .../engine/planner/physical/TestNLJoinExec.java |    6 +-
 .../planner/physical/TestPhysicalPlanner.java   |   23 +-
 .../physical/TestProgressExternalSortExec.java  |    3 +-
 .../physical/TestRightOuterHashJoinExec.java    |    9 +-
 .../physical/TestRightOuterMergeJoinExec.java   |   16 +-
 .../engine/planner/physical/TestSortExec.java   |    5 +-
 .../tajo/engine/query/TestJoinBroadcast.java    |    4 +-
 .../org/apache/tajo/jdbc/TestResultSet.java     |    5 +-
 .../org/apache/tajo/storage/TestRowFile.java    |    6 +-
 .../tajo/worker/TestRangeRetrieverHandler.java  |    4 +-
 tajo-dist/pom.xml                               |    7 +-
 tajo-jdbc/pom.xml                               |    6 +-
 tajo-project/pom.xml                            |   16 +
 tajo-storage/pom.xml                            |  328 +---
 .../java/org/apache/tajo/storage/Appender.java  |   41 -
 .../tajo/storage/BaseTupleComparator.java       |  206 --
 .../storage/BinarySerializerDeserializer.java   |  258 ---
 .../org/apache/tajo/storage/BufferPool.java     |   74 -
 .../tajo/storage/ByteBufInputChannel.java       |   76 -
 .../java/org/apache/tajo/storage/CSVFile.java   |  588 ------
 .../tajo/storage/CompressedSplitLineReader.java |  182 --
 .../org/apache/tajo/storage/DataLocation.java   |   45 -
 .../org/apache/tajo/storage/DiskDeviceInfo.java |   62 -
 .../java/org/apache/tajo/storage/DiskInfo.java  |   75 -
 .../org/apache/tajo/storage/DiskMountInfo.java  |  101 -
 .../java/org/apache/tajo/storage/DiskUtil.java  |  207 --
 .../storage/FieldSerializerDeserializer.java    |   35 -
 .../org/apache/tajo/storage/FileAppender.java   |   85 -
 .../org/apache/tajo/storage/FileScanner.java    |  124 --
 .../apache/tajo/storage/FileStorageManager.java |  832 --------
 .../org/apache/tajo/storage/FrameTuple.java     |  225 ---
 .../tajo/storage/HashShuffleAppender.java       |  209 --
 .../storage/HashShuffleAppenderManager.java     |  225 ---
 .../java/org/apache/tajo/storage/LazyTuple.java |  270 ---
 .../org/apache/tajo/storage/LineReader.java     |  559 ------
 .../org/apache/tajo/storage/MemoryUtil.java     |  163 --
 .../org/apache/tajo/storage/MergeScanner.java   |  201 --
 .../org/apache/tajo/storage/NullScanner.java    |   62 -
 .../tajo/storage/NumericPathComparator.java     |   34 -
 .../java/org/apache/tajo/storage/RawFile.java   |  774 --------
 .../java/org/apache/tajo/storage/RowFile.java   |  498 -----
 .../org/apache/tajo/storage/RowStoreUtil.java   |  377 ----
 .../java/org/apache/tajo/storage/Scanner.java   |  103 -
 .../apache/tajo/storage/SeekableScanner.java    |   28 -
 .../tajo/storage/SerializerDeserializer.java    |   34 -
 .../apache/tajo/storage/SplitLineReader.java    |   39 -
 .../java/org/apache/tajo/storage/Storage.java   |   45 -
 .../org/apache/tajo/storage/StorageManager.java |  926 ---------
 .../apache/tajo/storage/StorageProperty.java    |   40 -
 .../org/apache/tajo/storage/StorageUtil.java    |  248 ---
 .../apache/tajo/storage/TableStatistics.java    |  129 --
 .../storage/TextSerializerDeserializer.java     |  226 ---
 .../apache/tajo/storage/TupleComparator.java    |   32 -
 .../org/apache/tajo/storage/TupleRange.java     |  112 --
 .../storage/annotation/ForSplitableStore.java   |   29 -
 .../apache/tajo/storage/avro/AvroAppender.java  |  221 ---
 .../apache/tajo/storage/avro/AvroScanner.java   |  287 ---
 .../org/apache/tajo/storage/avro/AvroUtil.java  |   77 -
 .../apache/tajo/storage/avro/package-info.java  |   85 -
 .../apache/tajo/storage/compress/CodecPool.java |  185 --
 .../AlreadyExistsStorageException.java          |   39 -
 .../exception/UnknownCodecException.java        |   32 -
 .../exception/UnknownDataTypeException.java     |   32 -
 .../exception/UnsupportedFileTypeException.java |   36 -
 .../tajo/storage/fragment/FileFragment.java     |  237 ---
 .../apache/tajo/storage/fragment/Fragment.java  |   39 -
 .../storage/fragment/FragmentConvertor.java     |  129 --
 .../storage/hbase/AbstractHBaseAppender.java    |  223 ---
 .../storage/hbase/AddSortForInsertRewriter.java |   87 -
 .../tajo/storage/hbase/ColumnMapping.java       |  236 ---
 .../HBaseBinarySerializerDeserializer.java      |   97 -
 .../tajo/storage/hbase/HBaseFragment.java       |  198 --
 .../tajo/storage/hbase/HBasePutAppender.java    |  120 --
 .../apache/tajo/storage/hbase/HBaseScanner.java |  445 -----
 .../storage/hbase/HBaseStorageConstants.java    |   33 -
 .../tajo/storage/hbase/HBaseStorageManager.java | 1126 -----------
 .../hbase/HBaseTextSerializerDeserializer.java  |   71 -
 .../tajo/storage/hbase/HFileAppender.java       |  167 --
 .../tajo/storage/hbase/IndexPredication.java    |   61 -
 .../tajo/storage/hbase/RowKeyMapping.java       |   40 -
 .../apache/tajo/storage/index/IndexMethod.java  |   33 -
 .../apache/tajo/storage/index/IndexReader.java  |   35 -
 .../apache/tajo/storage/index/IndexWriter.java  |   33 -
 .../tajo/storage/index/OrderIndexReader.java    |   45 -
 .../apache/tajo/storage/index/bst/BSTIndex.java |  623 ------
 .../tajo/storage/parquet/ParquetAppender.java   |  151 --
 .../tajo/storage/parquet/ParquetScanner.java    |  119 --
 .../tajo/storage/parquet/TajoParquetReader.java |   85 -
 .../tajo/storage/parquet/TajoParquetWriter.java |  104 -
 .../tajo/storage/parquet/TajoReadSupport.java   |  101 -
 .../storage/parquet/TajoRecordConverter.java    |  386 ----
 .../storage/parquet/TajoRecordMaterializer.java |   78 -
 .../storage/parquet/TajoSchemaConverter.java    |  206 --
 .../tajo/storage/parquet/TajoWriteSupport.java  |  149 --
 .../tajo/storage/parquet/package-info.java      |   96 -
 .../storage/rcfile/BytesRefArrayWritable.java   |  261 ---
 .../tajo/storage/rcfile/BytesRefWritable.java   |  248 ---
 .../storage/rcfile/ColumnProjectionUtils.java   |  117 --
 .../rcfile/LazyDecompressionCallback.java       |   32 -
 .../rcfile/NonSyncByteArrayInputStream.java     |  113 --
 .../rcfile/NonSyncByteArrayOutputStream.java    |  144 --
 .../storage/rcfile/NonSyncDataInputBuffer.java  |  507 -----
 .../storage/rcfile/NonSyncDataOutputBuffer.java |   91 -
 .../org/apache/tajo/storage/rcfile/RCFile.java  | 1808 ------------------
 .../SchemaAwareCompressionInputStream.java      |   43 -
 .../SchemaAwareCompressionOutputStream.java     |   44 -
 .../sequencefile/SequenceFileAppender.java      |  274 ---
 .../sequencefile/SequenceFileScanner.java       |  336 ----
 .../tajo/storage/text/ByteBufLineReader.java    |  154 --
 .../tajo/storage/text/DelimitedLineReader.java  |  157 --
 .../tajo/storage/text/DelimitedTextFile.java    |  468 -----
 .../tajo/storage/text/FieldSplitProcessor.java  |   38 -
 .../tajo/storage/text/LineSplitProcessor.java   |   45 -
 .../text/TextFieldSerializerDeserializer.java   |  223 ---
 .../thirdparty/parquet/CodecFactory.java        |  196 --
 .../parquet/ColumnChunkPageWriteStore.java      |  211 --
 .../parquet/InternalParquetRecordReader.java    |  187 --
 .../parquet/InternalParquetRecordWriter.java    |  160 --
 .../thirdparty/parquet/ParquetFileWriter.java   |  504 -----
 .../thirdparty/parquet/ParquetReader.java       |  151 --
 .../thirdparty/parquet/ParquetWriter.java       |  224 ---
 .../org/apache/tajo/tuple/BaseTupleBuilder.java |  112 --
 .../org/apache/tajo/tuple/RowBlockReader.java   |   33 -
 .../org/apache/tajo/tuple/TupleBuilder.java     |   26 -
 .../tajo/tuple/offheap/DirectBufTuple.java      |   41 -
 .../tajo/tuple/offheap/FixedSizeLimitSpec.java  |   32 -
 .../apache/tajo/tuple/offheap/HeapTuple.java    |  272 ---
 .../tajo/tuple/offheap/OffHeapMemory.java       |  102 -
 .../tajo/tuple/offheap/OffHeapRowBlock.java     |  176 --
 .../tuple/offheap/OffHeapRowBlockReader.java    |   63 -
 .../tuple/offheap/OffHeapRowBlockUtils.java     |   54 -
 .../tuple/offheap/OffHeapRowBlockWriter.java    |   58 -
 .../tajo/tuple/offheap/OffHeapRowWriter.java    |  232 ---
 .../tajo/tuple/offheap/ResizableLimitSpec.java  |  142 --
 .../apache/tajo/tuple/offheap/RowWriter.java    |   73 -
 .../apache/tajo/tuple/offheap/UnSafeTuple.java  |  311 ---
 .../offheap/UnSafeTupleBytesComparator.java     |   99 -
 .../tajo/tuple/offheap/ZeroCopyTuple.java       |   35 -
 tajo-storage/src/main/proto/IndexProtos.proto   |   31 -
 .../src/main/proto/StorageFragmentProtos.proto  |   35 -
 .../src/main/resources/storage-default.xml      |  180 --
 .../java/org/apache/tajo/HttpFileServer.java    |   84 -
 .../org/apache/tajo/HttpFileServerHandler.java  |  184 --
 .../tajo/HttpFileServerPipelineFactory.java     |   54 -
 .../tajo/storage/TestCompressionStorages.java   |  185 --
 .../apache/tajo/storage/TestFileSystems.java    |  130 --
 .../org/apache/tajo/storage/TestFrameTuple.java |   84 -
 .../org/apache/tajo/storage/TestLazyTuple.java  |  258 ---
 .../org/apache/tajo/storage/TestLineReader.java |  163 --
 .../apache/tajo/storage/TestMergeScanner.java   |  202 --
 .../apache/tajo/storage/TestSplitProcessor.java |   72 -
 .../apache/tajo/storage/TestStorageManager.java |  203 --
 .../org/apache/tajo/storage/TestStorages.java   |  857 ---------
 .../tajo/storage/TestTupleComparator.java       |   77 -
 .../org/apache/tajo/storage/TestVTuple.java     |  160 --
 .../apache/tajo/storage/avro/TestAvroUtil.java  |  108 --
 .../tajo/storage/hbase/TestColumnMapping.java   |   95 -
 .../storage/hbase/TestHBaseStorageManager.java  |  109 --
 .../apache/tajo/storage/index/TestBSTIndex.java |  946 ---------
 .../index/TestSingleCSVFileBSTIndex.java        |  248 ---
 .../tajo/storage/parquet/TestReadWrite.java     |  114 --
 .../storage/parquet/TestSchemaConverter.java    |  132 --
 .../java/org/apache/tajo/storage/s3/INode.java  |  124 --
 .../storage/s3/InMemoryFileSystemStore.java     |  175 --
 .../apache/tajo/storage/s3/S3OutputStream.java  |  234 ---
 .../tajo/storage/s3/SmallBlockS3FileSystem.java |  314 ---
 .../apache/tajo/tuple/TestBaseTupleBuilder.java |   76 -
 .../tajo/tuple/offheap/TestHeapTuple.java       |   45 -
 .../tajo/tuple/offheap/TestOffHeapRowBlock.java |  577 ------
 .../tajo/tuple/offheap/TestResizableSpec.java   |   59 -
 .../src/test/resources/storage-default.xml      |  154 --
 .../src/test/resources/testVariousTypes.avsc    |   21 -
 tajo-storage/tajo-storage-common/pom.xml        |  337 ++++
 .../java/org/apache/tajo/storage/Appender.java  |   41 +
 .../tajo/storage/BaseTupleComparator.java       |  206 ++
 .../storage/BinarySerializerDeserializer.java   |  258 +++
 .../org/apache/tajo/storage/BufferPool.java     |   74 +
 .../tajo/storage/ByteBufInputChannel.java       |   76 +
 .../org/apache/tajo/storage/DataLocation.java   |   45 +
 .../org/apache/tajo/storage/DiskDeviceInfo.java |   62 +
 .../java/org/apache/tajo/storage/DiskInfo.java  |   75 +
 .../org/apache/tajo/storage/DiskMountInfo.java  |  101 +
 .../java/org/apache/tajo/storage/DiskUtil.java  |  207 ++
 .../storage/FieldSerializerDeserializer.java    |   35 +
 .../org/apache/tajo/storage/FrameTuple.java     |  225 +++
 .../java/org/apache/tajo/storage/LazyTuple.java |  270 +++
 .../org/apache/tajo/storage/MemoryUtil.java     |  163 ++
 .../org/apache/tajo/storage/MergeScanner.java   |  201 ++
 .../org/apache/tajo/storage/NullScanner.java    |  109 ++
 .../tajo/storage/NumericPathComparator.java     |   34 +
 .../org/apache/tajo/storage/RowStoreUtil.java   |  377 ++++
 .../java/org/apache/tajo/storage/Scanner.java   |  103 +
 .../apache/tajo/storage/SeekableScanner.java    |   28 +
 .../tajo/storage/SerializerDeserializer.java    |   34 +
 .../org/apache/tajo/storage/StorageManager.java |  933 +++++++++
 .../apache/tajo/storage/StorageProperty.java    |   40 +
 .../org/apache/tajo/storage/StorageUtil.java    |  221 +++
 .../apache/tajo/storage/TableStatistics.java    |  129 ++
 .../storage/TextSerializerDeserializer.java     |  226 +++
 .../apache/tajo/storage/TupleComparator.java    |   32 +
 .../org/apache/tajo/storage/TupleRange.java     |  112 ++
 .../storage/annotation/ForSplitableStore.java   |   29 +
 .../apache/tajo/storage/compress/CodecPool.java |  185 ++
 .../AlreadyExistsStorageException.java          |   39 +
 .../exception/UnknownCodecException.java        |   32 +
 .../exception/UnknownDataTypeException.java     |   32 +
 .../exception/UnsupportedFileTypeException.java |   36 +
 .../apache/tajo/storage/fragment/Fragment.java  |   39 +
 .../storage/fragment/FragmentConvertor.java     |  129 ++
 .../org/apache/tajo/tuple/BaseTupleBuilder.java |  112 ++
 .../org/apache/tajo/tuple/RowBlockReader.java   |   33 +
 .../org/apache/tajo/tuple/TupleBuilder.java     |   26 +
 .../tajo/tuple/offheap/DirectBufTuple.java      |   41 +
 .../tajo/tuple/offheap/FixedSizeLimitSpec.java  |   32 +
 .../apache/tajo/tuple/offheap/HeapTuple.java    |  272 +++
 .../tajo/tuple/offheap/OffHeapMemory.java       |  102 +
 .../tajo/tuple/offheap/OffHeapRowBlock.java     |  176 ++
 .../tuple/offheap/OffHeapRowBlockReader.java    |   63 +
 .../tuple/offheap/OffHeapRowBlockUtils.java     |   54 +
 .../tuple/offheap/OffHeapRowBlockWriter.java    |   58 +
 .../tajo/tuple/offheap/OffHeapRowWriter.java    |  232 +++
 .../tajo/tuple/offheap/ResizableLimitSpec.java  |  142 ++
 .../apache/tajo/tuple/offheap/RowWriter.java    |   73 +
 .../apache/tajo/tuple/offheap/UnSafeTuple.java  |  311 +++
 .../offheap/UnSafeTupleBytesComparator.java     |   99 +
 .../tajo/tuple/offheap/ZeroCopyTuple.java       |   35 +
 .../src/main/proto/IndexProtos.proto            |   31 +
 .../src/main/resources/storage-default.xml      |  184 ++
 .../org/apache/tajo/storage/TestFrameTuple.java |   84 +
 .../org/apache/tajo/storage/TestLazyTuple.java  |  258 +++
 .../tajo/storage/TestTupleComparator.java       |   77 +
 .../org/apache/tajo/storage/TestVTuple.java     |  160 ++
 .../src/test/resources/storage-default.xml      |  164 ++
 tajo-storage/tajo-storage-hbase/pom.xml         |  349 ++++
 .../storage/hbase/AbstractHBaseAppender.java    |  223 +++
 .../storage/hbase/AddSortForInsertRewriter.java |   90 +
 .../tajo/storage/hbase/ColumnMapping.java       |  236 +++
 .../HBaseBinarySerializerDeserializer.java      |   97 +
 .../tajo/storage/hbase/HBaseFragment.java       |  198 ++
 .../tajo/storage/hbase/HBasePutAppender.java    |  120 ++
 .../apache/tajo/storage/hbase/HBaseScanner.java |  449 +++++
 .../storage/hbase/HBaseStorageConstants.java    |   33 +
 .../tajo/storage/hbase/HBaseStorageManager.java | 1135 +++++++++++
 .../hbase/HBaseTextSerializerDeserializer.java  |   71 +
 .../tajo/storage/hbase/HFileAppender.java       |  167 ++
 .../tajo/storage/hbase/IndexPredication.java    |   61 +
 .../tajo/storage/hbase/RowKeyMapping.java       |   40 +
 .../src/main/proto/StorageFragmentProtos.proto  |   35 +
 .../tajo/storage/hbase/TestColumnMapping.java   |   93 +
 .../storage/hbase/TestHBaseStorageManager.java  |  109 ++
 tajo-storage/tajo-storage-hdfs/pom.xml          |  380 ++++
 .../java/org/apache/tajo/storage/CSVFile.java   |  587 ++++++
 .../tajo/storage/CompressedSplitLineReader.java |  182 ++
 .../org/apache/tajo/storage/FileAppender.java   |   87 +
 .../org/apache/tajo/storage/FileScanner.java    |  124 ++
 .../apache/tajo/storage/FileStorageManager.java |  854 +++++++++
 .../tajo/storage/HashShuffleAppender.java       |  209 ++
 .../storage/HashShuffleAppenderManager.java     |  225 +++
 .../org/apache/tajo/storage/LineReader.java     |  559 ++++++
 .../java/org/apache/tajo/storage/RawFile.java   |  773 ++++++++
 .../java/org/apache/tajo/storage/RowFile.java   |  498 +++++
 .../apache/tajo/storage/SplitLineReader.java    |   39 +
 .../apache/tajo/storage/avro/AvroAppender.java  |  219 +++
 .../apache/tajo/storage/avro/AvroScanner.java   |  286 +++
 .../org/apache/tajo/storage/avro/AvroUtil.java  |   77 +
 .../apache/tajo/storage/avro/package-info.java  |   85 +
 .../tajo/storage/fragment/FileFragment.java     |  237 +++
 .../apache/tajo/storage/index/IndexMethod.java  |   32 +
 .../apache/tajo/storage/index/IndexReader.java  |   35 +
 .../apache/tajo/storage/index/IndexWriter.java  |   33 +
 .../tajo/storage/index/OrderIndexReader.java    |   45 +
 .../apache/tajo/storage/index/bst/BSTIndex.java |  623 ++++++
 .../tajo/storage/parquet/ParquetAppender.java   |  151 ++
 .../tajo/storage/parquet/ParquetScanner.java    |  119 ++
 .../tajo/storage/parquet/TajoParquetReader.java |   85 +
 .../tajo/storage/parquet/TajoParquetWriter.java |  104 +
 .../tajo/storage/parquet/TajoReadSupport.java   |   99 +
 .../storage/parquet/TajoRecordConverter.java    |  380 ++++
 .../storage/parquet/TajoRecordMaterializer.java |   77 +
 .../storage/parquet/TajoSchemaConverter.java    |  206 ++
 .../tajo/storage/parquet/TajoWriteSupport.java  |  148 ++
 .../tajo/storage/parquet/package-info.java      |   96 +
 .../storage/rcfile/BytesRefArrayWritable.java   |  261 +++
 .../tajo/storage/rcfile/BytesRefWritable.java   |  248 +++
 .../storage/rcfile/ColumnProjectionUtils.java   |  117 ++
 .../rcfile/LazyDecompressionCallback.java       |   32 +
 .../rcfile/NonSyncByteArrayInputStream.java     |  113 ++
 .../rcfile/NonSyncByteArrayOutputStream.java    |  144 ++
 .../storage/rcfile/NonSyncDataInputBuffer.java  |  507 +++++
 .../storage/rcfile/NonSyncDataOutputBuffer.java |   91 +
 .../org/apache/tajo/storage/rcfile/RCFile.java  | 1807 +++++++++++++++++
 .../SchemaAwareCompressionInputStream.java      |   43 +
 .../SchemaAwareCompressionOutputStream.java     |   44 +
 .../sequencefile/SequenceFileAppender.java      |  274 +++
 .../sequencefile/SequenceFileScanner.java       |  336 ++++
 .../tajo/storage/text/ByteBufLineReader.java    |  154 ++
 .../tajo/storage/text/DelimitedLineReader.java  |  157 ++
 .../tajo/storage/text/DelimitedTextFile.java    |  468 +++++
 .../tajo/storage/text/FieldSplitProcessor.java  |   38 +
 .../tajo/storage/text/LineSplitProcessor.java   |   45 +
 .../text/TextFieldSerializerDeserializer.java   |  223 +++
 .../thirdparty/parquet/CodecFactory.java        |  190 ++
 .../parquet/ColumnChunkPageWriteStore.java      |  206 ++
 .../parquet/InternalParquetRecordReader.java    |  188 ++
 .../parquet/InternalParquetRecordWriter.java    |  160 ++
 .../thirdparty/parquet/ParquetFileWriter.java   |  492 +++++
 .../thirdparty/parquet/ParquetReader.java       |  146 ++
 .../thirdparty/parquet/ParquetWriter.java       |  224 +++
 .../src/main/proto/StorageFragmentProtos.proto  |   34 +
 .../java/org/apache/tajo/HttpFileServer.java    |   84 +
 .../org/apache/tajo/HttpFileServerHandler.java  |  184 ++
 .../tajo/HttpFileServerPipelineFactory.java     |   54 +
 .../tajo/storage/TestCompressionStorages.java   |  185 ++
 .../tajo/storage/TestFileStorageManager.java    |  203 ++
 .../apache/tajo/storage/TestFileSystems.java    |  129 ++
 .../apache/tajo/storage/TestMergeScanner.java   |  202 ++
 .../org/apache/tajo/storage/TestStorages.java   |  867 +++++++++
 .../apache/tajo/storage/avro/TestAvroUtil.java  |  106 +
 .../apache/tajo/storage/index/TestBSTIndex.java |  947 +++++++++
 .../index/TestSingleCSVFileBSTIndex.java        |  248 +++
 .../tajo/storage/parquet/TestReadWrite.java     |  109 ++
 .../storage/parquet/TestSchemaConverter.java    |  130 ++
 .../java/org/apache/tajo/storage/s3/INode.java  |  124 ++
 .../storage/s3/InMemoryFileSystemStore.java     |  176 ++
 .../apache/tajo/storage/s3/S3OutputStream.java  |  234 +++
 .../tajo/storage/s3/SmallBlockS3FileSystem.java |  314 +++
 .../src/test/resources/storage-default.xml      |  164 ++
 .../src/test/resources/testVariousTypes.avsc    |   21 +
 tajo-yarn-pullserver/pom.xml                    |    7 +-
 362 files changed, 30704 insertions(+), 30655 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 235b374..87ef9a6 100644
--- a/CHANGES
+++ b/CHANGES
@@ -17,6 +17,9 @@ Release 0.9.1 - unreleased
 
   IMPROVEMENT
 
+    TAJO-1122: Refactor the tajo-storage project structure.
+    (Hyoungjun Kim)
+
     TAJO-1195: Remove unused CachedDNSResolver Class. (DaeMyung Kang via jaehwa)
 
     TAJO-1184: Upgrade netty-buffer to 4.0.24.Final. (jinho)

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index b3ae9be..92437af 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -73,15 +73,6 @@ message FragmentProto {
   required bytes contents = 3;
 }
 
-message FileFragmentProto {
-  required string id = 1;
-  required string path = 2;
-  required int64 startOffset = 3;
-  required int64 length = 4;
-  repeated string hosts = 7;
-  repeated int32 diskIds = 8;
-}
-
 message TableProto {
   required StoreType storeType = 1;
   optional KeyValueSetProto params = 2;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
index 2950a96..4187150 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hcatalog/pom.xml
@@ -113,7 +113,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
     </dependency>
     <dependency>
       <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-client/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-client/pom.xml b/tajo-client/pom.xml
index 7fc8c74..c023db8 100644
--- a/tajo-client/pom.xml
+++ b/tajo-client/pom.xml
@@ -195,7 +195,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
index c0b01e2..b6e47c7 100644
--- a/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
+++ b/tajo-client/src/main/java/org/apache/tajo/jdbc/TajoResultSet.java
@@ -30,10 +30,7 @@ import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.client.QueryClient;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.MergeScanner;
-import org.apache.tajo.storage.Scanner;
-import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.Fragment;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
index 0ceb2b2..dcf5c4e 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
@@ -119,6 +119,15 @@ public class TUtil {
     return new ArrayList<T>();
   }
 
+  public static List<Integer> newIntList(int...items) {
+    List<Integer> list = new ArrayList<Integer>();
+    for (int item: items) {
+      list.add(item);
+    }
+
+    return list;
+  }
+
   public static <T> List<T> newList(T...items) {
     List<T> list = new ArrayList<T>();
     for (T t : items) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml
index b58ae89..361b342 100644
--- a/tajo-core/pom.xml
+++ b/tajo-core/pom.xml
@@ -245,7 +245,16 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-storage</artifactId>
+      <artifactId>tajo-storage-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hdfs</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-hbase</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
@@ -482,8 +491,8 @@
         <dependency>
           <groupId>org.apache.tajo</groupId>
           <artifactId>tajo-hcatalog</artifactId>
-          <scope>test</scope>
           <version>${tajo.version}</version>
+          <scope>test</scope>
           <exclusions>
             <exclusion>
               <groupId>com.google.protobuf</groupId>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 98a621e..3880bcb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
@@ -49,11 +49,7 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.SortSpecAr
 import org.apache.tajo.plan.LogicalPlan;
 import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.FileStorageManager;
-import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
@@ -921,7 +917,7 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
           if (broadcastFlag) {
             PartitionedTableScanNode partitionedTableScanNode = (PartitionedTableScanNode) scanNode;
             List<Fragment> fileFragments = TUtil.newList();
-            FileStorageManager fileStorageManager = StorageManager.getFileStorageManager(ctx.getConf());
+            FileStorageManager fileStorageManager = (FileStorageManager)StorageManager.getFileStorageManager(ctx.getConf());
             for (Path path : partitionedTableScanNode.getInputPaths()) {
               fileFragments.addAll(TUtil.newList(fileStorageManager.split(scanNode.getCanonicalName(), path)));
             }
@@ -1186,8 +1182,8 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
         FragmentConvertor.convert(ctx.getConf(), fragmentProtos);
 
     String indexName = IndexUtil.getIndexNameOfFrag(fragments.get(0), annotation.getSortKeys());
-    Path indexPath = new Path(
-        StorageManager.getFileStorageManager(ctx.getConf()).getTablePath(annotation.getTableName()), "index");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(ctx.getConf());
+    Path indexPath = new Path(sm.getTablePath(annotation.getTableName()), "index");
 
     TupleComparator comp = new BaseTupleComparator(annotation.getKeySchema(),
         annotation.getSortKeys());

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 c5df5f9..8ee4e2f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/ColPartitionStoreExec.java
@@ -35,6 +35,7 @@ import org.apache.tajo.plan.logical.InsertNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.StoreTableNode;
 import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.FileStorageManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.unit.StorageUnit;
@@ -161,7 +162,7 @@ public abstract class ColPartitionStoreExec extends UnaryPhysicalExec {
       actualFilePath = new Path(lastFileName + "_" + suffixId);
     }
 
-    appender = StorageManager.getFileStorageManager(context.getConf())
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(context.getConf()))
         .getAppender(meta, outSchema, actualFilePath);
 
     appender.enableStats();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 818bd2e..f38ae3c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
@@ -34,11 +34,7 @@ import org.apache.tajo.plan.expr.EvalNode;
 import org.apache.tajo.plan.logical.NodeType;
 import org.apache.tajo.plan.logical.PersistentStoreNode;
 import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.storage.FileStorageManager;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 568c6ec..585d73a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/RangeShuffleFileWriteExec.java
@@ -80,8 +80,8 @@ public class RangeShuffleFileWriteExec extends UnaryPhysicalExec {
         context.getDataChannel().getStoreType() : CatalogProtos.StoreType.RAW);
     FileSystem fs = new RawLocalFileSystem();
     fs.mkdirs(storeTablePath);
-    this.appender = (FileAppender) StorageManager.getFileStorageManager(context.getConf()).getAppender(meta,
-        outSchema, new Path(storeTablePath, "output"));
+    this.appender = (FileAppender) ((FileStorageManager)StorageManager.getFileStorageManager(context.getConf()))
+        .getAppender(meta, outSchema, new Path(storeTablePath, "output"));
     this.appender.enableStats();
     this.appender.init();
     this.indexWriter = bst.getIndexWriter(new Path(storeTablePath, "index"),

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 a5e0b5d..3d3da5c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/StoreTableExec.java
@@ -31,6 +31,7 @@ import org.apache.tajo.plan.logical.InsertNode;
 import org.apache.tajo.plan.logical.PersistentStoreNode;
 import org.apache.tajo.plan.util.PlannerUtil;
 import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.FileStorageManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.unit.StorageUnit;
@@ -92,7 +93,8 @@ public class StoreTableExec extends UnaryPhysicalExec {
         lastFileName = new Path(lastFileName + "_" + suffixId);
       }
 
-      appender = StorageManager.getFileStorageManager(context.getConf()).getAppender(meta, appenderSchema, lastFileName);
+      appender = ((FileStorageManager)StorageManager.getFileStorageManager(context.getConf()))
+          .getAppender(meta, appenderSchema, lastFileName);
 
       if (suffixId > 0) {
         LOG.info(prevFile + " exceeds " + SessionVars.MAX_OUTPUT_FILE_SIZE.keyname() + " (" + maxPerFileSize + " MB), " +

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 0d3ae19..87646de 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
@@ -324,6 +324,7 @@ public class GlobalEngine extends AbstractService {
         responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
         responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
         responseBuilder.setErrorMessage("Fail starting QueryMaster.");
+        LOG.error("Fail starting QueryMaster: " + sql);
       } else {
         responseBuilder.setIsForwarded(true);
         responseBuilder.setQueryId(queryInfo.getQueryId().getProto());
@@ -332,7 +333,8 @@ public class GlobalEngine extends AbstractService {
           responseBuilder.setQueryMasterHost(queryInfo.getQueryMasterHost());
         }
         responseBuilder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
-        LOG.info("Query is forwarded to " + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
+        LOG.info("Query " + queryInfo.getQueryId().toString() + "," + queryInfo.getSql() + "," +
+            " is forwarded to " + queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
       }
     }
     SubmitQueryResponse response = responseBuilder.build();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
index aff4b7d..038c334 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/LazyTaskScheduler.java
@@ -42,6 +42,7 @@ import org.apache.tajo.master.querymaster.QueryUnitAttempt;
 import org.apache.tajo.master.querymaster.SubQuery;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.fragment.Fragment;
 import org.apache.tajo.util.NetUtils;
 import org.apache.tajo.worker.FetchImpl;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 d015efe..49a6d92 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
@@ -167,7 +167,7 @@ public class TajoMaster extends CompositeService {
 
       // check the system directory and create if they are not created.
       checkAndInitializeSystemDirectories();
-      this.storeManager = StorageManager.getFileStorageManager(systemConf, null);
+      this.storeManager = (FileStorageManager)StorageManager.getFileStorageManager(systemConf, null);
 
       catalogServer = new CatalogServer(FunctionLoader.load());
       addIfService(catalogServer);

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
index 8164869..a240ace 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -97,7 +97,8 @@ public class Repartitioner {
     for (int i = 0; i < scans.length; i++) {
       TableDesc tableDesc = masterContext.getTableDescMap().get(scans[i].getCanonicalName());
       if (tableDesc == null) { // if it is a real table stored on storage
-        FileStorageManager storageManager = StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+        FileStorageManager storageManager =
+            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
 
         tablePath = storageManager.getTablePath(scans[i].getTableName());
         if (execBlock.getUnionScanMap() != null && !execBlock.getUnionScanMap().isEmpty()) {
@@ -410,7 +411,8 @@ public class Repartitioner {
         Path[] partitionScanPaths = null;
         TableDesc tableDesc = masterContext.getTableDescMap().get(eachScan.getCanonicalName());
         if (eachScan.getType() == NodeType.PARTITIONS_SCAN) {
-          FileStorageManager storageManager = StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+          FileStorageManager storageManager =
+              (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
 
           PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)eachScan;
           partitionScanPaths = partitionScan.getInputPaths();
@@ -535,7 +537,7 @@ public class Repartitioner {
         partitionScanPaths = partitionScan.getInputPaths();
         // set null to inputPaths in getFragmentsFromPartitionedTable()
         FileStorageManager storageManager =
-            StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
         scanFragments = getFragmentsFromPartitionedTable(storageManager, scan, desc);
       } else {
         StorageManager storageManager =
@@ -644,7 +646,8 @@ public class Repartitioner {
     ExecutionBlock execBlock = subQuery.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
     Path tablePath;
-    tablePath = StorageManager.getFileStorageManager(subQuery.getContext().getConf()).getTablePath(scan.getTableName());
+    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf()))
+        .getTablePath(scan.getTableName());
 
     ExecutionBlock sampleChildBlock = masterPlan.getChild(subQuery.getId(), 0);
     SortNode sortNode = PlannerUtil.findTopNode(sampleChildBlock.getPlan(), NodeType.SORT);
@@ -809,7 +812,8 @@ public class Repartitioner {
     ExecutionBlock execBlock = subQuery.getBlock();
     ScanNode scan = execBlock.getScanNodes()[0];
     Path tablePath;
-    tablePath = StorageManager.getFileStorageManager(subQuery.getContext().getConf()).getTablePath(scan.getTableName());
+    tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf()))
+        .getTablePath(scan.getTableName());
 
     Fragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
     List<Fragment> fragments = new ArrayList<Fragment>();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 6676072..745456a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -1051,7 +1051,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
       if (scan.getType() == NodeType.PARTITIONS_SCAN) {
         // After calling this method, partition paths are removed from the physical plan.
         FileStorageManager storageManager =
-            StorageManager.getFileStorageManager(subQuery.getContext().getConf());
+            (FileStorageManager)StorageManager.getFileStorageManager(subQuery.getContext().getConf());
         fragments = Repartitioner.getFragmentsFromPartitionedTable(storageManager, scan, table);
       } else {
         StorageManager storageManager =

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
index 0e3ccad..5d7472f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -38,6 +38,7 @@ import org.apache.tajo.master.TajoMaster;
 import org.apache.tajo.master.querymaster.QueryInProgress;
 import org.apache.tajo.rpc.CallFuture;
 import org.apache.tajo.util.ApplicationIdUtils;
+import org.apache.tajo.util.StringUtils;
 
 import java.io.IOException;
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
index 0657e66..00eabcc 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java
@@ -52,11 +52,7 @@ import org.apache.tajo.plan.logical.*;
 import org.apache.tajo.pullserver.TajoPullServerService;
 import org.apache.tajo.pullserver.retriever.FileChunk;
 import org.apache.tajo.rpc.NullCallback;
-import org.apache.tajo.storage.BaseTupleComparator;
-import org.apache.tajo.storage.HashShuffleAppenderManager;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.util.NetUtils;
 import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
@@ -157,8 +153,8 @@ public class Task {
         this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys());
       }
     } else {
-      Path outFilePath = StorageManager.getFileStorageManager(systemConf).getAppenderFilePath(
-          taskId, queryContext.getStagingDir());
+      Path outFilePath = ((FileStorageManager)StorageManager.getFileStorageManager(systemConf))
+          .getAppenderFilePath(taskId, queryContext.getStagingDir());
       LOG.info("Output File Path: " + outFilePath);
       context.setOutputPath(outFilePath);
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 0959f2e..fb98be2 100644
--- a/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/BackendTestingUtil.java
@@ -47,7 +47,7 @@ public class BackendTestingUtil {
 
   public static void writeTmpTable(TajoConf conf, Path tablePath)
       throws IOException {
-    FileStorageManager sm = StorageManager.getFileStorageManager(conf, tablePath);
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, tablePath);
     FileSystem fs = sm.getFileSystem();
 
     Appender appender;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 211f1fe..9ce7b5b 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
@@ -40,10 +40,10 @@ import org.apache.tajo.plan.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.fragment.FileFragment;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleComparator;
 import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.storage.fragment.FileFragment;
 import org.apache.tajo.storage.fragment.FragmentConvertor;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.KeyValueSet;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
index 64d7cee..3803c7a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/global/TestBroadcastJoinPlan.java
@@ -42,10 +42,7 @@ import org.apache.tajo.plan.LogicalPlanner;
 import org.apache.tajo.plan.PlanningException;
 import org.apache.tajo.engine.query.QueryContext;
 import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.Appender;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.*;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -140,8 +137,8 @@ public class TestBroadcastJoinPlan {
         contentsData += j;
       }
     }
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(tableMeta, schema,
-        dataPath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(tableMeta, schema, dataPath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     int writtenSize = 0;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 9517e1b..6a6aafb 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
@@ -89,7 +89,8 @@ public class TestBNLJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < OUTER_TUPLE_NUM; i++) {
@@ -110,7 +111,8 @@ public class TestBNLJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getFileStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < INNER_TUPLE_NUM; i += 2) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 61f32da..dc3c28d 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
@@ -91,7 +91,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 = StorageManager.getFileStorageManager(conf, workDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, workDir);
 
     idxPath = new Path(workDir, "test.idx");
 
@@ -117,8 +117,7 @@ public class TestBSTIndexExec {
     fs = tablePath.getFileSystem(conf);
     fs.mkdirs(tablePath.getParent());
 
-    FileAppender appender = (FileAppender)StorageManager.getFileStorageManager(conf).getAppender(meta, schema,
-        tablePath);
+    FileAppender appender = (FileAppender)sm.getAppender(meta, schema, tablePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < 10000; i++) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 00d023e..c0bf6ce 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
@@ -76,7 +76,7 @@ public class TestExternalSortExec {
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(TajoConstants.DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf.setVar(TajoConf.ConfVars.WORKER_TEMPORAL_DIR, testDir.toString());
-    sm = StorageManager.getFileStorageManager(conf, testDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, testDir);
 
     Schema schema = new Schema();
     schema.addColumn("managerid", Type.INT4);
@@ -85,7 +85,8 @@ public class TestExternalSortExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
     Tuple tuple = new VTuple(schema.size());

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 3a4e963..ecd1c23 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterHashJoinExec.java
@@ -84,7 +84,7 @@ public class TestFullOuterHashJoinExec {
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
     conf = util.getConfiguration();
-    sm = StorageManager.getFileStorageManager(conf, testDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, testDir);
 
     //----------------- dep3 ------------------------------
     // dep_id | dep_name  | loc_id
@@ -107,7 +107,8 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -136,7 +137,8 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -175,7 +177,8 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getFileStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -227,8 +230,8 @@ public class TestFullOuterHashJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getFileStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
     appender5.flush();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 6684002..a81979f 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestFullOuterMergeJoinExec.java
@@ -112,7 +112,8 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -150,7 +151,8 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = StorageManager.getFileStorageManager(conf).getAppender(dep4Meta, dep4Schema, dep4Path);
+    Appender appender4 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     Tuple tuple4 = new VTuple(dep4Schema.size());
     for (int i = 0; i < 11; i++) {
@@ -181,7 +183,8 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -220,7 +223,8 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getFileStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -272,8 +276,8 @@ public class TestFullOuterMergeJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getFileStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     appender5.flush();
     appender5.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 48c1cae..4fe6ff2 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
@@ -86,8 +86,8 @@ public class TestHashAntiJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
 
@@ -112,7 +112,8 @@ public class TestHashAntiJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getFileStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 10; i += 2) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 dcd18cd..55e87d4 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
@@ -88,8 +88,8 @@ public class TestHashJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
     for (int i = 0; i < 10; i++) {
@@ -111,7 +111,8 @@ public class TestHashJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getFileStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 10; i += 2) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 5cb1a7d..a2f1155 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
@@ -87,8 +87,8 @@ public class TestHashSemiJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
 
@@ -113,7 +113,8 @@ public class TestHashSemiJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getFileStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     // make 27 tuples

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 d709554..0477771 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterHashJoinExec.java
@@ -108,7 +108,8 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -137,7 +138,8 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -176,7 +178,8 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getFileStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -228,8 +231,8 @@ public class TestLeftOuterHashJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getFileStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     
     appender5.flush();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
index 89f5feb..36dd77e 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestLeftOuterNLJoinExec.java
@@ -104,7 +104,8 @@ public class TestLeftOuterNLJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -133,7 +134,8 @@ public class TestLeftOuterNLJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -172,7 +174,8 @@ public class TestLeftOuterNLJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getFileStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -224,8 +227,8 @@ public class TestLeftOuterNLJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getFileStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
     
     appender5.flush();

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 12509f1..10d4d33 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
@@ -89,8 +89,8 @@ public class TestMergeJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
     for (int i = 0; i < 10; i++) {
@@ -118,7 +118,8 @@ public class TestMergeJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getFileStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 10; i += 2) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 abace81..e976456 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
@@ -86,7 +86,8 @@ public class TestNLJoinExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(schema.size());
     for (int i = 0; i < 50; i++) {
@@ -109,7 +110,8 @@ public class TestNLJoinExec {
     peopleSchema.addColumn("age", Type.INT4);
     TableMeta peopleMeta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path peoplePath = new Path(testDir, "people.csv");
-    appender = StorageManager.getFileStorageManager(conf).getAppender(peopleMeta, peopleSchema, peoplePath);
+    appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(peopleMeta, peopleSchema, peoplePath);
     appender.init();
     tuple = new VTuple(peopleSchema.size());
     for (int i = 1; i < 50; i += 2) {

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 6aad738..cce4ba7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -107,7 +107,7 @@ public class TestPhysicalPlanner {
     util.startCatalogCluster();
     conf = util.getConfiguration();
     testDir = CommonTestingUtil.getTestDir("target/test-data/TestPhysicalPlanner");
-    sm = StorageManager.getFileStorageManager(conf, testDir);
+    sm = (FileStorageManager)StorageManager.getFileStorageManager(conf, testDir);
     catalog = util.getMiniCatalogCluster().getCatalog();
     catalog.createTablespace(DEFAULT_TABLESPACE_NAME, testDir.toUri().toString());
     catalog.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME);
@@ -130,8 +130,7 @@ public class TestPhysicalPlanner {
 
 
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, employeeSchema,
-        employeePath);
+    Appender appender = sm.getAppender(employeeMeta, employeeSchema, employeePath);
     appender.init();
     Tuple tuple = new VTuple(employeeSchema.size());
     for (int i = 0; i < 100; i++) {
@@ -149,7 +148,7 @@ public class TestPhysicalPlanner {
 
     Path scorePath = new Path(testDir, "score");
     TableMeta scoreMeta = CatalogUtil.newTableMeta(StoreType.CSV, new KeyValueSet());
-    appender = StorageManager.getFileStorageManager(conf).getAppender(scoreMeta, scoreSchema, scorePath);
+    appender = sm.getAppender(scoreMeta, scoreSchema, scorePath);
     appender.init();
     score = new TableDesc(
         CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "score"), scoreSchema, scoreMeta,
@@ -190,8 +189,8 @@ public class TestPhysicalPlanner {
 
     Schema scoreSchmea = score.getSchema();
     TableMeta scoreLargeMeta = CatalogUtil.newTableMeta(StoreType.RAW, new KeyValueSet());
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(scoreLargeMeta, scoreSchmea,
-        scoreLargePath);
+    Appender appender =  ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(scoreLargeMeta, scoreSchmea, scoreLargePath);
     appender.enableStats();
     appender.init();
     largeScore = new TableDesc(
@@ -452,8 +451,8 @@ public class TestPhysicalPlanner {
     exec.next();
     exec.close();
 
-    Scanner scanner = StorageManager.getFileStorageManager(conf).getFileScanner(outputMeta, rootNode.getOutSchema(),
-        ctx.getOutputPath());
+    Scanner scanner =  ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getFileScanner(outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;
     int i = 0;
@@ -512,7 +511,7 @@ public class TestPhysicalPlanner {
     // checking the file contents
     long totalNum = 0;
     for (FileStatus status : fs.listStatus(ctx.getOutputPath().getParent())) {
-      Scanner scanner = StorageManager.getFileStorageManager(conf).getFileScanner(
+      Scanner scanner =  ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getFileScanner(
           CatalogUtil.newTableMeta(StoreType.CSV),
           rootNode.getOutSchema(),
           status.getPath());
@@ -549,8 +548,8 @@ public class TestPhysicalPlanner {
     exec.next();
     exec.close();
 
-    Scanner scanner = StorageManager.getFileStorageManager(conf).getFileScanner(outputMeta, rootNode.getOutSchema(),
-        ctx.getOutputPath());
+    Scanner scanner = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getFileScanner(
+        outputMeta, rootNode.getOutSchema(), ctx.getOutputPath());
     scanner.init();
     Tuple tuple;
     int i = 0;
@@ -1084,7 +1083,7 @@ public class TestPhysicalPlanner {
     Path outputPath = StorageUtil.concatPath(workDir, "output", "output");
     TableMeta meta = CatalogUtil.newTableMeta(channel.getStoreType(), new KeyValueSet());
     SeekableScanner scanner =
-        StorageManager.getSeekableScanner(conf, meta, exec.getSchema(), outputPath);
+        FileStorageManager.getSeekableScanner(conf, meta, exec.getSchema(), outputPath);
     scanner.init();
 
     int cnt = 0;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 f3936b1..3c78b12 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java
@@ -89,7 +89,8 @@ public class TestProgressExternalSortExec {
 
     TableMeta employeeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.RAW);
     Path employeePath = new Path(testDir, "employee.csv");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(employeeMeta, schema, employeePath);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(employeeMeta, schema, employeePath);
     appender.enableStats();
     appender.init();
     Tuple tuple = new VTuple(schema.size());

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 400d00f..879ca21 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java
@@ -103,7 +103,8 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -132,7 +133,8 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -171,7 +173,8 @@ public class TestRightOuterHashJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getFileStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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 2708273..8bc00cc 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java
@@ -110,7 +110,8 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta dep3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep3Path = new Path(testDir, "dep3.csv");
-    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(dep3Meta, dep3Schema, dep3Path);
+    Appender appender1 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep3Meta, dep3Schema, dep3Path);
     appender1.init();
     Tuple tuple = new VTuple(dep3Schema.size());
     for (int i = 0; i < 10; i++) {
@@ -148,7 +149,8 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta dep4Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path dep4Path = new Path(testDir, "dep4.csv");
-    Appender appender4 = StorageManager.getFileStorageManager(conf).getAppender(dep4Meta, dep4Schema, dep4Path);
+    Appender appender4 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(dep4Meta, dep4Schema, dep4Path);
     appender4.init();
     Tuple tuple4 = new VTuple(dep4Schema.size());
     for (int i = 0; i < 11; i++) {
@@ -179,7 +181,8 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta job3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path job3Path = new Path(testDir, "job3.csv");
-    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(job3Meta, job3Schema, job3Path);
+    Appender appender2 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(job3Meta, job3Schema, job3Path);
     appender2.init();
     Tuple tuple2 = new VTuple(job3Schema.size());
     for (int i = 1; i < 4; i++) {
@@ -218,7 +221,8 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta emp3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path emp3Path = new Path(testDir, "emp3.csv");
-    Appender appender3 = StorageManager.getFileStorageManager(conf).getAppender(emp3Meta, emp3Schema, emp3Path);
+    Appender appender3 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(emp3Meta, emp3Schema, emp3Path);
     appender3.init();
     Tuple tuple3 = new VTuple(emp3Schema.size());
 
@@ -270,8 +274,8 @@ public class TestRightOuterMergeJoinExec {
 
     TableMeta phone3Meta = CatalogUtil.newTableMeta(StoreType.CSV);
     Path phone3Path = new Path(testDir, "phone3.csv");
-    Appender appender5 = StorageManager.getFileStorageManager(conf).getAppender(phone3Meta, phone3Schema,
-        phone3Path);
+    Appender appender5 = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(phone3Meta, phone3Schema, phone3Path);
     appender5.init();
 
     appender5.flush();


[17/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
deleted file mode 100644
index ad5f408..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ /dev/null
@@ -1,857 +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.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoIdProtos;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.ProtobufDatumFactory;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.rcfile.RCFile;
-import org.apache.tajo.storage.sequencefile.SequenceFileScanner;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.KeyValueSet;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-@RunWith(Parameterized.class)
-public class TestStorages {
-	private TajoConf conf;
-	private static String TEST_PATH = "target/test-data/TestStorages";
-
-  private static String TEST_PROJECTION_AVRO_SCHEMA =
-      "{\n" +
-      "  \"type\": \"record\",\n" +
-      "  \"namespace\": \"org.apache.tajo\",\n" +
-      "  \"name\": \"testProjection\",\n" +
-      "  \"fields\": [\n" +
-      "    { \"name\": \"id\", \"type\": \"int\" },\n" +
-      "    { \"name\": \"age\", \"type\": \"long\" },\n" +
-      "    { \"name\": \"score\", \"type\": \"float\" }\n" +
-      "  ]\n" +
-      "}\n";
-
-  private static String TEST_NULL_HANDLING_TYPES_AVRO_SCHEMA =
-      "{\n" +
-      "  \"type\": \"record\",\n" +
-      "  \"namespace\": \"org.apache.tajo\",\n" +
-      "  \"name\": \"testNullHandlingTypes\",\n" +
-      "  \"fields\": [\n" +
-      "    { \"name\": \"col1\", \"type\": [\"null\", \"boolean\"] },\n" +
-      "    { \"name\": \"col2\", \"type\": [\"null\", \"int\"] },\n" +
-      "    { \"name\": \"col3\", \"type\": [\"null\", \"string\"] },\n" +
-      "    { \"name\": \"col4\", \"type\": [\"null\", \"int\"] },\n" +
-      "    { \"name\": \"col5\", \"type\": [\"null\", \"int\"] },\n" +
-      "    { \"name\": \"col6\", \"type\": [\"null\", \"long\"] },\n" +
-      "    { \"name\": \"col7\", \"type\": [\"null\", \"float\"] },\n" +
-      "    { \"name\": \"col8\", \"type\": [\"null\", \"double\"] },\n" +
-      "    { \"name\": \"col9\", \"type\": [\"null\", \"string\"] },\n" +
-      "    { \"name\": \"col10\", \"type\": [\"null\", \"bytes\"] },\n" +
-      "    { \"name\": \"col11\", \"type\": [\"null\", \"bytes\"] },\n" +
-      "    { \"name\": \"col12\", \"type\": \"null\" },\n" +
-      "    { \"name\": \"col13\", \"type\": [\"null\", \"bytes\"] }\n" +
-      "  ]\n" +
-      "}\n";
-
-  private StoreType storeType;
-  private boolean splitable;
-  private boolean statsable;
-  private boolean seekable;
-  private Path testDir;
-  private FileSystem fs;
-
-  public TestStorages(StoreType type, boolean splitable, boolean statsable, boolean seekable) throws IOException {
-    this.storeType = type;
-    this.splitable = splitable;
-    this.statsable = statsable;
-    this.seekable = seekable;
-
-    conf = new TajoConf();
-
-    if (storeType == StoreType.RCFILE) {
-      conf.setInt(RCFile.RECORD_INTERVAL_CONF_STR, 100);
-    }
-
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> generateParameters() {
-    return Arrays.asList(new Object[][] {
-        //type, splitable, statsable, seekable
-        {StoreType.CSV, true, true, true},
-        {StoreType.RAW, false, true, true},
-        {StoreType.RCFILE, true, true, false},
-        {StoreType.PARQUET, false, false, false},
-        {StoreType.SEQUENCEFILE, true, true, false},
-        {StoreType.AVRO, false, false, false},
-        {StoreType.TEXTFILE, true, true, false},
-    });
-  }
-
-	@Test
-  public void testSplitable() throws IOException {
-    if (splitable) {
-      Schema schema = new Schema();
-      schema.addColumn("id", Type.INT4);
-      schema.addColumn("age", Type.INT8);
-
-      TableMeta meta = CatalogUtil.newTableMeta(storeType);
-      Path tablePath = new Path(testDir, "Splitable.data");
-      Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-      appender.enableStats();
-      appender.init();
-      int tupleNum = 10000;
-      VTuple vTuple;
-
-      for (int i = 0; i < tupleNum; i++) {
-        vTuple = new VTuple(2);
-        vTuple.put(0, DatumFactory.createInt4(i + 1));
-        vTuple.put(1, DatumFactory.createInt8(25l));
-        appender.addTuple(vTuple);
-      }
-      appender.close();
-      TableStats stat = appender.getStats();
-      assertEquals(tupleNum, stat.getNumRows().longValue());
-
-      FileStatus status = fs.getFileStatus(tablePath);
-      long fileLen = status.getLen();
-      long randomNum = (long) (Math.random() * fileLen) + 1;
-
-      FileFragment[] tablets = new FileFragment[2];
-      tablets[0] = new FileFragment("Splitable", tablePath, 0, randomNum);
-      tablets[1] = new FileFragment("Splitable", tablePath, randomNum, (fileLen - randomNum));
-
-      Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-      assertTrue(scanner.isSplittable());
-      scanner.init();
-      int tupleCnt = 0;
-      while (scanner.next() != null) {
-        tupleCnt++;
-      }
-      scanner.close();
-
-      scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
-      assertTrue(scanner.isSplittable());
-      scanner.init();
-      while (scanner.next() != null) {
-        tupleCnt++;
-      }
-      scanner.close();
-
-      assertEquals(tupleNum, tupleCnt);
-    }
-	}
-
-  @Test
-  public void testRCFileSplitable() throws IOException {
-    if (storeType == StoreType.RCFILE) {
-      Schema schema = new Schema();
-      schema.addColumn("id", Type.INT4);
-      schema.addColumn("age", Type.INT8);
-
-      TableMeta meta = CatalogUtil.newTableMeta(storeType);
-      Path tablePath = new Path(testDir, "Splitable.data");
-      Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-      appender.enableStats();
-      appender.init();
-      int tupleNum = 10000;
-      VTuple vTuple;
-
-      for (int i = 0; i < tupleNum; i++) {
-        vTuple = new VTuple(2);
-        vTuple.put(0, DatumFactory.createInt4(i + 1));
-        vTuple.put(1, DatumFactory.createInt8(25l));
-        appender.addTuple(vTuple);
-      }
-      appender.close();
-      TableStats stat = appender.getStats();
-      assertEquals(tupleNum, stat.getNumRows().longValue());
-
-      FileStatus status = fs.getFileStatus(tablePath);
-      long fileLen = status.getLen();
-      long randomNum = 122; // header size
-
-      FileFragment[] tablets = new FileFragment[2];
-      tablets[0] = new FileFragment("Splitable", tablePath, 0, randomNum);
-      tablets[1] = new FileFragment("Splitable", tablePath, randomNum, (fileLen - randomNum));
-
-      Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-      assertTrue(scanner.isSplittable());
-      scanner.init();
-      int tupleCnt = 0;
-      while (scanner.next() != null) {
-        tupleCnt++;
-      }
-      scanner.close();
-
-      scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
-      assertTrue(scanner.isSplittable());
-      scanner.init();
-      while (scanner.next() != null) {
-        tupleCnt++;
-      }
-      scanner.close();
-
-      assertEquals(tupleNum, tupleCnt);
-    }
-  }
-
-  @Test
-  public void testProjection() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4);
-    schema.addColumn("age", Type.INT8);
-    schema.addColumn("score", Type.FLOAT4);
-
-    TableMeta meta = CatalogUtil.newTableMeta(storeType);
-    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
-    if (storeType == StoreType.AVRO) {
-      meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL,
-                     TEST_PROJECTION_AVRO_SCHEMA);
-    }
-
-    Path tablePath = new Path(testDir, "testProjection.data");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    int tupleNum = 10000;
-    VTuple vTuple;
-
-    for (int i = 0; i < tupleNum; i++) {
-      vTuple = new VTuple(3);
-      vTuple.put(0, DatumFactory.createInt4(i + 1));
-      vTuple.put(1, DatumFactory.createInt8(i + 2));
-      vTuple.put(2, DatumFactory.createFloat4(i + 3));
-      appender.addTuple(vTuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    FileFragment fragment = new FileFragment("testReadAndWrite", tablePath, 0, status.getLen());
-
-    Schema target = new Schema();
-    target.addColumn("age", Type.INT8);
-    target.addColumn("score", Type.FLOAT4);
-    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment, target);
-    scanner.init();
-    int tupleCnt = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      if (storeType == StoreType.RCFILE
-          || storeType == StoreType.CSV
-          || storeType == StoreType.PARQUET
-          || storeType == StoreType.SEQUENCEFILE
-          || storeType == StoreType.AVRO) {
-        assertTrue(tuple.get(0) == null);
-      }
-      assertTrue(tupleCnt + 2 == tuple.get(1).asInt8());
-      assertTrue(tupleCnt + 3 == tuple.get(2).asFloat4());
-      tupleCnt++;
-    }
-    scanner.close();
-
-    assertEquals(tupleNum, tupleCnt);
-  }
-
-  @Test
-  public void testVariousTypes() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("col1", Type.BOOLEAN);
-    schema.addColumn("col2", Type.BIT);
-    schema.addColumn("col3", Type.CHAR, 7);
-    schema.addColumn("col4", Type.INT2);
-    schema.addColumn("col5", Type.INT4);
-    schema.addColumn("col6", Type.INT8);
-    schema.addColumn("col7", Type.FLOAT4);
-    schema.addColumn("col8", Type.FLOAT8);
-    schema.addColumn("col9", Type.TEXT);
-    schema.addColumn("col10", Type.BLOB);
-    schema.addColumn("col11", Type.INET4);
-    schema.addColumn("col12", Type.NULL_TYPE);
-    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
-
-    KeyValueSet options = new KeyValueSet();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
-    if (storeType == StoreType.AVRO) {
-      String path = FileUtil.getResourcePath("testVariousTypes.avsc").toString();
-      meta.putOption(StorageConstants.AVRO_SCHEMA_URL, path);
-    }
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-
-    QueryId queryid = new QueryId("12345", 5);
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-
-    Tuple tuple = new VTuple(13);
-    tuple.put(new Datum[] {
-        DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
-        DatumFactory.createChar("hyunsik"),
-        DatumFactory.createInt2((short) 17),
-        DatumFactory.createInt4(59),
-        DatumFactory.createInt8(23l),
-        DatumFactory.createFloat4(77.9f),
-        DatumFactory.createFloat8(271.9f),
-        DatumFactory.createText("hyunsik"),
-        DatumFactory.createBlob("hyunsik".getBytes()),
-        DatumFactory.createInet4("192.168.0.1"),
-        NullDatum.get(),
-        factory.createDatum(queryid.getProto())
-    });
-    appender.addTuple(tuple);
-    appender.flush();
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
-    scanner.init();
-
-    Tuple retrieved;
-    while ((retrieved = scanner.next()) != null) {
-      for (int i = 0; i < tuple.size(); i++) {
-        assertEquals(tuple.get(i), retrieved.get(i));
-      }
-    }
-    scanner.close();
-  }
-
-  @Test
-  public void testNullHandlingTypes() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("col1", Type.BOOLEAN);
-    schema.addColumn("col2", Type.BIT);
-    schema.addColumn("col3", Type.CHAR, 7);
-    schema.addColumn("col4", Type.INT2);
-    schema.addColumn("col5", Type.INT4);
-    schema.addColumn("col6", Type.INT8);
-    schema.addColumn("col7", Type.FLOAT4);
-    schema.addColumn("col8", Type.FLOAT8);
-    schema.addColumn("col9", Type.TEXT);
-    schema.addColumn("col10", Type.BLOB);
-    schema.addColumn("col11", Type.INET4);
-    schema.addColumn("col12", Type.NULL_TYPE);
-    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
-
-    KeyValueSet options = new KeyValueSet();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
-    meta.putOption(StorageConstants.TEXT_NULL, "\\\\N");
-    meta.putOption(StorageConstants.RCFILE_NULL, "\\\\N");
-    meta.putOption(StorageConstants.RCFILE_SERDE, TextSerializerDeserializer.class.getName());
-    meta.putOption(StorageConstants.SEQUENCEFILE_NULL, "\\");
-    if (storeType == StoreType.AVRO) {
-      meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL,
-                     TEST_NULL_HANDLING_TYPES_AVRO_SCHEMA);
-    }
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-
-    QueryId queryid = new QueryId("12345", 5);
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-
-    Tuple seedTuple = new VTuple(13);
-    seedTuple.put(new Datum[]{
-        DatumFactory.createBool(true),                // 0
-        DatumFactory.createBit((byte) 0x99),          // 1
-        DatumFactory.createChar("hyunsik"),           // 2
-        DatumFactory.createInt2((short) 17),          // 3
-        DatumFactory.createInt4(59),                  // 4
-        DatumFactory.createInt8(23l),                 // 5
-        DatumFactory.createFloat4(77.9f),             // 6
-        DatumFactory.createFloat8(271.9f),            // 7
-        DatumFactory.createText("hyunsik"),           // 8
-        DatumFactory.createBlob("hyunsik".getBytes()),// 9
-        DatumFactory.createInet4("192.168.0.1"),      // 10
-        NullDatum.get(),                              // 11
-        factory.createDatum(queryid.getProto())       // 12
-    });
-
-    // Making tuples with different null column positions
-    Tuple tuple;
-    for (int i = 0; i < 13; i++) {
-      tuple = new VTuple(13);
-      for (int j = 0; j < 13; j++) {
-        if (i == j) { // i'th column will have NULL value
-          tuple.put(j, NullDatum.get());
-        } else {
-          tuple.put(j, seedTuple.get(j));
-        }
-      }
-      appender.addTuple(tuple);
-    }
-    appender.flush();
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
-    scanner.init();
-
-    Tuple retrieved;
-    int i = 0;
-    while ((retrieved = scanner.next()) != null) {
-      assertEquals(13, retrieved.size());
-      for (int j = 0; j < 13; j++) {
-        if (i == j) {
-          assertEquals(NullDatum.get(), retrieved.get(j));
-        } else {
-          assertEquals(seedTuple.get(j), retrieved.get(j));
-        }
-      }
-
-      i++;
-    }
-    scanner.close();
-  }
-
-  @Test
-  public void testRCFileTextSerializeDeserialize() throws IOException {
-    if(storeType != StoreType.RCFILE) return;
-
-    Schema schema = new Schema();
-    schema.addColumn("col1", Type.BOOLEAN);
-    schema.addColumn("col2", Type.BIT);
-    schema.addColumn("col3", Type.CHAR, 7);
-    schema.addColumn("col4", Type.INT2);
-    schema.addColumn("col5", Type.INT4);
-    schema.addColumn("col6", Type.INT8);
-    schema.addColumn("col7", Type.FLOAT4);
-    schema.addColumn("col8", Type.FLOAT8);
-    schema.addColumn("col9", Type.TEXT);
-    schema.addColumn("col10", Type.BLOB);
-    schema.addColumn("col11", Type.INET4);
-    schema.addColumn("col12", Type.NULL_TYPE);
-    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
-
-    KeyValueSet options = new KeyValueSet();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.putOption(StorageConstants.CSVFILE_SERDE, TextSerializerDeserializer.class.getName());
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-    appender.init();
-
-    QueryId queryid = new QueryId("12345", 5);
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-
-    Tuple tuple = new VTuple(13);
-    tuple.put(new Datum[] {
-        DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
-        DatumFactory.createChar("jinho"),
-        DatumFactory.createInt2((short) 17),
-        DatumFactory.createInt4(59),
-        DatumFactory.createInt8(23l),
-        DatumFactory.createFloat4(77.9f),
-        DatumFactory.createFloat8(271.9f),
-        DatumFactory.createText("jinho"),
-        DatumFactory.createBlob("hyunsik babo".getBytes()),
-        DatumFactory.createInet4("192.168.0.1"),
-        NullDatum.get(),
-        factory.createDatum(queryid.getProto())
-    });
-    appender.addTuple(tuple);
-    appender.flush();
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
-
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
-    scanner.init();
-
-    Tuple retrieved;
-    while ((retrieved=scanner.next()) != null) {
-      for (int i = 0; i < tuple.size(); i++) {
-        assertEquals(tuple.get(i), retrieved.get(i));
-      }
-    }
-    scanner.close();
-    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
-    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
-  }
-
-  @Test
-  public void testRCFileBinarySerializeDeserialize() throws IOException {
-    if(storeType != StoreType.RCFILE) return;
-
-    Schema schema = new Schema();
-    schema.addColumn("col1", Type.BOOLEAN);
-    schema.addColumn("col2", Type.BIT);
-    schema.addColumn("col3", Type.CHAR, 7);
-    schema.addColumn("col4", Type.INT2);
-    schema.addColumn("col5", Type.INT4);
-    schema.addColumn("col6", Type.INT8);
-    schema.addColumn("col7", Type.FLOAT4);
-    schema.addColumn("col8", Type.FLOAT8);
-    schema.addColumn("col9", Type.TEXT);
-    schema.addColumn("col10", Type.BLOB);
-    schema.addColumn("col11", Type.INET4);
-    schema.addColumn("col12", Type.NULL_TYPE);
-    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
-
-    KeyValueSet options = new KeyValueSet();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.putOption(StorageConstants.RCFILE_SERDE, BinarySerializerDeserializer.class.getName());
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-    appender.init();
-
-    QueryId queryid = new QueryId("12345", 5);
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-
-    Tuple tuple = new VTuple(13);
-    tuple.put(new Datum[] {
-        DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
-        DatumFactory.createChar("jinho"),
-        DatumFactory.createInt2((short) 17),
-        DatumFactory.createInt4(59),
-        DatumFactory.createInt8(23l),
-        DatumFactory.createFloat4(77.9f),
-        DatumFactory.createFloat8(271.9f),
-        DatumFactory.createText("jinho"),
-        DatumFactory.createBlob("hyunsik babo".getBytes()),
-        DatumFactory.createInet4("192.168.0.1"),
-        NullDatum.get(),
-        factory.createDatum(queryid.getProto())
-    });
-    appender.addTuple(tuple);
-    appender.flush();
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
-
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
-    scanner.init();
-
-    Tuple retrieved;
-    while ((retrieved=scanner.next()) != null) {
-      for (int i = 0; i < tuple.size(); i++) {
-        assertEquals(tuple.get(i), retrieved.get(i));
-      }
-    }
-    scanner.close();
-    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
-    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
-  }
-
-  @Test
-  public void testSequenceFileTextSerializeDeserialize() throws IOException {
-    if(storeType != StoreType.SEQUENCEFILE) return;
-
-    Schema schema = new Schema();
-    schema.addColumn("col1", Type.BOOLEAN);
-    schema.addColumn("col2", Type.BIT);
-    schema.addColumn("col3", Type.CHAR, 7);
-    schema.addColumn("col4", Type.INT2);
-    schema.addColumn("col5", Type.INT4);
-    schema.addColumn("col6", Type.INT8);
-    schema.addColumn("col7", Type.FLOAT4);
-    schema.addColumn("col8", Type.FLOAT8);
-    schema.addColumn("col9", Type.TEXT);
-    schema.addColumn("col10", Type.BLOB);
-    schema.addColumn("col11", Type.INET4);
-    schema.addColumn("col12", Type.NULL_TYPE);
-    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
-
-    KeyValueSet options = new KeyValueSet();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-    appender.init();
-
-    QueryId queryid = new QueryId("12345", 5);
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-
-    Tuple tuple = new VTuple(13);
-    tuple.put(new Datum[] {
-        DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
-        DatumFactory.createChar("jinho"),
-        DatumFactory.createInt2((short) 17),
-        DatumFactory.createInt4(59),
-        DatumFactory.createInt8(23l),
-        DatumFactory.createFloat4(77.9f),
-        DatumFactory.createFloat8(271.9f),
-        DatumFactory.createText("jinho"),
-        DatumFactory.createBlob("hyunsik babo".getBytes()),
-        DatumFactory.createInet4("192.168.0.1"),
-        NullDatum.get(),
-        factory.createDatum(queryid.getProto())
-    });
-    appender.addTuple(tuple);
-    appender.flush();
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
-
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
-    scanner.init();
-
-    assertTrue(scanner instanceof SequenceFileScanner);
-    Writable key = ((SequenceFileScanner) scanner).getKey();
-    assertEquals(key.getClass().getCanonicalName(), LongWritable.class.getCanonicalName());
-
-    Tuple retrieved;
-    while ((retrieved=scanner.next()) != null) {
-      for (int i = 0; i < tuple.size(); i++) {
-        assertEquals(tuple.get(i), retrieved.get(i));
-      }
-    }
-    scanner.close();
-    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
-    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
-  }
-
-  @Test
-  public void testSequenceFileBinarySerializeDeserialize() throws IOException {
-    if(storeType != StoreType.SEQUENCEFILE) return;
-
-    Schema schema = new Schema();
-    schema.addColumn("col1", Type.BOOLEAN);
-    schema.addColumn("col2", Type.BIT);
-    schema.addColumn("col3", Type.CHAR, 7);
-    schema.addColumn("col4", Type.INT2);
-    schema.addColumn("col5", Type.INT4);
-    schema.addColumn("col6", Type.INT8);
-    schema.addColumn("col7", Type.FLOAT4);
-    schema.addColumn("col8", Type.FLOAT8);
-    schema.addColumn("col9", Type.TEXT);
-    schema.addColumn("col10", Type.BLOB);
-    schema.addColumn("col11", Type.INET4);
-    schema.addColumn("col12", Type.NULL_TYPE);
-    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
-
-    KeyValueSet options = new KeyValueSet();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, BinarySerializerDeserializer.class.getName());
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-    appender.init();
-
-    QueryId queryid = new QueryId("12345", 5);
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-
-    Tuple tuple = new VTuple(13);
-    tuple.put(new Datum[] {
-        DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
-        DatumFactory.createChar("jinho"),
-        DatumFactory.createInt2((short) 17),
-        DatumFactory.createInt4(59),
-        DatumFactory.createInt8(23l),
-        DatumFactory.createFloat4(77.9f),
-        DatumFactory.createFloat8(271.9f),
-        DatumFactory.createText("jinho"),
-        DatumFactory.createBlob("hyunsik babo".getBytes()),
-        DatumFactory.createInet4("192.168.0.1"),
-        NullDatum.get(),
-        factory.createDatum(queryid.getProto())
-    });
-    appender.addTuple(tuple);
-    appender.flush();
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
-
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
-    scanner.init();
-
-    assertTrue(scanner instanceof SequenceFileScanner);
-    Writable key = ((SequenceFileScanner) scanner).getKey();
-    assertEquals(key.getClass().getCanonicalName(), BytesWritable.class.getCanonicalName());
-
-    Tuple retrieved;
-    while ((retrieved=scanner.next()) != null) {
-      for (int i = 0; i < tuple.size(); i++) {
-        assertEquals(tuple.get(i), retrieved.get(i));
-      }
-    }
-    scanner.close();
-    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
-    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
-  }
-
-  @Test
-  public void testTime() throws IOException {
-    if (storeType == StoreType.CSV || storeType == StoreType.RAW) {
-      Schema schema = new Schema();
-      schema.addColumn("col1", Type.DATE);
-      schema.addColumn("col2", Type.TIME);
-      schema.addColumn("col3", Type.TIMESTAMP);
-
-      KeyValueSet options = new KeyValueSet();
-      TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-
-      Path tablePath = new Path(testDir, "testTime.data");
-      Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-      appender.init();
-
-      Tuple tuple = new VTuple(3);
-      tuple.put(new Datum[]{
-          DatumFactory.createDate("1980-04-01"),
-          DatumFactory.createTime("12:34:56"),
-          DatumFactory.createTimestmpDatumWithUnixTime((int)(System.currentTimeMillis() / 1000))
-      });
-      appender.addTuple(tuple);
-      appender.flush();
-      appender.close();
-
-      FileStatus status = fs.getFileStatus(tablePath);
-      FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-      Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
-      scanner.init();
-
-      Tuple retrieved;
-      while ((retrieved = scanner.next()) != null) {
-        for (int i = 0; i < tuple.size(); i++) {
-          assertEquals(tuple.get(i), retrieved.get(i));
-        }
-      }
-      scanner.close();
-    }
-  }
-
-  @Test
-  public void testSeekableScanner() throws IOException {
-    if (!seekable) {
-      return;
-    }
-
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4);
-    schema.addColumn("age", Type.INT8);
-    schema.addColumn("comment", Type.TEXT);
-
-    TableMeta meta = CatalogUtil.newTableMeta(storeType);
-    Path tablePath = new Path(testDir, "Seekable.data");
-    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(meta, schema,
-        tablePath);
-    appender.enableStats();
-    appender.init();
-    int tupleNum = 100000;
-    VTuple vTuple;
-
-    List<Long> offsets = Lists.newArrayList();
-    offsets.add(0L);
-    for (int i = 0; i < tupleNum; i++) {
-      vTuple = new VTuple(3);
-      vTuple.put(0, DatumFactory.createInt4(i + 1));
-      vTuple.put(1, DatumFactory.createInt8(25l));
-      vTuple.put(2, DatumFactory.createText("test" + i));
-      appender.addTuple(vTuple);
-
-      // find a seek position
-      if (i % (tupleNum / 3) == 0) {
-        offsets.add(appender.getOffset());
-      }
-    }
-
-    // end of file
-    if (!offsets.contains(appender.getOffset())) {
-      offsets.add(appender.getOffset());
-    }
-
-    appender.close();
-    if (statsable) {
-      TableStats stat = appender.getStats();
-      assertEquals(tupleNum, stat.getNumRows().longValue());
-    }
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    assertEquals(status.getLen(), appender.getOffset());
-
-    Scanner scanner;
-    int tupleCnt = 0;
-    long prevOffset = 0;
-    long readBytes = 0;
-    long readRows = 0;
-    for (long offset : offsets) {
-      scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema,
-	        new FileFragment("table", tablePath, prevOffset, offset - prevOffset), schema);
-      scanner.init();
-
-      while (scanner.next() != null) {
-        tupleCnt++;
-      }
-
-      scanner.close();
-      if (statsable) {
-        readBytes += scanner.getInputStats().getNumBytes();
-        readRows += scanner.getInputStats().getNumRows();
-      }
-      prevOffset = offset;
-    }
-
-    assertEquals(tupleNum, tupleCnt);
-    if (statsable) {
-      assertEquals(appender.getStats().getNumBytes().longValue(), readBytes);
-      assertEquals(appender.getStats().getNumRows().longValue(), readRows);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
deleted file mode 100644
index 639ca04..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SortSpec;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestTupleComparator {
-
-  @Before
-  public void setUp() throws Exception {
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
-  @Test
-  public final void testCompare() {
-    Schema schema = new Schema();
-    schema.addColumn("col1", Type.INT4);
-    schema.addColumn("col2", Type.INT4);
-    schema.addColumn("col3", Type.INT4);
-    schema.addColumn("col4", Type.INT4);
-    schema.addColumn("col5", Type.TEXT);
-    
-    Tuple tuple1 = new VTuple(5);
-    Tuple tuple2 = new VTuple(5);
-
-    tuple1.put(
-        new Datum[] {
-        DatumFactory.createInt4(9),
-        DatumFactory.createInt4(3),
-        DatumFactory.createInt4(33),
-        DatumFactory.createInt4(4),
-        DatumFactory.createText("abc")});
-    tuple2.put(
-        new Datum[] {
-        DatumFactory.createInt4(1),
-        DatumFactory.createInt4(25),
-        DatumFactory.createInt4(109),
-        DatumFactory.createInt4(4),
-        DatumFactory.createText("abd")});
-
-    SortSpec sortKey1 = new SortSpec(schema.getColumn("col4"), true, false);
-    SortSpec sortKey2 = new SortSpec(schema.getColumn("col5"), true, false);
-
-    BaseTupleComparator tc = new BaseTupleComparator(schema,
-        new SortSpec[] {sortKey1, sortKey2});
-    assertEquals(-1, tc.compare(tuple1, tuple2));
-    assertEquals(1, tc.compare(tuple2, tuple1));
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java
deleted file mode 100644
index 9837fd1..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-
-import org.junit.Before;
-import org.junit.Test;
-import org.apache.tajo.datum.DatumFactory;
-
-import static org.junit.Assert.*;
-
-public class TestVTuple {
-
-	/**
-	 * @throws java.lang.Exception
-	 */
-	@Before
-	public void setUp() throws Exception {
-		
-	}
-	
-	@Test
-	public void testContain() {
-		VTuple t1 = new VTuple(260);
-		t1.put(0, DatumFactory.createInt4(1));
-		t1.put(1, DatumFactory.createInt4(1));
-		t1.put(27, DatumFactory.createInt4(1));
-		t1.put(96, DatumFactory.createInt4(1));
-		t1.put(257, DatumFactory.createInt4(1));
-		
-		assertTrue(t1.contains(0));
-		assertTrue(t1.contains(1));
-		assertFalse(t1.contains(2));
-		assertFalse(t1.contains(3));
-		assertFalse(t1.contains(4));
-		assertTrue(t1.contains(27));
-		assertFalse(t1.contains(28));
-		assertFalse(t1.contains(95));
-		assertTrue(t1.contains(96));
-		assertFalse(t1.contains(97));
-		assertTrue(t1.contains(257));
-	}
-	
-	@Test
-	public void testPut() {
-		VTuple t1 = new VTuple(260);
-		t1.put(0, DatumFactory.createText("str"));
-		t1.put(1, DatumFactory.createInt4(2));
-		t1.put(257, DatumFactory.createFloat4(0.76f));
-		
-		assertTrue(t1.contains(0));
-		assertTrue(t1.contains(1));
-		
-		assertEquals(t1.getText(0),"str");
-		assertEquals(t1.get(1).asInt4(),2);
-		assertTrue(t1.get(257).asFloat4() == 0.76f);
-	}
-
-  @Test
-	public void testEquals() {
-	  Tuple t1 = new VTuple(5);
-	  Tuple t2 = new VTuple(5);
-	  
-	  t1.put(0, DatumFactory.createInt4(1));
-	  t1.put(1, DatumFactory.createInt4(2));
-	  t1.put(3, DatumFactory.createInt4(2));
-	  
-	  t2.put(0, DatumFactory.createInt4(1));
-    t2.put(1, DatumFactory.createInt4(2));
-    t2.put(3, DatumFactory.createInt4(2));
-    
-    assertEquals(t1,t2);
-    
-    Tuple t3 = new VTuple(5);
-    t2.put(0, DatumFactory.createInt4(1));
-    t2.put(1, DatumFactory.createInt4(2));
-    t2.put(4, DatumFactory.createInt4(2));
-    
-    assertNotSame(t1,t3);
-	}
-	
-	@Test
-	public void testHashCode() {
-	  Tuple t1 = new VTuple(5);
-    Tuple t2 = new VTuple(5);
-    
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(3, DatumFactory.createInt4(2));
-    t1.put(4, DatumFactory.createText("hyunsik"));
-    
-    t2.put(0, DatumFactory.createInt4(1));
-    t2.put(1, DatumFactory.createInt4(2));
-    t2.put(3, DatumFactory.createInt4(2));
-    t2.put(4, DatumFactory.createText("hyunsik"));
-    
-    assertEquals(t1.hashCode(),t2.hashCode());
-    
-    Tuple t3 = new VTuple(5);
-    t3.put(0, DatumFactory.createInt4(1));
-    t3.put(1, DatumFactory.createInt4(2));
-    t3.put(4, DatumFactory.createInt4(2));
-    
-    assertNotSame(t1.hashCode(),t3.hashCode());
-	}
-
-  @Test
-  public void testPutTuple() {
-    Tuple t1 = new VTuple(5);
-
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(2, DatumFactory.createInt4(3));
-
-    Tuple t2 = new VTuple(2);
-    t2.put(0, DatumFactory.createInt4(4));
-    t2.put(1, DatumFactory.createInt4(5));
-
-    t1.put(3, t2);
-
-    for (int i = 0; i < 5; i++) {
-      assertEquals(i+1, t1.get(i).asInt4());
-    }
-  }
-
-  @Test
-  public void testClone() throws CloneNotSupportedException {
-    Tuple t1 = new VTuple(5);
-
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(3, DatumFactory.createInt4(2));
-    t1.put(4, DatumFactory.createText("str"));
-
-    VTuple t2 = (VTuple) t1.clone();
-    assertNotSame(t1, t2);
-    assertEquals(t1, t2);
-
-    assertSame(t1.get(4), t2.get(4));
-
-    t1.clear();
-    assertFalse(t1.equals(t2));
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java b/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
deleted file mode 100644
index 6186e9e..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.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.avro;
-
-import org.apache.avro.Schema;
-import org.apache.tajo.HttpFileServer;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.NetUtils;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URISyntaxException;
-import java.net.URL;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for {@link org.apache.tajo.storage.avro.AvroUtil}.
- */
-public class TestAvroUtil {
-  private Schema expected;
-  private URL schemaUrl;
-
-  @Before
-  public void setUp() throws Exception {
-    schemaUrl = FileUtil.getResourcePath("testVariousTypes.avsc");
-    assertNotNull(schemaUrl);
-
-    File file = new File(schemaUrl.getPath());
-    assertTrue(file.exists());
-
-    expected = new Schema.Parser().parse(file);
-  }
-
-  @Test
-  public void testGetSchema() throws IOException, URISyntaxException {
-    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
-    meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL, FileUtil.readTextFile(new File(schemaUrl.getPath())));
-    Schema schema = AvroUtil.getAvroSchema(meta, new TajoConf());
-    assertEquals(expected, schema);
-
-    meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
-    meta.putOption(StorageConstants.AVRO_SCHEMA_URL, schemaUrl.getPath());
-    schema = AvroUtil.getAvroSchema(meta, new TajoConf());
-    assertEquals(expected, schema);
-
-    HttpFileServer server = new HttpFileServer(NetUtils.createSocketAddr("127.0.0.1:0"));
-    try {
-      server.start();
-      InetSocketAddress addr = server.getBindAddress();
-
-      String url = "http://127.0.0.1:" + addr.getPort() + schemaUrl.getPath();
-      meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
-      meta.putOption(StorageConstants.AVRO_SCHEMA_URL, url);
-      schema = AvroUtil.getAvroSchema(meta, new TajoConf());
-    } finally {
-      server.stop();
-    }
-    assertEquals(expected, schema);
-  }
-
-  @Test
-  public void testGetSchemaFromHttp() throws IOException, URISyntaxException {
-    HttpFileServer server = new HttpFileServer(NetUtils.createSocketAddr("127.0.0.1:0"));
-    try {
-      server.start();
-      InetSocketAddress addr = server.getBindAddress();
-
-      Schema schema = AvroUtil.getAvroSchemaFromHttp("http://127.0.0.1:" + addr.getPort() + schemaUrl.getPath());
-      assertEquals(expected, schema);
-    } finally {
-      server.stop();
-    }
-  }
-
-  @Test
-  public void testGetSchemaFromFileSystem() throws IOException, URISyntaxException {
-    Schema schema = AvroUtil.getAvroSchemaFromFileSystem(schemaUrl.toString(), new TajoConf());
-
-    assertEquals(expected, schema);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java b/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java
deleted file mode 100644
index 39d28b3..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java
+++ /dev/null
@@ -1,95 +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.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.util.KeyValueSet;
-import org.junit.Test;
-
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class TestColumnMapping {
-  @Test
-  public void testColumnKeyValueMapping() throws Exception {
-    KeyValueSet keyValueSet = new KeyValueSet();
-    keyValueSet.set(HBaseStorageConstants.META_TABLE_KEY, "test");
-    keyValueSet.set(HBaseStorageConstants.META_COLUMNS_KEY, ":key,col2:key:,col2:value:#b,col3:");
-
-    Schema schema = new Schema();
-    schema.addColumn("c1", Type.TEXT);
-    schema.addColumn("c2", Type.TEXT);
-    schema.addColumn("c3", Type.TEXT);
-    schema.addColumn("c4", Type.TEXT);
-
-    TableMeta tableMeta = new TableMeta(StoreType.HBASE, keyValueSet);
-
-    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
-
-    List<String> cfNames = columnMapping.getColumnFamilyNames();
-    assertEquals(2, cfNames.size());
-    assertEquals("col2", cfNames.get(0));
-    assertEquals("col3", cfNames.get(1));
-
-    for (int i = 0; i < columnMapping.getIsBinaryColumns().length; i++) {
-      if (i == 2) {
-        assertTrue(columnMapping.getIsBinaryColumns()[i]);
-      } else {
-        assertFalse(columnMapping.getIsBinaryColumns()[i]);
-      }
-    }
-
-    for (int i = 0; i < columnMapping.getIsRowKeyMappings().length; i++) {
-      if (i == 0) {
-        assertTrue(columnMapping.getIsRowKeyMappings()[i]);
-      } else {
-        assertFalse(columnMapping.getIsRowKeyMappings()[i]);
-      }
-    }
-
-    String[] expectedColumnNames = { null, null, null, null};
-    for (int i = 0; i < schema.size(); i++) {
-      String columnName = columnMapping.getMappingColumns()[i][1] == null ? null :
-          new String(columnMapping.getMappingColumns()[i][1]);
-      assertEquals(expectedColumnNames[i], columnName);
-    }
-
-    for (int i = 0; i < schema.size(); i++) {
-      if (i == 1) {
-        assertTrue(columnMapping.getIsColumnKeys()[i]);
-      } else {
-        assertFalse(columnMapping.getIsColumnKeys()[i]);
-      }
-    }
-
-    for (int i = 0; i < schema.size(); i++) {
-      if (i == 2) {
-        assertTrue(columnMapping.getIsColumnValues()[i]);
-      } else {
-        assertFalse(columnMapping.getIsColumnValues()[i]);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java b/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
deleted file mode 100644
index 1fc4065..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
+++ /dev/null
@@ -1,109 +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.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.plan.expr.*;
-import org.apache.tajo.plan.logical.ScanNode;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.util.Pair;
-import org.junit.Test;
-
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-public class TestHBaseStorageManager {
-  @Test
-  public void testGetIndexPredications() throws Exception {
-    Column rowkeyColumn = new Column("rk", Type.TEXT);
-    // where rk >= '020' and rk <= '055'
-    ScanNode scanNode = new ScanNode(1);
-    EvalNode evalNode1 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("020")));
-    EvalNode evalNode2 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("055")));
-    EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2);
-    scanNode.setQual(evalNodeA);
-
-    HBaseStorageManager storageManager =
-        (HBaseStorageManager) StorageManager.getStorageManager(new TajoConf(), StoreType.HBASE);
-    List<Set<EvalNode>> indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
-    assertNotNull(indexEvals);
-    assertEquals(1, indexEvals.size());
-    Pair<Datum, Datum> indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
-    assertEquals("020", indexPredicateValue.getFirst().asChars());
-    assertEquals("055", indexPredicateValue.getSecond().asChars());
-
-    // where (rk >= '020' and rk <= '055') or rk = '075'
-    EvalNode evalNode3 = new BinaryEval(EvalType.EQUAL, new FieldEval(rowkeyColumn),new ConstEval(new TextDatum("075")));
-    EvalNode evalNodeB = new BinaryEval(EvalType.OR, evalNodeA, evalNode3);
-    scanNode.setQual(evalNodeB);
-    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
-    assertEquals(2, indexEvals.size());
-    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
-    assertEquals("020", indexPredicateValue.getFirst().asChars());
-    assertEquals("055", indexPredicateValue.getSecond().asChars());
-
-    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
-    assertEquals("075", indexPredicateValue.getFirst().asChars());
-    assertEquals("075", indexPredicateValue.getSecond().asChars());
-
-    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078')
-    EvalNode evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("072")));
-    EvalNode evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("078")));
-    EvalNode evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
-    EvalNode evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC);
-    scanNode.setQual(evalNodeD);
-    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
-    assertEquals(2, indexEvals.size());
-
-    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
-    assertEquals("020", indexPredicateValue.getFirst().asChars());
-    assertEquals("055", indexPredicateValue.getSecond().asChars());
-
-    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
-    assertEquals("072", indexPredicateValue.getFirst().asChars());
-    assertEquals("078", indexPredicateValue.getSecond().asChars());
-
-    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078' and rk >= '073')
-    evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("072")));
-    evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("078")));
-    evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
-    EvalNode evalNode6 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("073")));
-    evalNodeD = new BinaryEval(EvalType.AND, evalNodeC, evalNode6);
-    EvalNode evalNodeE = new BinaryEval(EvalType.OR, evalNodeA, evalNodeD);
-    scanNode.setQual(evalNodeE);
-    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
-    assertEquals(2, indexEvals.size());
-
-    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
-    assertEquals("020", indexPredicateValue.getFirst().asChars());
-    assertEquals("055", indexPredicateValue.getSecond().asChars());
-
-    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
-    assertEquals("073", indexPredicateValue.getFirst().asChars());
-    assertEquals("078", indexPredicateValue.getSecond().asChars());
-  }
-}


[22/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
deleted file mode 100644
index 352776f..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
+++ /dev/null
@@ -1,117 +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.rcfile;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
-
-import java.util.ArrayList;
-
-/**
- * ColumnProjectionUtils.
- *
- */
-public final class ColumnProjectionUtils {
-
-  public static final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
-
-  /**
-   * Sets read columns' ids(start from zero) for RCFile's Reader. Once a column
-   * is included in the list, RCFile's reader will not skip its value.
-   *
-   */
-  public static void setReadColumnIDs(Configuration conf, ArrayList<Integer> ids) {
-    String id = toReadColumnIDString(ids);
-    setReadColumnIDConf(conf, id);
-  }
-
-  /**
-   * Sets read columns' ids(start from zero) for RCFile's Reader. Once a column
-   * is included in the list, RCFile's reader will not skip its value.
-   *
-   */
-  public static void appendReadColumnIDs(Configuration conf,
-                                         ArrayList<Integer> ids) {
-    String id = toReadColumnIDString(ids);
-    if (id != null) {
-      String old = conf.get(READ_COLUMN_IDS_CONF_STR, null);
-      String newConfStr = id;
-      if (old != null) {
-        newConfStr = newConfStr + StringUtils.COMMA_STR + old;
-      }
-
-      setReadColumnIDConf(conf, newConfStr);
-    }
-  }
-
-  private static void setReadColumnIDConf(Configuration conf, String id) {
-    if (id == null || id.length() <= 0) {
-      conf.set(READ_COLUMN_IDS_CONF_STR, "");
-      return;
-    }
-
-    conf.set(READ_COLUMN_IDS_CONF_STR, id);
-  }
-
-  private static String toReadColumnIDString(ArrayList<Integer> ids) {
-    String id = null;
-    if (ids != null) {
-      for (int i = 0; i < ids.size(); i++) {
-        if (i == 0) {
-          id = "" + ids.get(i);
-        } else {
-          id = id + StringUtils.COMMA_STR + ids.get(i);
-        }
-      }
-    }
-    return id;
-  }
-
-  /**
-   * Returns an array of column ids(start from zero) which is set in the given
-   * parameter <tt>conf</tt>.
-   */
-  public static ArrayList<Integer> getReadColumnIDs(Configuration conf) {
-    if (conf == null) {
-      return new ArrayList<Integer>(0);
-    }
-    String skips = conf.get(READ_COLUMN_IDS_CONF_STR, "");
-    String[] list = StringUtils.split(skips);
-    ArrayList<Integer> result = new ArrayList<Integer>(list.length);
-    for (String element : list) {
-      // it may contain duplicates, remove duplicates
-      Integer toAdd = Integer.parseInt(element);
-      if (!result.contains(toAdd)) {
-        result.add(toAdd);
-      }
-    }
-    return result;
-  }
-
-  /**
-   * Clears the read column ids set in the conf, and will read all columns.
-   */
-  public static void setFullyReadColumns(Configuration conf) {
-    conf.set(READ_COLUMN_IDS_CONF_STR, "");
-  }
-
-  private ColumnProjectionUtils() {
-    // prevent instantiation
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
deleted file mode 100644
index 707d55a..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
+++ /dev/null
@@ -1,32 +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.rcfile;
-
-import java.io.IOException;
-
-/**
- * Used to call back lazy decompression process.
- *
- * @see BytesRefWritable
- */
-public interface LazyDecompressionCallback {
-
-  byte[] decompress() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
deleted file mode 100644
index bb6af22..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
+++ /dev/null
@@ -1,113 +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.rcfile;
-
-import java.io.ByteArrayInputStream;
-
-/**
- * A thread-not-safe version of ByteArrayInputStream, which removes all
- * synchronized modifiers.
- */
-public class NonSyncByteArrayInputStream extends ByteArrayInputStream {
-  public NonSyncByteArrayInputStream() {
-    super(new byte[] {});
-  }
-
-  public NonSyncByteArrayInputStream(byte[] bs) {
-    super(bs);
-  }
-
-  public NonSyncByteArrayInputStream(byte[] buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public void reset(byte[] input, int start, int length) {
-    buf = input;
-    count = start + length;
-    mark = start;
-    pos = start;
-  }
-
-  public int getPosition() {
-    return pos;
-  }
-
-  public int getLength() {
-    return count;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public int read() {
-    return (pos < count) ? (buf[pos++] & 0xff) : -1;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public int read(byte b[], int off, int len) {
-    if (b == null) {
-      throw new NullPointerException();
-    } else if (off < 0 || len < 0 || len > b.length - off) {
-      throw new IndexOutOfBoundsException();
-    }
-    if (pos >= count) {
-      return -1;
-    }
-    if (pos + len > count) {
-      len = count - pos;
-    }
-    if (len <= 0) {
-      return 0;
-    }
-    System.arraycopy(buf, pos, b, off, len);
-    pos += len;
-    return len;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public long skip(long n) {
-    if (pos + n > count) {
-      n = count - pos;
-    }
-    if (n < 0) {
-      return 0;
-    }
-    pos += n;
-    return n;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public int available() {
-    return count - pos;
-  }
-
-  public void seek(int pos) {
-    this.pos = pos;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
deleted file mode 100644
index 53a3dca..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
+++ /dev/null
@@ -1,144 +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.rcfile;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * A thread-not-safe version of ByteArrayOutputStream, which removes all
- * synchronized modifiers.
- */
-public class NonSyncByteArrayOutputStream extends ByteArrayOutputStream {
-  public NonSyncByteArrayOutputStream(int size) {
-    super(size);
-  }
-
-  public NonSyncByteArrayOutputStream() {
-    super(64 * 1024);
-  }
-
-  public byte[] getData() {
-    return buf;
-  }
-
-  public int getLength() {
-    return count;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void reset() {
-    count = 0;
-  }
-
-  public void write(DataInput in, int length) throws IOException {
-    enLargeBuffer(length);
-    in.readFully(buf, count, length);
-    count += length;
-  }
-
-  private byte[] vLongBytes = new byte[9];
-
-  public int writeVLongToByteArray(byte[] bytes, int offset, long l) {
-    if (l >= -112 && l <= 127) {
-      bytes[offset] = (byte) l;
-      return 1;
-    }
-
-    int len = -112;
-    if (l < 0) {
-      l ^= -1L; // take one's complement'
-      len = -120;
-    }
-
-    long tmp = l;
-    while (tmp != 0) {
-      tmp = tmp >> 8;
-      len--;
-    }
-
-    bytes[offset++] = (byte) len;
-    len = (len < -120) ? -(len + 120) : -(len + 112);
-
-    for (int idx = len; idx != 0; idx--) {
-      int shiftbits = (idx - 1) * 8;
-      bytes[offset++] = (byte) ((l & (0xFFL << shiftbits)) >> shiftbits);
-    }
-    return 1 + len;
-  }
-
-  public int writeVLong(long l) {
-    int len = writeVLongToByteArray(vLongBytes, 0, l);
-    write(vLongBytes, 0, len);
-    return len;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void write(int b) {
-    enLargeBuffer(1);
-    buf[count] = (byte) b;
-    count += 1;
-  }
-
-  private int enLargeBuffer(int increment) {
-    int temp = count + increment;
-    int newLen = temp;
-    if (temp > buf.length) {
-      if ((buf.length << 1) > temp) {
-        newLen = buf.length << 1;
-      }
-      byte newbuf[] = new byte[newLen];
-      System.arraycopy(buf, 0, newbuf, 0, count);
-      buf = newbuf;
-    }
-    return newLen;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void write(byte b[], int off, int len) {
-    if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length)
-        || ((off + len) < 0)) {
-      throw new IndexOutOfBoundsException();
-    } else if (len == 0) {
-      return;
-    }
-    enLargeBuffer(len);
-    System.arraycopy(b, off, buf, count, len);
-    count += len;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void writeTo(OutputStream out) throws IOException {
-    out.write(buf, 0, count);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
deleted file mode 100644
index 46745ab..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
+++ /dev/null
@@ -1,507 +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.rcfile;
-
-import org.apache.hadoop.fs.Seekable;
-
-import java.io.*;
-
-/**
- * A thread-not-safe version of Hadoop's DataInputBuffer, which removes all
- * synchronized modifiers.
- */
-public class NonSyncDataInputBuffer extends FilterInputStream implements
-    DataInput, Seekable {
-
-  private final NonSyncByteArrayInputStream buffer;
-
-  byte[] buff = new byte[16];
-
-  /** Constructs a new empty buffer. */
-  public NonSyncDataInputBuffer() {
-    this(new NonSyncByteArrayInputStream());
-  }
-
-  private NonSyncDataInputBuffer(NonSyncByteArrayInputStream buffer) {
-    super(buffer);
-    this.buffer = buffer;
-  }
-
-  /** Resets the data that the buffer reads. */
-  public void reset(byte[] input, int length) {
-    buffer.reset(input, 0, length);
-  }
-
-  /** Resets the data that the buffer reads. */
-  public void reset(byte[] input, int start, int length) {
-    buffer.reset(input, start, length);
-  }
-
-  /** Returns the current position in the input. */
-  public int getPosition() {
-    return buffer.getPosition();
-  }
-
-  /** Returns the length of the input. */
-  public int getLength() {
-    return buffer.getLength();
-  }
-
-  /**
-   * Reads bytes from the source stream into the byte array <code>buffer</code>.
-   * The number of bytes actually read is returned.
-   *
-   * @param buffer
-   *          the buffer to read bytes into
-   * @return the number of bytes actually read or -1 if end of stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  @Override
-  public final int read(byte[] buffer) throws IOException {
-    return in.read(buffer, 0, buffer.length);
-  }
-
-  /**
-   * Read at most <code>length</code> bytes from this DataInputStream and stores
-   * them in byte array <code>buffer</code> starting at <code>offset</code>.
-   * Answer the number of bytes actually read or -1 if no bytes were read and
-   * end of stream was encountered.
-   *
-   * @param buffer
-   *          the byte array in which to store the read bytes.
-   * @param offset
-   *          the offset in <code>buffer</code> to store the read bytes.
-   * @param length
-   *          the maximum number of bytes to store in <code>buffer</code>.
-   * @return the number of bytes actually read or -1 if end of stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  @Deprecated
-  @Override
-  public final int read(byte[] buffer, int offset, int length)
-      throws IOException {
-    return in.read(buffer, offset, length);
-  }
-
-  /**
-   * Reads a boolean from this stream.
-   *
-   * @return the next boolean value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final boolean readBoolean() throws IOException {
-    int temp = in.read();
-    if (temp < 0) {
-      throw new EOFException();
-    }
-    return temp != 0;
-  }
-
-  /**
-   * Reads an 8-bit byte value from this stream.
-   *
-   * @return the next byte value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final byte readByte() throws IOException {
-    int temp = in.read();
-    if (temp < 0) {
-      throw new EOFException();
-    }
-    return (byte) temp;
-  }
-
-  /**
-   * Reads a 16-bit character value from this stream.
-   *
-   * @return the next <code>char</code> value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  private int readToBuff(int count) throws IOException {
-    int offset = 0;
-
-    while (offset < count) {
-      int bytesRead = in.read(buff, offset, count - offset);
-      if (bytesRead == -1) {
-        return bytesRead;
-      }
-      offset += bytesRead;
-    }
-    return offset;
-  }
-
-  public final char readChar() throws IOException {
-    if (readToBuff(2) < 0) {
-      throw new EOFException();
-    }
-    return (char) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
-
-  }
-
-  /**
-   * Reads a 64-bit <code>double</code> value from this stream.
-   *
-   * @return the next <code>double</code> value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final double readDouble() throws IOException {
-    return Double.longBitsToDouble(readLong());
-  }
-
-  /**
-   * Reads a 32-bit <code>float</code> value from this stream.
-   *
-   * @return the next <code>float</code> value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final float readFloat() throws IOException {
-    return Float.intBitsToFloat(readInt());
-  }
-
-  /**
-   * Reads bytes from this stream into the byte array <code>buffer</code>. This
-   * method will block until <code>buffer.length</code> number of bytes have
-   * been read.
-   *
-   * @param buffer
-   *          to read bytes into
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final void readFully(byte[] buffer) throws IOException {
-    readFully(buffer, 0, buffer.length);
-  }
-
-  /**
-   * Reads bytes from this stream and stores them in the byte array
-   * <code>buffer</code> starting at the position <code>offset</code>. This
-   * method blocks until <code>count</code> bytes have been read.
-   *
-   * @param buffer
-   *          the byte array into which the data is read
-   * @param offset
-   *          the offset the operation start at
-   * @param length
-   *          the maximum number of bytes to read
-   *
-   * @throws java.io.IOException
-   *           if a problem occurs while reading from this stream
-   * @throws java.io.EOFException
-   *           if reaches the end of the stream before enough bytes have been
-   *           read
-   */
-  public final void readFully(byte[] buffer, int offset, int length)
-      throws IOException {
-    if (length < 0) {
-      throw new IndexOutOfBoundsException();
-    }
-    if (length == 0) {
-      return;
-    }
-    if (in == null || buffer == null) {
-      throw new NullPointerException("Null Pointer to underlying input stream");
-    }
-
-    if (offset < 0 || offset > buffer.length - length) {
-      throw new IndexOutOfBoundsException();
-    }
-    while (length > 0) {
-      int result = in.read(buffer, offset, length);
-      if (result < 0) {
-        throw new EOFException();
-      }
-      offset += result;
-      length -= result;
-    }
-  }
-
-  /**
-   * Reads a 32-bit integer value from this stream.
-   *
-   * @return the next <code>int</code> value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final int readInt() throws IOException {
-    if (readToBuff(4) < 0) {
-      throw new EOFException();
-    }
-    return ((buff[0] & 0xff) << 24) | ((buff[1] & 0xff) << 16)
-        | ((buff[2] & 0xff) << 8) | (buff[3] & 0xff);
-  }
-
-  /**
-   * Answers a <code>String</code> representing the next line of text available
-   * in this BufferedReader. A line is represented by 0 or more characters
-   * followed by <code>'\n'</code>, <code>'\r'</code>, <code>"\n\r"</code> or
-   * end of stream. The <code>String</code> does not include the newline
-   * sequence.
-   *
-   * @return the contents of the line or null if no characters were read before
-   *         end of stream.
-   *
-   * @throws java.io.IOException
-   *           If the DataInputStream is already closed or some other IO error
-   *           occurs.
-   *
-   * @deprecated Use BufferedReader
-   */
-  @Deprecated
-  public final String readLine() throws IOException {
-    StringBuilder line = new StringBuilder(80); // Typical line length
-    boolean foundTerminator = false;
-    while (true) {
-      int nextByte = in.read();
-      switch (nextByte) {
-        case -1:
-          if (line.length() == 0 && !foundTerminator) {
-            return null;
-          }
-          return line.toString();
-        case (byte) '\r':
-          if (foundTerminator) {
-            ((PushbackInputStream) in).unread(nextByte);
-            return line.toString();
-          }
-          foundTerminator = true;
-        /* Have to be able to peek ahead one byte */
-          if (!(in.getClass() == PushbackInputStream.class)) {
-            in = new PushbackInputStream(in);
-          }
-          break;
-        case (byte) '\n':
-          return line.toString();
-        default:
-          if (foundTerminator) {
-            ((PushbackInputStream) in).unread(nextByte);
-            return line.toString();
-          }
-          line.append((char) nextByte);
-      }
-    }
-  }
-
-  /**
-   * Reads a 64-bit <code>long</code> value from this stream.
-   *
-   * @return the next <code>long</code> value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final long readLong() throws IOException {
-    if (readToBuff(8) < 0) {
-      throw new EOFException();
-    }
-    int i1 = ((buff[0] & 0xff) << 24) | ((buff[1] & 0xff) << 16)
-        | ((buff[2] & 0xff) << 8) | (buff[3] & 0xff);
-    int i2 = ((buff[4] & 0xff) << 24) | ((buff[5] & 0xff) << 16)
-        | ((buff[6] & 0xff) << 8) | (buff[7] & 0xff);
-
-    return ((i1 & 0xffffffffL) << 32) | (i2 & 0xffffffffL);
-  }
-
-  /**
-   * Reads a 16-bit <code>short</code> value from this stream.
-   *
-   * @return the next <code>short</code> value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final short readShort() throws IOException {
-    if (readToBuff(2) < 0) {
-      throw new EOFException();
-    }
-    return (short) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
-  }
-
-  /**
-   * Reads an unsigned 8-bit <code>byte</code> value from this stream and
-   * returns it as an int.
-   *
-   * @return the next unsigned byte value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final int readUnsignedByte() throws IOException {
-    int temp = in.read();
-    if (temp < 0) {
-      throw new EOFException();
-    }
-    return temp;
-  }
-
-  /**
-   * Reads a 16-bit unsigned <code>short</code> value from this stream and
-   * returns it as an int.
-   *
-   * @return the next unsigned <code>short</code> value from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final int readUnsignedShort() throws IOException {
-    if (readToBuff(2) < 0) {
-      throw new EOFException();
-    }
-    return (char) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
-  }
-
-  /**
-   * Reads a UTF format String from this Stream.
-   *
-   * @return the next UTF String from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public final String readUTF() throws IOException {
-    return decodeUTF(readUnsignedShort());
-  }
-
-  String decodeUTF(int utfSize) throws IOException {
-    return decodeUTF(utfSize, this);
-  }
-
-  private static String decodeUTF(int utfSize, DataInput in) throws IOException {
-    byte[] buf = new byte[utfSize];
-    char[] out = new char[utfSize];
-    in.readFully(buf, 0, utfSize);
-
-    return convertUTF8WithBuf(buf, out, 0, utfSize);
-  }
-
-  /**
-   * Reads a UTF format String from the DataInput Stream <code>in</code>.
-   *
-   * @param in
-   *          the input stream to read from
-   * @return the next UTF String from the source stream.
-   *
-   * @throws java.io.IOException
-   *           If a problem occurs reading from this DataInputStream.
-   *
-   */
-  public static final String readUTF(DataInput in) throws IOException {
-    return decodeUTF(in.readUnsignedShort(), in);
-  }
-
-  /**
-   * Skips <code>count</code> number of bytes in this stream. Subsequent
-   * <code>read()</code>'s will not return these bytes unless
-   * <code>reset()</code> is used.
-   *
-   * @param count
-   *          the number of bytes to skip.
-   * @return the number of bytes actually skipped.
-   *
-   * @throws java.io.IOException
-   *           If the stream is already closed or another IOException occurs.
-   */
-  public final int skipBytes(int count) throws IOException {
-    int skipped = 0;
-    long skip;
-    while (skipped < count && (skip = in.skip(count - skipped)) != 0) {
-      skipped += skip;
-    }
-    if (skipped < 0) {
-      throw new EOFException();
-    }
-    return skipped;
-  }
-
-  public static String convertUTF8WithBuf(byte[] buf, char[] out, int offset,
-                                          int utfSize) throws UTFDataFormatException {
-    int count = 0, s = 0, a;
-    while (count < utfSize) {
-      if ((out[s] = (char) buf[offset + count++]) < '\u0080') {
-        s++;
-      } else if (((a = out[s]) & 0xe0) == 0xc0) {
-        if (count >= utfSize) {
-          throw new UTFDataFormatException();
-        }
-        int b = buf[count++];
-        if ((b & 0xC0) != 0x80) {
-          throw new UTFDataFormatException();
-        }
-        out[s++] = (char) (((a & 0x1F) << 6) | (b & 0x3F));
-      } else if ((a & 0xf0) == 0xe0) {
-        if (count + 1 >= utfSize) {
-          throw new UTFDataFormatException();
-        }
-        int b = buf[count++];
-        int c = buf[count++];
-        if (((b & 0xC0) != 0x80) || ((c & 0xC0) != 0x80)) {
-          throw new UTFDataFormatException();
-        }
-        out[s++] = (char) (((a & 0x0F) << 12) | ((b & 0x3F) << 6) | (c & 0x3F));
-      } else {
-        throw new UTFDataFormatException();
-      }
-    }
-    return new String(out, 0, s);
-  }
-
-  @Override
-  public void seek(long pos) throws IOException {
-    buffer.seek((int)pos);
-  }
-
-  @Override
-  public long getPos() throws IOException {
-    return buffer.getPosition();
-  }
-
-  @Override
-  public boolean seekToNewSource(long targetPos) throws IOException {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
deleted file mode 100644
index 3944f38..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.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.rcfile;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-/**
- * A thread-not-safe version of Hadoop's DataOutputBuffer, which removes all
- * synchronized modifiers.
- */
-public class NonSyncDataOutputBuffer extends DataOutputStream {
-
-  private final NonSyncByteArrayOutputStream buffer;
-
-  /** Constructs a new empty buffer. */
-  public NonSyncDataOutputBuffer() {
-    this(new NonSyncByteArrayOutputStream());
-  }
-
-  private NonSyncDataOutputBuffer(NonSyncByteArrayOutputStream buffer) {
-    super(buffer);
-    this.buffer = buffer;
-  }
-
-  /**
-   * Returns the current contents of the buffer. Data is only valid to
-   * {@link #getLength()}.
-   */
-  public byte[] getData() {
-    return buffer.getData();
-  }
-
-  /** Returns the length of the valid data currently in the buffer. */
-  public int getLength() {
-    return buffer.getLength();
-  }
-
-  /** Resets the buffer to empty. */
-  public NonSyncDataOutputBuffer reset() {
-    written = 0;
-    buffer.reset();
-    return this;
-  }
-
-  /** Writes bytes from a DataInput directly into the buffer. */
-  public void write(DataInput in, int length) throws IOException {
-    buffer.write(in, length);
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    buffer.write(b);
-    incCount(1);
-  }
-
-  @Override
-  public void write(byte b[], int off, int len) throws IOException {
-    buffer.write(b, off, len);
-    incCount(len);
-  }
-
-  public void writeTo(DataOutputStream out) throws IOException {
-    buffer.writeTo(out);
-  }
-
-  private void incCount(int value) {
-    if (written + value < 0) {
-      written = Integer.MAX_VALUE;
-    } else {
-      written += value;
-    }
-  }
-}


[19/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
deleted file mode 100644
index 0fb2c3a..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetReader.java
+++ /dev/null
@@ -1,151 +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.thirdparty.parquet;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-import parquet.filter.UnboundRecordFilter;
-import parquet.hadoop.*;
-import parquet.hadoop.api.InitContext;
-import parquet.hadoop.api.ReadSupport;
-import parquet.hadoop.api.ReadSupport.ReadContext;
-import parquet.hadoop.metadata.BlockMetaData;
-import parquet.hadoop.metadata.GlobalMetaData;
-import parquet.schema.MessageType;
-
-/**
- * Read records from a Parquet file.
- */
-public class ParquetReader<T> implements Closeable {
-
-  private ReadSupport<T> readSupport;
-  private UnboundRecordFilter filter;
-  private Configuration conf;
-  private ReadContext readContext;
-  private Iterator<Footer> footersIterator;
-  private InternalParquetRecordReader<T> reader;
-  private GlobalMetaData globalMetaData;
-
-  /**
-   * @param file the file to read
-   * @param readSupport to materialize records
-   * @throws IOException
-   */
-  public ParquetReader(Path file, ReadSupport<T> readSupport) throws IOException {
-    this(file, readSupport, null);
-  }
-
-  /**
-   * @param conf the configuration
-   * @param file the file to read
-   * @param readSupport to materialize records
-   * @throws IOException
-   */
-  public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport) throws IOException {
-    this(conf, file, readSupport, null);
-  }
-
-  /**
-   * @param file the file to read
-   * @param readSupport to materialize records
-   * @param filter the filter to use to filter records
-   * @throws IOException
-   */
-  public ParquetReader(Path file, ReadSupport<T> readSupport, UnboundRecordFilter filter) throws IOException {
-    this(new Configuration(), file, readSupport, filter);
-  }
-
-  /**
-   * @param conf the configuration
-   * @param file the file to read
-   * @param readSupport to materialize records
-   * @param filter the filter to use to filter records
-   * @throws IOException
-   */
-  public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport, UnboundRecordFilter filter) throws IOException {
-    this.readSupport = readSupport;
-    this.filter = filter;
-    this.conf = conf;
-
-    FileSystem fs = file.getFileSystem(conf);
-    List<FileStatus> statuses = Arrays.asList(fs.listStatus(file));
-    List<Footer> footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(conf, statuses);
-    this.footersIterator = footers.iterator();
-    globalMetaData = ParquetFileWriter.getGlobalMetaData(footers);
-
-    List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
-    for (Footer footer : footers) {
-      blocks.addAll(footer.getParquetMetadata().getBlocks());
-    }
-
-    MessageType schema = globalMetaData.getSchema();
-    Map<String, Set<String>> extraMetadata = globalMetaData.getKeyValueMetaData();
-    readContext = readSupport.init(new InitContext(conf, extraMetadata, schema));
-  }
-
-  /**
-   * @return the next record or null if finished
-   * @throws IOException
-   */
-  public T read() throws IOException {
-    try {
-      if (reader != null && reader.nextKeyValue()) {
-        return reader.getCurrentValue();
-      } else {
-        initReader();
-        return reader == null ? null : read();
-      }
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-  private void initReader() throws IOException {
-    if (reader != null) {
-      reader.close();
-      reader = null;
-    }
-    if (footersIterator.hasNext()) {
-      Footer footer = footersIterator.next();
-      reader = new InternalParquetRecordReader<T>(readSupport, filter);
-      reader.initialize(
-          readContext.getRequestedSchema(), globalMetaData.getSchema(), footer.getParquetMetadata().getFileMetaData().getKeyValueMetaData(),
-          readContext.getReadSupportMetadata(), footer.getFile(), footer.getParquetMetadata().getBlocks(), conf);
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (reader != null) {
-      reader.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
deleted file mode 100644
index 0447a47..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetWriter.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.thirdparty.parquet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import parquet.column.ParquetProperties;
-import parquet.hadoop.api.WriteSupport;
-import parquet.hadoop.metadata.CompressionCodecName;
-import parquet.schema.MessageType;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public class ParquetWriter<T> implements Closeable {
-
-  public static final int DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024;
-  public static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024;
-  public static final CompressionCodecName DEFAULT_COMPRESSION_CODEC_NAME =
-      CompressionCodecName.UNCOMPRESSED;
-  public static final boolean DEFAULT_IS_DICTIONARY_ENABLED = true;
-  public static final boolean DEFAULT_IS_VALIDATING_ENABLED = false;
-  public static final ParquetProperties.WriterVersion DEFAULT_WRITER_VERSION =
-      ParquetProperties.WriterVersion.PARQUET_1_0;
-
-  private final InternalParquetRecordWriter<T> writer;
-
-  /**
-   * Create a new ParquetWriter.
-   * (with dictionary encoding enabled and validation off)
-   *
-   * @param file the file to create
-   * @param writeSupport the implementation to write a record to a RecordConsumer
-   * @param compressionCodecName the compression codec to use
-   * @param blockSize the block size threshold
-   * @param pageSize the page size threshold
-   * @throws java.io.IOException
-   * @see #ParquetWriter(org.apache.hadoop.fs.Path, parquet.hadoop.api.WriteSupport, CompressionCodecName, int, int, boolean, boolean)
-   */
-  public ParquetWriter(Path file, WriteSupport<T> writeSupport, CompressionCodecName compressionCodecName, int blockSize, int pageSize) throws IOException {
-    this(file, writeSupport, compressionCodecName, blockSize, pageSize,
-        DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED);
-  }
-
-  /**
-   * Create a new ParquetWriter.
-   *
-   * @param file the file to create
-   * @param writeSupport the implementation to write a record to a RecordConsumer
-   * @param compressionCodecName the compression codec to use
-   * @param blockSize the block size threshold
-   * @param pageSize the page size threshold (both data and dictionary)
-   * @param enableDictionary to turn dictionary encoding on
-   * @param validating to turn on validation using the schema
-   * @throws IOException
-   * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean)
-   */
-  public ParquetWriter(
-      Path file,
-      WriteSupport<T> writeSupport,
-      CompressionCodecName compressionCodecName,
-      int blockSize,
-      int pageSize,
-      boolean enableDictionary,
-      boolean validating) throws IOException {
-    this(file, writeSupport, compressionCodecName, blockSize, pageSize, pageSize, enableDictionary, validating);
-  }
-
-  /**
-   * Create a new ParquetWriter.
-   *
-   * @param file the file to create
-   * @param writeSupport the implementation to write a record to a RecordConsumer
-   * @param compressionCodecName the compression codec to use
-   * @param blockSize the block size threshold
-   * @param pageSize the page size threshold
-   * @param dictionaryPageSize the page size threshold for the dictionary pages
-   * @param enableDictionary to turn dictionary encoding on
-   * @param validating to turn on validation using the schema
-   * @throws IOException
-   * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion)
-   */
-  public ParquetWriter(
-      Path file,
-      WriteSupport<T> writeSupport,
-      CompressionCodecName compressionCodecName,
-      int blockSize,
-      int pageSize,
-      int dictionaryPageSize,
-      boolean enableDictionary,
-      boolean validating) throws IOException {
-    this(file, writeSupport, compressionCodecName, blockSize, pageSize,
-        dictionaryPageSize, enableDictionary, validating,
-        DEFAULT_WRITER_VERSION);
-  }
-
-  /**
-   * Create a new ParquetWriter.
-   *
-   * Directly instantiates a Hadoop {@link org.apache.hadoop.conf.Configuration} which reads
-   * configuration from the classpath.
-   *
-   * @param file the file to create
-   * @param writeSupport the implementation to write a record to a RecordConsumer
-   * @param compressionCodecName the compression codec to use
-   * @param blockSize the block size threshold
-   * @param pageSize the page size threshold
-   * @param dictionaryPageSize the page size threshold for the dictionary pages
-   * @param enableDictionary to turn dictionary encoding on
-   * @param validating to turn on validation using the schema
-   * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion}
-   * @throws IOException
-   * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean, parquet.column.ParquetProperties.WriterVersion, org.apache.hadoop.conf.Configuration)
-   */
-  public ParquetWriter(
-      Path file,
-      WriteSupport<T> writeSupport,
-      CompressionCodecName compressionCodecName,
-      int blockSize,
-      int pageSize,
-      int dictionaryPageSize,
-      boolean enableDictionary,
-      boolean validating,
-      ParquetProperties.WriterVersion writerVersion) throws IOException {
-    this(file, writeSupport, compressionCodecName, blockSize, pageSize, dictionaryPageSize, enableDictionary, validating, writerVersion, new Configuration());
-  }
-
-  /**
-   * Create a new ParquetWriter.
-   *
-   * @param file the file to create
-   * @param writeSupport the implementation to write a record to a RecordConsumer
-   * @param compressionCodecName the compression codec to use
-   * @param blockSize the block size threshold
-   * @param pageSize the page size threshold
-   * @param dictionaryPageSize the page size threshold for the dictionary pages
-   * @param enableDictionary to turn dictionary encoding on
-   * @param validating to turn on validation using the schema
-   * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion}
-   * @param conf Hadoop configuration to use while accessing the filesystem
-   * @throws IOException
-   */
-  public ParquetWriter(
-      Path file,
-      WriteSupport<T> writeSupport,
-      CompressionCodecName compressionCodecName,
-      int blockSize,
-      int pageSize,
-      int dictionaryPageSize,
-      boolean enableDictionary,
-      boolean validating,
-      ParquetProperties.WriterVersion writerVersion,
-      Configuration conf) throws IOException {
-
-    WriteSupport.WriteContext writeContext = writeSupport.init(conf);
-    MessageType schema = writeContext.getSchema();
-
-    ParquetFileWriter fileWriter = new ParquetFileWriter(conf, schema, file);
-    fileWriter.start();
-
-    CodecFactory codecFactory = new CodecFactory(conf);
-    CodecFactory.BytesCompressor compressor =	codecFactory.getCompressor(compressionCodecName, 0);
-    this.writer = new InternalParquetRecordWriter<T>(
-        fileWriter,
-        writeSupport,
-        schema,
-        writeContext.getExtraMetaData(),
-        blockSize,
-        pageSize,
-        compressor,
-        dictionaryPageSize,
-        enableDictionary,
-        validating,
-        writerVersion);
-  }
-
-  /**
-   * Create a new ParquetWriter.  The default block size is 50 MB.The default
-   * page size is 1 MB.  Default compression is no compression. Dictionary encoding is disabled.
-   *
-   * @param file the file to create
-   * @param writeSupport the implementation to write a record to a RecordConsumer
-   * @throws IOException
-   */
-  public ParquetWriter(Path file, WriteSupport<T> writeSupport) throws IOException {
-    this(file, writeSupport, DEFAULT_COMPRESSION_CODEC_NAME, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
-  }
-
-  public void write(T object) throws IOException {
-    try {
-      writer.write(object);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-  public long getEstimatedWrittenSize() throws IOException {
-    return this.writer.getEstimatedWrittenSize();
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      writer.close();
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
deleted file mode 100644
index c1835df..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/BaseTupleBuilder.java
+++ /dev/null
@@ -1,112 +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.tuple;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SchemaUtil;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.tuple.offheap.HeapTuple;
-import org.apache.tajo.tuple.offheap.OffHeapRowWriter;
-import org.apache.tajo.tuple.offheap.ZeroCopyTuple;
-import org.apache.tajo.unit.StorageUnit;
-import org.apache.tajo.util.Deallocatable;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.UnsafeUtil;
-import sun.misc.Unsafe;
-import sun.nio.ch.DirectBuffer;
-
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-
-public class BaseTupleBuilder extends OffHeapRowWriter implements TupleBuilder, Deallocatable {
-  private static final Log LOG = LogFactory.getLog(BaseTupleBuilder.class);
-
-  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
-
-  // buffer
-  private ByteBuffer buffer;
-  private long address;
-
-  public BaseTupleBuilder(Schema schema) {
-    super(SchemaUtil.toDataTypes(schema));
-    buffer = ByteBuffer.allocateDirect(64 * StorageUnit.KB).order(ByteOrder.nativeOrder());
-    address = UnsafeUtil.getAddress(buffer);
-  }
-
-  @Override
-  public long address() {
-    return address;
-  }
-
-  public void ensureSize(int size) {
-    if (buffer.remaining() - size < 0) { // check the remain size
-      // enlarge new buffer and copy writing data
-      int newBlockSize = UnsafeUtil.alignedSize(buffer.capacity() * 2);
-      ByteBuffer newByteBuf = ByteBuffer.allocateDirect(newBlockSize);
-      long newAddress = ((DirectBuffer)newByteBuf).address();
-      UNSAFE.copyMemory(this.address, newAddress, buffer.limit());
-      LOG.debug("Increase the buffer size to " + FileUtil.humanReadableByteCount(newBlockSize, false));
-
-      // release existing buffer and replace variables
-      UnsafeUtil.free(buffer);
-      buffer = newByteBuf;
-      address = newAddress;
-    }
-  }
-
-  @Override
-  public int position() {
-    return 0;
-  }
-
-  @Override
-  public void forward(int length) {
-  }
-
-  @Override
-  public void endRow() {
-    super.endRow();
-    buffer.position(0).limit(offset());
-  }
-
-  @Override
-  public Tuple build() {
-    return buildToHeapTuple();
-  }
-
-  public HeapTuple buildToHeapTuple() {
-    byte [] bytes = new byte[buffer.limit()];
-    UNSAFE.copyMemory(null, address, bytes, UnsafeUtil.ARRAY_BOOLEAN_BASE_OFFSET, buffer.limit());
-    return new HeapTuple(bytes, dataTypes());
-  }
-
-  public ZeroCopyTuple buildToZeroCopyTuple() {
-    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
-    zcTuple.set(buffer, 0, buffer.limit(), dataTypes());
-    return zcTuple;
-  }
-
-  public void release() {
-    UnsafeUtil.free(buffer);
-    buffer = null;
-    address = 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/RowBlockReader.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
deleted file mode 100644
index be734e1..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/RowBlockReader.java
+++ /dev/null
@@ -1,33 +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.tuple;
-
-import org.apache.tajo.storage.Tuple;
-
-public interface RowBlockReader<T extends Tuple> {
-
-  /**
-   * Return for each tuple
-   *
-   * @return True if tuple block is filled with tuples. Otherwise, It will return false.
-   */
-  public boolean next(T tuple);
-
-  public void reset();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/TupleBuilder.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
deleted file mode 100644
index c43c018..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/TupleBuilder.java
+++ /dev/null
@@ -1,26 +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.tuple;
-
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.tuple.offheap.RowWriter;
-
-public interface TupleBuilder extends RowWriter {
-  public Tuple build();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
deleted file mode 100644
index 9662d5a..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/DirectBufTuple.java
+++ /dev/null
@@ -1,41 +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.tuple.offheap;
-
-import org.apache.tajo.util.Deallocatable;
-import org.apache.tajo.util.UnsafeUtil;
-
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-
-import static org.apache.tajo.common.TajoDataTypes.DataType;
-
-public class DirectBufTuple extends UnSafeTuple implements Deallocatable {
-  private ByteBuffer bb;
-
-  public DirectBufTuple(int length, DataType[] types) {
-    bb = ByteBuffer.allocateDirect(length).order(ByteOrder.nativeOrder());
-    set(bb, 0, length, types);
-  }
-
-  @Override
-  public void release() {
-    UnsafeUtil.free(bb);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
deleted file mode 100644
index a327123..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/FixedSizeLimitSpec.java
+++ /dev/null
@@ -1,32 +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.tuple.offheap;
-
-/**
- * Fixed size limit specification
- */
-public class FixedSizeLimitSpec extends ResizableLimitSpec {
-  public FixedSizeLimitSpec(long size) {
-    super(size, size);
-  }
-
-  public FixedSizeLimitSpec(long size, float allowedOverflowRatio) {
-    super(size, size, allowedOverflowRatio);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
deleted file mode 100644
index 33f9f1c..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/HeapTuple.java
+++ /dev/null
@@ -1,272 +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.tuple.offheap;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.Message;
-
-import org.apache.tajo.datum.*;
-import org.apache.tajo.exception.UnsupportedException;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.util.SizeOf;
-import org.apache.tajo.util.StringUtils;
-import org.apache.tajo.util.UnsafeUtil;
-
-import sun.misc.Unsafe;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-
-import static org.apache.tajo.common.TajoDataTypes.DataType;
-
-public class HeapTuple implements Tuple {
-  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
-  private static final long BASE_OFFSET = UnsafeUtil.ARRAY_BYTE_BASE_OFFSET;
-
-  private final byte [] data;
-  private final DataType [] types;
-
-  public HeapTuple(final byte [] bytes, final DataType [] types) {
-    this.data = bytes;
-    this.types = types;
-  }
-
-  @Override
-  public int size() {
-    return data.length;
-  }
-
-  public ByteBuffer nioBuffer() {
-    return ByteBuffer.wrap(data);
-  }
-
-  private int getFieldOffset(int fieldId) {
-    return UNSAFE.getInt(data, BASE_OFFSET + SizeOf.SIZE_OF_INT + (fieldId * SizeOf.SIZE_OF_INT));
-  }
-
-  private int checkNullAndGetOffset(int fieldId) {
-    int offset = getFieldOffset(fieldId);
-    if (offset == OffHeapRowBlock.NULL_FIELD_OFFSET) {
-      throw new RuntimeException("Invalid Field Access: " + fieldId);
-    }
-    return offset;
-  }
-
-  @Override
-  public boolean contains(int fieldid) {
-    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
-  }
-
-  @Override
-  public boolean isNull(int fieldid) {
-    return getFieldOffset(fieldid) == OffHeapRowBlock.NULL_FIELD_OFFSET;
-  }
-
-  @Override
-  public boolean isNotNull(int fieldid) {
-    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
-  }
-
-  @Override
-  public void clear() {
-    // nothing to do
-  }
-
-  @Override
-  public void put(int fieldId, Datum value) {
-    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum).");
-  }
-
-  @Override
-  public void put(int fieldId, Datum[] values) {
-    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum []).");
-  }
-
-  @Override
-  public void put(int fieldId, Tuple tuple) {
-    throw new UnsupportedException("UnSafeTuple does not support put(int, Tuple).");
-  }
-
-  @Override
-  public void put(Datum[] values) {
-    throw new UnsupportedException("UnSafeTuple does not support put(Datum []).");
-  }
-
-  @Override
-  public Datum get(int fieldId) {
-    if (isNull(fieldId)) {
-      return NullDatum.get();
-    }
-
-    switch (types[fieldId].getType()) {
-    case BOOLEAN:
-      return DatumFactory.createBool(getBool(fieldId));
-    case INT1:
-    case INT2:
-      return DatumFactory.createInt2(getInt2(fieldId));
-    case INT4:
-      return DatumFactory.createInt4(getInt4(fieldId));
-    case INT8:
-      return DatumFactory.createInt8(getInt4(fieldId));
-    case FLOAT4:
-      return DatumFactory.createFloat4(getFloat4(fieldId));
-    case FLOAT8:
-      return DatumFactory.createFloat8(getFloat8(fieldId));
-    case TEXT:
-      return DatumFactory.createText(getText(fieldId));
-    case TIMESTAMP:
-      return DatumFactory.createTimestamp(getInt8(fieldId));
-    case DATE:
-      return DatumFactory.createDate(getInt4(fieldId));
-    case TIME:
-      return DatumFactory.createTime(getInt8(fieldId));
-    case INTERVAL:
-      return getInterval(fieldId);
-    case INET4:
-      return DatumFactory.createInet4(getInt4(fieldId));
-    case PROTOBUF:
-      return getProtobufDatum(fieldId);
-    default:
-      throw new UnsupportedException("Unknown type: " + types[fieldId]);
-    }
-  }
-
-  @Override
-  public void setOffset(long offset) {
-  }
-
-  @Override
-  public long getOffset() {
-    return 0;
-  }
-
-  @Override
-  public boolean getBool(int fieldId) {
-    return UNSAFE.getByte(data, BASE_OFFSET + checkNullAndGetOffset(fieldId)) == 0x01;
-  }
-
-  @Override
-  public byte getByte(int fieldId) {
-    return UNSAFE.getByte(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
-  }
-
-  @Override
-  public char getChar(int fieldId) {
-    return UNSAFE.getChar(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
-  }
-
-  @Override
-  public byte[] getBytes(int fieldId) {
-    long pos = checkNullAndGetOffset(fieldId);
-    int len = UNSAFE.getInt(data, BASE_OFFSET + pos);
-    pos += SizeOf.SIZE_OF_INT;
-
-    byte [] bytes = new byte[len];
-    UNSAFE.copyMemory(data, BASE_OFFSET + pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
-    return bytes;
-  }
-
-  @Override
-  public short getInt2(int fieldId) {
-    return UNSAFE.getShort(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
-  }
-
-  @Override
-  public int getInt4(int fieldId) {
-    return UNSAFE.getInt(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
-  }
-
-  @Override
-  public long getInt8(int fieldId) {
-    return UNSAFE.getLong(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
-  }
-
-  @Override
-  public float getFloat4(int fieldId) {
-    return UNSAFE.getFloat(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
-  }
-
-  @Override
-  public double getFloat8(int fieldId) {
-    return UNSAFE.getDouble(data, BASE_OFFSET + checkNullAndGetOffset(fieldId));
-  }
-
-  @Override
-  public String getText(int fieldId) {
-    return new String(getBytes(fieldId));
-  }
-
-  public IntervalDatum getInterval(int fieldId) {
-    long pos = checkNullAndGetOffset(fieldId);
-    int months = UNSAFE.getInt(data, BASE_OFFSET + pos);
-    pos += SizeOf.SIZE_OF_INT;
-    long millisecs = UNSAFE.getLong(data, BASE_OFFSET + pos);
-    return new IntervalDatum(months, millisecs);
-  }
-
-  @Override
-  public Datum getProtobufDatum(int fieldId) {
-    byte [] bytes = getBytes(fieldId);
-
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(types[fieldId].getCode());
-    Message.Builder builder = factory.newBuilder();
-    try {
-      builder.mergeFrom(bytes);
-    } catch (InvalidProtocolBufferException e) {
-      return NullDatum.get();
-    }
-
-    return new ProtobufDatum(builder.build());
-  }
-
-  @Override
-  public char[] getUnicodeChars(int fieldId) {
-    long pos = checkNullAndGetOffset(fieldId);
-    int len = UNSAFE.getInt(data, BASE_OFFSET + pos);
-    pos += SizeOf.SIZE_OF_INT;
-
-    byte [] bytes = new byte[len];
-    UNSAFE.copyMemory(data, BASE_OFFSET + pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
-    return StringUtils.convertBytesToChars(bytes, Charset.forName("UTF-8"));
-  }
-
-  @Override
-  public Tuple clone() throws CloneNotSupportedException {
-    return this;
-  }
-
-  @Override
-  public Datum[] getValues() {
-    Datum [] datums = new Datum[size()];
-    for (int i = 0; i < size(); i++) {
-      if (contains(i)) {
-        datums[i] = get(i);
-      } else {
-        datums[i] = NullDatum.get();
-      }
-    }
-    return datums;
-  }
-
-  @Override
-  public String toString() {
-    return VTuple.toDisplayString(getValues());
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
deleted file mode 100644
index 2f8e349..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapMemory.java
+++ /dev/null
@@ -1,102 +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.tuple.offheap;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.util.Deallocatable;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.UnsafeUtil;
-import sun.misc.Unsafe;
-import sun.nio.ch.DirectBuffer;
-
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-
-public class OffHeapMemory implements Deallocatable {
-  private static final Log LOG = LogFactory.getLog(OffHeapMemory.class);
-
-  protected static final Unsafe UNSAFE = UnsafeUtil.unsafe;
-
-  protected ByteBuffer buffer;
-  protected int memorySize;
-  protected ResizableLimitSpec limitSpec;
-  protected long address;
-
-  @VisibleForTesting
-  protected OffHeapMemory(ByteBuffer buffer, ResizableLimitSpec limitSpec) {
-    this.buffer = buffer;
-    this.address = ((DirectBuffer) buffer).address();
-    this.memorySize = buffer.limit();
-    this.limitSpec = limitSpec;
-  }
-
-  public OffHeapMemory(ResizableLimitSpec limitSpec) {
-    this(ByteBuffer.allocateDirect((int) limitSpec.initialSize()).order(ByteOrder.nativeOrder()), limitSpec);
-  }
-
-  public long address() {
-    return address;
-  }
-
-  public long size() {
-    return memorySize;
-  }
-
-  public void resize(int newSize) {
-    Preconditions.checkArgument(newSize > 0, "Size must be greater than 0 bytes");
-
-    if (newSize > limitSpec.limit()) {
-      throw new RuntimeException("Resize cannot exceed the size limit");
-    }
-
-    if (newSize < memorySize) {
-      LOG.warn("The size reduction is ignored.");
-    }
-
-    int newBlockSize = UnsafeUtil.alignedSize(newSize);
-    ByteBuffer newByteBuf = ByteBuffer.allocateDirect(newBlockSize);
-    long newAddress = ((DirectBuffer)newByteBuf).address();
-
-    UNSAFE.copyMemory(this.address, newAddress, memorySize);
-
-    UnsafeUtil.free(buffer);
-    this.memorySize = newSize;
-    this.buffer = newByteBuf;
-    this.address = newAddress;
-  }
-
-  public java.nio.Buffer nioBuffer() {
-    return (ByteBuffer) buffer.position(0).limit(memorySize);
-  }
-
-  @Override
-  public void release() {
-    UnsafeUtil.free(this.buffer);
-    this.buffer = null;
-    this.address = 0;
-    this.memorySize = 0;
-  }
-
-  public String toString() {
-    return "memory=" + FileUtil.humanReadableByteCount(memorySize, false) + "," + limitSpec;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
deleted file mode 100644
index 689efb7..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlock.java
+++ /dev/null
@@ -1,176 +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.tuple.offheap;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SchemaUtil;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.util.Deallocatable;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.SizeOf;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-
-import static org.apache.tajo.common.TajoDataTypes.DataType;
-
-public class OffHeapRowBlock extends OffHeapMemory implements Deallocatable {
-  private static final Log LOG = LogFactory.getLog(OffHeapRowBlock.class);
-
-  public static final int NULL_FIELD_OFFSET = -1;
-
-  DataType [] dataTypes;
-
-  // Basic States
-  private int maxRowNum = Integer.MAX_VALUE; // optional
-  private int rowNum;
-  protected int position = 0;
-
-  private OffHeapRowBlockWriter builder;
-
-  private OffHeapRowBlock(ByteBuffer buffer, Schema schema, ResizableLimitSpec limitSpec) {
-    super(buffer, limitSpec);
-    initialize(schema);
-  }
-
-  public OffHeapRowBlock(Schema schema, ResizableLimitSpec limitSpec) {
-    super(limitSpec);
-    initialize(schema);
-  }
-
-  private void initialize(Schema schema) {
-    dataTypes = SchemaUtil.toDataTypes(schema);
-
-    this.builder = new OffHeapRowBlockWriter(this);
-  }
-
-  @VisibleForTesting
-  public OffHeapRowBlock(Schema schema, int bytes) {
-    this(schema, new ResizableLimitSpec(bytes));
-  }
-
-  @VisibleForTesting
-  public OffHeapRowBlock(Schema schema, ByteBuffer buffer) {
-    this(buffer, schema, ResizableLimitSpec.DEFAULT_LIMIT);
-  }
-
-  public void position(int pos) {
-    this.position = pos;
-  }
-
-  public void clear() {
-    this.position = 0;
-    this.rowNum = 0;
-
-    builder.clear();
-  }
-
-  @Override
-  public ByteBuffer nioBuffer() {
-    return (ByteBuffer) buffer.position(0).limit(position);
-  }
-
-  public int position() {
-    return position;
-  }
-
-  public long usedMem() {
-    return position;
-  }
-
-  /**
-   * Ensure that this buffer has enough remaining space to add the size.
-   * Creates and copies to a new buffer if necessary
-   *
-   * @param size Size to add
-   */
-  public void ensureSize(int size) {
-    if (remain() - size < 0) {
-      if (!limitSpec.canIncrease(memorySize)) {
-        throw new RuntimeException("Cannot increase RowBlock anymore.");
-      }
-
-      int newBlockSize = limitSpec.increasedSize(memorySize);
-      resize(newBlockSize);
-      LOG.info("Increase DirectRowBlock to " + FileUtil.humanReadableByteCount(newBlockSize, false));
-    }
-  }
-
-  public long remain() {
-    return memorySize - position - builder.offset();
-  }
-
-  public int maxRowNum() {
-    return maxRowNum;
-  }
-  public int rows() {
-    return rowNum;
-  }
-
-  public void setRows(int rowNum) {
-    this.rowNum = rowNum;
-  }
-
-  public boolean copyFromChannel(FileChannel channel, TableStats stats) throws IOException {
-    if (channel.position() < channel.size()) {
-      clear();
-
-      buffer.clear();
-      channel.read(buffer);
-      memorySize = buffer.position();
-
-      while (position < memorySize) {
-        long recordPtr = address + position;
-
-        if (remain() < SizeOf.SIZE_OF_INT) {
-          channel.position(channel.position() - remain());
-          memorySize = (int) (memorySize - remain());
-          return true;
-        }
-
-        int recordSize = UNSAFE.getInt(recordPtr);
-
-        if (remain() < recordSize) {
-          channel.position(channel.position() - remain());
-          memorySize = (int) (memorySize - remain());
-          return true;
-        }
-
-        position += recordSize;
-        rowNum++;
-      }
-
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  public RowWriter getWriter() {
-    return builder;
-  }
-
-  public OffHeapRowBlockReader getReader() {
-    return new OffHeapRowBlockReader(this);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
deleted file mode 100644
index 4a9313f..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockReader.java
+++ /dev/null
@@ -1,63 +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.tuple.offheap;
-
-import org.apache.tajo.tuple.RowBlockReader;
-import org.apache.tajo.util.UnsafeUtil;
-import sun.misc.Unsafe;
-
-public class OffHeapRowBlockReader implements RowBlockReader<ZeroCopyTuple> {
-  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
-  OffHeapRowBlock rowBlock;
-
-  // Read States
-  private int curRowIdxForRead;
-  private int curPosForRead;
-
-  public OffHeapRowBlockReader(OffHeapRowBlock rowBlock) {
-    this.rowBlock = rowBlock;
-  }
-
-  public long remainForRead() {
-    return rowBlock.memorySize - curPosForRead;
-  }
-
-  @Override
-  public boolean next(ZeroCopyTuple tuple) {
-    if (curRowIdxForRead < rowBlock.rows()) {
-
-      long recordStartPtr = rowBlock.address() + curPosForRead;
-      int recordLen = UNSAFE.getInt(recordStartPtr);
-      tuple.set(rowBlock.buffer, curPosForRead, recordLen, rowBlock.dataTypes);
-
-      curPosForRead += recordLen;
-      curRowIdxForRead++;
-
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public void reset() {
-    curPosForRead = 0;
-    curRowIdxForRead = 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
deleted file mode 100644
index dbc3188..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockUtils.java
+++ /dev/null
@@ -1,54 +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.tuple.offheap;
-
-import com.google.common.collect.Lists;
-import org.apache.tajo.storage.Tuple;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-public class OffHeapRowBlockUtils {
-
-  public static List<Tuple> sort(OffHeapRowBlock rowBlock, Comparator<Tuple> comparator) {
-    List<Tuple> tupleList = Lists.newArrayList();
-    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-    while(reader.next(zcTuple)) {
-      tupleList.add(zcTuple);
-      zcTuple = new ZeroCopyTuple();
-    }
-    Collections.sort(tupleList, comparator);
-    return tupleList;
-  }
-
-  public static Tuple[] sortToArray(OffHeapRowBlock rowBlock, Comparator<Tuple> comparator) {
-    Tuple[] tuples = new Tuple[rowBlock.rows()];
-    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-    for (int i = 0; i < rowBlock.rows() && reader.next(zcTuple); i++) {
-      tuples[i] = zcTuple;
-      zcTuple = new ZeroCopyTuple();
-    }
-    Arrays.sort(tuples, comparator);
-    return tuples;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
deleted file mode 100644
index d177e0c..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowBlockWriter.java
+++ /dev/null
@@ -1,58 +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.tuple.offheap;
-
-import org.apache.tajo.common.TajoDataTypes;
-
-public class OffHeapRowBlockWriter extends OffHeapRowWriter {
-  OffHeapRowBlock rowBlock;
-
-  OffHeapRowBlockWriter(OffHeapRowBlock rowBlock) {
-    super(rowBlock.dataTypes);
-    this.rowBlock = rowBlock;
-  }
-
-  public long address() {
-    return rowBlock.address();
-  }
-
-  public int position() {
-    return rowBlock.position();
-  }
-
-  @Override
-  public void forward(int length) {
-    rowBlock.position(position() + length);
-  }
-
-  public void ensureSize(int size) {
-    rowBlock.ensureSize(size);
-  }
-
-  @Override
-  public void endRow() {
-    super.endRow();
-    rowBlock.setRows(rowBlock.rows() + 1);
-  }
-
-  @Override
-  public TajoDataTypes.DataType[] dataTypes() {
-    return rowBlock.dataTypes;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
deleted file mode 100644
index 85c7e0b..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/OffHeapRowWriter.java
+++ /dev/null
@@ -1,232 +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.tuple.offheap;
-
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.IntervalDatum;
-import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.util.SizeOf;
-import org.apache.tajo.util.UnsafeUtil;
-
-/**
- *
- * Row Record Structure
- *
- * | row length (4 bytes) | field 1 offset | field 2 offset | ... | field N offset| field 1 | field 2| ... | field N |
- *                              4 bytes          4 bytes               4 bytes
- *
- */
-public abstract class OffHeapRowWriter implements RowWriter {
-  /** record size + offset list */
-  private final int headerSize;
-  /** field offsets */
-  private final int [] fieldOffsets;
-  private final TajoDataTypes.DataType [] dataTypes;
-
-  private int curFieldIdx;
-  private int curOffset;
-
-  public OffHeapRowWriter(final TajoDataTypes.DataType [] dataTypes) {
-    this.dataTypes = dataTypes;
-    fieldOffsets = new int[dataTypes.length];
-    headerSize = SizeOf.SIZE_OF_INT * (dataTypes.length + 1);
-  }
-
-  public void clear() {
-    curOffset = 0;
-    curFieldIdx = 0;
-  }
-
-  public long recordStartAddr() {
-    return address() + position();
-  }
-
-  public abstract long address();
-
-  public abstract void ensureSize(int size);
-
-  public int offset() {
-    return curOffset;
-  }
-
-  /**
-   * Current position
-   *
-   * @return The position
-   */
-  public abstract int position();
-
-  /**
-   * Forward the address;
-   *
-   * @param length Length to be forwarded
-   */
-  public abstract void forward(int length);
-
-  @Override
-  public TajoDataTypes.DataType[] dataTypes() {
-    return dataTypes;
-  }
-
-  public boolean startRow() {
-    curOffset = headerSize;
-    curFieldIdx = 0;
-    return true;
-  }
-
-  public void endRow() {
-    long rowHeaderPos = address() + position();
-    OffHeapMemory.UNSAFE.putInt(rowHeaderPos, curOffset);
-    rowHeaderPos += SizeOf.SIZE_OF_INT;
-
-    for (int i = 0; i < curFieldIdx; i++) {
-      OffHeapMemory.UNSAFE.putInt(rowHeaderPos, fieldOffsets[i]);
-      rowHeaderPos += SizeOf.SIZE_OF_INT;
-    }
-    for (int i = curFieldIdx; i < dataTypes.length; i++) {
-      OffHeapMemory.UNSAFE.putInt(rowHeaderPos, OffHeapRowBlock.NULL_FIELD_OFFSET);
-      rowHeaderPos += SizeOf.SIZE_OF_INT;
-    }
-
-    // rowOffset is equivalent to a byte length of this row.
-    forward(curOffset);
-  }
-
-  public void skipField() {
-    fieldOffsets[curFieldIdx++] = OffHeapRowBlock.NULL_FIELD_OFFSET;
-  }
-
-  private void forwardField() {
-    fieldOffsets[curFieldIdx++] = curOffset;
-  }
-
-  public void putBool(boolean val) {
-    ensureSize(SizeOf.SIZE_OF_BOOL);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putByte(recordStartAddr() + curOffset, (byte) (val ? 0x01 : 0x00));
-
-    curOffset += SizeOf.SIZE_OF_BOOL;
-  }
-
-  public void putInt2(short val) {
-    ensureSize(SizeOf.SIZE_OF_SHORT);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putShort(recordStartAddr() + curOffset, val);
-    curOffset += SizeOf.SIZE_OF_SHORT;
-  }
-
-  public void putInt4(int val) {
-    ensureSize(SizeOf.SIZE_OF_INT);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putInt(recordStartAddr() + curOffset, val);
-    curOffset += SizeOf.SIZE_OF_INT;
-  }
-
-  public void putInt8(long val) {
-    ensureSize(SizeOf.SIZE_OF_LONG);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putLong(recordStartAddr() + curOffset, val);
-    curOffset += SizeOf.SIZE_OF_LONG;
-  }
-
-  public void putFloat4(float val) {
-    ensureSize(SizeOf.SIZE_OF_FLOAT);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putFloat(recordStartAddr() + curOffset, val);
-    curOffset += SizeOf.SIZE_OF_FLOAT;
-  }
-
-  public void putFloat8(double val) {
-    ensureSize(SizeOf.SIZE_OF_DOUBLE);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putDouble(recordStartAddr() + curOffset, val);
-    curOffset += SizeOf.SIZE_OF_DOUBLE;
-  }
-
-  public void putText(String val) {
-    byte[] bytes = val.getBytes(TextDatum.DEFAULT_CHARSET);
-    putText(bytes);
-  }
-
-  public void putText(byte[] val) {
-    int bytesLen = val.length;
-
-    ensureSize(SizeOf.SIZE_OF_INT + bytesLen);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putInt(recordStartAddr() + curOffset, bytesLen);
-    curOffset += SizeOf.SIZE_OF_INT;
-
-    OffHeapMemory.UNSAFE.copyMemory(val, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, null,
-        recordStartAddr() + curOffset, bytesLen);
-    curOffset += bytesLen;
-  }
-
-  public void putBlob(byte[] val) {
-    int bytesLen = val.length;
-
-    ensureSize(SizeOf.SIZE_OF_INT + bytesLen);
-    forwardField();
-
-    OffHeapMemory.UNSAFE.putInt(recordStartAddr() + curOffset, bytesLen);
-    curOffset += SizeOf.SIZE_OF_INT;
-
-    OffHeapMemory.UNSAFE.copyMemory(val, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, null,
-        recordStartAddr() + curOffset, bytesLen);
-    curOffset += bytesLen;
-  }
-
-  public void putTimestamp(long val) {
-    putInt8(val);
-  }
-
-  public void putDate(int val) {
-    putInt4(val);
-  }
-
-  public void putTime(long val) {
-    putInt8(val);
-  }
-
-  public void putInterval(IntervalDatum val) {
-    ensureSize(SizeOf.SIZE_OF_INT + SizeOf.SIZE_OF_LONG);
-    forwardField();
-
-    long offset = recordStartAddr() + curOffset;
-    OffHeapMemory.UNSAFE.putInt(offset, val.getMonths());
-    offset += SizeOf.SIZE_OF_INT;
-    OffHeapMemory.UNSAFE.putLong(offset, val.getMilliSeconds());
-    curOffset += SizeOf.SIZE_OF_INT + SizeOf.SIZE_OF_LONG;
-  }
-
-  public void putInet4(int val) {
-    putInt4(val);
-  }
-
-  public void putProtoDatum(ProtobufDatum val) {
-    putBlob(val.asByteArray());
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
deleted file mode 100644
index 14e67b2..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
+++ /dev/null
@@ -1,142 +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.tuple.offheap;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.util.FileUtil;
-
-/**
- * It specifies the maximum size or increasing ratio. In addition,
- * it guarantees that all numbers are less than or equal to Integer.MAX_VALUE 2^31
- * due to ByteBuffer.
- */
-public class ResizableLimitSpec {
-  private final Log LOG = LogFactory.getLog(ResizableLimitSpec.class);
-
-  public static final int MAX_SIZE_BYTES = Integer.MAX_VALUE;
-  public static final ResizableLimitSpec DEFAULT_LIMIT = new ResizableLimitSpec(Integer.MAX_VALUE);
-
-  private final long initSize;
-  private final long limitBytes;
-  private final float incRatio;
-  private final float allowedOVerflowRatio;
-  private final static float DEFAULT_ALLOWED_OVERFLOW_RATIO = 0.1f;
-  private final static float DEFAULT_INCREASE_RATIO = 1.0f;
-
-  public ResizableLimitSpec(long initSize) {
-    this(initSize, MAX_SIZE_BYTES, DEFAULT_ALLOWED_OVERFLOW_RATIO);
-  }
-
-  public ResizableLimitSpec(long initSize, long limitBytes) {
-    this(initSize, limitBytes, DEFAULT_ALLOWED_OVERFLOW_RATIO);
-  }
-
-  public ResizableLimitSpec(long initSize, long limitBytes, float allowedOverflow) {
-    this(initSize, limitBytes, allowedOverflow, DEFAULT_INCREASE_RATIO);
-  }
-
-  public ResizableLimitSpec(long initSize, long limitBytes, float allowedOverflowRatio, float incRatio) {
-    Preconditions.checkArgument(initSize > 0, "initial size must be greater than 0 bytes.");
-    Preconditions.checkArgument(initSize <= MAX_SIZE_BYTES, "The maximum initial size is 2GB.");
-    Preconditions.checkArgument(limitBytes > 0, "The limit size must be greater than 0 bytes.");
-    Preconditions.checkArgument(limitBytes <= MAX_SIZE_BYTES, "The maximum limit size is 2GB.");
-    Preconditions.checkArgument(incRatio > 0.0f, "Increase Ratio must be greater than 0.");
-
-    if (initSize == limitBytes) {
-      long overflowedSize = (long) (initSize + (initSize * allowedOverflowRatio));
-
-      if (overflowedSize > Integer.MAX_VALUE) {
-        overflowedSize = Integer.MAX_VALUE;
-      }
-
-      this.initSize = overflowedSize;
-      this.limitBytes = overflowedSize;
-    } else {
-      this.initSize = initSize;
-      limitBytes = (long) (limitBytes + (limitBytes * allowedOverflowRatio));
-
-      if (limitBytes > Integer.MAX_VALUE) {
-        this.limitBytes = Integer.MAX_VALUE;
-      } else {
-        this.limitBytes = limitBytes;
-      }
-    }
-
-    this.allowedOVerflowRatio = allowedOverflowRatio;
-    this.incRatio = incRatio;
-  }
-
-  public long initialSize() {
-    return initSize;
-  }
-
-  public long limit() {
-    return limitBytes;
-  }
-
-  public float remainRatio(long currentSize) {
-    Preconditions.checkArgument(currentSize > 0, "Size must be greater than 0 bytes.");
-    if (currentSize > Integer.MAX_VALUE) {
-      currentSize = Integer.MAX_VALUE;
-    }
-    return (float)currentSize / (float)limitBytes;
-  }
-
-  public boolean canIncrease(long currentSize) {
-    return remain(currentSize) > 0;
-  }
-
-  public long remain(long currentSize) {
-    Preconditions.checkArgument(currentSize > 0, "Size must be greater than 0 bytes.");
-    return limitBytes > Integer.MAX_VALUE ? Integer.MAX_VALUE - currentSize : limitBytes - currentSize;
-  }
-
-  public int increasedSize(int currentSize) {
-    if (currentSize < initSize) {
-      return (int) initSize;
-    }
-
-    if (currentSize > Integer.MAX_VALUE) {
-      LOG.warn("Current size already exceeds the maximum size (" + Integer.MAX_VALUE + " bytes)");
-      return Integer.MAX_VALUE;
-    }
-    long nextSize = (long) (currentSize + ((float) currentSize * incRatio));
-
-    if (nextSize > limitBytes) {
-      LOG.info("Increasing reaches size limit (" + FileUtil.humanReadableByteCount(limitBytes, false) + ")");
-      nextSize = limitBytes;
-    }
-
-    if (nextSize > Integer.MAX_VALUE) {
-      LOG.info("Increasing reaches maximum size (" + FileUtil.humanReadableByteCount(Integer.MAX_VALUE, false) + ")");
-      nextSize = Integer.MAX_VALUE;
-    }
-
-    return (int) nextSize;
-  }
-
-  @Override
-  public String toString() {
-    return "init=" + FileUtil.humanReadableByteCount(initSize, false) + ",limit="
-        + FileUtil.humanReadableByteCount(limitBytes, false) + ",overflow_ratio=" + allowedOVerflowRatio
-        + ",inc_ratio=" + incRatio;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
deleted file mode 100644
index a2b2561..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
+++ /dev/null
@@ -1,73 +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.tuple.offheap;
-
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.IntervalDatum;
-import org.apache.tajo.datum.ProtobufDatum;
-
-/**
- * The call sequence should be as follows:
- *
- * <pre>
- *   startRow() -->  skipField() or putXXX --> endRow()
- * </pre>
- *
- * The total number of skipField and putXXX invocations must be equivalent to the number of fields.
- */
-public interface RowWriter {
-
-  public TajoDataTypes.DataType [] dataTypes();
-
-  public boolean startRow();
-
-  public void endRow();
-
-  public void skipField();
-
-  public void putBool(boolean val);
-
-  public void putInt2(short val);
-
-  public void putInt4(int val);
-
-  public void putInt8(long val);
-
-  public void putFloat4(float val);
-
-  public void putFloat8(double val);
-
-  public void putText(String val);
-
-  public void putText(byte[] val);
-
-  public void putBlob(byte[] val);
-
-  public void putTimestamp(long val);
-
-  public void putTime(long val);
-
-  public void putDate(int val);
-
-  public void putInterval(IntervalDatum val);
-
-  public void putInet4(int val);
-
-  public void putProtoDatum(ProtobufDatum datum);
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
deleted file mode 100644
index b742e6d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
+++ /dev/null
@@ -1,311 +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.tuple.offheap;
-
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.Message;
-
-import org.apache.tajo.datum.*;
-import org.apache.tajo.exception.UnsupportedException;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.util.SizeOf;
-import org.apache.tajo.util.StringUtils;
-import org.apache.tajo.util.UnsafeUtil;
-
-import sun.misc.Unsafe;
-import sun.nio.ch.DirectBuffer;
-
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.charset.Charset;
-
-import static org.apache.tajo.common.TajoDataTypes.DataType;
-
-public abstract class UnSafeTuple implements Tuple {
-  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
-
-  private DirectBuffer bb;
-  private int relativePos;
-  private int length;
-  private DataType [] types;
-
-  protected void set(ByteBuffer bb, int relativePos, int length, DataType [] types) {
-    this.bb = (DirectBuffer) bb;
-    this.relativePos = relativePos;
-    this.length = length;
-    this.types = types;
-  }
-
-  void set(ByteBuffer bb, DataType [] types) {
-    set(bb, 0, bb.limit(), types);
-  }
-
-  @Override
-  public int size() {
-    return types.length;
-  }
-
-  public ByteBuffer nioBuffer() {
-    return ((ByteBuffer)((ByteBuffer)bb).duplicate().position(relativePos).limit(relativePos + length)).slice();
-  }
-
-  public HeapTuple toHeapTuple() {
-    byte [] bytes = new byte[length];
-    UNSAFE.copyMemory(null, bb.address() + relativePos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, length);
-    return new HeapTuple(bytes, types);
-  }
-
-  public void copyFrom(UnSafeTuple tuple) {
-    Preconditions.checkNotNull(tuple);
-
-    ((ByteBuffer) bb).clear();
-    if (length < tuple.length) {
-      UnsafeUtil.free((ByteBuffer) bb);
-      bb = (DirectBuffer) ByteBuffer.allocateDirect(tuple.length).order(ByteOrder.nativeOrder());
-      this.relativePos = 0;
-      this.length = tuple.length;
-    }
-
-    ((ByteBuffer) bb).put(tuple.nioBuffer());
-  }
-
-  private int getFieldOffset(int fieldId) {
-    return UNSAFE.getInt(bb.address() + relativePos + SizeOf.SIZE_OF_INT + (fieldId * SizeOf.SIZE_OF_INT));
-  }
-
-  public long getFieldAddr(int fieldId) {
-    int fieldOffset = getFieldOffset(fieldId);
-    if (fieldOffset == -1) {
-      throw new RuntimeException("Invalid Field Access: " + fieldId);
-    }
-    return bb.address() + relativePos + fieldOffset;
-  }
-
-  @Override
-  public boolean contains(int fieldid) {
-    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
-  }
-
-  @Override
-  public boolean isNull(int fieldid) {
-    return getFieldOffset(fieldid) == OffHeapRowBlock.NULL_FIELD_OFFSET;
-  }
-
-  @Override
-  public boolean isNotNull(int fieldid) {
-    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
-  }
-
-  @Override
-  public void clear() {
-    // nothing to do
-  }
-
-  @Override
-  public void put(int fieldId, Datum value) {
-    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum).");
-  }
-
-  @Override
-  public void put(int fieldId, Datum[] values) {
-    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum []).");
-  }
-
-  @Override
-  public void put(int fieldId, Tuple tuple) {
-    throw new UnsupportedException("UnSafeTuple does not support put(int, Tuple).");
-  }
-
-  @Override
-  public void put(Datum[] values) {
-    throw new UnsupportedException("UnSafeTuple does not support put(Datum []).");
-  }
-
-  @Override
-  public Datum get(int fieldId) {
-    if (isNull(fieldId)) {
-      return NullDatum.get();
-    }
-
-    switch (types[fieldId].getType()) {
-    case BOOLEAN:
-      return DatumFactory.createBool(getBool(fieldId));
-    case INT1:
-    case INT2:
-      return DatumFactory.createInt2(getInt2(fieldId));
-    case INT4:
-      return DatumFactory.createInt4(getInt4(fieldId));
-    case INT8:
-      return DatumFactory.createInt8(getInt4(fieldId));
-    case FLOAT4:
-      return DatumFactory.createFloat4(getFloat4(fieldId));
-    case FLOAT8:
-      return DatumFactory.createFloat8(getFloat8(fieldId));
-    case TEXT:
-      return DatumFactory.createText(getText(fieldId));
-    case TIMESTAMP:
-      return DatumFactory.createTimestamp(getInt8(fieldId));
-    case DATE:
-      return DatumFactory.createDate(getInt4(fieldId));
-    case TIME:
-      return DatumFactory.createTime(getInt8(fieldId));
-    case INTERVAL:
-      return getInterval(fieldId);
-    case INET4:
-      return DatumFactory.createInet4(getInt4(fieldId));
-    case PROTOBUF:
-      return getProtobufDatum(fieldId);
-    default:
-      throw new UnsupportedException("Unknown type: " + types[fieldId]);
-    }
-  }
-
-  @Override
-  public void setOffset(long offset) {
-  }
-
-  @Override
-  public long getOffset() {
-    return 0;
-  }
-
-  @Override
-  public boolean getBool(int fieldId) {
-    return UNSAFE.getByte(getFieldAddr(fieldId)) == 0x01;
-  }
-
-  @Override
-  public byte getByte(int fieldId) {
-    return UNSAFE.getByte(getFieldAddr(fieldId));
-  }
-
-  @Override
-  public char getChar(int fieldId) {
-    return UNSAFE.getChar(getFieldAddr(fieldId));
-  }
-
-  @Override
-  public byte[] getBytes(int fieldId) {
-    long pos = getFieldAddr(fieldId);
-    int len = UNSAFE.getInt(pos);
-    pos += SizeOf.SIZE_OF_INT;
-
-    byte [] bytes = new byte[len];
-    UNSAFE.copyMemory(null, pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
-    return bytes;
-  }
-
-  @Override
-  public short getInt2(int fieldId) {
-    long addr = getFieldAddr(fieldId);
-    return UNSAFE.getShort(addr);
-  }
-
-  @Override
-  public int getInt4(int fieldId) {
-    return UNSAFE.getInt(getFieldAddr(fieldId));
-  }
-
-  @Override
-  public long getInt8(int fieldId) {
-    return UNSAFE.getLong(getFieldAddr(fieldId));
-  }
-
-  @Override
-  public float getFloat4(int fieldId) {
-    return UNSAFE.getFloat(getFieldAddr(fieldId));
-  }
-
-  @Override
-  public double getFloat8(int fieldId) {
-    return UNSAFE.getDouble(getFieldAddr(fieldId));
-  }
-
-  @Override
-  public String getText(int fieldId) {
-    long pos = getFieldAddr(fieldId);
-    int len = UNSAFE.getInt(pos);
-    pos += SizeOf.SIZE_OF_INT;
-
-    byte [] bytes = new byte[len];
-    UNSAFE.copyMemory(null, pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
-    return new String(bytes);
-  }
-
-  public IntervalDatum getInterval(int fieldId) {
-    long pos = getFieldAddr(fieldId);
-    int months = UNSAFE.getInt(pos);
-    pos += SizeOf.SIZE_OF_INT;
-    long millisecs = UNSAFE.getLong(pos);
-    return new IntervalDatum(months, millisecs);
-  }
-
-  @Override
-  public Datum getProtobufDatum(int fieldId) {
-    byte [] bytes = getBytes(fieldId);
-
-    ProtobufDatumFactory factory = ProtobufDatumFactory.get(types[fieldId].getCode());
-    Message.Builder builder = factory.newBuilder();
-    try {
-      builder.mergeFrom(bytes);
-    } catch (InvalidProtocolBufferException e) {
-      return NullDatum.get();
-    }
-
-    return new ProtobufDatum(builder.build());
-  }
-
-  @Override
-  public char[] getUnicodeChars(int fieldId) {
-    long pos = getFieldAddr(fieldId);
-    int len = UNSAFE.getInt(pos);
-    pos += SizeOf.SIZE_OF_INT;
-
-    byte [] bytes = new byte[len];
-    UNSAFE.copyMemory(null, pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
-    return StringUtils.convertBytesToChars(bytes, Charset.forName("UTF-8"));
-  }
-
-  @Override
-  public Tuple clone() throws CloneNotSupportedException {
-    return toHeapTuple();
-  }
-
-  @Override
-  public Datum[] getValues() {
-    Datum [] datums = new Datum[size()];
-    for (int i = 0; i < size(); i++) {
-      if (contains(i)) {
-        datums[i] = get(i);
-      } else {
-        datums[i] = NullDatum.get();
-      }
-    }
-    return datums;
-  }
-
-  @Override
-  public String toString() {
-    return VTuple.toDisplayString(getValues());
-  }
-
-  public abstract void release();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
deleted file mode 100644
index 73e1e2f..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
+++ /dev/null
@@ -1,99 +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.tuple.offheap;
-
-import com.google.common.primitives.Longs;
-import com.google.common.primitives.UnsignedLongs;
-import org.apache.tajo.util.SizeOf;
-import org.apache.tajo.util.UnsafeUtil;
-import sun.misc.Unsafe;
-
-import java.nio.ByteOrder;
-
-/**
- * It directly access UTF bytes in UnSafeTuple without any copy. It is used by compiled TupleComparator.
- */
-public class UnSafeTupleBytesComparator {
-  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
-
-  static final boolean littleEndian =
-      ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN);
-
-  public static int compare(long ptr1, long ptr2) {
-    int lstrLen = UNSAFE.getInt(ptr1);
-    int rstrLen = UNSAFE.getInt(ptr2);
-
-    ptr1 += SizeOf.SIZE_OF_INT;
-    ptr2 += SizeOf.SIZE_OF_INT;
-
-    int minLength = Math.min(lstrLen, rstrLen);
-    int minWords = minLength / Longs.BYTES;
-
-        /*
-         * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a
-         * time is no slower than comparing 4 bytes at a time even on 32-bit.
-         * On the other hand, it is substantially faster on 64-bit.
-         */
-    for (int i = 0; i < minWords * Longs.BYTES; i += Longs.BYTES) {
-      long lw = UNSAFE.getLong(ptr1);
-      long rw = UNSAFE.getLong(ptr2);
-      long diff = lw ^ rw;
-
-      if (diff != 0) {
-        if (!littleEndian) {
-          return UnsignedLongs.compare(lw, rw);
-        }
-
-        // Use binary search
-        int n = 0;
-        int y;
-        int x = (int) diff;
-        if (x == 0) {
-          x = (int) (diff >>> 32);
-          n = 32;
-        }
-
-        y = x << 16;
-        if (y == 0) {
-          n += 16;
-        } else {
-          x = y;
-        }
-
-        y = x << 8;
-        if (y == 0) {
-          n += 8;
-        }
-        return (int) (((lw >>> n) & 0xFFL) - ((rw >>> n) & 0xFFL));
-      }
-
-      ptr1 += SizeOf.SIZE_OF_LONG;
-      ptr2 += SizeOf.SIZE_OF_LONG;
-    }
-
-    // The epilogue to cover the last (minLength % 8) elements.
-    for (int i = minWords * Longs.BYTES; i < minLength; i++) {
-      int result = UNSAFE.getByte(ptr1++) - UNSAFE.getByte(ptr2++);
-      if (result != 0) {
-        return result;
-      }
-    }
-    return lstrLen - rstrLen;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java b/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
deleted file mode 100644
index 51dbb29..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
+++ /dev/null
@@ -1,35 +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.tuple.offheap;
-
-import java.nio.ByteBuffer;
-
-import static org.apache.tajo.common.TajoDataTypes.DataType;
-
-public class ZeroCopyTuple extends UnSafeTuple {
-
-  public void set(ByteBuffer bb, int relativePos, int length, DataType [] types) {
-    super.set(bb, relativePos, length, types);
-  }
-
-  @Override
-  public void release() {
-    // nothing to do
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/proto/IndexProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/proto/IndexProtos.proto b/tajo-storage/src/main/proto/IndexProtos.proto
deleted file mode 100644
index f5c8a08..0000000
--- a/tajo-storage/src/main/proto/IndexProtos.proto
+++ /dev/null
@@ -1,31 +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.
- */
-
-option java_package = "org.apache.tajo.index";
-option java_outer_classname = "IndexProtos";
-option optimize_for = SPEED;
-option java_generic_services = false;
-option java_generate_equals_and_hash = true;
-
-import "CatalogProtos.proto";
-
-message TupleComparatorProto {
-  required SchemaProto schema = 1;
-  repeated SortSpecProto sortSpecs = 2;
-  repeated TupleComparatorSpecProto compSpecs = 3;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/proto/StorageFragmentProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/proto/StorageFragmentProtos.proto b/tajo-storage/src/main/proto/StorageFragmentProtos.proto
deleted file mode 100644
index dd79d74..0000000
--- a/tajo-storage/src/main/proto/StorageFragmentProtos.proto
+++ /dev/null
@@ -1,35 +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.
- */
-
-option java_package = "org.apache.tajo.storage.fragment";
-option java_outer_classname = "StorageFragmentProtos";
-option optimize_for = SPEED;
-option java_generic_services = false;
-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;
-}
\ No newline at end of file


[28/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/DiskUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/DiskUtil.java
deleted file mode 100644
index 2d68870..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/DiskUtil.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.server.common.Util;
-
-import java.io.*;
-import java.net.URI;
-import java.util.*;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-
-public class DiskUtil {
-
-  static String UNIX_DISK_DEVICE_PATH = "/proc/partitions";
-
-  public enum OSType {
-		OS_TYPE_UNIX, OS_TYPE_WINXP, OS_TYPE_SOLARIS, OS_TYPE_MAC
-	}
-
-	static private OSType getOSType() {
-		String osName = System.getProperty("os.name");
-		if (osName.contains("Windows")
-				&& (osName.contains("XP") || osName.contains("2003")
-						|| osName.contains("Vista")
-						|| osName.contains("Windows_7")
-						|| osName.contains("Windows 7") || osName
-							.contains("Windows7"))) {
-			return OSType.OS_TYPE_WINXP;
-		} else if (osName.contains("SunOS") || osName.contains("Solaris")) {
-			return OSType.OS_TYPE_SOLARIS;
-		} else if (osName.contains("Mac")) {
-			return OSType.OS_TYPE_MAC;
-		} else {
-			return OSType.OS_TYPE_UNIX;
-		}
-	}
-	
-	public static List<DiskDeviceInfo> getDiskDeviceInfos() throws IOException {
-		List<DiskDeviceInfo> deviceInfos;
-		
-		if(getOSType() == OSType.OS_TYPE_UNIX) {
-			deviceInfos = getUnixDiskDeviceInfos();
-			setDeviceMountInfo(deviceInfos);
-		} else {
-			deviceInfos = getDefaultDiskDeviceInfos();
-		}
-		
-		return deviceInfos;
-	}
-
-	private static List<DiskDeviceInfo> getUnixDiskDeviceInfos() {
-		List<DiskDeviceInfo> infos = new ArrayList<DiskDeviceInfo>();
-		
-		File file = new File(UNIX_DISK_DEVICE_PATH);
-		if(!file.exists()) {
-			System.out.println("No partition file:" + file.getAbsolutePath());
-			return getDefaultDiskDeviceInfos();
-		}
-		
-		BufferedReader reader = null;
-		try {
-			reader = new BufferedReader(new InputStreamReader(new FileInputStream(UNIX_DISK_DEVICE_PATH)));
-			String line = null;
-			
-			int count = 0;
-			Set<String> deviceNames = new TreeSet<String>();
-			while((line = reader.readLine()) != null) {
-				if(count > 0 && !line.trim().isEmpty()) {
-					String[] tokens = line.trim().split(" +");
-					if(tokens.length == 4) {
-						String deviceName = getDiskDeviceName(tokens[3]);
-						deviceNames.add(deviceName);
-					}
-				}
-				count++;
-			}
-			
-			int id = 0;
-			for(String eachDeviceName: deviceNames) {
-				DiskDeviceInfo diskDeviceInfo = new DiskDeviceInfo(id++);
-				diskDeviceInfo.setName(eachDeviceName);
-				
-				//TODO set addtional info
-				// /sys/block/sda/queue
-				infos.add(diskDeviceInfo);
-			}
-		} catch (Exception e) {
-			e.printStackTrace();
-		} finally {
-			if(reader != null) {
-				try {
-					reader.close();
-				} catch (IOException e) {
-				}
-			}
-		}
-		
-		return infos;
-	}
-	
-	private static String getDiskDeviceName(String partitionName) {
-		byte[] bytes = partitionName.getBytes();
-		
-		byte[] result = new byte[bytes.length];
-		int length = 0;
-		for(int i = 0; i < bytes.length; i++, length++) {
-			if(bytes[i] >= '0' && bytes[i] <= '9') {
-				break;
-			} else {
-				result[i] = bytes[i];
-			}
-		}
-		
-		return new String(result, 0, length);
-	}
-	
-	public static List<DiskDeviceInfo> getDefaultDiskDeviceInfos() {
-		DiskDeviceInfo diskDeviceInfo = new DiskDeviceInfo(0);
-		diskDeviceInfo.setName("default");
-		
-		List<DiskDeviceInfo> infos = new ArrayList<DiskDeviceInfo>();
-		
-		infos.add(diskDeviceInfo);
-		
-		return infos;
-	}
-	
-	
-	private static void setDeviceMountInfo(List<DiskDeviceInfo> deviceInfos) throws IOException {
-		Map<String, DiskDeviceInfo> deviceMap = new HashMap<String, DiskDeviceInfo>();
-		for(DiskDeviceInfo eachDevice: deviceInfos) {
-			deviceMap.put(eachDevice.getName(), eachDevice);
-		}
-		
-		BufferedReader mountOutput = null;
-		try {
-			Process mountProcess = Runtime.getRuntime().exec("mount");
-			mountOutput = new BufferedReader(new InputStreamReader(
-					mountProcess.getInputStream()));
-			while (true) {
-				String line = mountOutput.readLine();
-				if (line == null) {
-					break;
-				}
-
-				int indexStart = line.indexOf(" on /");
-				int indexEnd = line.indexOf(" ", indexStart + 4);
-
-				String deviceName = line.substring(0, indexStart).trim();
-				String[] deviceNameTokens = deviceName.split("/");
-				if(deviceNameTokens.length == 3) {
-					if("dev".equals(deviceNameTokens[1])) {
-						String realDeviceName = getDiskDeviceName(deviceNameTokens[2]);
-						String mountPath = new File(line.substring(indexStart + 4, indexEnd)).getAbsolutePath();
-						
-						DiskDeviceInfo diskDeviceInfo = deviceMap.get(realDeviceName);
-						if(diskDeviceInfo != null) {
-							diskDeviceInfo.addMountPath(new DiskMountInfo(diskDeviceInfo.getId(), mountPath));
-						}
-					}
-				}
-			}
-		} catch (IOException e) {
-			throw e;
-		} finally {
-			if (mountOutput != null) {
-				mountOutput.close();
-			}
-		}
-	}
-
-  public static int getDataNodeStorageSize(){
-    return getStorageDirs().size();
-  }
-
-  public static List<URI> getStorageDirs(){
-    Configuration conf = new HdfsConfiguration();
-    Collection<String> dirNames = conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
-    return Util.stringCollectionAsURIs(dirNames);
-  }
-
-	public static void main(String[] args) throws Exception {
-		System.out.println("/dev/sde1".split("/").length);
-		for(String eachToken: "/dev/sde1".split("/")) {
-			System.out.println(eachToken);
-		}
- 	}
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
deleted file mode 100644
index 7df4584..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FieldSerializerDeserializer.java
+++ /dev/null
@@ -1,35 +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 io.netty.buffer.ByteBuf;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.datum.Datum;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-
-public interface FieldSerializerDeserializer {
-
-  public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) throws IOException;
-
-  public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.java
deleted file mode 100644
index 117d3da..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.conf.TajoConf;
-
-import java.io.IOException;
-
-public abstract class FileAppender implements Appender {
-  private static final Log LOG = LogFactory.getLog(FileAppender.class);
-
-  protected boolean inited = false;
-
-  protected final Configuration conf;
-  protected final TableMeta meta;
-  protected final Schema schema;
-  protected final Path workDir;
-  protected final QueryUnitAttemptId taskAttemptId;
-
-  protected boolean enabledStats;
-  protected Path path;
-
-  public FileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId, Schema schema,
-                      TableMeta meta, Path workDir) {
-    this.conf = conf;
-    this.meta = meta;
-    this.schema = schema;
-    this.workDir = workDir;
-    this.taskAttemptId = taskAttemptId;
-
-    try {
-      if (taskAttemptId != null) {
-        this.path = StorageManager.getFileStorageManager((TajoConf) conf).getAppenderFilePath(taskAttemptId, workDir);
-      } else {
-        this.path = workDir;
-      }
-    } catch (IOException e) {
-      LOG.error(e.getMessage(), e);
-    }
-  }
-
-  public void init() throws IOException {
-    if (inited) {
-     throw new IllegalStateException("FileAppender is already initialized.");
-    }
-    inited = true;
-  }
-
-  public void enableStats() {
-    if (inited) {
-      throw new IllegalStateException("Should enable this option before init()");
-    }
-
-    this.enabledStats = true;
-  }
-
-  public long getEstimatedOutputSize() throws IOException {
-    return getOffset();
-  }
-
-  public abstract long getOffset() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
deleted file mode 100644
index 038f0f4..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.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.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.ColumnStats;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-
-public abstract class FileScanner implements Scanner {
-  private static final Log LOG = LogFactory.getLog(FileScanner.class);
-
-  protected boolean inited = false;
-  protected final Configuration conf;
-  protected final TableMeta meta;
-  protected final Schema schema;
-  protected final FileFragment fragment;
-  protected final int columnNum;
-
-  protected Column [] targets;
-
-  protected float progress;
-
-  protected TableStats tableStats;
-
-  public FileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment) {
-    this.conf = conf;
-    this.meta = meta;
-    this.schema = schema;
-    this.fragment = (FileFragment)fragment;
-    this.tableStats = new TableStats();
-    this.columnNum = this.schema.size();
-  }
-
-  public void init() throws IOException {
-    inited = true;
-    progress = 0.0f;
-
-    if (fragment != null) {
-      tableStats.setNumBytes(fragment.getLength());
-      tableStats.setNumBlocks(1);
-    }
-
-    if (schema != null) {
-      for(Column eachColumn: schema.getColumns()) {
-        ColumnStats columnStats = new ColumnStats(eachColumn);
-        tableStats.addColumnStat(columnStats);
-      }
-    }
-  }
-
-  @Override
-  public Schema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public void setTarget(Column[] targets) {
-    if (inited) {
-      throw new IllegalStateException("Should be called before init()");
-    }
-    this.targets = targets;
-  }
-
-  public void setSearchCondition(Object expr) {
-    if (inited) {
-      throw new IllegalStateException("Should be called before init()");
-    }
-  }
-
-  public static FileSystem getFileSystem(TajoConf tajoConf, Path path) throws IOException {
-    String tajoUser = tajoConf.getVar(TajoConf.ConfVars.USERNAME);
-    FileSystem fs;
-    if(tajoUser != null) {
-      try {
-        fs = FileSystem.get(path.toUri(), tajoConf, tajoUser);
-      } catch (InterruptedException e) {
-        LOG.warn("Occur InterruptedException while FileSystem initiating with user[" + tajoUser + "]");
-        fs = FileSystem.get(path.toUri(), tajoConf);
-      }
-    } else {
-      fs = FileSystem.get(path.toUri(), tajoConf);
-    }
-
-    return fs;
-  }
-
-  @Override
-  public float getProgress() {
-    return progress;
-  }
-
-  @Override
-  public TableStats getInputStats() {
-    return tableStats;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/FileStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FileStorageManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/FileStorageManager.java
deleted file mode 100644
index 442ed5e..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FileStorageManager.java
+++ /dev/null
@@ -1,832 +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.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.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.plan.logical.LogicalNode;
-import org.apache.tajo.plan.logical.ScanNode;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.*;
-
-import java.io.IOException;
-import java.text.NumberFormat;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class FileStorageManager extends StorageManager {
-  private final Log LOG = LogFactory.getLog(FileStorageManager.class);
-
-  static final String OUTPUT_FILE_PREFIX="part-";
-  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SUBQUERY =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(2);
-          return fmt;
-        }
-      };
-  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_TASK =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(6);
-          return fmt;
-        }
-      };
-
-  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SEQ =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(3);
-          return fmt;
-        }
-      };
-
-  protected FileSystem fs;
-  protected Path tableBaseDir;
-  protected boolean blocksMetadataEnabled;
-  private static final HdfsVolumeId zeroVolumeId = new HdfsVolumeId(Bytes.toBytes(0));
-
-  public FileStorageManager(StoreType storeType) {
-    super(storeType);
-  }
-
-  @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)
-      LOG.warn("does not support block metadata. ('dfs.datanode.hdfs-blocks-metadata.enabled')");
-  }
-
-  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);
-  }
-
-  public Scanner getFileScanner(TableMeta meta, Schema schema, Path path, FileStatus status)
-      throws IOException {
-    Fragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
-    return getScanner(meta, schema, fragment);
-  }
-
-  public FileSystem getFileSystem() {
-    return this.fs;
-  }
-
-  public Path getWarehouseDir() {
-    return this.tableBaseDir;
-  }
-
-  public void delete(Path tablePath) throws IOException {
-    FileSystem fs = tablePath.getFileSystem(conf);
-    fs.delete(tablePath, true);
-  }
-
-  public boolean exists(Path path) throws IOException {
-    FileSystem fileSystem = path.getFileSystem(conf);
-    return fileSystem.exists(path);
-  }
-
-  /**
-   * This method deletes only data contained in the given path.
-   *
-   * @param path The path in which data are deleted.
-   * @throws IOException
-   */
-  public void deleteData(Path path) throws IOException {
-    FileSystem fileSystem = path.getFileSystem(conf);
-    FileStatus[] fileLists = fileSystem.listStatus(path);
-    for (FileStatus status : fileLists) {
-      fileSystem.delete(status.getPath(), true);
-    }
-  }
-
-  public Path getTablePath(String tableName) {
-    return new Path(tableBaseDir, tableName);
-  }
-
-  @VisibleForTesting
-  public Appender getAppender(TableMeta meta, Schema schema, Path filePath)
-      throws IOException {
-    return getAppender(null, null, meta, schema, filePath);
-  }
-
-  public FileFragment[] split(String tableName) throws IOException {
-    Path tablePath = new Path(tableBaseDir, tableName);
-    return split(tableName, tablePath, fs.getDefaultBlockSize());
-  }
-
-  public FileFragment[] split(String tableName, long fragmentSize) throws IOException {
-    Path tablePath = new Path(tableBaseDir, tableName);
-    return split(tableName, tablePath, fragmentSize);
-  }
-
-  public FileFragment[] splitBroadcastTable(Path tablePath) throws IOException {
-    FileSystem fs = tablePath.getFileSystem(conf);
-    List<FileFragment> listTablets = new ArrayList<FileFragment>();
-    FileFragment tablet;
-
-    FileStatus[] fileLists = fs.listStatus(tablePath);
-    for (FileStatus file : fileLists) {
-      tablet = new FileFragment(tablePath.getName(), file.getPath(), 0, file.getLen());
-      listTablets.add(tablet);
-    }
-
-    FileFragment[] tablets = new FileFragment[listTablets.size()];
-    listTablets.toArray(tablets);
-
-    return tablets;
-  }
-
-  public FileFragment[] split(Path tablePath) throws IOException {
-    FileSystem fs = tablePath.getFileSystem(conf);
-    return split(tablePath.getName(), tablePath, fs.getDefaultBlockSize());
-  }
-
-  public FileFragment[] split(String tableName, Path tablePath) throws IOException {
-    return split(tableName, tablePath, fs.getDefaultBlockSize());
-  }
-
-  private FileFragment[] split(String tableName, Path tablePath, long size)
-      throws IOException {
-    FileSystem fs = tablePath.getFileSystem(conf);
-
-    long defaultBlockSize = size;
-    List<FileFragment> listTablets = new ArrayList<FileFragment>();
-    FileFragment tablet;
-
-    FileStatus[] fileLists = fs.listStatus(tablePath);
-    for (FileStatus file : fileLists) {
-      long remainFileSize = file.getLen();
-      long start = 0;
-      if (remainFileSize > defaultBlockSize) {
-        while (remainFileSize > defaultBlockSize) {
-          tablet = new FileFragment(tableName, file.getPath(), start, defaultBlockSize);
-          listTablets.add(tablet);
-          start += defaultBlockSize;
-          remainFileSize -= defaultBlockSize;
-        }
-        listTablets.add(new FileFragment(tableName, file.getPath(), start, remainFileSize));
-      } else {
-        listTablets.add(new FileFragment(tableName, file.getPath(), 0, remainFileSize));
-      }
-    }
-
-    FileFragment[] tablets = new FileFragment[listTablets.size()];
-    listTablets.toArray(tablets);
-
-    return tablets;
-  }
-
-  public static FileFragment[] splitNG(Configuration conf, String tableName, TableMeta meta,
-                                       Path tablePath, long size)
-      throws IOException {
-    FileSystem fs = tablePath.getFileSystem(conf);
-
-    long defaultBlockSize = size;
-    List<FileFragment> listTablets = new ArrayList<FileFragment>();
-    FileFragment tablet;
-
-    FileStatus[] fileLists = fs.listStatus(tablePath);
-    for (FileStatus file : fileLists) {
-      long remainFileSize = file.getLen();
-      long start = 0;
-      if (remainFileSize > defaultBlockSize) {
-        while (remainFileSize > defaultBlockSize) {
-          tablet = new FileFragment(tableName, file.getPath(), start, defaultBlockSize);
-          listTablets.add(tablet);
-          start += defaultBlockSize;
-          remainFileSize -= defaultBlockSize;
-        }
-        listTablets.add(new FileFragment(tableName, file.getPath(), start, remainFileSize));
-      } else {
-        listTablets.add(new FileFragment(tableName, file.getPath(), 0, remainFileSize));
-      }
-    }
-
-    FileFragment[] tablets = new FileFragment[listTablets.size()];
-    listTablets.toArray(tablets);
-
-    return tablets;
-  }
-
-  public long calculateSize(Path tablePath) throws IOException {
-    FileSystem fs = tablePath.getFileSystem(conf);
-    long totalSize = 0;
-
-    if (fs.exists(tablePath)) {
-      totalSize = fs.getContentSummary(tablePath).getLength();
-    }
-
-    return totalSize;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // FileInputFormat Area
-  /////////////////////////////////////////////////////////////////////////////
-
-  public static final PathFilter hiddenFileFilter = new PathFilter() {
-    public boolean accept(Path p) {
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
-
-  public Path getAppenderFilePath(QueryUnitAttemptId taskAttemptId, Path workDir) {
-    if (taskAttemptId == null) {
-      // For testcase
-      return workDir;
-    }
-    // The final result of a task will be written in a file named part-ss-nnnnnnn,
-    // where ss is the subquery id associated with this task, and nnnnnn is the task id.
-    Path outFilePath = StorageUtil.concatPath(workDir, TajoConstants.RESULT_DIR_NAME,
-        OUTPUT_FILE_PREFIX +
-            OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskAttemptId.getQueryUnitId().getExecutionBlockId().getId()) + "-" +
-            OUTPUT_FILE_FORMAT_TASK.get().format(taskAttemptId.getQueryUnitId().getId()) + "-" +
-            OUTPUT_FILE_FORMAT_SEQ.get().format(0));
-    LOG.info("Output File Path: " + outFilePath);
-
-    return outFilePath;
-  }
-
-  /**
-   * Proxy PathFilter that accepts a path only if all filters given in the
-   * constructor do. Used by the listPaths() to apply the built-in
-   * hiddenFileFilter together with a user provided one (if any).
-   */
-  private static class MultiPathFilter implements PathFilter {
-    private List<PathFilter> filters;
-
-    public MultiPathFilter(List<PathFilter> filters) {
-      this.filters = filters;
-    }
-
-    public boolean accept(Path path) {
-      for (PathFilter filter : filters) {
-        if (!filter.accept(path)) {
-          return false;
-        }
-      }
-      return true;
-    }
-  }
-
-  /**
-   * List input directories.
-   * Subclasses may override to, e.g., select only files matching a regular
-   * expression.
-   *
-   * @return array of FileStatus objects
-   * @throws IOException if zero items.
-   */
-  protected List<FileStatus> listStatus(Path... dirs) throws IOException {
-    List<FileStatus> result = new ArrayList<FileStatus>();
-    if (dirs.length == 0) {
-      throw new IOException("No input paths specified in job");
-    }
-
-    List<IOException> errors = new ArrayList<IOException>();
-
-    // creates a MultiPathFilter with the hiddenFileFilter and the
-    // user provided one (if any).
-    List<PathFilter> filters = new ArrayList<PathFilter>();
-    filters.add(hiddenFileFilter);
-
-    PathFilter inputFilter = new MultiPathFilter(filters);
-
-    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));
-      } else if (matches.length == 0) {
-        errors.add(new IOException("Input Pattern " + p + " matches 0 files"));
-      } else {
-        for (FileStatus globStat : matches) {
-          if (globStat.isDirectory()) {
-            for (FileStatus stat : fs.listStatus(globStat.getPath(),
-                inputFilter)) {
-              result.add(stat);
-            }
-          } else {
-            result.add(globStat);
-          }
-        }
-      }
-    }
-
-    if (!errors.isEmpty()) {
-      throw new InvalidInputException(errors);
-    }
-    LOG.info("Total input paths to process : " + result.size());
-    return result;
-  }
-
-  /**
-   * Is the given filename splitable? Usually, true, but if the file is
-   * stream compressed, it will not be.
-   * <p/>
-   * <code>FileInputFormat</code> implementations can override this and return
-   * <code>false</code> to ensure that individual input files are never split-up
-   * so that Mappers process entire files.
-   *
-   *
-   * @param path the file name to check
-   * @param status get the file length
-   * @return is this file isSplittable?
-   */
-  protected boolean isSplittable(TableMeta meta, Schema schema, Path path, FileStatus status) throws IOException {
-    Scanner scanner = getFileScanner(meta, schema, path, status);
-    boolean split = scanner.isSplittable();
-    scanner.close();
-    return split;
-  }
-
-  private static final double SPLIT_SLOP = 1.1;   // 10% slop
-
-  protected int getBlockIndex(BlockLocation[] blkLocations,
-                              long offset) {
-    for (int i = 0; i < blkLocations.length; i++) {
-      // is the offset inside this block?
-      if ((blkLocations[i].getOffset() <= offset) &&
-          (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())) {
-        return i;
-      }
-    }
-    BlockLocation last = blkLocations[blkLocations.length - 1];
-    long fileLength = last.getOffset() + last.getLength() - 1;
-    throw new IllegalArgumentException("Offset " + offset +
-        " is outside of file (0.." +
-        fileLength + ")");
-  }
-
-  /**
-   * A factory that makes the split for this class. It can be overridden
-   * by sub-classes to make sub-types
-   */
-  protected FileFragment makeSplit(String fragmentId, Path file, long start, long length) {
-    return new FileFragment(fragmentId, file, start, length);
-  }
-
-  protected FileFragment makeSplit(String fragmentId, Path file, long start, long length,
-                                   String[] hosts) {
-    return new FileFragment(fragmentId, file, start, length, hosts);
-  }
-
-  protected FileFragment makeSplit(String fragmentId, Path file, BlockLocation blockLocation)
-      throws IOException {
-    return new FileFragment(fragmentId, file, blockLocation);
-  }
-
-  // for Non Splittable. eg, compressed gzip TextFile
-  protected FileFragment makeNonSplit(String fragmentId, Path file, long start, long length,
-                                      BlockLocation[] blkLocations) throws IOException {
-
-    Map<String, Integer> hostsBlockMap = new HashMap<String, Integer>();
-    for (BlockLocation blockLocation : blkLocations) {
-      for (String host : blockLocation.getHosts()) {
-        if (hostsBlockMap.containsKey(host)) {
-          hostsBlockMap.put(host, hostsBlockMap.get(host) + 1);
-        } else {
-          hostsBlockMap.put(host, 1);
-        }
-      }
-    }
-
-    List<Map.Entry<String, Integer>> entries = new ArrayList<Map.Entry<String, Integer>>(hostsBlockMap.entrySet());
-    Collections.sort(entries, new Comparator<Map.Entry<String, Integer>>() {
-
-      @Override
-      public int compare(Map.Entry<String, Integer> v1, Map.Entry<String, Integer> v2) {
-        return v1.getValue().compareTo(v2.getValue());
-      }
-    });
-
-    String[] hosts = new String[blkLocations[0].getHosts().length];
-
-    for (int i = 0; i < hosts.length; i++) {
-      Map.Entry<String, Integer> entry = entries.get((entries.size() - 1) - i);
-      hosts[i] = entry.getKey();
-    }
-    return new FileFragment(fragmentId, file, start, length, hosts);
-  }
-
-  /**
-   * Get the minimum split size
-   *
-   * @return the minimum number of bytes that can be in a split
-   */
-  public long getMinSplitSize() {
-    return conf.getLongVar(TajoConf.ConfVars.MINIMUM_SPLIT_SIZE);
-  }
-
-  /**
-   * Get Disk Ids by Volume Bytes
-   */
-  private int[] getDiskIds(VolumeId[] volumeIds) {
-    int[] diskIds = new int[volumeIds.length];
-    for (int i = 0; i < volumeIds.length; i++) {
-      int diskId = -1;
-      if (volumeIds[i] != null && volumeIds[i].hashCode() > 0) {
-        diskId = volumeIds[i].hashCode() - zeroVolumeId.hashCode();
-      }
-      diskIds[i] = diskId;
-    }
-    return diskIds;
-  }
-
-  /**
-   * 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
-   */
-  public List<Fragment> getSplits(String tableName, TableMeta meta, Schema schema, Path... inputs)
-      throws IOException {
-    // generate splits'
-
-    List<Fragment> splits = Lists.newArrayList();
-    List<Fragment> volumeSplits = Lists.newArrayList();
-    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)));
-      } else {
-        files.addAll(listStatus(p));
-      }
-
-      int previousSplitSize = splits.size();
-      for (FileStatus file : files) {
-        Path path = file.getPath();
-        long length = file.getLen();
-        if (length > 0) {
-          // Get locations of blocks of file
-          BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
-          boolean splittable = isSplittable(meta, schema, path, file);
-          if (blocksMetadataEnabled && fs instanceof DistributedFileSystem) {
-
-            if (splittable) {
-              for (BlockLocation blockLocation : blkLocations) {
-                volumeSplits.add(makeSplit(tableName, path, blockLocation));
-              }
-              blockLocations.addAll(Arrays.asList(blkLocations));
-
-            } else { // Non splittable
-              long blockSize = blkLocations[0].getLength();
-              if (blockSize >= length) {
-                blockLocations.addAll(Arrays.asList(blkLocations));
-                for (BlockLocation blockLocation : blkLocations) {
-                  volumeSplits.add(makeSplit(tableName, path, blockLocation));
-                }
-              } else {
-                splits.add(makeNonSplit(tableName, path, 0, length, blkLocations));
-              }
-            }
-
-          } else {
-            if (splittable) {
-
-              long minSize = Math.max(getMinSplitSize(), 1);
-
-              long blockSize = file.getBlockSize(); // s3n rest api contained block size but blockLocations is one
-              long splitSize = Math.max(minSize, blockSize);
-              long bytesRemaining = length;
-
-              // for s3
-              while (((double) bytesRemaining) / splitSize > SPLIT_SLOP) {
-                int blkIndex = getBlockIndex(blkLocations, length - bytesRemaining);
-                splits.add(makeSplit(tableName, path, length - bytesRemaining, splitSize,
-                    blkLocations[blkIndex].getHosts()));
-                bytesRemaining -= splitSize;
-              }
-              if (bytesRemaining > 0) {
-                int blkIndex = getBlockIndex(blkLocations, length - bytesRemaining);
-                splits.add(makeSplit(tableName, path, length - bytesRemaining, bytesRemaining,
-                    blkLocations[blkIndex].getHosts()));
-              }
-            } else { // Non splittable
-              splits.add(makeNonSplit(tableName, path, 0, length, blkLocations));
-            }
-          }
-        } else {
-          //for zero length files
-          splits.add(makeSplit(tableName, path, 0, length));
-        }
-      }
-      if(LOG.isDebugEnabled()){
-        LOG.debug("# of splits per partition: " + (splits.size() - previousSplitSize));
-      }
-    }
-
-    // Combine original fileFragments with new VolumeId information
-    setVolumeMeta(volumeSplits, blockLocations);
-    splits.addAll(volumeSplits);
-    LOG.info("Total # of splits: " + splits.size());
-    return splits;
-  }
-
-  private void setVolumeMeta(List<Fragment> splits, final List<BlockLocation> blockLocations)
-      throws IOException {
-
-    int locationSize = blockLocations.size();
-    int splitSize = splits.size();
-    if (locationSize == 0 || splitSize == 0) return;
-
-    if (locationSize != splitSize) {
-      // splits and locations don't match up
-      LOG.warn("Number of block locations not equal to number of splits: "
-          + "#locations=" + locationSize
-          + " #splits=" + splitSize);
-      return;
-    }
-
-    DistributedFileSystem fs = (DistributedFileSystem)DistributedFileSystem.get(conf);
-    int lsLimit = conf.getInt(DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
-    int blockLocationIdx = 0;
-
-    Iterator<Fragment> iter = splits.iterator();
-    while (locationSize > blockLocationIdx) {
-
-      int subSize = Math.min(locationSize - blockLocationIdx, lsLimit);
-      List<BlockLocation> locations = blockLocations.subList(blockLocationIdx, blockLocationIdx + subSize);
-      //BlockStorageLocation containing additional volume location information for each replica of each block.
-      BlockStorageLocation[] blockStorageLocations = fs.getFileBlockStorageLocations(locations);
-
-      for (BlockStorageLocation blockStorageLocation : blockStorageLocations) {
-        ((FileFragment)iter.next()).setDiskIds(getDiskIds(blockStorageLocation.getVolumeIds()));
-        blockLocationIdx++;
-      }
-    }
-    LOG.info("# of splits with volumeId " + splitSize);
-  }
-
-  private static class InvalidInputException extends IOException {
-    List<IOException> errors;
-    public InvalidInputException(List<IOException> errors) {
-      this.errors = errors;
-    }
-
-    @Override
-    public String getMessage(){
-      StringBuffer sb = new StringBuffer();
-      int messageLimit = Math.min(errors.size(), 10);
-      for (int i = 0; i < messageLimit ; i ++) {
-        sb.append(errors.get(i).getMessage()).append("\n");
-      }
-
-      if(messageLimit < errors.size())
-        sb.append("skipped .....").append("\n");
-
-      return sb.toString();
-    }
-  }
-
-  @Override
-  public List<Fragment> getSplits(String tableName, TableDesc table, ScanNode scanNode) throws IOException {
-    return getSplits(tableName, table.getMeta(), table.getSchema(), new Path(table.getPath()));
-  }
-
-  @Override
-  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
-    if (!tableDesc.isExternal()) {
-      String [] splitted = CatalogUtil.splitFQTableName(tableDesc.getName());
-      String databaseName = splitted[0];
-      String simpleTableName = splitted[1];
-
-      // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} )
-      Path tablePath = StorageUtil.concatPath(tableBaseDir, databaseName, simpleTableName);
-      tableDesc.setPath(tablePath.toUri());
-    } else {
-      Preconditions.checkState(tableDesc.getPath() != null, "ERROR: LOCATION must be given.");
-    }
-
-    Path path = new Path(tableDesc.getPath());
-
-    FileSystem fs = path.getFileSystem(conf);
-    TableStats stats = new TableStats();
-    if (tableDesc.isExternal()) {
-      if (!fs.exists(path)) {
-        LOG.error(path.toUri() + " does not exist");
-        throw new IOException("ERROR: " + path.toUri() + " does not exist");
-      }
-    } else {
-      fs.mkdirs(path);
-    }
-
-    long totalSize = 0;
-
-    try {
-      totalSize = calculateSize(path);
-    } catch (IOException e) {
-      LOG.warn("Cannot calculate the size of the relation", e);
-    }
-
-    stats.setNumBytes(totalSize);
-
-    if (tableDesc.isExternal()) { // if it is an external table, there is no way to know the exact row number without processing.
-      stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
-    }
-
-    tableDesc.setStats(stats);
-  }
-
-  @Override
-  public void purgeTable(TableDesc tableDesc) throws IOException {
-    try {
-      Path path = new Path(tableDesc.getPath());
-      FileSystem fs = path.getFileSystem(conf);
-      LOG.info("Delete table data dir: " + path);
-      fs.delete(path, true);
-    } catch (IOException e) {
-      throw new InternalError(e.getMessage());
-    }
-  }
-
-  @Override
-  public List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments) throws IOException {
-    // Listing table data file which is not empty.
-    // If the table is a partitioned table, return file list which has same partition key.
-    Path tablePath = new Path(tableDesc.getPath());
-    FileSystem fs = tablePath.getFileSystem(conf);
-
-    List<FileStatus> nonZeroLengthFiles = new ArrayList<FileStatus>();
-    if (fs.exists(tablePath)) {
-      getNonZeroLengthDataFiles(fs, tablePath, nonZeroLengthFiles, currentPage, numFragments,
-          new AtomicInteger(0));
-    }
-
-    List<Fragment> fragments = new ArrayList<Fragment>();
-
-    //In the case of partitioned table, return same partition key data files.
-    int numPartitionColumns = 0;
-    if (tableDesc.hasPartition()) {
-      numPartitionColumns = tableDesc.getPartitionMethod().getExpressionSchema().getColumns().size();
-    }
-    String[] previousPartitionPathNames = null;
-    for (FileStatus eachFile: nonZeroLengthFiles) {
-      FileFragment fileFragment = new FileFragment(tableDesc.getName(), eachFile.getPath(), 0, eachFile.getLen(), null);
-
-      if (numPartitionColumns > 0) {
-        // finding partition key;
-        Path filePath = fileFragment.getPath();
-        Path parentPath = filePath;
-        String[] parentPathNames = new String[numPartitionColumns];
-        for (int i = 0; i < numPartitionColumns; i++) {
-          parentPath = parentPath.getParent();
-          parentPathNames[numPartitionColumns - i - 1] = parentPath.getName();
-        }
-
-        // If current partitionKey == previousPartitionKey, add to result.
-        if (previousPartitionPathNames == null) {
-          fragments.add(fileFragment);
-        } else if (previousPartitionPathNames != null && Arrays.equals(previousPartitionPathNames, parentPathNames)) {
-          fragments.add(fileFragment);
-        } else {
-          break;
-        }
-        previousPartitionPathNames = parentPathNames;
-      } else {
-        fragments.add(fileFragment);
-      }
-    }
-
-    return fragments;
-  }
-
-  private void getNonZeroLengthDataFiles(FileSystem fs, Path path, List<FileStatus> result,
-                                                int startFileIndex, int numResultFiles,
-                                                AtomicInteger currentFileIndex) throws IOException {
-    if (fs.isDirectory(path)) {
-      FileStatus[] files = fs.listStatus(path, FileStorageManager.hiddenFileFilter);
-      if (files != null && files.length > 0) {
-        for (FileStatus eachFile : files) {
-          if (result.size() >= numResultFiles) {
-            return;
-          }
-          if (eachFile.isDirectory()) {
-            getNonZeroLengthDataFiles(fs, eachFile.getPath(), result, startFileIndex, numResultFiles,
-                currentFileIndex);
-          } else if (eachFile.isFile() && eachFile.getLen() > 0) {
-            if (currentFileIndex.get() >= startFileIndex) {
-              result.add(eachFile);
-            }
-            currentFileIndex.incrementAndGet();
-          }
-        }
-      }
-    } else {
-      FileStatus fileStatus = fs.getFileStatus(path);
-      if (fileStatus != null && fileStatus.getLen() > 0) {
-        if (currentFileIndex.get() >= startFileIndex) {
-          result.add(fileStatus);
-        }
-        currentFileIndex.incrementAndGet();
-        if (result.size() >= numResultFiles) {
-          return;
-        }
-      }
-    }
-  }
-
-  @Override
-  public StorageProperty getStorageProperty() {
-    StorageProperty storageProperty = new StorageProperty();
-    storageProperty.setSortedInsert(false);
-    if (storeType == StoreType.RAW) {
-      storageProperty.setSupportsInsertInto(false);
-    } else {
-      storageProperty.setSupportsInsertInto(true);
-    }
-
-    return storageProperty;
-  }
-
-  @Override
-  public void closeStorageManager() {
-  }
-
-  @Override
-  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
-  }
-
-  @Override
-  public void rollbackOutputCommit(LogicalNode node) throws IOException {
-  }
-
-  @Override
-  public TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
-                                          Schema inputSchema, SortSpec[] sortSpecs, TupleRange dataRange)
-      throws IOException {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java b/tajo-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
deleted file mode 100644
index 8b7e2e0..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
+++ /dev/null
@@ -1,225 +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.base.Preconditions;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.IntervalDatum;
-import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.exception.UnsupportedException;
-
-/**
- * An instance of FrameTuple is an immutable tuple.
- * It contains two tuples and pretends to be one instance of Tuple for
- * join qual evaluatations.
- */
-public class FrameTuple implements Tuple, Cloneable {
-  private int size;
-  private int leftSize;
-  
-  private Tuple left;
-  private Tuple right;
-  
-  public FrameTuple() {}
-  
-  public FrameTuple(Tuple left, Tuple right) {
-    set(left, right);
-  }
-  
-  public void set(Tuple left, Tuple right) {
-    this.size = left.size() + right.size();
-    this.left = left;
-    this.leftSize = left.size();
-    this.right = right;
-  }
-
-  @Override
-  public int size() {
-    return size;
-  }
-
-  @Override
-  public boolean contains(int fieldId) {
-    Preconditions.checkArgument(fieldId < size, 
-        "Out of field access: " + fieldId);
-    
-    if (fieldId < leftSize) {
-      return left.contains(fieldId);
-    } else {
-      return right.contains(fieldId - leftSize);
-    }
-  }
-
-  @Override
-  public boolean isNull(int fieldid) {
-    return get(fieldid).isNull();
-  }
-
-  @Override
-  public boolean isNotNull(int fieldid) {
-    return !isNull(fieldid);
-  }
-
-  @Override
-  public void clear() {
-    throw new UnsupportedException();
-  }
-
-  @Override
-  public void put(int fieldId, Datum value) {
-    throw new UnsupportedException();
-  }
-
-  @Override
-  public void put(int fieldId, Datum[] values) {
-    throw new UnsupportedException();
-  }
-
-  @Override
-  public void put(int fieldId, Tuple tuple) {
-    throw new UnsupportedException();
-  }
-
-  @Override
-  public void setOffset(long offset) {
-    throw new UnsupportedException();
-  }
-  
-  @Override
-  public long getOffset() {
-    throw new UnsupportedException();
-  }
-
-  @Override
-  public void put(Datum [] values) {
-    throw new UnsupportedException();
-  }
-
-  @Override
-  public Datum get(int fieldId) {
-    Preconditions.checkArgument(fieldId < size, 
-        "Out of field access: " + fieldId);
-    
-    if (fieldId < leftSize) {
-      return left.get(fieldId);
-    } else {
-      return right.get(fieldId - leftSize);
-    }
-  }
-
-  @Override
-  public boolean getBool(int fieldId) {
-    return get(fieldId).asBool();
-  }
-
-  @Override
-  public byte getByte(int fieldId) {
-    return get(fieldId).asByte();
-  }
-
-  @Override
-  public char getChar(int fieldId) {
-    return get(fieldId).asChar();
-  }
-
-  @Override
-  public byte [] getBytes(int fieldId) {
-    return get(fieldId).asByteArray();
-  }
-
-  @Override
-  public short getInt2(int fieldId) {
-    return get(fieldId).asInt2();
-  }
-
-  @Override
-  public int getInt4(int fieldId) {
-    return get(fieldId).asInt4();
-  }
-
-  @Override
-  public long getInt8(int fieldId) {
-    return get(fieldId).asInt8();
-  }
-
-  @Override
-  public float getFloat4(int fieldId) {
-    return get(fieldId).asFloat4();
-  }
-
-  @Override
-  public double getFloat8(int fieldId) {
-    return get(fieldId).asFloat8();
-  }
-
-  @Override
-  public String getText(int fieldId) {
-    return get(fieldId).asChars();
-  }
-
-  @Override
-  public ProtobufDatum getProtobufDatum(int fieldId) {
-    return (ProtobufDatum) get(fieldId);
-  }
-
-  @Override
-  public IntervalDatum getInterval(int fieldId) {
-    return (IntervalDatum) get(fieldId);
-  }
-
-  @Override
-  public char [] getUnicodeChars(int fieldId) {
-    return get(fieldId).asUnicodeChars();
-  }
-
-  @Override
-  public Tuple clone() throws CloneNotSupportedException {
-    FrameTuple frameTuple = (FrameTuple) super.clone();
-    frameTuple.set(this.left.clone(), this.right.clone());
-    return frameTuple;
-  }
-
-  @Override
-  public Datum[] getValues(){
-    throw new UnsupportedException();
-  }
-
-  public String toString() {
-    boolean first = true;
-    StringBuilder str = new StringBuilder();
-    str.append("(");
-    for(int i=0; i < size(); i++) {      
-      if(contains(i)) {
-        if(first) {
-          first = false;
-        } else {
-          str.append(", ");
-        }
-        str.append(i)
-        .append("=>")
-        .append(get(i));
-      }
-    }
-    str.append(")");
-    return str.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
deleted file mode 100644
index 40cad32..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.util.Pair;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class HashShuffleAppender implements Appender {
-  private static Log LOG = LogFactory.getLog(HashShuffleAppender.class);
-
-  private FileAppender appender;
-  private AtomicBoolean closed = new AtomicBoolean(false);
-  private int partId;
-
-  private TableStats tableStats;
-
-  //<taskId,<page start offset,<task start, task end>>>
-  private Map<QueryUnitAttemptId, List<Pair<Long, Pair<Integer, Integer>>>> taskTupleIndexes;
-
-  //page start offset, length
-  private List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
-
-  private Pair<Long, Integer> currentPage;
-
-  private int pageSize; //MB
-
-  private int rowNumInPage;
-
-  private int totalRows;
-
-  private long offset;
-
-  private ExecutionBlockId ebId;
-
-  public HashShuffleAppender(ExecutionBlockId ebId, int partId, int pageSize, FileAppender appender) {
-    this.ebId = ebId;
-    this.partId = partId;
-    this.appender = appender;
-    this.pageSize = pageSize;
-  }
-
-  @Override
-  public void init() throws IOException {
-    currentPage = new Pair(0L, 0);
-    taskTupleIndexes = new HashMap<QueryUnitAttemptId, List<Pair<Long, Pair<Integer, Integer>>>>();
-    rowNumInPage = 0;
-  }
-
-  /**
-   * Write multiple tuples. Each tuple is written by a FileAppender which is responsible specified partition.
-   * After writing if a current page exceeds pageSize, pageOffset will be added.
-   * @param taskId
-   * @param tuples
-   * @return written bytes
-   * @throws IOException
-   */
-  public int addTuples(QueryUnitAttemptId taskId, List<Tuple> tuples) throws IOException {
-    synchronized(appender) {
-      if (closed.get()) {
-        return 0;
-      }
-      long currentPos = appender.getOffset();
-
-      for (Tuple eachTuple: tuples) {
-        appender.addTuple(eachTuple);
-      }
-      long posAfterWritten = appender.getOffset();
-
-      int writtenBytes = (int)(posAfterWritten - currentPos);
-
-      int nextRowNum = rowNumInPage + tuples.size();
-      List<Pair<Long, Pair<Integer, Integer>>> taskIndexes = taskTupleIndexes.get(taskId);
-      if (taskIndexes == null) {
-        taskIndexes = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
-        taskTupleIndexes.put(taskId, taskIndexes);
-      }
-      taskIndexes.add(
-          new Pair<Long, Pair<Integer, Integer>>(currentPage.getFirst(), new Pair(rowNumInPage, nextRowNum)));
-      rowNumInPage = nextRowNum;
-
-      if (posAfterWritten - currentPage.getFirst() > pageSize) {
-        nextPage(posAfterWritten);
-        rowNumInPage = 0;
-      }
-
-      totalRows += tuples.size();
-      return writtenBytes;
-    }
-  }
-
-  public long getOffset() throws IOException {
-    if (closed.get()) {
-      return offset;
-    } else {
-      return appender.getOffset();
-    }
-  }
-
-  private void nextPage(long pos) {
-    currentPage.setSecond((int) (pos - currentPage.getFirst()));
-    pages.add(currentPage);
-    currentPage = new Pair(pos, 0);
-  }
-
-  @Override
-  public void addTuple(Tuple t) throws IOException {
-    throw new IOException("Not support addTuple, use addTuples()");
-  }
-
-  @Override
-  public void flush() throws IOException {
-    synchronized(appender) {
-      if (closed.get()) {
-        return;
-      }
-      appender.flush();
-    }
-  }
-
-  @Override
-  public long getEstimatedOutputSize() throws IOException {
-    return pageSize * pages.size();
-  }
-
-  @Override
-  public void close() throws IOException {
-    synchronized(appender) {
-      if (closed.get()) {
-        return;
-      }
-      appender.flush();
-      offset = appender.getOffset();
-      if (offset > currentPage.getFirst()) {
-        nextPage(offset);
-      }
-      appender.close();
-      if (LOG.isDebugEnabled()) {
-        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=" + pages.size());
-        }
-      }
-      closed.set(true);
-      tableStats = appender.getStats();
-    }
-  }
-
-  @Override
-  public void enableStats() {
-  }
-
-  @Override
-  public TableStats getStats() {
-    synchronized(appender) {
-      return appender.getStats();
-    }
-  }
-
-  public List<Pair<Long, Integer>> getPages() {
-    return pages;
-  }
-
-  public Map<QueryUnitAttemptId, List<Pair<Long, Pair<Integer, Integer>>>> getTaskTupleIndexes() {
-    return taskTupleIndexes;
-  }
-
-  public List<Pair<Long, Pair<Integer, Integer>>> getMergedTupleIndexes() {
-    List<Pair<Long, Pair<Integer, Integer>>> merged = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
-
-    for (List<Pair<Long, Pair<Integer, Integer>>> eachFailureIndex: taskTupleIndexes.values()) {
-      merged.addAll(eachFailureIndex);
-    }
-
-    return merged;
-  }
-
-  public void taskFinished(QueryUnitAttemptId taskId) {
-    taskTupleIndexes.remove(taskId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
deleted file mode 100644
index 33a9233..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryUnitAttemptId;
-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;
-
-public class HashShuffleAppenderManager {
-  private static final Log LOG = LogFactory.getLog(HashShuffleAppenderManager.class);
-
-  private Map<ExecutionBlockId, Map<Integer, PartitionAppenderMeta>> appenderMap =
-      new ConcurrentHashMap<ExecutionBlockId, Map<Integer, PartitionAppenderMeta>>();
-  private TajoConf systemConf;
-  private FileSystem defaultFS;
-  private FileSystem localFS;
-  private LocalDirAllocator lDirAllocator;
-  private int pageSize;
-
-  public HashShuffleAppenderManager(TajoConf systemConf) throws IOException {
-    this.systemConf = systemConf;
-
-    // initialize LocalDirAllocator
-    lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
-
-    // initialize DFS and LocalFileSystems
-    defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf);
-    localFS = FileSystem.getLocal(systemConf);
-    pageSize = systemConf.getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME) * 1024 * 1024;
-  }
-
-  public HashShuffleAppender getAppender(TajoConf tajoConf, ExecutionBlockId ebId, int partId,
-                              TableMeta meta, Schema outSchema) throws IOException {
-    synchronized (appenderMap) {
-      Map<Integer, PartitionAppenderMeta> partitionAppenderMap = appenderMap.get(ebId);
-
-      if (partitionAppenderMap == null) {
-        partitionAppenderMap = new ConcurrentHashMap<Integer, PartitionAppenderMeta>();
-        appenderMap.put(ebId, partitionAppenderMap);
-      }
-
-      PartitionAppenderMeta partitionAppenderMeta = partitionAppenderMap.get(partId);
-      if (partitionAppenderMeta == null) {
-        Path dataFile = getDataFile(ebId, partId);
-        FileSystem fs = dataFile.getFileSystem(systemConf);
-        if (fs.exists(dataFile)) {
-          FileStatus status = fs.getFileStatus(dataFile);
-          LOG.info("File " + dataFile + " already exists, size=" + status.getLen());
-        }
-
-        if (!fs.exists(dataFile.getParent())) {
-          fs.mkdirs(dataFile.getParent());
-        }
-        FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(
-            tajoConf, null).getAppender(meta, outSchema, dataFile);
-        appender.enableStats();
-        appender.init();
-
-        partitionAppenderMeta = new PartitionAppenderMeta();
-        partitionAppenderMeta.partId = partId;
-        partitionAppenderMeta.dataFile = dataFile;
-        partitionAppenderMeta.appender = new HashShuffleAppender(ebId, partId, pageSize, appender);
-        partitionAppenderMeta.appender.init();
-        partitionAppenderMap.put(partId, partitionAppenderMeta);
-
-        LOG.info("Create Hash shuffle file(partId=" + partId + "): " + dataFile);
-      }
-
-      return partitionAppenderMeta.appender;
-    }
-  }
-
-  public static int getPartParentId(int partId, TajoConf tajoConf) {
-    return partId % tajoConf.getIntVar(TajoConf.ConfVars.HASH_SHUFFLE_PARENT_DIRS);
-  }
-
-  private Path getDataFile(ExecutionBlockId ebId, int partId) throws IOException {
-    try {
-      // the base dir for an output dir
-      String executionBlockBaseDir = ebId.getQueryId().toString() + "/output" + "/" + ebId.getId() + "/hash-shuffle";
-      Path baseDirPath = localFS.makeQualified(lDirAllocator.getLocalPathForWrite(executionBlockBaseDir, systemConf));
-      //LOG.info(ebId + "'s basedir is created (" + baseDirPath + ")");
-
-      // If EB has many partition, too many shuffle file are in single directory.
-      return StorageUtil.concatPath(baseDirPath, "" + getPartParentId(partId, systemConf), "" + partId);
-    } catch (Exception e) {
-      LOG.error(e.getMessage(), e);
-      throw new IOException(e);
-    }
-  }
-
-  public List<HashShuffleIntermediate> close(ExecutionBlockId ebId) throws IOException {
-    Map<Integer, PartitionAppenderMeta> partitionAppenderMap = null;
-    synchronized (appenderMap) {
-      partitionAppenderMap = appenderMap.remove(ebId);
-    }
-
-    if (partitionAppenderMap == null) {
-      LOG.info("Close HashShuffleAppender:" + ebId + ", not a hash shuffle");
-      return null;
-    }
-
-    // Send Intermediate data to QueryMaster.
-    List<HashShuffleIntermediate> intermEntries = new ArrayList<HashShuffleIntermediate>();
-    for (PartitionAppenderMeta eachMeta : partitionAppenderMap.values()) {
-      try {
-        eachMeta.appender.close();
-        HashShuffleIntermediate intermediate =
-            new HashShuffleIntermediate(eachMeta.partId, eachMeta.appender.getOffset(),
-                eachMeta.appender.getPages(),
-                eachMeta.appender.getMergedTupleIndexes());
-        intermEntries.add(intermediate);
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-        throw e;
-      }
-    }
-
-    LOG.info("Close HashShuffleAppender:" + ebId + ", intermediates=" + intermEntries.size());
-
-    return intermEntries;
-  }
-
-  public void finalizeTask(QueryUnitAttemptId taskId) {
-    synchronized (appenderMap) {
-      Map<Integer, PartitionAppenderMeta> partitionAppenderMap =
-        appenderMap.get(taskId.getQueryUnitId().getExecutionBlockId());
-      if (partitionAppenderMap == null) {
-        return;
-      }
-
-      for (PartitionAppenderMeta eachAppender: partitionAppenderMap.values()) {
-        eachAppender.appender.taskFinished(taskId);
-      }
-    }
-  }
-
-  public static class HashShuffleIntermediate {
-    private int partId;
-
-    private long volume;
-
-    //[<page start offset,<task start, task end>>]
-    private Collection<Pair<Long, Pair<Integer, Integer>>> failureTskTupleIndexes;
-
-    //[<page start offset, length>]
-    private List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
-
-    public HashShuffleIntermediate(int partId, long volume,
-                                   List<Pair<Long, Integer>> pages,
-                                   Collection<Pair<Long, Pair<Integer, Integer>>> failureTskTupleIndexes) {
-      this.partId = partId;
-      this.volume = volume;
-      this.failureTskTupleIndexes = failureTskTupleIndexes;
-      this.pages = pages;
-    }
-
-    public int getPartId() {
-      return partId;
-    }
-
-    public long getVolume() {
-      return volume;
-    }
-
-    public Collection<Pair<Long, Pair<Integer, Integer>>> getFailureTskTupleIndexes() {
-      return failureTskTupleIndexes;
-    }
-
-    public List<Pair<Long, Integer>> getPages() {
-      return pages;
-    }
-  }
-
-  static class PartitionAppenderMeta {
-    int partId;
-    HashShuffleAppender appender;
-    Path dataFile;
-
-    public int getPartId() {
-      return partId;
-    }
-
-    public HashShuffleAppender getAppender() {
-      return appender;
-    }
-
-    public Path getDataFile() {
-      return dataFile;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java b/tajo-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
deleted file mode 100644
index bfbe478..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.IntervalDatum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.exception.UnsupportedException;
-
-import java.util.Arrays;
-
-public class LazyTuple implements Tuple, Cloneable {
-  private long offset;
-  private Datum[] values;
-  private byte[][] textBytes;
-  private Schema schema;
-  private byte[] nullBytes;
-  private SerializerDeserializer serializeDeserialize;
-
-  public LazyTuple(Schema schema, byte[][] textBytes, long offset) {
-    this(schema, textBytes, offset, NullDatum.get().asTextBytes(), new TextSerializerDeserializer());
-  }
-
-  public LazyTuple(Schema schema, byte[][] textBytes, long offset, byte[] nullBytes, SerializerDeserializer serde) {
-    this.schema = schema;
-    this.textBytes = textBytes;
-    this.values = new Datum[schema.size()];
-    this.offset = offset;
-    this.nullBytes = nullBytes;
-    this.serializeDeserialize = serde;
-  }
-
-  public LazyTuple(LazyTuple tuple) {
-    this.values = tuple.getValues();
-    this.offset = tuple.offset;
-    this.schema = tuple.schema;
-    this.textBytes = new byte[size()][];
-    this.nullBytes = tuple.nullBytes;
-    this.serializeDeserialize = tuple.serializeDeserialize;
-  }
-
-  @Override
-  public int size() {
-    return values.length;
-  }
-
-  @Override
-  public boolean contains(int fieldid) {
-    return textBytes[fieldid] != null || values[fieldid] != null;
-  }
-
-  @Override
-  public boolean isNull(int fieldid) {
-    return get(fieldid).isNull();
-  }
-
-  @Override
-  public boolean isNotNull(int fieldid) {
-    return !isNull(fieldid);
-  }
-
-  @Override
-  public void clear() {
-    for (int i = 0; i < values.length; i++) {
-      values[i] = null;
-      textBytes[i] = null;
-    }
-  }
-
-  //////////////////////////////////////////////////////
-  // Setter
-  //////////////////////////////////////////////////////
-  @Override
-  public void put(int fieldId, Datum value) {
-    values[fieldId] = value;
-    textBytes[fieldId] = null;
-  }
-
-  @Override
-  public void put(int fieldId, Datum[] values) {
-    for (int i = fieldId, j = 0; j < values.length; i++, j++) {
-      this.values[i] = values[j];
-    }
-    this.textBytes = new byte[values.length][];
-  }
-
-  @Override
-  public void put(int fieldId, Tuple tuple) {
-    for (int i = fieldId, j = 0; j < tuple.size(); i++, j++) {
-      values[i] = tuple.get(j);
-      textBytes[i] = null;
-    }
-  }
-
-  @Override
-  public void put(Datum[] values) {
-    System.arraycopy(values, 0, this.values, 0, size());
-    this.textBytes = new byte[values.length][];
-  }
-
-  //////////////////////////////////////////////////////
-  // Getter
-  //////////////////////////////////////////////////////
-  @Override
-  public Datum get(int fieldId) {
-    if (values[fieldId] != null)
-      return values[fieldId];
-    else if (textBytes.length <= fieldId) {
-      values[fieldId] = NullDatum.get();  // split error. (col : 3, separator: ',', row text: "a,")
-    } else if (textBytes[fieldId] != null) {
-      try {
-        values[fieldId] = serializeDeserialize.deserialize(schema.getColumn(fieldId),
-            textBytes[fieldId], 0, textBytes[fieldId].length, nullBytes);
-      } catch (Exception e) {
-        values[fieldId] = NullDatum.get();
-      }
-      textBytes[fieldId] = null;
-    } else {
-      //non-projection
-    }
-    return values[fieldId];
-  }
-
-  @Override
-  public void setOffset(long offset) {
-    this.offset = offset;
-  }
-
-  @Override
-  public long getOffset() {
-    return this.offset;
-  }
-
-  @Override
-  public boolean getBool(int fieldId) {
-    return get(fieldId).asBool();
-  }
-
-  @Override
-  public byte getByte(int fieldId) {
-    return get(fieldId).asByte();
-  }
-
-  @Override
-  public char getChar(int fieldId) {
-    return get(fieldId).asChar();
-  }
-
-  @Override
-  public byte [] getBytes(int fieldId) {
-    return get(fieldId).asByteArray();
-  }
-
-  @Override
-  public short getInt2(int fieldId) {
-    return get(fieldId).asInt2();
-  }
-
-  @Override
-  public int getInt4(int fieldId) {
-    return get(fieldId).asInt4();
-  }
-
-  @Override
-  public long getInt8(int fieldId) {
-    return get(fieldId).asInt8();
-  }
-
-  @Override
-  public float getFloat4(int fieldId) {
-    return get(fieldId).asFloat4();
-  }
-
-  @Override
-  public double getFloat8(int fieldId) {
-    return get(fieldId).asFloat8();
-  }
-
-  @Override
-  public String getText(int fieldId) {
-    return get(fieldId).asChars();
-  }
-
-  @Override
-  public ProtobufDatum getProtobufDatum(int fieldId) {
-    throw new UnsupportedException();
-  }
-
-  @Override
-  public IntervalDatum getInterval(int fieldId) {
-    return (IntervalDatum) get(fieldId);
-  }
-
-  @Override
-  public char[] getUnicodeChars(int fieldId) {
-    return get(fieldId).asUnicodeChars();
-  }
-
-  public String toString() {
-    boolean first = true;
-    StringBuilder str = new StringBuilder();
-    str.append("(");
-    Datum d;
-    for (int i = 0; i < values.length; i++) {
-      d = get(i);
-      if (d != null) {
-        if (first) {
-          first = false;
-        } else {
-          str.append(", ");
-        }
-        str.append(i)
-            .append("=>")
-            .append(d);
-      }
-    }
-    str.append(")");
-    return str.toString();
-  }
-
-  @Override
-  public int hashCode() {
-    return Arrays.hashCode(values);
-  }
-
-  @Override
-  public Datum[] getValues() {
-    Datum[] datums = new Datum[values.length];
-    for (int i = 0; i < values.length; i++) {
-      datums[i] = get(i);
-    }
-    return datums;
-  }
-
-  @Override
-  public Tuple clone() throws CloneNotSupportedException {
-    LazyTuple lazyTuple = (LazyTuple) super.clone();
-
-    lazyTuple.values = getValues(); //shallow copy
-    lazyTuple.textBytes = new byte[size()][];
-    return lazyTuple;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof Tuple) {
-      Tuple other = (Tuple) obj;
-      return Arrays.equals(getValues(), other.getValues());
-    }
-    return false;
-  }
-}


[07/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
new file mode 100644
index 0000000..81a1ffd
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
@@ -0,0 +1,623 @@
+/**
+ * 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.index.bst;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
+import org.apache.tajo.storage.index.IndexMethod;
+import org.apache.tajo.storage.index.IndexWriter;
+import org.apache.tajo.storage.index.OrderIndexReader;
+
+import java.io.Closeable;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.Set;
+import java.util.TreeMap;
+
+import static org.apache.tajo.index.IndexProtos.TupleComparatorProto;
+
+/**
+ * This is two-level binary search tree index. This is one of the value-list 
+ * index structure. Thus, it is inefficient in the case where 
+ * the many of the values are same. Also, the BST shows the fast performance 
+ * when the selectivity of rows to be retrieved is less than 5%.
+ * BSTIndexWriter is not thread-safe, whereas BSTIndexReader is thread-safe.
+ */
+public class BSTIndex implements IndexMethod {
+  private static final Log LOG = LogFactory.getLog(BSTIndex.class);
+
+  public static final int ONE_LEVEL_INDEX = 1;
+  public static final int TWO_LEVEL_INDEX = 2;
+
+  private final Configuration conf;
+
+  public BSTIndex(final Configuration conf) {
+    this.conf = conf;
+  }
+  
+  @Override
+  public BSTIndexWriter getIndexWriter(final Path fileName, int level, Schema keySchema,
+      TupleComparator comparator) throws IOException {
+    return new BSTIndexWriter(fileName, level, keySchema, comparator);
+  }
+
+  @Override
+  public BSTIndexReader getIndexReader(Path fileName, Schema keySchema, TupleComparator comparator) throws IOException {
+    return new BSTIndexReader(fileName, keySchema, comparator);
+  }
+
+  public BSTIndexReader getIndexReader(Path fileName) throws IOException {
+    return new BSTIndexReader(fileName);
+  }
+
+  public class BSTIndexWriter extends IndexWriter implements Closeable {
+    private FSDataOutputStream out;
+    private FileSystem fs;
+    private int level;
+    private int loadNum = 4096;
+    private Path fileName;
+
+    private final Schema keySchema;
+    private final TupleComparator compartor;
+    private final KeyOffsetCollector collector;
+    private KeyOffsetCollector rootCollector;
+
+    private Tuple firstKey;
+    private Tuple lastKey;
+
+    private RowStoreEncoder rowStoreEncoder;
+
+    // private Tuple lastestKey = null;
+
+    /**
+     * constructor
+     *
+     * @param level
+     *          : IndexCreater.ONE_LEVEL_INDEX or IndexCreater.TWO_LEVEL_INDEX
+     * @throws java.io.IOException
+     */
+    public BSTIndexWriter(final Path fileName, int level, Schema keySchema,
+        TupleComparator comparator) throws IOException {
+      this.fileName = fileName;
+      this.level = level;
+      this.keySchema = keySchema;
+      this.compartor = comparator;
+      this.collector = new KeyOffsetCollector(comparator);
+      this.rowStoreEncoder = RowStoreUtil.createEncoder(keySchema);
+    }
+
+   public void setLoadNum(int loadNum) {
+      this.loadNum = loadNum;
+    }
+
+    public void open() throws IOException {
+      fs = fileName.getFileSystem(conf);
+      if (fs.exists(fileName)) {
+        throw new IOException("ERROR: index file (" + fileName + " already exists");
+      }
+      out = fs.create(fileName);
+    }
+
+    @Override
+    public void write(Tuple key, long offset) throws IOException {
+      if (firstKey == null || compartor.compare(key, firstKey) < 0) {
+        firstKey = key;
+      }
+      if (lastKey == null || compartor.compare(lastKey, key) < 0) {
+        lastKey = key;
+      }
+
+      collector.put(key, offset);
+    }
+
+    public TupleComparator getComparator() {
+      return this.compartor;
+    }
+
+    public void flush() throws IOException {
+      out.flush();
+    }
+
+    public void writeHeader(int entryNum) throws IOException {
+      // schema
+      byte [] schemaBytes = keySchema.getProto().toByteArray();
+      out.writeInt(schemaBytes.length);
+      out.write(schemaBytes);
+
+      // comparator
+      byte [] comparatorBytes = compartor.getProto().toByteArray();
+      out.writeInt(comparatorBytes.length);
+      out.write(comparatorBytes);
+
+      // level
+      out.writeInt(this.level);
+      // entry
+      out.writeInt(entryNum);
+      if (entryNum > 0) {
+        byte [] minBytes = rowStoreEncoder.toBytes(firstKey);
+        out.writeInt(minBytes.length);
+        out.write(minBytes);
+        byte [] maxBytes = rowStoreEncoder.toBytes(lastKey);
+        out.writeInt(maxBytes.length);
+        out.write(maxBytes);
+      }
+      out.flush();
+    }
+
+    public void close() throws IOException {
+      /* two level initialize */
+      if (this.level == TWO_LEVEL_INDEX) {
+        rootCollector = new KeyOffsetCollector(this.compartor);
+      }
+
+      /* data writing phase */
+      TreeMap<Tuple, LinkedList<Long>> keyOffsetMap = collector.getMap();
+      Set<Tuple> keySet = keyOffsetMap.keySet();
+
+      int entryNum = keySet.size();
+      writeHeader(entryNum);
+
+      int loadCount = this.loadNum - 1;
+      for (Tuple key : keySet) {
+
+        if (this.level == TWO_LEVEL_INDEX) {
+          loadCount++;
+          if (loadCount == this.loadNum) {
+            rootCollector.put(key, out.getPos());
+            loadCount = 0;
+          }
+        }
+        /* key writing */
+        byte[] buf = rowStoreEncoder.toBytes(key);
+        out.writeInt(buf.length);
+        out.write(buf);
+
+        /**/
+        LinkedList<Long> offsetList = keyOffsetMap.get(key);
+        /* offset num writing */
+        int offsetSize = offsetList.size();
+        out.writeInt(offsetSize);
+        /* offset writing */
+        for (Long offset : offsetList) {
+          out.writeLong(offset);
+        }
+      }
+
+      out.flush();
+      out.close();
+      keySet.clear();
+      collector.clear();
+
+      FSDataOutputStream rootOut = null;
+      /* root index creating phase */
+      if (this.level == TWO_LEVEL_INDEX) {
+        TreeMap<Tuple, LinkedList<Long>> rootMap = rootCollector.getMap();
+        keySet = rootMap.keySet();
+
+        rootOut = fs.create(new Path(fileName + ".root"));
+        rootOut.writeInt(this.loadNum);
+        rootOut.writeInt(keySet.size());
+
+        /* root key writing */
+        for (Tuple key : keySet) {
+          byte[] buf = rowStoreEncoder.toBytes(key);
+          rootOut.writeInt(buf.length);
+          rootOut.write(buf);
+
+          LinkedList<Long> offsetList = rootMap.get(key);
+          if (offsetList.size() > 1 || offsetList.size() == 0) {
+            throw new IOException("Why root index doen't have one offset?");
+          }
+          rootOut.writeLong(offsetList.getFirst());
+
+        }
+        rootOut.flush();
+        rootOut.close();
+
+        keySet.clear();
+        rootCollector.clear();
+      }
+    }
+
+    private class KeyOffsetCollector {
+      private TreeMap<Tuple, LinkedList<Long>> map;
+
+      public KeyOffsetCollector(TupleComparator comparator) {
+        map = new TreeMap<Tuple, LinkedList<Long>>(comparator);
+      }
+
+      public void put(Tuple key, long offset) {
+        if (map.containsKey(key)) {
+          map.get(key).add(offset);
+        } else {
+          LinkedList<Long> list = new LinkedList<Long>();
+          list.add(offset);
+          map.put(key, list);
+        }
+      }
+
+      public TreeMap<Tuple, LinkedList<Long>> getMap() {
+        return this.map;
+      }
+
+      public void clear() {
+        this.map.clear();
+      }
+    }
+  }
+
+  /**
+   * BSTIndexReader is thread-safe.
+   */
+  public class BSTIndexReader implements OrderIndexReader , Closeable{
+    private Path fileName;
+    private Schema keySchema;
+    private TupleComparator comparator;
+
+    private FileSystem fs;
+    private FSDataInputStream indexIn;
+    private FSDataInputStream subIn;
+
+    private int level;
+    private int entryNum;
+    private int loadNum = -1;
+    private Tuple firstKey;
+    private Tuple lastKey;
+
+    // the cursors of BST
+    private int rootCursor;
+    private int keyCursor;
+    private int offsetCursor;
+
+    // mutex
+    private final Object mutex = new Object();
+
+    private RowStoreDecoder rowStoreDecoder;
+
+    /**
+     *
+     * @param fileName
+     * @param keySchema
+     * @param comparator
+     * @throws java.io.IOException
+     */
+    public BSTIndexReader(final Path fileName, Schema keySchema, TupleComparator comparator) throws IOException {
+      this.fileName = fileName;
+      this.keySchema = keySchema;
+      this.comparator = comparator;
+      this.rowStoreDecoder = RowStoreUtil.createDecoder(keySchema);
+    }
+
+    public BSTIndexReader(final Path fileName) throws IOException {
+      this.fileName = fileName;
+    }
+
+    public Schema getKeySchema() {
+      return this.keySchema;
+    }
+
+    public TupleComparator getComparator() {
+      return this.comparator;
+    }
+
+    private void readHeader() throws IOException {
+      // schema
+      int schemaByteSize = indexIn.readInt();
+      byte [] schemaBytes = new byte[schemaByteSize];
+      StorageUtil.readFully(indexIn, schemaBytes, 0, schemaByteSize);
+
+      SchemaProto.Builder builder = SchemaProto.newBuilder();
+      builder.mergeFrom(schemaBytes);
+      SchemaProto proto = builder.build();
+      this.keySchema = new Schema(proto);
+      this.rowStoreDecoder = RowStoreUtil.createDecoder(keySchema);
+
+      // comparator
+      int compByteSize = indexIn.readInt();
+      byte [] compBytes = new byte[compByteSize];
+      StorageUtil.readFully(indexIn, compBytes, 0, compByteSize);
+
+      TupleComparatorProto.Builder compProto = TupleComparatorProto.newBuilder();
+      compProto.mergeFrom(compBytes);
+      this.comparator = new BaseTupleComparator(compProto.build());
+
+      // level
+      this.level = indexIn.readInt();
+      // entry
+      this.entryNum = indexIn.readInt();
+      if (entryNum > 0) { // if there is no any entry, do not read firstKey/lastKey values
+        byte [] minBytes = new byte[indexIn.readInt()];
+        StorageUtil.readFully(indexIn, minBytes, 0, minBytes.length);
+        this.firstKey = rowStoreDecoder.toTuple(minBytes);
+
+        byte [] maxBytes = new byte[indexIn.readInt()];
+        StorageUtil.readFully(indexIn, maxBytes, 0, maxBytes.length);
+        this.lastKey = rowStoreDecoder.toTuple(maxBytes);
+      }
+    }
+
+    public void open()
+        throws IOException {
+      /* init the index file */
+      fs = fileName.getFileSystem(conf);
+      if (!fs.exists(fileName)) {
+        throw new FileNotFoundException("ERROR: does not exist " + fileName.toString());
+      }
+
+      indexIn = fs.open(this.fileName);
+      readHeader();
+      fillData();
+    }
+
+    private void fillData() throws IOException {
+      /* load on memory */
+      if (this.level == TWO_LEVEL_INDEX) {
+
+        Path rootPath = new Path(this.fileName + ".root");
+        if (!fs.exists(rootPath)) {
+          throw new FileNotFoundException("root index did not created");
+        }
+
+        subIn = indexIn;
+        indexIn = fs.open(rootPath);
+        /* root index header reading : type => loadNum => indexSize */
+        this.loadNum = indexIn.readInt();
+        this.entryNum = indexIn.readInt();
+        /**/
+        fillRootIndex(entryNum, indexIn);
+
+      } else {
+        fillLeafIndex(entryNum, indexIn, -1);
+      }
+    }
+
+    /**
+     *
+     * @return
+     * @throws java.io.IOException
+     */
+    public long find(Tuple key) throws IOException {
+      return find(key, false);
+    }
+
+    @Override
+    public long find(Tuple key, boolean nextKey) throws IOException {
+      synchronized (mutex) {
+        int pos = -1;
+        if (this.level == ONE_LEVEL_INDEX) {
+            pos = oneLevBS(key);
+        } else if (this.level == TWO_LEVEL_INDEX) {
+            pos = twoLevBS(key, this.loadNum + 1);
+        } else {
+          throw new IOException("More than TWL_LEVEL_INDEX is not supported.");
+        }
+
+        if (nextKey) {
+          if (pos + 1 >= this.offsetSubIndex.length) {
+            return -1;
+          }
+          keyCursor = pos + 1;
+          offsetCursor = 0;
+        } else {
+          if (correctable) {
+            keyCursor = pos;
+            offsetCursor = 0;
+          } else {
+            return -1;
+          }
+        }
+
+        return this.offsetSubIndex[keyCursor][offsetCursor];
+      }
+    }
+
+    public long next() throws IOException {
+      synchronized (mutex) {
+        if (offsetSubIndex[keyCursor].length - 1 > offsetCursor) {
+          offsetCursor++;
+        } else {
+          if (offsetSubIndex.length - 1 > keyCursor) {
+            keyCursor++;
+            offsetCursor = 0;
+          } else {
+            if (offsetIndex.length -1 > rootCursor) {
+              rootCursor++;
+              fillLeafIndex(loadNum + 1, subIn, this.offsetIndex[rootCursor]);
+              keyCursor = 1;
+              offsetCursor = 0;
+            } else {
+              return -1;
+            }
+          }
+        }
+
+        return this.offsetSubIndex[keyCursor][offsetCursor];
+      }
+    }
+    
+    public boolean isCurInMemory() {
+      return (offsetSubIndex[keyCursor].length - 1 >= offsetCursor);
+    }
+
+    private void fillLeafIndex(int entryNum, FSDataInputStream in, long pos)
+        throws IOException {
+      int counter = 0;
+      try {
+        if (pos != -1) {
+          in.seek(pos);
+        }
+        this.dataSubIndex = new Tuple[entryNum];
+        this.offsetSubIndex = new long[entryNum][];
+
+        byte[] buf;
+        for (int i = 0; i < entryNum; i++) {
+          counter++;
+          buf = new byte[in.readInt()];
+          StorageUtil.readFully(in, buf, 0, buf.length);
+          dataSubIndex[i] = rowStoreDecoder.toTuple(buf);
+
+          int offsetNum = in.readInt();
+          this.offsetSubIndex[i] = new long[offsetNum];
+          for (int j = 0; j < offsetNum; j++) {
+            this.offsetSubIndex[i][j] = in.readLong();
+          }
+
+        }
+
+      } catch (IOException e) {
+        counter--;
+        if (pos != -1) {
+          in.seek(pos);
+        }
+        this.dataSubIndex = new Tuple[counter];
+        this.offsetSubIndex = new long[counter][];
+
+        byte[] buf;
+        for (int i = 0; i < counter; i++) {
+          buf = new byte[in.readInt()];
+          StorageUtil.readFully(in, buf, 0, buf.length);
+          dataSubIndex[i] = rowStoreDecoder.toTuple(buf);
+
+          int offsetNum = in.readInt();
+          this.offsetSubIndex[i] = new long[offsetNum];
+          for (int j = 0; j < offsetNum; j++) {
+            this.offsetSubIndex[i][j] = in.readLong();
+          }
+
+        }
+      }
+    }
+
+    public Tuple getFirstKey() {
+      return this.firstKey;
+    }
+
+    public Tuple getLastKey() {
+      return this.lastKey;
+    }
+
+    private void fillRootIndex(int entryNum, FSDataInputStream in)
+        throws IOException {
+      this.dataIndex = new Tuple[entryNum];
+      this.offsetIndex = new long[entryNum];
+      Tuple keyTuple;
+      byte[] buf;
+      for (int i = 0; i < entryNum; i++) {
+        buf = new byte[in.readInt()];
+        StorageUtil.readFully(in, buf, 0, buf.length);
+        keyTuple = rowStoreDecoder.toTuple(buf);
+        dataIndex[i] = keyTuple;
+        this.offsetIndex[i] = in.readLong();
+      }
+    }
+
+    /* memory index, only one is used. */
+    private Tuple[] dataIndex = null;
+    private Tuple[] dataSubIndex = null;
+
+    /* offset index */
+    private long[] offsetIndex = null;
+    private long[][] offsetSubIndex = null;
+
+    private boolean correctable = true;
+
+    private int oneLevBS(Tuple key) throws IOException {
+      correctable = true;
+      int pos = binarySearch(this.dataSubIndex, key, 0, this.dataSubIndex.length);
+      return pos;
+    }
+
+    private int twoLevBS(Tuple key, int loadNum) throws IOException {
+      int pos = binarySearch(this.dataIndex, key, 0, this.dataIndex.length);
+      if(pos > 0) {
+        rootCursor = pos;
+      } else {
+        rootCursor = 0;
+      }
+      fillLeafIndex(loadNum, subIn, this.offsetIndex[rootCursor]);
+      pos = binarySearch(this.dataSubIndex, key, 0, this.dataSubIndex.length);
+       
+      return pos;
+    }
+
+    private int binarySearch(Tuple[] arr, Tuple key, int startPos, int endPos) {
+      int offset = -1;
+      int start = startPos;
+      int end = endPos;
+
+      //http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6412541
+      int centerPos = (start + end) >>> 1;
+      while (true) {
+        if (comparator.compare(arr[centerPos], key) > 0) {
+          if (centerPos == 0) {
+            correctable = false;
+            break;
+          } else if (comparator.compare(arr[centerPos - 1], key) < 0) {
+            correctable = false;
+            offset = centerPos - 1;
+            break;
+          } else {
+            end = centerPos;
+            centerPos = (start + end) / 2;
+          }
+        } else if (comparator.compare(arr[centerPos], key) < 0) {
+          if (centerPos == arr.length - 1) {
+            correctable = false;
+            offset = centerPos;
+            break;
+          } else if (comparator.compare(arr[centerPos + 1], key) > 0) {
+            correctable = false;
+            offset = centerPos;
+            break;
+          } else {
+            start = centerPos + 1;
+            centerPos = (start + end) / 2;
+          }
+        } else {
+          correctable = true;
+          offset = centerPos;
+          break;
+        }
+      }
+      return offset;
+    }
+
+    @Override
+    public void close() throws IOException {
+      this.indexIn.close();
+      this.subIn.close();
+    }
+
+    @Override
+    public String toString() {
+      return "BSTIndex (" + firstKey + ", " + lastKey + ") " + fileName;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
new file mode 100644
index 0000000..b10d423
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
@@ -0,0 +1,151 @@
+/**
+ * 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.parquet;
+
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.storage.StorageConstants;
+import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.metadata.CompressionCodecName;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.storage.FileAppender;
+import org.apache.tajo.storage.TableStatistics;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+
+/**
+ * FileAppender for writing to Parquet files.
+ */
+public class ParquetAppender extends FileAppender {
+  private TajoParquetWriter writer;
+  private int blockSize;
+  private int pageSize;
+  private CompressionCodecName compressionCodecName;
+  private boolean enableDictionary;
+  private boolean validating;
+  private TableStatistics stats;
+
+  /**
+   * Creates a new ParquetAppender.
+   *
+   * @param conf Configuration properties.
+   * @param schema The table schema.
+   * @param meta The table metadata.
+   * @param workDir The path of the Parquet file to write to.
+   */
+  public ParquetAppender(Configuration conf, QueryUnitAttemptId taskAttemptId, Schema schema, TableMeta meta,
+                         Path workDir) throws IOException {
+    super(conf, taskAttemptId, schema, meta, workDir);
+    this.blockSize = Integer.parseInt(
+        meta.getOption(ParquetOutputFormat.BLOCK_SIZE, StorageConstants.PARQUET_DEFAULT_BLOCK_SIZE));
+    this.pageSize = Integer.parseInt(
+        meta.getOption(ParquetOutputFormat.PAGE_SIZE, StorageConstants.PARQUET_DEFAULT_PAGE_SIZE));
+    this.compressionCodecName = CompressionCodecName.fromConf(
+        meta.getOption(ParquetOutputFormat.COMPRESSION, StorageConstants.PARQUET_DEFAULT_COMPRESSION_CODEC_NAME));
+    this.enableDictionary = Boolean.parseBoolean(
+        meta.getOption(ParquetOutputFormat.ENABLE_DICTIONARY, StorageConstants.PARQUET_DEFAULT_IS_DICTIONARY_ENABLED));
+    this.validating = Boolean.parseBoolean(
+        meta.getOption(ParquetOutputFormat.VALIDATION, StorageConstants.PARQUET_DEFAULT_IS_VALIDATION_ENABLED));
+  }
+
+  /**
+   * Initializes the Appender. This method creates a new TajoParquetWriter
+   * and initializes the table statistics if enabled.
+   */
+  public void init() throws IOException {
+    writer = new TajoParquetWriter(path,
+                                   schema,
+                                   compressionCodecName,
+                                   blockSize,
+                                   pageSize,
+                                   enableDictionary,
+                                   validating);
+    if (enabledStats) {
+      this.stats = new TableStatistics(schema);
+    }
+    super.init();
+  }
+
+  /**
+   * Gets the current offset. Tracking offsets is currenly not implemented, so
+   * this method always returns 0.
+   *
+   * @return 0
+   */
+  @Override
+  public long getOffset() throws IOException {
+    return 0;
+  }
+
+  /**
+   * Write a Tuple to the Parquet file.
+   *
+   * @param tuple The Tuple to write.
+   */
+  @Override
+  public void addTuple(Tuple tuple) throws IOException {
+    if (enabledStats) {
+      for (int i = 0; i < schema.size(); ++i) {
+        stats.analyzeField(i, tuple.get(i));
+      }
+    }
+    writer.write(tuple);
+    if (enabledStats) {
+      stats.incrementRow();
+    }
+  }
+
+  /**
+   * The ParquetWriter does not need to be flushed, so this is a no-op.
+   */
+  @Override
+  public void flush() throws IOException {
+  }
+
+  /**
+   * Closes the Appender.
+   */
+  @Override
+  public void close() throws IOException {
+    writer.close();
+  }
+
+  public long getEstimatedOutputSize() throws IOException {
+    return writer.getEstimatedWrittenSize();
+  }
+
+  /**
+   * If table statistics is enabled, retrieve the table statistics.
+   *
+   * @return Table statistics if enabled or null otherwise.
+   */
+  @Override
+  public TableStats getStats() {
+    if (enabledStats) {
+      return stats.getTableStat();
+    } else {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
new file mode 100644
index 0000000..2f8efcf
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
@@ -0,0 +1,119 @@
+/**
+ * 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.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.FileScanner;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+
+/**
+ * FileScanner for reading Parquet files
+ */
+public class ParquetScanner extends FileScanner {
+  private TajoParquetReader reader;
+
+  /**
+   * Creates a new ParquetScanner.
+   *
+   * @param conf
+   * @param schema
+   * @param meta
+   * @param fragment
+   */
+  public ParquetScanner(Configuration conf, final Schema schema,
+                        final TableMeta meta, final Fragment fragment) {
+    super(conf, schema, meta, fragment);
+  }
+
+  /**
+   * Initializes the ParquetScanner. This method initializes the
+   * TajoParquetReader.
+   */
+  @Override
+  public void init() throws IOException {
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+    reader = new TajoParquetReader(fragment.getPath(), schema, new Schema(targets));
+    super.init();
+  }
+
+  /**
+   * Reads the next Tuple from the Parquet file.
+   *
+   * @return The next Tuple from the Parquet file or null if end of file is
+   *         reached.
+   */
+  @Override
+  public Tuple next() throws IOException {
+    return reader.read();
+  }
+
+  /**
+   * Resets the scanner
+   */
+  @Override
+  public void reset() throws IOException {
+  }
+
+  /**
+   * Closes the scanner.
+   */
+  @Override
+  public void close() throws IOException {
+    if (reader != null) {
+      reader.close();
+    }
+  }
+
+  /**
+   * Returns whether this scanner is projectable.
+   *
+   * @return true
+   */
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  /**
+   * Returns whether this scanner is selectable.
+   *
+   * @return false
+   */
+  @Override
+  public boolean isSelectable() {
+    return false;
+  }
+
+  /**
+   * Returns whether this scanner is splittable.
+   *
+   * @return false
+   */
+  @Override
+  public boolean isSplittable() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
new file mode 100644
index 0000000..a765f48
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
@@ -0,0 +1,85 @@
+/**
+ * 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.parquet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.thirdparty.parquet.ParquetReader;
+import parquet.filter.UnboundRecordFilter;
+
+import java.io.IOException;
+
+/**
+ * Tajo implementation of {@link ParquetReader} to read Tajo records from a
+ * Parquet file. Users should use {@link ParquetScanner} and not this class
+ * directly.
+ */
+public class TajoParquetReader extends ParquetReader<Tuple> {
+  /**
+   * Creates a new TajoParquetReader.
+   *
+   * @param file The file to read from.
+   * @param readSchema Tajo schema of the table.
+   */
+  public TajoParquetReader(Path file, Schema readSchema) throws IOException {
+    super(file, new TajoReadSupport(readSchema));
+  }
+
+  /**
+   * Creates a new TajoParquetReader.
+   *
+   * @param file The file to read from.
+   * @param readSchema Tajo schema of the table.
+   * @param requestedSchema Tajo schema of the projection.
+   */
+  public TajoParquetReader(Path file, Schema readSchema,
+                           Schema requestedSchema) throws IOException {
+    super(file, new TajoReadSupport(readSchema, requestedSchema));
+  }
+
+  /**
+   * Creates a new TajoParquetReader.
+   *
+   * @param file The file to read from.
+   * @param readSchema Tajo schema of the table.
+   * @param recordFilter Record filter.
+   */
+  public TajoParquetReader(Path file, Schema readSchema,
+                           UnboundRecordFilter recordFilter)
+      throws IOException {
+    super(file, new TajoReadSupport(readSchema), recordFilter);
+  }
+
+  /**
+   * Creates a new TajoParquetReader.
+   *
+   * @param file The file to read from.
+   * @param readSchema Tajo schema of the table.
+   * @param requestedSchema Tajo schema of the projection.
+   * @param recordFilter Record filter.
+   */
+  public TajoParquetReader(Path file, Schema readSchema,
+                           Schema requestedSchema,
+                           UnboundRecordFilter recordFilter)
+      throws IOException {
+    super(file, new TajoReadSupport(readSchema, requestedSchema),
+          recordFilter);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
new file mode 100644
index 0000000..5f220c5
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
@@ -0,0 +1,104 @@
+/**
+ * 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.parquet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.thirdparty.parquet.ParquetWriter;
+import parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+/**
+ * Tajo implementation of {@link ParquetWriter} to write Tajo records to a
+ * Parquet file. Users should use {@link ParquetAppender} and not this class
+ * directly.
+ */
+public class TajoParquetWriter extends ParquetWriter<Tuple> {
+  /**
+   * Create a new TajoParquetWriter
+   *
+   * @param file The file name to write to.
+   * @param schema The Tajo schema of the table.
+   * @param compressionCodecName Compression codec to use, or
+   *                             CompressionCodecName.UNCOMPRESSED.
+   * @param blockSize The block size threshold.
+   * @param pageSize See parquet write up. Blocks are subdivided into pages
+   *                 for alignment.
+   * @throws java.io.IOException
+   */
+  public TajoParquetWriter(Path file,
+                           Schema schema,
+                           CompressionCodecName compressionCodecName,
+                           int blockSize,
+                           int pageSize) throws IOException {
+    super(file,
+          new TajoWriteSupport(schema),
+          compressionCodecName,
+          blockSize,
+          pageSize);
+  }
+
+  /**
+   * Create a new TajoParquetWriter.
+   *
+   * @param file The file name to write to.
+   * @param schema The Tajo schema of the table.
+   * @param compressionCodecName Compression codec to use, or
+   *                             CompressionCodecName.UNCOMPRESSED.
+   * @param blockSize The block size threshold.
+   * @param pageSize See parquet write up. Blocks are subdivided into pages
+   *                 for alignment.
+   * @param enableDictionary Whether to use a dictionary to compress columns.
+   * @param validating Whether to turn on validation.
+   * @throws java.io.IOException
+   */
+  public TajoParquetWriter(Path file,
+                           Schema schema,
+                           CompressionCodecName compressionCodecName,
+                           int blockSize,
+                           int pageSize,
+                           boolean enableDictionary,
+                           boolean validating) throws IOException {
+    super(file,
+          new TajoWriteSupport(schema),
+          compressionCodecName,
+          blockSize,
+          pageSize,
+          enableDictionary,
+          validating);
+  }
+
+  /**
+   * Creates a new TajoParquetWriter. The default block size is 128 MB.
+   * The default page size is 1 MB. Default compression is no compression.
+   *
+   * @param file The Path of the file to write to.
+   * @param schema The Tajo schema of the table.
+   * @throws java.io.IOException
+   */
+  public TajoParquetWriter(Path file, Schema schema) throws IOException {
+    this(file,
+         schema,
+         CompressionCodecName.UNCOMPRESSED,
+         DEFAULT_BLOCK_SIZE,
+         DEFAULT_PAGE_SIZE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
new file mode 100644
index 0000000..a64e987
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
@@ -0,0 +1,99 @@
+/**
+ * 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.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
+import parquet.Log;
+import parquet.hadoop.api.InitContext;
+import parquet.hadoop.api.ReadSupport;
+import parquet.io.api.RecordMaterializer;
+import parquet.schema.MessageType;
+
+import java.util.Map;
+
+/**
+ * Tajo implementation of {@link parquet.hadoop.api.ReadSupport} for {@link org.apache.tajo.storage.Tuple}s.
+ * Users should use {@link org.apache.tajo.storage.parquet.ParquetScanner} and not this class directly.
+ */
+public class TajoReadSupport extends ReadSupport<Tuple> {
+  private static final Log LOG = Log.getLog(TajoReadSupport.class);
+
+  private Schema readSchema;
+  private Schema requestedSchema;
+
+  /**
+   * Creates a new TajoReadSupport.
+   *
+   * @param requestedSchema The Tajo schema of the requested projection passed
+   *        down by ParquetScanner.
+   */
+  public TajoReadSupport(Schema readSchema, Schema requestedSchema) {
+    super();
+    this.readSchema = readSchema;
+    this.requestedSchema = requestedSchema;
+  }
+
+  /**
+   * Creates a new TajoReadSupport.
+   *
+   * @param readSchema The schema of the table.
+   */
+  public TajoReadSupport(Schema readSchema) {
+    super();
+    this.readSchema = readSchema;
+    this.requestedSchema = readSchema;
+  }
+
+  /**
+   * Initializes the ReadSupport.
+   *
+   * @param context The InitContext.
+   * @return A ReadContext that defines how to read the file.
+   */
+  @Override
+  public ReadContext init(InitContext context) {
+    if (requestedSchema == null) {
+      throw new RuntimeException("requestedSchema is null.");
+    }
+    MessageType requestedParquetSchema =
+      new TajoSchemaConverter().convert(requestedSchema);
+    LOG.debug("Reading data with projection:\n" + requestedParquetSchema);
+    return new ReadContext(requestedParquetSchema);
+  }
+
+  /**
+   * Prepares for read.
+   *
+   * @param configuration The job configuration.
+   * @param keyValueMetaData App-specific metadata from the file.
+   * @param fileSchema The schema of the Parquet file.
+   * @param readContext Returned by the init method.
+   */
+  @Override
+  public RecordMaterializer<Tuple> prepareForRead(
+      Configuration configuration,
+      Map<String, String> keyValueMetaData,
+      MessageType fileSchema,
+      ReadContext readContext) {
+    MessageType parquetRequestedSchema = readContext.getRequestedSchema();
+    return new TajoRecordMaterializer(parquetRequestedSchema, requestedSchema, readSchema);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
new file mode 100644
index 0000000..4375fa4
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
@@ -0,0 +1,380 @@
+/**
+ * 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.parquet;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import parquet.io.api.Binary;
+import parquet.io.api.Converter;
+import parquet.io.api.GroupConverter;
+import parquet.io.api.PrimitiveConverter;
+import parquet.schema.GroupType;
+import parquet.schema.Type;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Converter to convert a Parquet record into a Tajo Tuple.
+ */
+public class TajoRecordConverter extends GroupConverter {
+  private final GroupType parquetSchema;
+  private final Schema tajoReadSchema;
+  private final int[] projectionMap;
+  private final int tupleSize;
+
+  private final Converter[] converters;
+
+  private Tuple currentTuple;
+
+  /**
+   * Creates a new TajoRecordConverter.
+   *
+   * @param parquetSchema The Parquet schema of the projection.
+   * @param tajoReadSchema The Tajo schema of the table.
+   * @param projectionMap An array mapping the projection column to the column
+   *                      index in the table.
+   */
+  public TajoRecordConverter(GroupType parquetSchema, Schema tajoReadSchema,
+                             int[] projectionMap) {
+    this.parquetSchema = parquetSchema;
+    this.tajoReadSchema = tajoReadSchema;
+    this.projectionMap = projectionMap;
+    this.tupleSize = tajoReadSchema.size();
+
+    // The projectionMap.length does not match parquetSchema.getFieldCount()
+    // when the projection contains NULL_TYPE columns. We will skip over the
+    // NULL_TYPE columns when we construct the converters and populate the
+    // NULL_TYPE columns with NullDatums in start().
+    int index = 0;
+    this.converters = new Converter[parquetSchema.getFieldCount()];
+    for (int i = 0; i < projectionMap.length; ++i) {
+      final int projectionIndex = projectionMap[i];
+      Column column = tajoReadSchema.getColumn(projectionIndex);
+      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE) {
+        continue;
+      }
+      Type type = parquetSchema.getType(index);
+      converters[index] = newConverter(column, type, new ParentValueContainer() {
+        @Override
+        void add(Object value) {
+          TajoRecordConverter.this.set(projectionIndex, value);
+        }
+      });
+      ++index;
+    }
+  }
+
+  private void set(int index, Object value) {
+    currentTuple.put(index, (Datum)value);
+  }
+
+  private Converter newConverter(Column column, Type type,
+                                 ParentValueContainer parent) {
+    DataType dataType = column.getDataType();
+    switch (dataType.getType()) {
+      case BOOLEAN:
+        return new FieldBooleanConverter(parent);
+      case BIT:
+        return new FieldBitConverter(parent);
+      case CHAR:
+        return new FieldCharConverter(parent);
+      case INT2:
+        return new FieldInt2Converter(parent);
+      case INT4:
+        return new FieldInt4Converter(parent);
+      case INT8:
+        return new FieldInt8Converter(parent);
+      case FLOAT4:
+        return new FieldFloat4Converter(parent);
+      case FLOAT8:
+        return new FieldFloat8Converter(parent);
+      case INET4:
+        return new FieldInet4Converter(parent);
+      case INET6:
+        throw new RuntimeException("No converter for INET6");
+      case TEXT:
+        return new FieldTextConverter(parent);
+      case PROTOBUF:
+        return new FieldProtobufConverter(parent, dataType);
+      case BLOB:
+        return new FieldBlobConverter(parent);
+      case NULL_TYPE:
+        throw new RuntimeException("No converter for NULL_TYPE.");
+      default:
+        throw new RuntimeException("Unsupported data type");
+    }
+  }
+
+  /**
+   * Gets the converter for a specific field.
+   *
+   * @param fieldIndex Index of the field in the projection.
+   * @return The converter for the field.
+   */
+  @Override
+  public Converter getConverter(int fieldIndex) {
+    return converters[fieldIndex];
+  }
+
+  /**
+   * Called before processing fields. This method fills any fields that have
+   * NULL values or have type NULL_TYPE with a NullDatum.
+   */
+  @Override
+  public void start() {
+    currentTuple = new VTuple(tupleSize);
+  }
+
+  /**
+   * Called after all fields have been processed.
+   */
+  @Override
+  public void end() {
+    for (int i = 0; i < projectionMap.length; ++i) {
+      final int projectionIndex = projectionMap[i];
+      Column column = tajoReadSchema.getColumn(projectionIndex);
+      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE
+          || currentTuple.get(projectionIndex) == null) {
+        set(projectionIndex, NullDatum.get());
+      }
+    }
+  }
+
+  /**
+   * Returns the current record converted by this converter.
+   *
+   * @return The current record.
+   */
+  public Tuple getCurrentRecord() {
+    return currentTuple;
+  }
+
+  static abstract class ParentValueContainer {
+    /**
+     * Adds the value to the parent.
+     *
+     * @param value The value to add.
+     */
+    abstract void add(Object value);
+  }
+
+  static final class FieldBooleanConverter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldBooleanConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBoolean(boolean value) {
+      parent.add(DatumFactory.createBool(value));
+    }
+  }
+
+  static final class FieldBitConverter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldBitConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addInt(int value) {
+      parent.add(DatumFactory.createBit((byte)(value & 0xff)));
+    }
+  }
+
+  static final class FieldCharConverter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldCharConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      parent.add(DatumFactory.createChar(value.toStringUsingUTF8()));
+    }
+  }
+
+  static final class FieldInt2Converter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldInt2Converter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addInt(int value) {
+      parent.add(DatumFactory.createInt2((short)value));
+    }
+  }
+
+  static final class FieldInt4Converter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldInt4Converter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addInt(int value) {
+      parent.add(DatumFactory.createInt4(value));
+    }
+  }
+
+  static final class FieldInt8Converter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldInt8Converter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addLong(long value) {
+      parent.add(DatumFactory.createInt8(value));
+    }
+
+    @Override
+    final public void addInt(int value) {
+      parent.add(DatumFactory.createInt8(Long.valueOf(value)));
+    }
+  }
+
+  static final class FieldFloat4Converter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldFloat4Converter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addInt(int value) {
+      parent.add(DatumFactory.createFloat4(Float.valueOf(value)));
+    }
+
+    @Override
+    final public void addLong(long value) {
+      parent.add(DatumFactory.createFloat4(Float.valueOf(value)));
+    }
+
+    @Override
+    final public void addFloat(float value) {
+      parent.add(DatumFactory.createFloat4(value));
+    }
+  }
+
+  static final class FieldFloat8Converter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldFloat8Converter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addInt(int value) {
+      parent.add(DatumFactory.createFloat8(Double.valueOf(value)));
+    }
+
+    @Override
+    final public void addLong(long value) {
+      parent.add(DatumFactory.createFloat8(Double.valueOf(value)));
+    }
+
+    @Override
+    final public void addFloat(float value) {
+      parent.add(DatumFactory.createFloat8(Double.valueOf(value)));
+    }
+
+    @Override
+    final public void addDouble(double value) {
+      parent.add(DatumFactory.createFloat8(value));
+    }
+  }
+
+  static final class FieldInet4Converter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldInet4Converter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      parent.add(DatumFactory.createInet4(value.getBytes()));
+    }
+  }
+
+  static final class FieldTextConverter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldTextConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      parent.add(DatumFactory.createText(value.toStringUsingUTF8()));
+    }
+  }
+
+  static final class FieldBlobConverter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+
+    public FieldBlobConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      parent.add(new BlobDatum(ByteBuffer.wrap(value.getBytes())));
+    }
+  }
+
+  static final class FieldProtobufConverter extends PrimitiveConverter {
+    private final ParentValueContainer parent;
+    private final DataType dataType;
+
+    public FieldProtobufConverter(ParentValueContainer parent,
+                                  DataType dataType) {
+      this.parent = parent;
+      this.dataType = dataType;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      try {
+        ProtobufDatumFactory factory =
+            ProtobufDatumFactory.get(dataType.getCode());
+        Message.Builder builder = factory.newBuilder();
+        builder.mergeFrom(value.getBytes());
+        parent.add(factory.createDatum(builder));
+      } catch (InvalidProtocolBufferException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
new file mode 100644
index 0000000..436159c
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
@@ -0,0 +1,77 @@
+/**
+ * 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.parquet;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.storage.Tuple;
+import parquet.io.api.GroupConverter;
+import parquet.io.api.RecordMaterializer;
+import parquet.schema.MessageType;
+
+/**
+ * Materializes a Tajo Tuple from a stream of Parquet data.
+ */
+class TajoRecordMaterializer extends RecordMaterializer<Tuple> {
+  private final TajoRecordConverter root;
+
+  /**
+   * Creates a new TajoRecordMaterializer.
+   *
+   * @param parquetSchema The Parquet schema of the projection.
+   * @param tajoSchema The Tajo schema of the projection.
+   * @param tajoReadSchema The Tajo schema of the table.
+   */
+  public TajoRecordMaterializer(MessageType parquetSchema, Schema tajoSchema,
+                                Schema tajoReadSchema) {
+    int[] projectionMap = getProjectionMap(tajoReadSchema, tajoSchema);
+    this.root = new TajoRecordConverter(parquetSchema, tajoReadSchema,
+                                        projectionMap);
+  }
+
+  private int[] getProjectionMap(Schema schema, Schema projection) {
+    Column[] targets = projection.toArray();
+    int[] projectionMap = new int[targets.length];
+    for (int i = 0; i < targets.length; ++i) {
+      int tid = schema.getColumnId(targets[i].getQualifiedName());
+      projectionMap[i] = tid;
+    }
+    return projectionMap;
+  }
+
+  /**
+   * Returns the current record being materialized.
+   *
+   * @return The record being materialized.
+   */
+  @Override
+  public Tuple getCurrentRecord() {
+    return root.getCurrentRecord();
+  }
+
+  /**
+   * Returns the root converter.
+   *
+   * @return The root converter
+   */
+  @Override
+  public GroupConverter getRootConverter() {
+    return root;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
new file mode 100644
index 0000000..555b623
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
@@ -0,0 +1,206 @@
+/**
+ * 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.parquet;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import parquet.schema.MessageType;
+import parquet.schema.OriginalType;
+import parquet.schema.PrimitiveType;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+import parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Converts between Parquet and Tajo schemas. See package documentation for
+ * details on the mapping.
+ */
+public class TajoSchemaConverter {
+  private static final String TABLE_SCHEMA = "table_schema";
+
+  /**
+   * Creates a new TajoSchemaConverter.
+   */
+  public TajoSchemaConverter() {
+  }
+
+  /**
+   * Converts a Parquet schema to a Tajo schema.
+   *
+   * @param parquetSchema The Parquet schema to convert.
+   * @return The resulting Tajo schema.
+   */
+  public Schema convert(MessageType parquetSchema) {
+    return convertFields(parquetSchema.getFields());
+  }
+
+  private Schema convertFields(List<Type> parquetFields) {
+    List<Column> columns = new ArrayList<Column>();
+    for (int i = 0; i < parquetFields.size(); ++i) {
+      Type fieldType = parquetFields.get(i);
+      if (fieldType.isRepetition(Type.Repetition.REPEATED)) {
+        throw new RuntimeException("REPEATED not supported outside LIST or" +
+            " MAP. Type: " + fieldType);
+      }
+      columns.add(convertField(fieldType));
+    }
+    Column[] columnsArray = new Column[columns.size()];
+    columnsArray = columns.toArray(columnsArray);
+    return new Schema(columnsArray);
+  }
+
+  private Column convertField(final Type fieldType) {
+    if (fieldType.isPrimitive()) {
+      return convertPrimitiveField(fieldType);
+    } else {
+      return convertComplexField(fieldType);
+    }
+  }
+
+  private Column convertPrimitiveField(final Type fieldType) {
+    final String fieldName = fieldType.getName();
+    final PrimitiveTypeName parquetPrimitiveTypeName =
+        fieldType.asPrimitiveType().getPrimitiveTypeName();
+    final OriginalType originalType = fieldType.getOriginalType();
+    return parquetPrimitiveTypeName.convert(
+        new PrimitiveType.PrimitiveTypeNameConverter<Column, RuntimeException>() {
+      @Override
+      public Column convertBOOLEAN(PrimitiveTypeName primitiveTypeName) {
+        return new Column(fieldName, TajoDataTypes.Type.BOOLEAN);
+      }
+
+      @Override
+      public Column convertINT32(PrimitiveTypeName primitiveTypeName) {
+        return new Column(fieldName, TajoDataTypes.Type.INT4);
+      }
+
+      @Override
+      public Column convertINT64(PrimitiveTypeName primitiveTypeName) {
+        return new Column(fieldName, TajoDataTypes.Type.INT8);
+      }
+
+      @Override
+      public Column convertFLOAT(PrimitiveTypeName primitiveTypeName) {
+        return new Column(fieldName, TajoDataTypes.Type.FLOAT4);
+      }
+
+      @Override
+      public Column convertDOUBLE(PrimitiveTypeName primitiveTypeName) {
+        return new Column(fieldName, TajoDataTypes.Type.FLOAT8);
+      }
+
+      @Override
+      public Column convertFIXED_LEN_BYTE_ARRAY(
+          PrimitiveTypeName primitiveTypeName) {
+        return new Column(fieldName, TajoDataTypes.Type.BLOB);
+      }
+
+      @Override
+      public Column convertBINARY(PrimitiveTypeName primitiveTypeName) {
+        if (originalType == OriginalType.UTF8) {
+          return new Column(fieldName, TajoDataTypes.Type.TEXT);
+        } else {
+          return new Column(fieldName, TajoDataTypes.Type.BLOB);
+        }
+      }
+
+      @Override
+      public Column convertINT96(PrimitiveTypeName primitiveTypeName) {
+        throw new RuntimeException("Converting from INT96 not supported.");
+      }
+    });
+  }
+
+  private Column convertComplexField(final Type fieldType) {
+    throw new RuntimeException("Complex types not supported.");
+  }
+
+  /**
+   * Converts a Tajo schema to a Parquet schema.
+   *
+   * @param tajoSchema The Tajo schema to convert.
+   * @return The resulting Parquet schema.
+   */
+  public MessageType convert(Schema tajoSchema) {
+    List<Type> types = new ArrayList<Type>();
+    for (int i = 0; i < tajoSchema.size(); ++i) {
+      Column column = tajoSchema.getColumn(i);
+      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE) {
+        continue;
+      }
+      types.add(convertColumn(column));
+    }
+    return new MessageType(TABLE_SCHEMA, types);
+  }
+
+  private Type convertColumn(Column column) {
+    TajoDataTypes.Type type = column.getDataType().getType();
+    switch (type) {
+      case BOOLEAN:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.BOOLEAN);
+      case BIT:
+      case INT2:
+      case INT4:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.INT32);
+      case INT8:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.INT64);
+      case FLOAT4:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.FLOAT);
+      case FLOAT8:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.DOUBLE);
+      case CHAR:
+      case TEXT:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.BINARY,
+                         OriginalType.UTF8);
+      case PROTOBUF:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.BINARY);
+      case BLOB:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.BINARY);
+      case INET4:
+      case INET6:
+        return primitive(column.getSimpleName(),
+                         PrimitiveTypeName.BINARY);
+      default:
+        throw new RuntimeException("Cannot convert Tajo type: " + type);
+    }
+  }
+
+  private PrimitiveType primitive(String name,
+                                  PrimitiveTypeName primitive) {
+    return new PrimitiveType(Type.Repetition.OPTIONAL, primitive, name, null);
+  }
+
+  private PrimitiveType primitive(String name,
+                                  PrimitiveTypeName primitive,
+                                  OriginalType originalType) {
+    return new PrimitiveType(Type.Repetition.OPTIONAL, primitive, name,
+                             originalType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
new file mode 100644
index 0000000..00aadf4
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
@@ -0,0 +1,148 @@
+/**
+ * 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.parquet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Tuple;
+import parquet.hadoop.api.WriteSupport;
+import parquet.io.api.Binary;
+import parquet.io.api.RecordConsumer;
+import parquet.schema.GroupType;
+import parquet.schema.MessageType;
+import parquet.schema.Type;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tajo implementation of {@link parquet.hadoop.api.WriteSupport} for {@link org.apache.tajo.storage.Tuple}s.
+ * Users should use {@link ParquetAppender} and not this class directly.
+ */
+public class TajoWriteSupport extends WriteSupport<Tuple> {
+  private RecordConsumer recordConsumer;
+  private MessageType rootSchema;
+  private Schema rootTajoSchema;
+
+  /**
+   * Creates a new TajoWriteSupport.
+   *
+   * @param tajoSchema The Tajo schema for the table.
+   */
+  public TajoWriteSupport(Schema tajoSchema) {
+    this.rootSchema = new TajoSchemaConverter().convert(tajoSchema);
+    this.rootTajoSchema = tajoSchema;
+  }
+
+  /**
+   * Initializes the WriteSupport.
+   *
+   * @param configuration The job's configuration.
+   * @return A WriteContext that describes how to write the file.
+   */
+  @Override
+  public WriteContext init(Configuration configuration) {
+    Map<String, String> extraMetaData = new HashMap<String, String>();
+    return new WriteContext(rootSchema, extraMetaData);
+  }
+
+  /**
+   * Called once per row group.
+   *
+   * @param recordConsumer The {@link parquet.io.api.RecordConsumer} to write to.
+   */
+  @Override
+  public void prepareForWrite(RecordConsumer recordConsumer) {
+    this.recordConsumer = recordConsumer;
+  }
+
+  /**
+   * Writes a Tuple to the file.
+   *
+   * @param tuple The Tuple to write to the file.
+   */
+  @Override
+  public void write(Tuple tuple) {
+    recordConsumer.startMessage();
+    writeRecordFields(rootSchema, rootTajoSchema, tuple);
+    recordConsumer.endMessage();
+  }
+
+  private void writeRecordFields(GroupType schema, Schema tajoSchema,
+                                 Tuple tuple) {
+    List<Type> fields = schema.getFields();
+    // Parquet ignores Tajo NULL_TYPE columns, so the index may differ.
+    int index = 0;
+    for (int tajoIndex = 0; tajoIndex < tajoSchema.size(); ++tajoIndex) {
+      Column column = tajoSchema.getColumn(tajoIndex);
+      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE) {
+        continue;
+      }
+      Datum datum = tuple.get(tajoIndex);
+      Type fieldType = fields.get(index);
+      if (!tuple.isNull(tajoIndex)) {
+        recordConsumer.startField(fieldType.getName(), index);
+        writeValue(fieldType, column, datum);
+        recordConsumer.endField(fieldType.getName(), index);
+      } else if (fieldType.isRepetition(Type.Repetition.REQUIRED)) {
+        throw new RuntimeException("Null-value for required field: " +
+            column.getSimpleName());
+      }
+      ++index;
+    }
+  }
+
+  private void writeValue(Type fieldType, Column column, Datum datum) {
+    switch (column.getDataType().getType()) {
+      case BOOLEAN:
+        recordConsumer.addBoolean((Boolean) datum.asBool());
+        break;
+      case BIT:
+      case INT2:
+      case INT4:
+        recordConsumer.addInteger(datum.asInt4());
+        break;
+      case INT8:
+        recordConsumer.addLong(datum.asInt8());
+        break;
+      case FLOAT4:
+        recordConsumer.addFloat(datum.asFloat4());
+        break;
+      case FLOAT8:
+        recordConsumer.addDouble(datum.asFloat8());
+        break;
+      case CHAR:
+      case TEXT:
+        recordConsumer.addBinary(Binary.fromString(datum.asChars()));
+        break;
+      case PROTOBUF:
+      case BLOB:
+      case INET4:
+      case INET6:
+        recordConsumer.addBinary(Binary.fromByteArray(datum.asByteArray()));
+        break;
+      default:
+        break;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java
new file mode 100644
index 0000000..d7d16b7
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java
@@ -0,0 +1,96 @@
+/**
+ * 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.
+ */
+/**
+ * <p>
+ * Provides read and write support for Parquet files. Tajo schemas are
+ * converted to Parquet schemas according to the following mapping of Tajo
+ * and Parquet types:
+ * </p>
+ *
+ * <table>
+ *   <tr>
+ *     <th>Tajo type</th>
+ *     <th>Parquet type</th>
+ *   </tr>
+ *   <tr>
+ *     <td>NULL_TYPE</td>
+ *     <td>No type. The field is not encoded in Parquet.</td>
+ *   </tr>
+ *   <tr>
+ *     <td>BOOLEAN</td>
+ *     <td>BOOLEAN</td>
+ *   </tr>
+ *   <tr>
+ *     <td>BIT</td>
+ *     <td>INT32</td>
+ *   </tr>
+ *   <tr>
+ *     <td>INT2</td>
+ *     <td>INT32</td>
+ *   </tr>
+ *   <tr>
+ *     <td>INT4</td>
+ *     <td>INT32</td>
+ *   </tr>
+ *   <tr>
+ *     <td>INT8</td>
+ *     <td>INT64</td>
+ *   </tr>
+ *   <tr>
+ *     <td>FLOAT4</td>
+ *     <td>FLOAT</td>
+ *   </tr>
+ *   <tr>
+ *     <td>FLOAT8</td>
+ *     <td>DOUBLE</td>
+ *   </tr>
+ *   <tr>
+ *     <td>CHAR</td>
+ *     <td>BINARY (with OriginalType UTF8)</td>
+ *   </tr>
+ *   <tr>
+ *     <td>TEXT</td>
+ *     <td>BINARY (with OriginalType UTF8)</td>
+ *   </tr>
+ *   <tr>
+ *     <td>PROTOBUF</td>
+ *     <td>BINARY</td>
+ *   </tr>
+ *   <tr>
+ *     <td>BLOB</td>
+ *     <td>BINARY</td>
+ *   </tr>
+ *   <tr>
+ *     <td>INET4</td>
+ *     <td>BINARY</td>
+ *   </tr>
+ * </table>
+ *
+ * <p>
+ * Because Tajo fields can be NULL, all Parquet fields are marked as optional.
+ * </p>
+ *
+ * <p>
+ * The conversion from Tajo to Parquet is lossy without the original Tajo
+ * schema. As a result, Parquet files are read using the Tajo schema saved in
+ * the Tajo catalog for the table the Parquet files belong to, which was
+ * defined when the table was created.
+ * </p>
+ */
+
+package org.apache.tajo.storage.parquet;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
new file mode 100644
index 0000000..5e200a0
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
@@ -0,0 +1,261 @@
+/**
+ * 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.rcfile;
+
+import com.google.common.base.Objects;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * <tt>BytesRefArrayWritable</tt> holds an array reference to BytesRefWritable,
+ * and is able to resize without recreating new array if not necessary.
+ * <p>
+ *
+ * Each <tt>BytesRefArrayWritable holds</tt> instance has a <i>valid</i> field,
+ * which is the desired valid number of <tt>BytesRefWritable</tt> it holds.
+ * <tt>resetValid</tt> can reset the valid, but it will not care the underlying
+ * BytesRefWritable.
+ */
+
+public class BytesRefArrayWritable implements Writable,
+    Comparable<BytesRefArrayWritable> {
+
+  private BytesRefWritable[] bytesRefWritables = null;
+
+  private int valid = 0;
+
+  /**
+   * Constructs an empty array with the specified capacity.
+   *
+   * @param capacity
+   *          initial capacity
+   * @exception IllegalArgumentException
+   *              if the specified initial capacity is negative
+   */
+  public BytesRefArrayWritable(int capacity) {
+    if (capacity < 0) {
+      throw new IllegalArgumentException("Capacity can not be negative.");
+    }
+    bytesRefWritables = new BytesRefWritable[0];
+    ensureCapacity(capacity);
+  }
+
+  /**
+   * Constructs an empty array with a capacity of ten.
+   */
+  public BytesRefArrayWritable() {
+    this(10);
+  }
+
+  /**
+   * Returns the number of valid elements.
+   *
+   * @return the number of valid elements
+   */
+  public int size() {
+    return valid;
+  }
+
+  /**
+   * Gets the BytesRefWritable at the specified position. Make sure the position
+   * is valid by first call resetValid.
+   *
+   * @param index
+   *          the position index, starting from zero
+   * @throws IndexOutOfBoundsException
+   */
+  public BytesRefWritable get(int index) {
+    if (index >= valid) {
+      throw new IndexOutOfBoundsException(
+          "This BytesRefArrayWritable only has " + valid + " valid values.");
+    }
+    return bytesRefWritables[index];
+  }
+
+  /**
+   * Gets the BytesRefWritable at the specified position without checking.
+   *
+   * @param index
+   *          the position index, starting from zero
+   * @throws IndexOutOfBoundsException
+   */
+  public BytesRefWritable unCheckedGet(int index) {
+    return bytesRefWritables[index];
+  }
+
+  /**
+   * Set the BytesRefWritable at the specified position with the specified
+   * BytesRefWritable.
+   *
+   * @param index
+   *          index position
+   * @param bytesRefWritable
+   *          the new element
+   * @throws IllegalArgumentException
+   *           if the specified new element is null
+   */
+  public void set(int index, BytesRefWritable bytesRefWritable) {
+    if (bytesRefWritable == null) {
+      throw new IllegalArgumentException("Can not assign null.");
+    }
+    ensureCapacity(index + 1);
+    bytesRefWritables[index] = bytesRefWritable;
+    if (valid <= index) {
+      valid = index + 1;
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public int compareTo(BytesRefArrayWritable other) {
+    if (other == null) {
+      throw new IllegalArgumentException("Argument can not be null.");
+    }
+    if (this == other) {
+      return 0;
+    }
+    int sizeDiff = valid - other.valid;
+    if (sizeDiff != 0) {
+      return sizeDiff;
+    }
+    for (int i = 0; i < valid; i++) {
+      if (other.contains(bytesRefWritables[i])) {
+        continue;
+      } else {
+        return 1;
+      }
+    }
+    return 0;
+  }
+
+  @Override
+  public int hashCode(){
+    return Objects.hashCode(bytesRefWritables);
+  }
+  /**
+   * Returns <tt>true</tt> if this instance contains one or more the specified
+   * BytesRefWritable.
+   *
+   * @param bytesRefWritable
+   *          BytesRefWritable element to be tested
+   * @return <tt>true</tt> if contains the specified element
+   * @throws IllegalArgumentException
+   *           if the specified element is null
+   */
+  public boolean contains(BytesRefWritable bytesRefWritable) {
+    if (bytesRefWritable == null) {
+      throw new IllegalArgumentException("Argument can not be null.");
+    }
+    for (int i = 0; i < valid; i++) {
+      if (bytesRefWritables[i].equals(bytesRefWritable)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (o == null || !(o instanceof BytesRefArrayWritable)) {
+      return false;
+    }
+    return compareTo((BytesRefArrayWritable) o) == 0;
+  }
+
+  /**
+   * Removes all elements.
+   */
+  public void clear() {
+    valid = 0;
+  }
+
+  /**
+   * enlarge the capacity if necessary, to ensure that it can hold the number of
+   * elements specified by newValidCapacity argument. It will also narrow the
+   * valid capacity when needed. Notice: it only enlarge or narrow the valid
+   * capacity with no care of the already stored invalid BytesRefWritable.
+   *
+   * @param newValidCapacity
+   *          the desired capacity
+   */
+  public void resetValid(int newValidCapacity) {
+    ensureCapacity(newValidCapacity);
+    valid = newValidCapacity;
+  }
+
+  protected void ensureCapacity(int newCapacity) {
+    int size = bytesRefWritables.length;
+    if (size < newCapacity) {
+      bytesRefWritables = Arrays.copyOf(bytesRefWritables, newCapacity);
+      while (size < newCapacity) {
+        bytesRefWritables[size] = new BytesRefWritable();
+        size++;
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int count = in.readInt();
+    ensureCapacity(count);
+    for (int i = 0; i < count; i++) {
+      bytesRefWritables[i].readFields(in);
+    }
+    valid = count;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(valid);
+
+    for (int i = 0; i < valid; i++) {
+      BytesRefWritable cu = bytesRefWritables[i];
+      cu.write(out);
+    }
+  }
+
+  static {
+    WritableFactories.setFactory(BytesRefArrayWritable.class,
+        new WritableFactory() {
+
+          @Override
+          public Writable newInstance() {
+            return new BytesRefArrayWritable();
+          }
+
+        });
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
new file mode 100644
index 0000000..158c740
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
@@ -0,0 +1,248 @@
+/**
+ * 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.rcfile;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * <tt>BytesRefWritable</tt> referenced a section of byte array. It can be used
+ * to avoid unnecessary byte copy.
+ */
+public class BytesRefWritable implements Writable, Comparable<BytesRefWritable> {
+
+  private static final byte[] EMPTY_BYTES = new byte[0];
+  public static final BytesRefWritable ZeroBytesRefWritable = new BytesRefWritable();
+
+  int start = 0;
+  int length = 0;
+  byte[] bytes = null;
+
+  LazyDecompressionCallback lazyDecompressObj;
+
+  /**
+   * Create a zero-size bytes.
+   */
+  public BytesRefWritable() {
+    this(EMPTY_BYTES);
+  }
+
+  /**
+   * Create a BytesRefWritable with <tt>length</tt> bytes.
+   */
+  public BytesRefWritable(int length) {
+    assert length > 0;
+    this.length = length;
+    bytes = new byte[this.length];
+    start = 0;
+  }
+
+  /**
+   * Create a BytesRefWritable referenced to the given bytes.
+   */
+  public BytesRefWritable(byte[] bytes) {
+    this.bytes = bytes;
+    length = bytes.length;
+    start = 0;
+  }
+
+  /**
+   * Create a BytesRefWritable referenced to one section of the given bytes. The
+   * section is determined by argument <tt>offset</tt> and <tt>len</tt>.
+   */
+  public BytesRefWritable(byte[] data, int offset, int len) {
+    bytes = data;
+    start = offset;
+    length = len;
+  }
+
+  /**
+   * Create a BytesRefWritable referenced to one section of the given bytes. The
+   * argument <tt>lazyDecompressData</tt> refers to a LazyDecompressionCallback
+   * object. The arguments <tt>offset</tt> and <tt>len</tt> are referred to
+   * uncompressed bytes of <tt>lazyDecompressData</tt>. Use <tt>offset</tt> and
+   * <tt>len</tt> after uncompressing the data.
+   */
+  public BytesRefWritable(LazyDecompressionCallback lazyDecompressData,
+                          int offset, int len) {
+    lazyDecompressObj = lazyDecompressData;
+    start = offset;
+    length = len;
+  }
+
+  private void lazyDecompress() throws IOException {
+    if (bytes == null && lazyDecompressObj != null) {
+      bytes = lazyDecompressObj.decompress();
+    }
+  }
+
+  /**
+   * Returns a copy of the underlying bytes referenced by this instance.
+   *
+   * @return a new copied byte array
+   * @throws java.io.IOException
+   */
+  public byte[] getBytesCopy() throws IOException {
+    lazyDecompress();
+    byte[] bb = new byte[length];
+    System.arraycopy(bytes, start, bb, 0, length);
+    return bb;
+  }
+
+  /**
+   * Returns the underlying bytes.
+   *
+   * @throws java.io.IOException
+   */
+  public byte[] getData() throws IOException {
+    lazyDecompress();
+    return bytes;
+  }
+
+  /**
+   * readFields() will corrupt the array. So use the set method whenever
+   * possible.
+   *
+   * @see #readFields(java.io.DataInput)
+   */
+  public void set(byte[] newData, int offset, int len) {
+    bytes = newData;
+    start = offset;
+    length = len;
+    lazyDecompressObj = null;
+  }
+
+  /**
+   * readFields() will corrupt the array. So use the set method whenever
+   * possible.
+   *
+   * @see #readFields(java.io.DataInput)
+   */
+  public void set(LazyDecompressionCallback newData, int offset, int len) {
+    bytes = null;
+    start = offset;
+    length = len;
+    lazyDecompressObj = newData;
+  }
+
+  public void writeDataTo(DataOutput out) throws IOException {
+    lazyDecompress();
+    out.write(bytes, start, length);
+  }
+
+  /**
+   * Always reuse the bytes array if length of bytes array is equal or greater
+   * to the current record, otherwise create a new one. readFields will corrupt
+   * the array. Please use set() whenever possible.
+   *
+   * @see #set(byte[], int, int)
+   */
+  public void readFields(DataInput in) throws IOException {
+    int len = in.readInt();
+    if (len > bytes.length) {
+      bytes = new byte[len];
+    }
+    start = 0;
+    length = len;
+    in.readFully(bytes, start, length);
+  }
+
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    lazyDecompress();
+    out.writeInt(length);
+    out.write(bytes, start, length);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(3 * length);
+    for (int idx = start; idx < length; idx++) {
+      // if not the first, put a blank separator in
+      if (idx != 0) {
+        sb.append(' ');
+      }
+      String num = Integer.toHexString(0xff & bytes[idx]);
+      // if it is only one digit, add a leading 0.
+      if (num.length() < 2) {
+        sb.append('0');
+      }
+      sb.append(num);
+    }
+    return sb.toString();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int compareTo(BytesRefWritable other) {
+    if (other == null) {
+      throw new IllegalArgumentException("Argument can not be null.");
+    }
+    if (this == other) {
+      return 0;
+    }
+    try {
+      return WritableComparator.compareBytes(getData(), start, getLength(),
+          other.getData(), other.start, other.getLength());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public boolean equals(Object right_obj) {
+    if (right_obj == null || !(right_obj instanceof BytesRefWritable)) {
+      return false;
+    }
+    return compareTo((BytesRefWritable) right_obj) == 0;
+  }
+
+  static {
+    WritableFactories.setFactory(BytesRefWritable.class, new WritableFactory() {
+
+      @Override
+      public Writable newInstance() {
+        return new BytesRefWritable();
+      }
+
+    });
+  }
+
+  public int getLength() {
+    return length;
+  }
+
+  public int getStart() {
+    return start;
+  }
+}


[20/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
deleted file mode 100644
index 1448885..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/ByteBufLineReader.java
+++ /dev/null
@@ -1,154 +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.text;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.util.CharsetUtil;
-import org.apache.tajo.storage.BufferPool;
-import org.apache.tajo.storage.ByteBufInputChannel;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class ByteBufLineReader implements Closeable {
-  private static int DEFAULT_BUFFER = 64 * 1024;
-
-  private int bufferSize;
-  private long readBytes;
-  private ByteBuf buffer;
-  private final ByteBufInputChannel channel;
-  private final AtomicInteger tempReadBytes = new AtomicInteger();
-  private final LineSplitProcessor processor = new LineSplitProcessor();
-
-  public ByteBufLineReader(ByteBufInputChannel channel) {
-    this(channel, BufferPool.directBuffer(DEFAULT_BUFFER));
-  }
-
-  public ByteBufLineReader(ByteBufInputChannel channel, ByteBuf buf) {
-    this.readBytes = 0;
-    this.channel = channel;
-    this.buffer = buf;
-    this.bufferSize = buf.capacity();
-  }
-
-  public long readBytes() {
-    return readBytes - buffer.readableBytes();
-  }
-
-  public long available() throws IOException {
-    return channel.available() + buffer.readableBytes();
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (this.buffer.refCnt() > 0) {
-      this.buffer.release();
-    }
-    this.channel.close();
-  }
-
-  public String readLine() throws IOException {
-    ByteBuf buf = readLineBuf(tempReadBytes);
-    if (buf != null) {
-      return buf.toString(CharsetUtil.UTF_8);
-    }
-    return null;
-  }
-
-  private void fillBuffer() throws IOException {
-
-    int tailBytes = 0;
-    if (this.readBytes > 0) {
-      this.buffer.markReaderIndex();
-      this.buffer.discardSomeReadBytes();  // compact the buffer
-      tailBytes = this.buffer.writerIndex();
-      if (!this.buffer.isWritable()) {
-        // a line bytes is large than the buffer
-        BufferPool.ensureWritable(buffer, bufferSize);
-        this.bufferSize = buffer.capacity();
-      }
-    }
-
-    boolean release = true;
-    try {
-      int readBytes = tailBytes;
-      for (; ; ) {
-        int localReadBytes = buffer.writeBytes(channel, bufferSize - readBytes);
-        if (localReadBytes < 0) {
-          break;
-        }
-        readBytes += localReadBytes;
-        if (readBytes == bufferSize) {
-          break;
-        }
-      }
-      this.readBytes += (readBytes - tailBytes);
-      release = false;
-      this.buffer.readerIndex(this.buffer.readerIndex() + tailBytes); //skip past buffer (tail)
-    } finally {
-      if (release) {
-        buffer.release();
-      }
-    }
-  }
-
-  /**
-   * Read a line terminated by one of CR, LF, or CRLF.
-   */
-  public ByteBuf readLineBuf(AtomicInteger reads) throws IOException {
-    int startIndex = buffer.readerIndex();
-    int readBytes;
-    int readable;
-    int newlineLength; //length of terminating newline
-
-    loop:
-    while (true) {
-      readable = buffer.readableBytes();
-      if (readable <= 0) {
-        buffer.readerIndex(startIndex);
-        fillBuffer(); //compact and fill buffer
-        if (!buffer.isReadable()) {
-          return null;
-        } else {
-          startIndex = 0; // reset the line start position
-        }
-        readable = buffer.readableBytes();
-      }
-
-      int endIndex = buffer.forEachByte(buffer.readerIndex(), readable, processor);
-      if (endIndex < 0) {
-        buffer.readerIndex(buffer.writerIndex());
-      } else {
-        buffer.readerIndex(endIndex + 1);
-        readBytes = buffer.readerIndex() - startIndex;
-        if (processor.isPrevCharCR() && buffer.isReadable()
-            && buffer.getByte(buffer.readerIndex()) == LineSplitProcessor.LF) {
-          buffer.skipBytes(1);
-          newlineLength = 2;
-        } else {
-          newlineLength = 1;
-        }
-        break loop;
-      }
-    }
-    reads.set(readBytes);
-    return buffer.slice(startIndex, readBytes - newlineLength);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
deleted file mode 100644
index d9e2016..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java
+++ /dev/null
@@ -1,157 +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.text;
-
-import io.netty.buffer.ByteBuf;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.compress.SplittableCompressionCodec;
-import org.apache.tajo.common.exception.NotImplementedException;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.ByteBufInputChannel;
-import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.BufferPool;
-import org.apache.tajo.storage.compress.CodecPool;
-import org.apache.tajo.storage.fragment.FileFragment;
-
-import java.io.Closeable;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class DelimitedLineReader implements Closeable {
-  private static final Log LOG = LogFactory.getLog(DelimitedLineReader.class);
-  private final static int DEFAULT_PAGE_SIZE = 128 * 1024;
-
-  private FileSystem fs;
-  private FSDataInputStream fis;
-  private InputStream is; //decompressd stream
-  private CompressionCodecFactory factory;
-  private CompressionCodec codec;
-  private Decompressor decompressor;
-
-  private long startOffset, end, pos;
-  private boolean eof = true;
-  private ByteBufLineReader lineReader;
-  private AtomicInteger tempReadBytes = new AtomicInteger();
-  private FileFragment fragment;
-  private Configuration conf;
-
-  public DelimitedLineReader(Configuration conf, final FileFragment fragment) throws IOException {
-    this.fragment = fragment;
-    this.conf = conf;
-    this.factory = new CompressionCodecFactory(conf);
-    this.codec = factory.getCodec(fragment.getPath());
-    if (this.codec instanceof SplittableCompressionCodec) {
-      throw new NotImplementedException(); // bzip2 does not support multi-thread model
-    }
-  }
-
-  public void init() throws IOException {
-    if (fs == null) {
-      fs = FileScanner.getFileSystem((TajoConf) conf, fragment.getPath());
-    }
-    if (fis == null) fis = fs.open(fragment.getPath());
-    pos = startOffset = fragment.getStartKey();
-    end = startOffset + fragment.getLength();
-
-    if (codec != null) {
-      decompressor = CodecPool.getDecompressor(codec);
-      is = new DataInputStream(codec.createInputStream(fis, decompressor));
-      ByteBufInputChannel channel = new ByteBufInputChannel(is);
-      lineReader = new ByteBufLineReader(channel, BufferPool.directBuffer(DEFAULT_PAGE_SIZE));
-    } else {
-      fis.seek(startOffset);
-      is = fis;
-
-      ByteBufInputChannel channel = new ByteBufInputChannel(is);
-      lineReader = new ByteBufLineReader(channel,
-          BufferPool.directBuffer((int) Math.min(DEFAULT_PAGE_SIZE, end)));
-    }
-    eof = false;
-  }
-
-  public long getCompressedPosition() throws IOException {
-    long retVal;
-    if (isCompressed()) {
-      retVal = fis.getPos();
-    } else {
-      retVal = pos;
-    }
-    return retVal;
-  }
-
-  public long getUnCompressedPosition() throws IOException {
-    return pos;
-  }
-
-  public long getReadBytes() {
-    return pos - startOffset;
-  }
-
-  public boolean isReadable() {
-    return !eof;
-  }
-
-  public ByteBuf readLine() throws IOException {
-    if (eof) {
-      return null;
-    }
-
-    ByteBuf buf = lineReader.readLineBuf(tempReadBytes);
-    if (buf == null) {
-      eof = true;
-    } else {
-      pos += tempReadBytes.get();
-    }
-
-    if (!isCompressed() && getCompressedPosition() > end) {
-      eof = true;
-    }
-    return buf;
-  }
-
-  public boolean isCompressed() {
-    return codec != null;
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      IOUtils.cleanup(LOG, lineReader, is, fis);
-      fs = null;
-      is = null;
-      fis = null;
-      lineReader = null;
-    } finally {
-      if (decompressor != null) {
-        CodecPool.returnDecompressor(decompressor);
-        decompressor = null;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
deleted file mode 100644
index a337509..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java
+++ /dev/null
@@ -1,468 +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.text;
-
-import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.compress.CodecPool;
-import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
-
-import java.io.BufferedOutputStream;
-import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Arrays;
-
-public class DelimitedTextFile {
-
-  public static final byte LF = '\n';
-  public static int EOF = -1;
-
-  private static final Log LOG = LogFactory.getLog(DelimitedTextFile.class);
-
-  public static class DelimitedTextFileAppender extends FileAppender {
-    private final TableMeta meta;
-    private final Schema schema;
-    private final int columnNum;
-    private final FileSystem fs;
-    private FSDataOutputStream fos;
-    private DataOutputStream outputStream;
-    private CompressionOutputStream deflateFilter;
-    private char delimiter;
-    private TableStatistics stats = null;
-    private Compressor compressor;
-    private CompressionCodecFactory codecFactory;
-    private CompressionCodec codec;
-    private Path compressedPath;
-    private byte[] nullChars;
-    private int BUFFER_SIZE = 128 * 1024;
-    private int bufferedBytes = 0;
-    private long pos = 0;
-
-    private NonSyncByteArrayOutputStream os;
-    private FieldSerializerDeserializer serde;
-
-    public DelimitedTextFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
-                                     final Schema schema, final TableMeta meta, final Path path)
-        throws IOException {
-      super(conf, taskAttemptId, schema, meta, path);
-      this.fs = path.getFileSystem(conf);
-      this.meta = meta;
-      this.schema = schema;
-      this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_DELIMITER,
-          StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
-      this.columnNum = schema.size();
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.TEXT_NULL,
-          NullDatum.DEFAULT_TEXT));
-      if (StringUtils.isEmpty(nullCharacters)) {
-        nullChars = NullDatum.get().asTextBytes();
-      } else {
-        nullChars = nullCharacters.getBytes();
-      }
-    }
-
-    @Override
-    public void init() throws IOException {
-      if (!fs.exists(path.getParent())) {
-        throw new FileNotFoundException(path.toString());
-      }
-
-      if (this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
-        String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
-        codecFactory = new CompressionCodecFactory(conf);
-        codec = codecFactory.getCodecByClassName(codecName);
-        compressor = CodecPool.getCompressor(codec);
-        if (compressor != null) compressor.reset();  //builtin gzip is null
-
-        String extension = codec.getDefaultExtension();
-        compressedPath = path.suffix(extension);
-
-        if (fs.exists(compressedPath)) {
-          throw new AlreadyExistsStorageException(compressedPath);
-        }
-
-        fos = fs.create(compressedPath);
-        deflateFilter = codec.createOutputStream(fos, compressor);
-        outputStream = new DataOutputStream(deflateFilter);
-
-      } else {
-        if (fs.exists(path)) {
-          throw new AlreadyExistsStorageException(path);
-        }
-        fos = fs.create(path);
-        outputStream = new DataOutputStream(new BufferedOutputStream(fos));
-      }
-
-      if (enabledStats) {
-        this.stats = new TableStatistics(this.schema);
-      }
-
-      serde = new TextFieldSerializerDeserializer();
-
-      if (os == null) {
-        os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
-      }
-
-      os.reset();
-      pos = fos.getPos();
-      bufferedBytes = 0;
-      super.init();
-    }
-
-
-    @Override
-    public void addTuple(Tuple tuple) throws IOException {
-      Datum datum;
-      int rowBytes = 0;
-
-      for (int i = 0; i < columnNum; i++) {
-        datum = tuple.get(i);
-        rowBytes += serde.serialize(os, datum, schema.getColumn(i), i, nullChars);
-
-        if (columnNum - 1 > i) {
-          os.write((byte) delimiter);
-          rowBytes += 1;
-        }
-      }
-      os.write(LF);
-      rowBytes += 1;
-
-      pos += rowBytes;
-      bufferedBytes += rowBytes;
-      if (bufferedBytes > BUFFER_SIZE) {
-        flushBuffer();
-      }
-      // Statistical section
-      if (enabledStats) {
-        stats.incrementRow();
-      }
-    }
-
-    private void flushBuffer() throws IOException {
-      if (os.getLength() > 0) {
-        os.writeTo(outputStream);
-        os.reset();
-        bufferedBytes = 0;
-      }
-    }
-
-    @Override
-    public long getOffset() throws IOException {
-      return pos;
-    }
-
-    @Override
-    public void flush() throws IOException {
-      flushBuffer();
-      outputStream.flush();
-    }
-
-    @Override
-    public void close() throws IOException {
-
-      try {
-        if(outputStream != null){
-          flush();
-        }
-
-        // Statistical section
-        if (enabledStats) {
-          stats.setNumBytes(getOffset());
-        }
-
-        if (deflateFilter != null) {
-          deflateFilter.finish();
-          deflateFilter.resetState();
-          deflateFilter = null;
-        }
-
-        os.close();
-      } finally {
-        IOUtils.cleanup(LOG, fos);
-        if (compressor != null) {
-          CodecPool.returnCompressor(compressor);
-          compressor = null;
-        }
-      }
-    }
-
-    @Override
-    public TableStats getStats() {
-      if (enabledStats) {
-        return stats.getTableStat();
-      } else {
-        return null;
-      }
-    }
-
-    public boolean isCompress() {
-      return compressor != null;
-    }
-
-    public String getExtension() {
-      return codec != null ? codec.getDefaultExtension() : "";
-    }
-  }
-
-  public static class DelimitedTextFileScanner extends FileScanner {
-
-    private boolean splittable = false;
-    private final long startOffset;
-    private final long endOffset;
-
-    private int recordCount = 0;
-    private int[] targetColumnIndexes;
-
-    private ByteBuf nullChars;
-    private FieldSerializerDeserializer serde;
-    private DelimitedLineReader reader;
-    private FieldSplitProcessor processor;
-
-    public DelimitedTextFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
-                                    final Fragment fragment)
-        throws IOException {
-      super(conf, schema, meta, fragment);
-      reader = new DelimitedLineReader(conf, this.fragment);
-      if (!reader.isCompressed()) {
-        splittable = true;
-      }
-
-      startOffset = this.fragment.getStartKey();
-      endOffset = startOffset + fragment.getLength();
-
-      //Delimiter
-      String delim = meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-      this.processor = new FieldSplitProcessor(StringEscapeUtils.unescapeJava(delim).charAt(0));
-    }
-
-    @Override
-    public void init() throws IOException {
-      if (nullChars != null) {
-        nullChars.release();
-      }
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_NULL,
-          NullDatum.DEFAULT_TEXT));
-      byte[] bytes;
-      if (StringUtils.isEmpty(nullCharacters)) {
-        bytes = NullDatum.get().asTextBytes();
-      } else {
-        bytes = nullCharacters.getBytes();
-      }
-
-      nullChars = BufferPool.directBuffer(bytes.length, bytes.length);
-      nullChars.writeBytes(bytes);
-
-      if (reader != null) {
-        reader.close();
-      }
-      reader = new DelimitedLineReader(conf, fragment);
-      reader.init();
-      recordCount = 0;
-
-      if (targets == null) {
-        targets = schema.toArray();
-      }
-
-      targetColumnIndexes = new int[targets.length];
-      for (int i = 0; i < targets.length; i++) {
-        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
-      }
-
-      serde = new TextFieldSerializerDeserializer();
-
-      super.init();
-      Arrays.sort(targetColumnIndexes);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("DelimitedTextFileScanner open:" + fragment.getPath() + "," + startOffset + "," + endOffset);
-      }
-
-      if (startOffset > 0) {
-        reader.readLine();  // skip first line;
-      }
-    }
-
-    public ByteBuf readLine() throws IOException {
-      ByteBuf buf = reader.readLine();
-      if (buf == null) {
-        return null;
-      } else {
-        recordCount++;
-      }
-
-      return buf;
-    }
-
-    @Override
-    public float getProgress() {
-      try {
-        if (!reader.isReadable()) {
-          return 1.0f;
-        }
-        long filePos = reader.getCompressedPosition();
-        if (startOffset == filePos) {
-          return 0.0f;
-        } else {
-          long readBytes = filePos - startOffset;
-          long remainingBytes = Math.max(endOffset - filePos, 0);
-          return Math.min(1.0f, (float) (readBytes) / (float) (readBytes + remainingBytes));
-        }
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-        return 0.0f;
-      }
-    }
-
-    @Override
-    public Tuple next() throws IOException {
-      try {
-        if (!reader.isReadable()) return null;
-
-        ByteBuf buf = readLine();
-        if (buf == null) return null;
-
-        if (targets.length == 0) {
-          return EmptyTuple.get();
-        }
-
-        VTuple tuple = new VTuple(schema.size());
-        fillTuple(schema, tuple, buf, targetColumnIndexes);
-        return tuple;
-      } catch (Throwable t) {
-        LOG.error("Tuple list current index: " + recordCount + " file offset:" + reader.getCompressedPosition(), t);
-        throw new IOException(t);
-      }
-    }
-
-    private void fillTuple(Schema schema, Tuple dst, ByteBuf lineBuf, int[] target) throws IOException {
-      int[] projection = target;
-      if (lineBuf == null || target == null || target.length == 0) {
-        return;
-      }
-
-      final int rowLength = lineBuf.readableBytes();
-      int start = 0, fieldLength = 0, end = 0;
-
-      //Projection
-      int currentTarget = 0;
-      int currentIndex = 0;
-
-      while (end != -1) {
-        end = lineBuf.forEachByte(start, rowLength - start, processor);
-
-        if (end < 0) {
-          fieldLength = rowLength - start;
-        } else {
-          fieldLength = end - start;
-        }
-
-        if (projection.length > currentTarget && currentIndex == projection[currentTarget]) {
-          lineBuf.setIndex(start, start + fieldLength);
-          Datum datum = serde.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars);
-          dst.put(currentIndex, datum);
-          currentTarget++;
-        }
-
-        if (projection.length == currentTarget) {
-          break;
-        }
-
-        start = end + 1;
-        currentIndex++;
-      }
-    }
-
-    @Override
-    public void reset() throws IOException {
-      init();
-    }
-
-    @Override
-    public void close() throws IOException {
-      try {
-        if (nullChars != null) {
-          nullChars.release();
-          nullChars = null;
-        }
-
-        if (tableStats != null && reader != null) {
-          tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
-          tableStats.setNumRows(recordCount);
-        }
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("DelimitedTextFileScanner processed record:" + recordCount);
-        }
-      } finally {
-        IOUtils.cleanup(LOG, reader);
-        reader = null;
-      }
-    }
-
-    @Override
-    public boolean isProjectable() {
-      return true;
-    }
-
-    @Override
-    public boolean isSelectable() {
-      return false;
-    }
-
-    @Override
-    public void setSearchCondition(Object expr) {
-    }
-
-    @Override
-    public boolean isSplittable() {
-      return splittable;
-    }
-
-    @Override
-    public TableStats getInputStats() {
-      if (tableStats != null && reader != null) {
-        tableStats.setReadBytes(reader.getReadBytes());  //Actual Processed Bytes. (decompressed bytes + overhead)
-        tableStats.setNumRows(recordCount);
-        tableStats.setNumBytes(fragment.getLength());
-      }
-      return tableStats;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
deleted file mode 100644
index a5ac142..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/FieldSplitProcessor.java
+++ /dev/null
@@ -1,38 +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.text;
-
-import io.netty.buffer.ByteBufProcessor;
-
-public class FieldSplitProcessor implements ByteBufProcessor {
-  private char delimiter; //the ascii separate character
-
-  public FieldSplitProcessor(char recordDelimiterByte) {
-    this.delimiter = recordDelimiterByte;
-  }
-
-  @Override
-  public boolean process(byte value) throws Exception {
-    return delimiter != value;
-  }
-
-  public char getDelimiter() {
-    return delimiter;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
deleted file mode 100644
index a130527..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/LineSplitProcessor.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.text;
-
-import io.netty.buffer.ByteBufProcessor;
-
-public class LineSplitProcessor implements ByteBufProcessor {
-  public static final byte CR = '\r';
-  public static final byte LF = '\n';
-  private boolean prevCharCR = false; //true of prev char was CR
-
-  @Override
-  public boolean process(byte value) throws Exception {
-    switch (value) {
-      case LF:
-        return false;
-      case CR:
-        prevCharCR = true;
-        return false;
-      default:
-        prevCharCR = false;
-        return true;
-    }
-  }
-
-  public boolean isPrevCharCR() {
-    return prevCharCR;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
deleted file mode 100644
index 9722959..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java
+++ /dev/null
@@ -1,223 +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.text;
-
-import com.google.protobuf.Message;
-import io.netty.buffer.ByteBuf;
-import io.netty.util.CharsetUtil;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.*;
-import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
-import org.apache.tajo.storage.FieldSerializerDeserializer;
-import org.apache.tajo.util.NumberUtil;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.CharsetDecoder;
-
-//Compatibility with Apache Hive
-public class TextFieldSerializerDeserializer implements FieldSerializerDeserializer {
-  public static final byte[] trueBytes = "true".getBytes();
-  public static final byte[] falseBytes = "false".getBytes();
-  private ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
-  private final CharsetDecoder decoder = CharsetUtil.getDecoder(CharsetUtil.UTF_8);
-
-  private static boolean isNull(ByteBuf val, ByteBuf nullBytes) {
-    return !val.isReadable() || nullBytes.equals(val);
-  }
-
-  private static boolean isNullText(ByteBuf val, ByteBuf nullBytes) {
-    return val.readableBytes() > 0 && nullBytes.equals(val);
-  }
-
-  @Override
-  public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) throws IOException {
-    byte[] bytes;
-    int length = 0;
-    TajoDataTypes.DataType dataType = col.getDataType();
-
-    if (datum == null || datum instanceof NullDatum) {
-      switch (dataType.getType()) {
-        case CHAR:
-        case TEXT:
-          length = nullChars.length;
-          out.write(nullChars);
-          break;
-        default:
-          break;
-      }
-      return length;
-    }
-
-    switch (dataType.getType()) {
-      case BOOLEAN:
-        out.write(datum.asBool() ? trueBytes : falseBytes);
-        length = trueBytes.length;
-        break;
-      case CHAR:
-        byte[] pad = new byte[dataType.getLength() - datum.size()];
-        bytes = datum.asTextBytes();
-        out.write(bytes);
-        out.write(pad);
-        length = bytes.length + pad.length;
-        break;
-      case TEXT:
-      case BIT:
-      case INT2:
-      case INT4:
-      case INT8:
-      case FLOAT4:
-      case FLOAT8:
-      case INET4:
-      case DATE:
-      case INTERVAL:
-        bytes = datum.asTextBytes();
-        length = bytes.length;
-        out.write(bytes);
-        break;
-      case TIME:
-        bytes = ((TimeDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
-        length = bytes.length;
-        out.write(bytes);
-        break;
-      case TIMESTAMP:
-        bytes = ((TimestampDatum) datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
-        length = bytes.length;
-        out.write(bytes);
-        break;
-      case INET6:
-      case BLOB:
-        bytes = Base64.encodeBase64(datum.asByteArray(), false);
-        length = bytes.length;
-        out.write(bytes, 0, length);
-        break;
-      case PROTOBUF:
-        ProtobufDatum protobuf = (ProtobufDatum) datum;
-        byte[] protoBytes = protobufJsonFormat.printToString(protobuf.get()).getBytes();
-        length = protoBytes.length;
-        out.write(protoBytes, 0, protoBytes.length);
-        break;
-      case NULL_TYPE:
-      default:
-        break;
-    }
-    return length;
-  }
-
-  @Override
-  public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars) throws IOException {
-    Datum datum;
-    TajoDataTypes.Type type = col.getDataType().getType();
-    boolean nullField;
-    if (type == TajoDataTypes.Type.TEXT || type == TajoDataTypes.Type.CHAR) {
-      nullField = isNullText(buf, nullChars);
-    } else {
-      nullField = isNull(buf, nullChars);
-    }
-
-    if (nullField) {
-      datum = NullDatum.get();
-    } else {
-      switch (type) {
-        case BOOLEAN:
-          byte bool = buf.readByte();
-          datum = DatumFactory.createBool(bool == 't' || bool == 'T');
-          break;
-        case BIT:
-          datum = DatumFactory.createBit(Byte.parseByte(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()));
-          break;
-        case CHAR:
-          datum = DatumFactory.createChar(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString().trim());
-          break;
-        case INT1:
-        case INT2:
-          datum = DatumFactory.createInt2((short) NumberUtil.parseInt(buf));
-          break;
-        case INT4:
-          datum = DatumFactory.createInt4(NumberUtil.parseInt(buf));
-          break;
-        case INT8:
-          datum = DatumFactory.createInt8(NumberUtil.parseLong(buf));
-          break;
-        case FLOAT4:
-          datum = DatumFactory.createFloat4(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
-          break;
-        case FLOAT8:
-          datum = DatumFactory.createFloat8(NumberUtil.parseDouble(buf));
-          break;
-        case TEXT: {
-          byte[] bytes = new byte[buf.readableBytes()];
-          buf.readBytes(bytes);
-          datum = DatumFactory.createText(bytes);
-          break;
-        }
-        case DATE:
-          datum = DatumFactory.createDate(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
-          break;
-        case TIME:
-          datum = DatumFactory.createTime(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
-          break;
-        case TIMESTAMP:
-          datum = DatumFactory.createTimestamp(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
-          break;
-        case INTERVAL:
-          datum = DatumFactory.createInterval(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
-          break;
-        case PROTOBUF: {
-          ProtobufDatumFactory factory = ProtobufDatumFactory.get(col.getDataType());
-          Message.Builder builder = factory.newBuilder();
-          try {
-            byte[] bytes = new byte[buf.readableBytes()];
-            buf.readBytes(bytes);
-            protobufJsonFormat.merge(bytes, builder);
-            datum = factory.createDatum(builder.build());
-          } catch (IOException e) {
-            e.printStackTrace();
-            throw new RuntimeException(e);
-          }
-          break;
-        }
-        case INET4:
-          datum = DatumFactory.createInet4(
-              decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString());
-          break;
-        case BLOB: {
-          byte[] bytes = new byte[buf.readableBytes()];
-          buf.readBytes(bytes);
-          datum = DatumFactory.createBlob(Base64.decodeBase64(bytes));
-          break;
-        }
-        default:
-          datum = NullDatum.get();
-          break;
-      }
-    }
-    return datum;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java b/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
deleted file mode 100644
index 543336f..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/CodecFactory.java
+++ /dev/null
@@ -1,196 +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.thirdparty.parquet;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.HashMap;
-import java.util.Map;
-
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.util.ReflectionUtils;
-
-import parquet.bytes.BytesInput;
-import parquet.hadoop.BadConfigurationException;
-import parquet.hadoop.metadata.CompressionCodecName;
-
-class CodecFactory {
-
-  public class BytesDecompressor {
-
-    private final CompressionCodec codec;
-    private final Decompressor decompressor;
-
-    public BytesDecompressor(CompressionCodec codec) {
-      this.codec = codec;
-      if (codec != null) {
-        decompressor = CodecPool.getDecompressor(codec);
-      } else {
-        decompressor = null;
-      }
-    }
-
-    public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException {
-      final BytesInput decompressed;
-      if (codec != null) {
-        decompressor.reset();
-        InputStream is = codec.createInputStream(new ByteArrayInputStream(bytes.toByteArray()), decompressor);
-        decompressed = BytesInput.from(is, uncompressedSize);
-      } else {
-        decompressed = bytes;
-      }
-      return decompressed;
-    }
-
-    private void release() {
-      if (decompressor != null) {
-        CodecPool.returnDecompressor(decompressor);
-      }
-    }
-  }
-
-  /**
-   * Encapsulates the logic around hadoop compression
-   *
-   * @author Julien Le Dem
-   *
-   */
-  public static class BytesCompressor {
-
-    private final CompressionCodec codec;
-    private final Compressor compressor;
-    private final ByteArrayOutputStream compressedOutBuffer;
-    private final CompressionCodecName codecName;
-
-    public BytesCompressor(CompressionCodecName codecName, CompressionCodec codec, int pageSize) {
-      this.codecName = codecName;
-      this.codec = codec;
-      if (codec != null) {
-        this.compressor = CodecPool.getCompressor(codec);
-        this.compressedOutBuffer = new ByteArrayOutputStream(pageSize);
-      } else {
-        this.compressor = null;
-        this.compressedOutBuffer = null;
-      }
-    }
-
-    public BytesInput compress(BytesInput bytes) throws IOException {
-      final BytesInput compressedBytes;
-      if (codec == null) {
-        compressedBytes = bytes;
-      } else {
-        compressedOutBuffer.reset();
-        if (compressor != null) {
-          // null compressor for non-native gzip
-          compressor.reset();
-        }
-        CompressionOutputStream cos = codec.createOutputStream(compressedOutBuffer, compressor);
-        bytes.writeAllTo(cos);
-        cos.finish();
-        cos.close();
-        compressedBytes = BytesInput.from(compressedOutBuffer);
-      }
-      return compressedBytes;
-    }
-
-    private void release() {
-      if (compressor != null) {
-        CodecPool.returnCompressor(compressor);
-      }
-    }
-
-    public CompressionCodecName getCodecName() {
-      return codecName;
-    }
-
-  }
-
-  private final Map<CompressionCodecName, BytesCompressor> compressors = new HashMap<CompressionCodecName, BytesCompressor>();
-  private final Map<CompressionCodecName, BytesDecompressor> decompressors = new HashMap<CompressionCodecName, BytesDecompressor>();
-  private final Map<String, CompressionCodec> codecByName = new HashMap<String, CompressionCodec>();
-  private final Configuration configuration;
-
-  public CodecFactory(Configuration configuration) {
-    this.configuration = configuration;
-  }
-
-  /**
-   *
-   * @param codecName the requested codec
-   * @return the corresponding hadoop codec. null if UNCOMPRESSED
-   */
-  private CompressionCodec getCodec(CompressionCodecName codecName) {
-    String codecClassName = codecName.getHadoopCompressionCodecClassName();
-    if (codecClassName == null) {
-      return null;
-    }
-    CompressionCodec codec = codecByName.get(codecClassName);
-    if (codec != null) {
-      return codec;
-    }
-
-    try {
-      Class<?> codecClass = Class.forName(codecClassName);
-      codec = (CompressionCodec)ReflectionUtils.newInstance(codecClass, configuration);
-      codecByName.put(codecClassName, codec);
-      return codec;
-    } catch (ClassNotFoundException e) {
-      throw new BadConfigurationException("Class " + codecClassName + " was not found", e);
-    }
-  }
-
-  public BytesCompressor getCompressor(CompressionCodecName codecName, int pageSize) {
-    BytesCompressor comp = compressors.get(codecName);
-    if (comp == null) {
-      CompressionCodec codec = getCodec(codecName);
-      comp = new BytesCompressor(codecName, codec, pageSize);
-      compressors.put(codecName, comp);
-    }
-    return comp;
-  }
-
-  public BytesDecompressor getDecompressor(CompressionCodecName codecName) {
-    BytesDecompressor decomp = decompressors.get(codecName);
-    if (decomp == null) {
-      CompressionCodec codec = getCodec(codecName);
-      decomp = new BytesDecompressor(codec);
-      decompressors.put(codecName, decomp);
-    }
-    return decomp;
-  }
-
-  public void release() {
-    for (BytesCompressor compressor : compressors.values()) {
-      compressor.release();
-    }
-    compressors.clear();
-    for (BytesDecompressor decompressor : decompressors.values()) {
-      decompressor.release();
-    }
-    decompressors.clear();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java b/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
deleted file mode 100644
index 5f89ead..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ColumnChunkPageWriteStore.java
+++ /dev/null
@@ -1,211 +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.thirdparty.parquet;
-
-import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
-import static parquet.Log.INFO;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import parquet.Log;
-import parquet.bytes.BytesInput;
-import parquet.bytes.CapacityByteArrayOutputStream;
-import parquet.column.ColumnDescriptor;
-import parquet.column.Encoding;
-import parquet.column.page.DictionaryPage;
-import parquet.column.page.PageWriteStore;
-import parquet.column.page.PageWriter;
-import parquet.column.statistics.Statistics;
-import parquet.column.statistics.BooleanStatistics;
-import parquet.format.converter.ParquetMetadataConverter;
-import parquet.io.ParquetEncodingException;
-import parquet.schema.MessageType;
-
-class ColumnChunkPageWriteStore implements PageWriteStore {
-  private static final Log LOG = Log.getLog(ColumnChunkPageWriteStore.class);
-
-  private static ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
-
-  private static final class ColumnChunkPageWriter implements PageWriter {
-
-    private final ColumnDescriptor path;
-    private final BytesCompressor compressor;
-
-    private final CapacityByteArrayOutputStream buf;
-    private DictionaryPage dictionaryPage;
-
-    private long uncompressedLength;
-    private long compressedLength;
-    private long totalValueCount;
-    private int pageCount;
-
-    private Set<Encoding> encodings = new HashSet<Encoding>();
-
-    private Statistics totalStatistics;
-
-    private ColumnChunkPageWriter(ColumnDescriptor path, BytesCompressor compressor, int initialSize) {
-      this.path = path;
-      this.compressor = compressor;
-      this.buf = new CapacityByteArrayOutputStream(initialSize);
-      this.totalStatistics = Statistics.getStatsBasedOnType(this.path.getType());
-    }
-
-    @Deprecated
-    @Override
-    public void writePage(BytesInput bytes,
-                          int valueCount,
-                          Encoding rlEncoding,
-                          Encoding dlEncoding,
-                          Encoding valuesEncoding) throws IOException {
-      long uncompressedSize = bytes.size();
-      BytesInput compressedBytes = compressor.compress(bytes);
-      long compressedSize = compressedBytes.size();
-      BooleanStatistics statistics = new BooleanStatistics(); // dummy stats object
-      parquetMetadataConverter.writeDataPageHeader(
-          (int)uncompressedSize,
-          (int)compressedSize,
-          valueCount,
-          statistics,
-          rlEncoding,
-          dlEncoding,
-          valuesEncoding,
-          buf);
-      this.uncompressedLength += uncompressedSize;
-      this.compressedLength += compressedSize;
-      this.totalValueCount += valueCount;
-      this.pageCount += 1;
-      compressedBytes.writeAllTo(buf);
-      encodings.add(rlEncoding);
-      encodings.add(dlEncoding);
-      encodings.add(valuesEncoding);
-    }
-
-    @Override
-    public void writePage(BytesInput bytes,
-                          int valueCount,
-                          Statistics statistics,
-                          Encoding rlEncoding,
-                          Encoding dlEncoding,
-                          Encoding valuesEncoding) throws IOException {
-      long uncompressedSize = bytes.size();
-      BytesInput compressedBytes = compressor.compress(bytes);
-      long compressedSize = compressedBytes.size();
-      parquetMetadataConverter.writeDataPageHeader(
-          (int)uncompressedSize,
-          (int)compressedSize,
-          valueCount,
-          statistics,
-          rlEncoding,
-          dlEncoding,
-          valuesEncoding,
-          buf);
-      this.uncompressedLength += uncompressedSize;
-      this.compressedLength += compressedSize;
-      this.totalValueCount += valueCount;
-      this.pageCount += 1;
-      this.totalStatistics.mergeStatistics(statistics);
-      compressedBytes.writeAllTo(buf);
-      encodings.add(rlEncoding);
-      encodings.add(dlEncoding);
-      encodings.add(valuesEncoding);
-    }
-
-    @Override
-    public long getMemSize() {
-      return buf.size();
-    }
-
-    public void writeToFileWriter(ParquetFileWriter writer) throws IOException {
-      writer.startColumn(path, totalValueCount, compressor.getCodecName());
-      if (dictionaryPage != null) {
-        writer.writeDictionaryPage(dictionaryPage);
-        encodings.add(dictionaryPage.getEncoding());
-      }
-      writer.writeDataPages(BytesInput.from(buf), uncompressedLength, compressedLength, totalStatistics, new ArrayList<Encoding>(encodings));
-      writer.endColumn();
-      if (INFO) {
-        LOG.info(
-            String.format(
-                "written %,dB for %s: %,d values, %,dB raw, %,dB comp, %d pages, encodings: %s",
-                buf.size(), path, totalValueCount, uncompressedLength, compressedLength, pageCount, encodings)
-                + (dictionaryPage != null ? String.format(
-                ", dic { %,d entries, %,dB raw, %,dB comp}",
-                dictionaryPage.getDictionarySize(), dictionaryPage.getUncompressedSize(), dictionaryPage.getDictionarySize())
-                : ""));
-      }
-      encodings.clear();
-      pageCount = 0;
-    }
-
-    @Override
-    public long allocatedSize() {
-      return buf.getCapacity();
-    }
-
-    @Override
-    public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException {
-      if (this.dictionaryPage != null) {
-        throw new ParquetEncodingException("Only one dictionary page is allowed");
-      }
-      BytesInput dictionaryBytes = dictionaryPage.getBytes();
-      int uncompressedSize = (int)dictionaryBytes.size();
-      BytesInput compressedBytes = compressor.compress(dictionaryBytes);
-      this.dictionaryPage = new DictionaryPage(BytesInput.copy(compressedBytes), uncompressedSize, dictionaryPage.getDictionarySize(), dictionaryPage.getEncoding());
-    }
-
-    @Override
-    public String memUsageString(String prefix) {
-      return buf.memUsageString(prefix + " ColumnChunkPageWriter");
-    }
-  }
-
-  private final Map<ColumnDescriptor, ColumnChunkPageWriter> writers = new HashMap<ColumnDescriptor, ColumnChunkPageWriter>();
-  private final MessageType schema;
-  private final BytesCompressor compressor;
-  private final int initialSize;
-
-  public ColumnChunkPageWriteStore(BytesCompressor compressor, MessageType schema, int initialSize) {
-    this.compressor = compressor;
-    this.schema = schema;
-    this.initialSize = initialSize;
-  }
-
-  @Override
-  public PageWriter getPageWriter(ColumnDescriptor path) {
-    if (!writers.containsKey(path)) {
-      writers.put(path,  new ColumnChunkPageWriter(path, compressor, initialSize));
-    }
-    return writers.get(path);
-  }
-
-  public void flushToFileWriter(ParquetFileWriter writer) throws IOException {
-    List<ColumnDescriptor> columns = schema.getColumns();
-    for (ColumnDescriptor columnDescriptor : columns) {
-      ColumnChunkPageWriter pageWriter = writers.get(columnDescriptor);
-      pageWriter.writeToFileWriter(writer);
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
deleted file mode 100644
index 61567e5..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordReader.java
+++ /dev/null
@@ -1,187 +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.thirdparty.parquet;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import parquet.Log;
-import parquet.column.ColumnDescriptor;
-import parquet.column.page.PageReadStore;
-import parquet.filter.UnboundRecordFilter;
-import parquet.hadoop.ParquetFileReader;
-import parquet.hadoop.api.ReadSupport;
-import parquet.hadoop.metadata.BlockMetaData;
-import parquet.hadoop.util.counters.BenchmarkCounter;
-import parquet.io.ColumnIOFactory;
-import parquet.io.MessageColumnIO;
-import parquet.io.ParquetDecodingException;
-import parquet.io.api.RecordMaterializer;
-import parquet.schema.GroupType;
-import parquet.schema.MessageType;
-import parquet.schema.Type;
-
-import static java.lang.String.format;
-import static parquet.Log.DEBUG;
-
-class InternalParquetRecordReader<T> {
-  private static final Log LOG = Log.getLog(InternalParquetRecordReader.class);
-
-  private final ColumnIOFactory columnIOFactory = new ColumnIOFactory();
-
-  private MessageType requestedSchema;
-  private MessageType fileSchema;
-  private int columnCount;
-  private final ReadSupport<T> readSupport;
-
-  private RecordMaterializer<T> recordConverter;
-
-  private T currentValue;
-  private long total;
-  private int current = 0;
-  private int currentBlock = -1;
-  private ParquetFileReader reader;
-  private parquet.io.RecordReader<T> recordReader;
-  private UnboundRecordFilter recordFilter;
-
-  private long totalTimeSpentReadingBytes;
-  private long totalTimeSpentProcessingRecords;
-  private long startedAssemblingCurrentBlockAt;
-
-  private long totalCountLoadedSoFar = 0;
-
-  private Path file;
-
-  /**
-   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
-   */
-  public InternalParquetRecordReader(ReadSupport<T> readSupport) {
-    this(readSupport, null);
-  }
-
-  /**
-   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
-   * @param filter Optional filter for only returning matching records.
-   */
-  public InternalParquetRecordReader(ReadSupport<T> readSupport, UnboundRecordFilter
-      filter) {
-    this.readSupport = readSupport;
-    this.recordFilter = filter;
-  }
-
-  private void checkRead() throws IOException {
-    if (current == totalCountLoadedSoFar) {
-      if (current != 0) {
-        long timeAssembling = System.currentTimeMillis() - startedAssemblingCurrentBlockAt;
-        totalTimeSpentProcessingRecords += timeAssembling;
-        LOG.info("Assembled and processed " + totalCountLoadedSoFar + " records from " + columnCount + " columns in " + totalTimeSpentProcessingRecords + " ms: "+((float)totalCountLoadedSoFar / totalTimeSpentProcessingRecords) + " rec/ms, " + ((float)totalCountLoadedSoFar * columnCount / totalTimeSpentProcessingRecords) + " cell/ms");
-        long totalTime = totalTimeSpentProcessingRecords + totalTimeSpentReadingBytes;
-        long percentReading = 100 * totalTimeSpentReadingBytes / totalTime;
-        long percentProcessing = 100 * totalTimeSpentProcessingRecords / totalTime;
-        LOG.info("time spent so far " + percentReading + "% reading ("+totalTimeSpentReadingBytes+" ms) and " + percentProcessing + "% processing ("+totalTimeSpentProcessingRecords+" ms)");
-      }
-
-      LOG.info("at row " + current + ". reading next block");
-      long t0 = System.currentTimeMillis();
-      PageReadStore pages = reader.readNextRowGroup();
-      if (pages == null) {
-        throw new IOException("expecting more rows but reached last block. Read " + current + " out of " + total);
-      }
-      long timeSpentReading = System.currentTimeMillis() - t0;
-      totalTimeSpentReadingBytes += timeSpentReading;
-      BenchmarkCounter.incrementTime(timeSpentReading);
-      LOG.info("block read in memory in " + timeSpentReading + " ms. row count = " + pages.getRowCount());
-      if (Log.DEBUG) LOG.debug("initializing Record assembly with requested schema " + requestedSchema);
-      MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema, fileSchema);
-      recordReader = columnIO.getRecordReader(pages, recordConverter, recordFilter);
-      startedAssemblingCurrentBlockAt = System.currentTimeMillis();
-      totalCountLoadedSoFar += pages.getRowCount();
-      ++ currentBlock;
-    }
-  }
-
-  public void close() throws IOException {
-    reader.close();
-  }
-
-  public Void getCurrentKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  public T getCurrentValue() throws IOException,
-      InterruptedException {
-    return currentValue;
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return (float) current / total;
-  }
-
-  public void initialize(MessageType requestedSchema, MessageType fileSchema,
-                         Map<String, String> extraMetadata, Map<String, String> readSupportMetadata,
-                         Path file, List<BlockMetaData> blocks, Configuration configuration)
-      throws IOException {
-    this.requestedSchema = requestedSchema;
-    this.fileSchema = fileSchema;
-    this.file = file;
-    this.columnCount = this.requestedSchema.getPaths().size();
-    this.recordConverter = readSupport.prepareForRead(
-        configuration, extraMetadata, fileSchema,
-        new ReadSupport.ReadContext(requestedSchema, readSupportMetadata));
-
-    List<ColumnDescriptor> columns = requestedSchema.getColumns();
-    reader = new ParquetFileReader(configuration, file, blocks, columns);
-    for (BlockMetaData block : blocks) {
-      total += block.getRowCount();
-    }
-    LOG.info("RecordReader initialized will read a total of " + total + " records.");
-  }
-
-  private boolean contains(GroupType group, String[] path, int index) {
-    if (index == path.length) {
-      return false;
-    }
-    if (group.containsField(path[index])) {
-      Type type = group.getType(path[index]);
-      if (type.isPrimitive()) {
-        return index + 1 == path.length;
-      } else {
-        return contains(type.asGroupType(), path, index + 1);
-      }
-    }
-    return false;
-  }
-
-  public boolean nextKeyValue() throws IOException, InterruptedException {
-    if (current < total) {
-      try {
-        checkRead();
-        currentValue = recordReader.read();
-        if (DEBUG) LOG.debug("read value: " + currentValue);
-        current ++;
-      } catch (RuntimeException e) {
-        throw new ParquetDecodingException(format("Can not read value at %d in block %d in file %s", current, currentBlock, file), e);
-      }
-      return true;
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
deleted file mode 100644
index 7410d2b..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/InternalParquetRecordWriter.java
+++ /dev/null
@@ -1,160 +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.thirdparty.parquet;
-
-import static java.lang.Math.max;
-import static java.lang.Math.min;
-import static java.lang.String.format;
-import static org.apache.tajo.storage.thirdparty.parquet.CodecFactory.BytesCompressor;
-import static parquet.Log.DEBUG;
-import static parquet.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.util.Map;
-
-import parquet.Log;
-import parquet.column.ParquetProperties.WriterVersion;
-import parquet.column.impl.ColumnWriteStoreImpl;
-import parquet.hadoop.api.WriteSupport;
-import parquet.io.ColumnIOFactory;
-import parquet.io.MessageColumnIO;
-import parquet.schema.MessageType;
-
-class InternalParquetRecordWriter<T> {
-  private static final Log LOG = Log.getLog(InternalParquetRecordWriter.class);
-
-  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
-  private static final int MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
-  private static final int MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
-
-  private final ParquetFileWriter w;
-  private final WriteSupport<T> writeSupport;
-  private final MessageType schema;
-  private final Map<String, String> extraMetaData;
-  private final int blockSize;
-  private final int pageSize;
-  private final BytesCompressor compressor;
-  private final int dictionaryPageSize;
-  private final boolean enableDictionary;
-  private final boolean validating;
-  private final WriterVersion writerVersion;
-
-  private long recordCount = 0;
-  private long recordCountForNextMemCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
-
-  private ColumnWriteStoreImpl store;
-  private ColumnChunkPageWriteStore pageStore;
-
-  /**
-   * @param w the file to write to
-   * @param writeSupport the class to convert incoming records
-   * @param schema the schema of the records
-   * @param extraMetaData extra meta data to write in the footer of the file
-   * @param blockSize the size of a block in the file (this will be approximate)
-   * @param codec the codec used to compress
-   */
-  public InternalParquetRecordWriter(
-      ParquetFileWriter w,
-      WriteSupport<T> writeSupport,
-      MessageType schema,
-      Map<String, String> extraMetaData,
-      int blockSize,
-      int pageSize,
-      BytesCompressor compressor,
-      int dictionaryPageSize,
-      boolean enableDictionary,
-      boolean validating,
-      WriterVersion writerVersion) {
-    this.w = w;
-    this.writeSupport = checkNotNull(writeSupport, "writeSupport");
-    this.schema = schema;
-    this.extraMetaData = extraMetaData;
-    this.blockSize = blockSize;
-    this.pageSize = pageSize;
-    this.compressor = compressor;
-    this.dictionaryPageSize = dictionaryPageSize;
-    this.enableDictionary = enableDictionary;
-    this.validating = validating;
-    this.writerVersion = writerVersion;
-    initStore();
-  }
-
-  private void initStore() {
-    // we don't want this number to be too small
-    // ideally we divide the block equally across the columns
-    // it is unlikely all columns are going to be the same size.
-    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize / schema.getColumns().size() / 5);
-    pageStore = new ColumnChunkPageWriteStore(compressor, schema, initialBlockBufferSize);
-    // we don't want this number to be too small either
-    // ideally, slightly bigger than the page size, but not bigger than the block buffer
-    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize + pageSize / 10, initialBlockBufferSize));
-    store = new ColumnWriteStoreImpl(pageStore, pageSize, initialPageBufferSize, dictionaryPageSize, enableDictionary, writerVersion);
-    MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(schema);
-    writeSupport.prepareForWrite(columnIO.getRecordWriter(store));
-  }
-
-  public void close() throws IOException, InterruptedException {
-    flushStore();
-    w.end(extraMetaData);
-  }
-
-  public void write(T value) throws IOException, InterruptedException {
-    writeSupport.write(value);
-    ++ recordCount;
-    checkBlockSizeReached();
-  }
-
-  private void checkBlockSizeReached() throws IOException {
-    if (recordCount >= recordCountForNextMemCheck) { // checking the memory size is relatively expensive, so let's not do it for every record.
-      long memSize = store.memSize();
-      if (memSize > blockSize) {
-        LOG.info(format("mem size %,d > %,d: flushing %,d records to disk.", memSize, blockSize, recordCount));
-        flushStore();
-        initStore();
-        recordCountForNextMemCheck = min(max(MINIMUM_RECORD_COUNT_FOR_CHECK, recordCount / 2), MAXIMUM_RECORD_COUNT_FOR_CHECK);
-      } else {
-        float recordSize = (float) memSize / recordCount;
-        recordCountForNextMemCheck = min(
-            max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(blockSize / recordSize)) / 2), // will check halfway
-            recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead
-        );
-        if (DEBUG) LOG.debug(format("Checked mem at %,d will check again at: %,d ", recordCount, recordCountForNextMemCheck));
-      }
-    }
-  }
-
-  public long getEstimatedWrittenSize() throws IOException {
-    return w.getPos() + store.memSize();
-  }
-
-  private void flushStore()
-      throws IOException {
-    LOG.info(format("Flushing mem store to file. allocated memory: %,d", store.allocatedSize()));
-    if (store.allocatedSize() > 3 * blockSize) {
-      LOG.warn("Too much memory used: " + store.memUsageString());
-    }
-    w.startBlock(recordCount);
-    store.flush();
-    pageStore.flushToFileWriter(w);
-    recordCount = 0;
-    w.endBlock();
-    store = null;
-    pageStore = null;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
deleted file mode 100644
index 73ce7c2..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/thirdparty/parquet/ParquetFileWriter.java
+++ /dev/null
@@ -1,504 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.thirdparty.parquet;
-
-import static parquet.Log.DEBUG;
-import static parquet.format.Util.writeFileMetaData;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-import parquet.Log;
-import parquet.Version;
-import parquet.bytes.BytesInput;
-import parquet.bytes.BytesUtils;
-import parquet.column.ColumnDescriptor;
-import parquet.column.page.DictionaryPage;
-import parquet.column.statistics.Statistics;
-import parquet.format.converter.ParquetMetadataConverter;
-import parquet.hadoop.Footer;
-import parquet.hadoop.metadata.BlockMetaData;
-import parquet.hadoop.metadata.ColumnChunkMetaData;
-import parquet.hadoop.metadata.ColumnPath;
-import parquet.hadoop.metadata.CompressionCodecName;
-import parquet.hadoop.metadata.FileMetaData;
-import parquet.hadoop.metadata.GlobalMetaData;
-import parquet.hadoop.metadata.ParquetMetadata;
-import parquet.io.ParquetEncodingException;
-import parquet.schema.MessageType;
-import parquet.schema.PrimitiveType.PrimitiveTypeName;
-
-/**
- * Internal implementation of the Parquet file writer as a block container
- *
- * @author Julien Le Dem
- *
- */
-public class ParquetFileWriter {
-  private static final Log LOG = Log.getLog(ParquetFileWriter.class);
-
-  public static final String PARQUET_METADATA_FILE = "_metadata";
-  public static final byte[] MAGIC = "PAR1".getBytes(Charset.forName("ASCII"));
-  public static final int CURRENT_VERSION = 1;
-
-  private static final ParquetMetadataConverter metadataConverter = new ParquetMetadataConverter();
-
-  private final MessageType schema;
-  private final FSDataOutputStream out;
-  private BlockMetaData currentBlock;
-  private ColumnChunkMetaData currentColumn;
-  private long currentRecordCount;
-  private List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
-  private long uncompressedLength;
-  private long compressedLength;
-  private Set<parquet.column.Encoding> currentEncodings;
-
-  private CompressionCodecName currentChunkCodec;
-  private ColumnPath currentChunkPath;
-  private PrimitiveTypeName currentChunkType;
-  private long currentChunkFirstDataPage;
-  private long currentChunkDictionaryPageOffset;
-  private long currentChunkValueCount;
-
-  private Statistics currentStatistics;
-
-  /**
-   * Captures the order in which methods should be called
-   *
-   * @author Julien Le Dem
-   *
-   */
-  private enum STATE {
-    NOT_STARTED {
-      STATE start() {
-        return STARTED;
-      }
-    },
-    STARTED {
-      STATE startBlock() {
-        return BLOCK;
-      }
-      STATE end() {
-        return ENDED;
-      }
-    },
-    BLOCK  {
-      STATE startColumn() {
-        return COLUMN;
-      }
-      STATE endBlock() {
-        return STARTED;
-      }
-    },
-    COLUMN {
-      STATE endColumn() {
-        return BLOCK;
-      };
-      STATE write() {
-        return this;
-      }
-    },
-    ENDED;
-
-    STATE start() throws IOException { return error(); }
-    STATE startBlock() throws IOException { return error(); }
-    STATE startColumn() throws IOException { return error(); }
-    STATE write() throws IOException { return error(); }
-    STATE endColumn() throws IOException { return error(); }
-    STATE endBlock() throws IOException { return error(); }
-    STATE end() throws IOException { return error(); }
-
-    private final STATE error() throws IOException {
-      throw new IOException("The file being written is in an invalid state. Probably caused by an error thrown previously. Current state: " + this.name());
-    }
-  }
-
-  private STATE state = STATE.NOT_STARTED;
-
-  /**
-   *
-   * @param schema the schema of the data
-   * @param out the file to write to
-   * @param codec the codec to use to compress blocks
-   * @throws IOException if the file can not be created
-   */
-  public ParquetFileWriter(Configuration configuration, MessageType schema, Path file) throws IOException {
-    super();
-    this.schema = schema;
-    FileSystem fs = file.getFileSystem(configuration);
-    this.out = fs.create(file, false);
-  }
-
-  /**
-   * start the file
-   * @throws IOException
-   */
-  public void start() throws IOException {
-    state = state.start();
-    if (DEBUG) LOG.debug(out.getPos() + ": start");
-    out.write(MAGIC);
-  }
-
-  /**
-   * start a block
-   * @param recordCount the record count in this block
-   * @throws IOException
-   */
-  public void startBlock(long recordCount) throws IOException {
-    state = state.startBlock();
-    if (DEBUG) LOG.debug(out.getPos() + ": start block");
-//    out.write(MAGIC); // TODO: add a magic delimiter
-    currentBlock = new BlockMetaData();
-    currentRecordCount = recordCount;
-  }
-
-  /**
-   * start a column inside a block
-   * @param descriptor the column descriptor
-   * @param valueCount the value count in this column
-   * @param statistics the statistics in this column
-   * @param compressionCodecName
-   * @throws IOException
-   */
-  public void startColumn(ColumnDescriptor descriptor,
-                          long valueCount,
-                          CompressionCodecName compressionCodecName) throws IOException {
-    state = state.startColumn();
-    if (DEBUG) LOG.debug(out.getPos() + ": start column: " + descriptor + " count=" + valueCount);
-    currentEncodings = new HashSet<parquet.column.Encoding>();
-    currentChunkPath = ColumnPath.get(descriptor.getPath());
-    currentChunkType = descriptor.getType();
-    currentChunkCodec = compressionCodecName;
-    currentChunkValueCount = valueCount;
-    currentChunkFirstDataPage = out.getPos();
-    compressedLength = 0;
-    uncompressedLength = 0;
-    // need to know what type of stats to initialize to
-    // better way to do this?
-    currentStatistics = Statistics.getStatsBasedOnType(currentChunkType);
-  }
-
-  /**
-   * writes a dictionary page page
-   * @param dictionaryPage the dictionary page
-   */
-  public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException {
-    state = state.write();
-    if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page: " + dictionaryPage.getDictionarySize() + " values");
-    currentChunkDictionaryPageOffset = out.getPos();
-    int uncompressedSize = dictionaryPage.getUncompressedSize();
-    int compressedPageSize = (int)dictionaryPage.getBytes().size(); // TODO: fix casts
-    metadataConverter.writeDictionaryPageHeader(
-        uncompressedSize,
-        compressedPageSize,
-        dictionaryPage.getDictionarySize(),
-        dictionaryPage.getEncoding(),
-        out);
-    long headerSize = out.getPos() - currentChunkDictionaryPageOffset;
-    this.uncompressedLength += uncompressedSize + headerSize;
-    this.compressedLength += compressedPageSize + headerSize;
-    if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page content " + compressedPageSize);
-    dictionaryPage.getBytes().writeAllTo(out);
-    currentEncodings.add(dictionaryPage.getEncoding());
-  }
-
-
-  /**
-   * writes a single page
-   * @param valueCount count of values
-   * @param uncompressedPageSize the size of the data once uncompressed
-   * @param bytes the compressed data for the page without header
-   * @param rlEncoding encoding of the repetition level
-   * @param dlEncoding encoding of the definition level
-   * @param valuesEncoding encoding of values
-   */
-  @Deprecated
-  public void writeDataPage(
-      int valueCount, int uncompressedPageSize,
-      BytesInput bytes,
-      parquet.column.Encoding rlEncoding,
-      parquet.column.Encoding dlEncoding,
-      parquet.column.Encoding valuesEncoding) throws IOException {
-    state = state.write();
-    long beforeHeader = out.getPos();
-    if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values");
-    int compressedPageSize = (int)bytes.size();
-    metadataConverter.writeDataPageHeader(
-        uncompressedPageSize, compressedPageSize,
-        valueCount,
-        rlEncoding,
-        dlEncoding,
-        valuesEncoding,
-        out);
-    long headerSize = out.getPos() - beforeHeader;
-    this.uncompressedLength += uncompressedPageSize + headerSize;
-    this.compressedLength += compressedPageSize + headerSize;
-    if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize);
-    bytes.writeAllTo(out);
-    currentEncodings.add(rlEncoding);
-    currentEncodings.add(dlEncoding);
-    currentEncodings.add(valuesEncoding);
-  }
-
-  /**
-   * writes a single page
-   * @param valueCount count of values
-   * @param uncompressedPageSize the size of the data once uncompressed
-   * @param bytes the compressed data for the page without header
-   * @param rlEncoding encoding of the repetition level
-   * @param dlEncoding encoding of the definition level
-   * @param valuesEncoding encoding of values
-   */
-  public void writeDataPage(
-      int valueCount, int uncompressedPageSize,
-      BytesInput bytes,
-      Statistics statistics,
-      parquet.column.Encoding rlEncoding,
-      parquet.column.Encoding dlEncoding,
-      parquet.column.Encoding valuesEncoding) throws IOException {
-    state = state.write();
-    long beforeHeader = out.getPos();
-    if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values");
-    int compressedPageSize = (int)bytes.size();
-    metadataConverter.writeDataPageHeader(
-        uncompressedPageSize, compressedPageSize,
-        valueCount,
-        statistics,
-        rlEncoding,
-        dlEncoding,
-        valuesEncoding,
-        out);
-    long headerSize = out.getPos() - beforeHeader;
-    this.uncompressedLength += uncompressedPageSize + headerSize;
-    this.compressedLength += compressedPageSize + headerSize;
-    if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize);
-    bytes.writeAllTo(out);
-    currentStatistics.mergeStatistics(statistics);
-    currentEncodings.add(rlEncoding);
-    currentEncodings.add(dlEncoding);
-    currentEncodings.add(valuesEncoding);
-  }
-
-  /**
-   * writes a number of pages at once
-   * @param bytes bytes to be written including page headers
-   * @param uncompressedTotalPageSize total uncompressed size (without page headers)
-   * @param compressedTotalPageSize total compressed size (without page headers)
-   * @throws IOException
-   */
-  void writeDataPages(BytesInput bytes,
-                      long uncompressedTotalPageSize,
-                      long compressedTotalPageSize,
-                      Statistics totalStats,
-                      List<parquet.column.Encoding> encodings) throws IOException {
-    state = state.write();
-    if (DEBUG) LOG.debug(out.getPos() + ": write data pages");
-    long headersSize = bytes.size() - compressedTotalPageSize;
-    this.uncompressedLength += uncompressedTotalPageSize + headersSize;
-    this.compressedLength += compressedTotalPageSize + headersSize;
-    if (DEBUG) LOG.debug(out.getPos() + ": write data pages content");
-    bytes.writeAllTo(out);
-    currentEncodings.addAll(encodings);
-    currentStatistics = totalStats;
-  }
-
-  /**
-   * end a column (once all rep, def and data have been written)
-   * @throws IOException
-   */
-  public void endColumn() throws IOException {
-    state = state.endColumn();
-    if (DEBUG) LOG.debug(out.getPos() + ": end column");
-    currentBlock.addColumn(ColumnChunkMetaData.get(
-        currentChunkPath,
-        currentChunkType,
-        currentChunkCodec,
-        currentEncodings,
-        currentStatistics,
-        currentChunkFirstDataPage,
-        currentChunkDictionaryPageOffset,
-        currentChunkValueCount,
-        compressedLength,
-        uncompressedLength));
-    if (DEBUG) LOG.info("ended Column chumk: " + currentColumn);
-    currentColumn = null;
-    this.currentBlock.setTotalByteSize(currentBlock.getTotalByteSize() + uncompressedLength);
-    this.uncompressedLength = 0;
-    this.compressedLength = 0;
-  }
-
-  /**
-   * ends a block once all column chunks have been written
-   * @throws IOException
-   */
-  public void endBlock() throws IOException {
-    state = state.endBlock();
-    if (DEBUG) LOG.debug(out.getPos() + ": end block");
-    currentBlock.setRowCount(currentRecordCount);
-    blocks.add(currentBlock);
-    currentBlock = null;
-  }
-
-  /**
-   * ends a file once all blocks have been written.
-   * closes the file.
-   * @param extraMetaData the extra meta data to write in the footer
-   * @throws IOException
-   */
-  public void end(Map<String, String> extraMetaData) throws IOException {
-    state = state.end();
-    if (DEBUG) LOG.debug(out.getPos() + ": end");
-    ParquetMetadata footer = new ParquetMetadata(new FileMetaData(schema, extraMetaData, Version.FULL_VERSION), blocks);
-    serializeFooter(footer, out);
-    out.close();
-  }
-
-  private static void serializeFooter(ParquetMetadata footer, FSDataOutputStream out) throws IOException {
-    long footerIndex = out.getPos();
-    parquet.format.FileMetaData parquetMetadata = new ParquetMetadataConverter().toParquetMetadata(CURRENT_VERSION, footer);
-    writeFileMetaData(parquetMetadata, out);
-    if (DEBUG) LOG.debug(out.getPos() + ": footer length = " + (out.getPos() - footerIndex));
-    BytesUtils.writeIntLittleEndian(out, (int)(out.getPos() - footerIndex));
-    out.write(MAGIC);
-  }
-
-  /**
-   * writes a _metadata file
-   * @param configuration the configuration to use to get the FileSystem
-   * @param outputPath the directory to write the _metadata file to
-   * @param footers the list of footers to merge
-   * @throws IOException
-   */
-  public static void writeMetadataFile(Configuration configuration, Path outputPath, List<Footer> footers) throws IOException {
-    Path metaDataPath = new Path(outputPath, PARQUET_METADATA_FILE);
-    FileSystem fs = outputPath.getFileSystem(configuration);
-    outputPath = outputPath.makeQualified(fs);
-    FSDataOutputStream metadata = fs.create(metaDataPath);
-    metadata.write(MAGIC);
-    ParquetMetadata metadataFooter = mergeFooters(outputPath, footers);
-    serializeFooter(metadataFooter, metadata);
-    metadata.close();
-  }
-
-  private static ParquetMetadata mergeFooters(Path root, List<Footer> footers) {
-    String rootPath = root.toString();
-    GlobalMetaData fileMetaData = null;
-    List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
-    for (Footer footer : footers) {
-      String path = footer.getFile().toString();
-      if (!path.startsWith(rootPath)) {
-        throw new ParquetEncodingException(path + " invalid: all the files must be contained in the root " + root);
-      }
-      path = path.substring(rootPath.length());
-      while (path.startsWith("/")) {
-        path = path.substring(1);
-      }
-      fileMetaData = mergeInto(footer.getParquetMetadata().getFileMetaData(), fileMetaData);
-      for (BlockMetaData block : footer.getParquetMetadata().getBlocks()) {
-        block.setPath(path);
-        blocks.add(block);
-      }
-    }
-    return new ParquetMetadata(fileMetaData.merge(), blocks);
-  }
-
-  /**
-   * @return the current position in the underlying file
-   * @throws IOException
-   */
-  public long getPos() throws IOException {
-    return out.getPos();
-  }
-
-  /**
-   * Will merge the metadata of all the footers together
-   * @param footers the list files footers to merge
-   * @return the global meta data for all the footers
-   */
-  static GlobalMetaData getGlobalMetaData(List<Footer> footers) {
-    GlobalMetaData fileMetaData = null;
-    for (Footer footer : footers) {
-      ParquetMetadata currentMetadata = footer.getParquetMetadata();
-      fileMetaData = mergeInto(currentMetadata.getFileMetaData(), fileMetaData);
-    }
-    return fileMetaData;
-  }
-
-  /**
-   * Will return the result of merging toMerge into mergedMetadata
-   * @param toMerge the metadata toMerge
-   * @param mergedMetadata the reference metadata to merge into
-   * @return the result of the merge
-   */
-  static GlobalMetaData mergeInto(
-      FileMetaData toMerge,
-      GlobalMetaData mergedMetadata) {
-    MessageType schema = null;
-    Map<String, Set<String>> newKeyValues = new HashMap<String, Set<String>>();
-    Set<String> createdBy = new HashSet<String>();
-    if (mergedMetadata != null) {
-      schema = mergedMetadata.getSchema();
-      newKeyValues.putAll(mergedMetadata.getKeyValueMetaData());
-      createdBy.addAll(mergedMetadata.getCreatedBy());
-    }
-    if ((schema == null && toMerge.getSchema() != null)
-        || (schema != null && !schema.equals(toMerge.getSchema()))) {
-      schema = mergeInto(toMerge.getSchema(), schema);
-    }
-    for (Entry<String, String> entry : toMerge.getKeyValueMetaData().entrySet()) {
-      Set<String> values = newKeyValues.get(entry.getKey());
-      if (values == null) {
-        values = new HashSet<String>();
-        newKeyValues.put(entry.getKey(), values);
-      }
-      values.add(entry.getValue());
-    }
-    createdBy.add(toMerge.getCreatedBy());
-    return new GlobalMetaData(
-        schema,
-        newKeyValues,
-        createdBy);
-  }
-
-  /**
-   * will return the result of merging toMerge into mergedSchema
-   * @param toMerge the schema to merge into mergedSchema
-   * @param mergedSchema the schema to append the fields to
-   * @return the resulting schema
-   */
-  static MessageType mergeInto(MessageType toMerge, MessageType mergedSchema) {
-    if (mergedSchema == null) {
-      return toMerge;
-    }
-    return mergedSchema.union(toMerge);
-  }
-
-}


[18/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/resources/storage-default.xml b/tajo-storage/src/main/resources/storage-default.xml
deleted file mode 100644
index 85e5f41..0000000
--- a/tajo-storage/src/main/resources/storage-default.xml
+++ /dev/null
@@ -1,180 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<!--
-  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.
-  -->
-
-<configuration>
-  <property>
-    <name>tajo.storage.manager.maxReadBytes</name>
-    <value>8388608</value>
-    <description></description>
-  </property>
-
-  <property>
-    <name>tajo.storage.manager.concurrency.perDisk</name>
-    <value>1</value>
-    <description></description>
-  </property>
-
-  <!--- Registered Scanner Handler -->
-  <property>
-    <name>tajo.storage.scanner-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
-  </property>
-
-  <!--- Fragment Class Configurations -->
-  <property>
-    <name>tajo.storage.fragment.textfile.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.csv.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.raw.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.rcfile.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.row.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.parquet.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.sequencefile.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.avro.class</name>
-    <value>org.apache.tajo.storage.fragment.FileFragment</value>
-  </property>
-  <property>
-    <name>tajo.storage.fragment.hbase.class</name>
-    <value>org.apache.tajo.storage.hbase.HBaseFragment</value>
-  </property>
-
-  <!--- Scanner Handler -->
-  <property>
-    <name>tajo.storage.scanner-handler.textfile.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.csv.class</name>
-    <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.raw.class</name>
-    <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.rcfile.class</name>
-    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.rowfile.class</name>
-    <value>org.apache.tajo.storage.RowFile$RowFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.parquet.class</name>
-    <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.sequencefile.class</name>
-    <value>org.apache.tajo.storage.sequencefile.SequenceFileScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.avro.class</name>
-    <value>org.apache.tajo.storage.avro.AvroScanner</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.scanner-handler.hbase.class</name>
-    <value>org.apache.tajo.storage.hbase.HBaseScanner</value>
-  </property>
-  
-  <!--- Appender Handler -->
-  <property>
-    <name>tajo.storage.appender-handler</name>
-    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.textfile.class</name>
-    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.csv.class</name>
-    <value>org.apache.tajo.storage.CSVFile$CSVAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.raw.class</name>
-    <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.rcfile.class</name>
-    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.rowfile.class</name>
-    <value>org.apache.tajo.storage.RowFile$RowFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.parquet.class</name>
-    <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.sequencefile.class</name>
-    <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.avro.class</name>
-    <value>org.apache.tajo.storage.avro.AvroAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.hbase.class</name>
-    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
-  </property>
-
-  <property>
-    <name>tajo.storage.appender-handler.hfile.class</name>
-    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/HttpFileServer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServer.java b/tajo-storage/src/test/java/org/apache/tajo/HttpFileServer.java
deleted file mode 100644
index cf8a54e..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServer.java
+++ /dev/null
@@ -1,84 +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;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.net.NetUtils;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFactory;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.ChannelGroupFuture;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
-
-import java.net.InetSocketAddress;
-import java.util.concurrent.Executors;
-
-public class HttpFileServer {
-  private final static Log LOG = LogFactory.getLog(HttpFileServer.class);
-
-  private final InetSocketAddress addr;
-  private InetSocketAddress bindAddr;
-  private ServerBootstrap bootstrap = null;
-  private ChannelFactory factory = null;
-  private ChannelGroup channelGroup = null;
-
-  public HttpFileServer(final InetSocketAddress addr) {
-    this.addr = addr;
-    this.factory = new NioServerSocketChannelFactory(
-        Executors.newCachedThreadPool(), Executors.newCachedThreadPool(),
-        2);
-
-    // Configure the server.
-    this.bootstrap = new ServerBootstrap(factory);
-    // Set up the event pipeline factory.
-    this.bootstrap.setPipelineFactory(new HttpFileServerPipelineFactory());
-    this.channelGroup = new DefaultChannelGroup();
-  }
-
-  public HttpFileServer(String bindaddr) {
-    this(NetUtils.createSocketAddr(bindaddr));
-  }
-
-  public void start() {
-    // Bind and start to accept incoming connections.
-    Channel channel = bootstrap.bind(addr);
-    channelGroup.add(channel);    
-    this.bindAddr = (InetSocketAddress) channel.getLocalAddress();
-    LOG.info("HttpFileServer starts up ("
-        + this.bindAddr.getAddress().getHostAddress() + ":" + this.bindAddr.getPort()
-        + ")");
-  }
-  
-  public InetSocketAddress getBindAddress() {
-    return this.bindAddr;
-  }
-
-  public void stop() {
-    ChannelGroupFuture future = channelGroup.close();
-    future.awaitUninterruptibly();
-    factory.releaseExternalResources();
-
-    LOG.info("HttpFileServer shutdown ("
-        + this.bindAddr.getAddress().getHostAddress() + ":"
-        + this.bindAddr.getPort() + ")");
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerHandler.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerHandler.java b/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerHandler.java
deleted file mode 100644
index 6c77317..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerHandler.java
+++ /dev/null
@@ -1,184 +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;
-
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.*;
-import org.jboss.netty.handler.codec.frame.TooLongFrameException;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.handler.stream.ChunkedFile;
-import org.jboss.netty.util.CharsetUtil;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.RandomAccessFile;
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-
-import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
-import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-
-/**
- * this is an implementation copied from HttpStaticFileServerHandler.java of netty 3.6
- */
-public class HttpFileServerHandler extends SimpleChannelUpstreamHandler {
-
-  @Override
-  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
-    HttpRequest request = (HttpRequest) e.getMessage();
-    if (request.getMethod() != GET) {
-      sendError(ctx, METHOD_NOT_ALLOWED);
-      return;
-    }
-
-    final String path = sanitizeUri(request.getUri());
-    if (path == null) {
-      sendError(ctx, FORBIDDEN);
-      return;
-    }
-
-    File file = new File(path);
-    if (file.isHidden() || !file.exists()) {
-      sendError(ctx, NOT_FOUND);
-      return;
-    }
-    if (!file.isFile()) {
-      sendError(ctx, FORBIDDEN);
-      return;
-    }
-
-    RandomAccessFile raf;
-    try {
-      raf = new RandomAccessFile(file, "r");
-    } catch (FileNotFoundException fnfe) {
-      sendError(ctx, NOT_FOUND);
-      return;
-    }
-    long fileLength = raf.length();
-
-    HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-    setContentLength(response, fileLength);
-    setContentTypeHeader(response);
-
-    Channel ch = e.getChannel();
-
-    // Write the initial line and the header.
-    ch.write(response);
-
-    // Write the content.
-    ChannelFuture writeFuture;
-    if (ch.getPipeline().get(SslHandler.class) != null) {
-      // Cannot use zero-copy with HTTPS.
-      writeFuture = ch.write(new ChunkedFile(raf, 0, fileLength, 8192));
-    } else {
-      // No encryption - use zero-copy.
-      final FileRegion region =
-          new DefaultFileRegion(raf.getChannel(), 0, fileLength);
-      writeFuture = ch.write(region);
-      writeFuture.addListener(new ChannelFutureProgressListener() {
-        public void operationComplete(ChannelFuture future) {
-          region.releaseExternalResources();
-        }
-
-        public void operationProgressed(
-            ChannelFuture future, long amount, long current, long total) {
-          System.out.printf("%s: %d / %d (+%d)%n", path, current, total, amount);
-        }
-      });
-    }
-
-    // Decide whether to close the connection or not.
-    if (!isKeepAlive(request)) {
-      // Close the connection when the whole content is written out.
-      writeFuture.addListener(ChannelFutureListener.CLOSE);
-    }
-  }
-
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
-      throws Exception {
-    Channel ch = e.getChannel();
-    Throwable cause = e.getCause();
-    if (cause instanceof TooLongFrameException) {
-      sendError(ctx, BAD_REQUEST);
-      return;
-    }
-
-    cause.printStackTrace();
-    if (ch.isConnected()) {
-      sendError(ctx, INTERNAL_SERVER_ERROR);
-    }
-  }
-
-  private static String sanitizeUri(String uri) {
-    // Decode the path.
-    try {
-      uri = URLDecoder.decode(uri, "UTF-8");
-    } catch (UnsupportedEncodingException e) {
-      try {
-        uri = URLDecoder.decode(uri, "ISO-8859-1");
-      } catch (UnsupportedEncodingException e1) {
-        throw new Error();
-      }
-    }
-
-    // Convert file separators.
-    uri = uri.replace('/', File.separatorChar);
-
-    // Simplistic dumb security check.
-    // You will have to do something serious in the production environment.
-    if (uri.contains(File.separator + '.') ||
-        uri.contains('.' + File.separator) ||
-        uri.startsWith(".") || uri.endsWith(".")) {
-      return null;
-    }
-
-    return uri;
-  }
-
-  private static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
-    HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
-    response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
-    response.setContent(ChannelBuffers.copiedBuffer(
-        "Failure: " + status.toString() + "\r\n",
-        CharsetUtil.UTF_8));
-
-    // Close the connection as soon as the error message is sent.
-    ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
-  }
-
-  /**
-   * Sets the content type header for the HTTP Response
-   *
-   * @param response
-   *            HTTP response
-   */
-  private static void setContentTypeHeader(HttpResponse response) {
-    response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java b/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
deleted file mode 100644
index cecf93b..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/HttpFileServerPipelineFactory.java
+++ /dev/null
@@ -1,54 +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;
-
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
-import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
-import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
-import org.jboss.netty.handler.stream.ChunkedWriteHandler;
-
-import static org.jboss.netty.channel.Channels.pipeline;
-
-// Uncomment the following lines if you want HTTPS
-//import javax.net.ssl.SSLEngine;
-//import org.jboss.netty.example.securechat.SecureChatSslContextFactory;
-//import org.jboss.netty.handler.ssl.SslHandler;
-
-//this class is copied from HttpStaticFileServerPipelineFactory.java of netty 3.6
-public class HttpFileServerPipelineFactory implements ChannelPipelineFactory {
-  public ChannelPipeline getPipeline() throws Exception {
-    // Create a default pipeline implementation.
-    ChannelPipeline pipeline = pipeline();
-
-    // Uncomment the following lines if you want HTTPS
-    //SSLEngine engine = SecureChatSslContextFactory.getServerContext().createSSLEngine();
-    //engine.setUseClientMode(false);
-    //pipeline.addLast("ssl", new SslHandler(engine));
-
-    pipeline.addLast("decoder", new HttpRequestDecoder());
-    pipeline.addLast("aggregator", new HttpChunkAggregator(65536));
-    pipeline.addLast("encoder", new HttpResponseEncoder());
-    pipeline.addLast("chunkedWriter", new ChunkedWriteHandler());
-
-    pipeline.addLast("handler", new HttpFileServerHandler());
-    return pipeline;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
deleted file mode 100644
index ea46fa7..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.compress.*;
-import org.apache.hadoop.io.compress.zlib.ZlibFactory;
-import org.apache.hadoop.util.NativeCodeLoader;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.sequencefile.SequenceFileScanner;
-import org.apache.tajo.storage.text.DelimitedTextFile;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-
-import static org.junit.Assert.*;
-
-@RunWith(Parameterized.class)
-public class TestCompressionStorages {
-  private TajoConf conf;
-  private static String TEST_PATH = "target/test-data/TestCompressionStorages";
-
-  private StoreType storeType;
-  private Path testDir;
-  private FileSystem fs;
-
-  public TestCompressionStorages(StoreType type) throws IOException {
-    this.storeType = type;
-    conf = new TajoConf();
-
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> generateParameters() {
-    return Arrays.asList(new Object[][]{
-        {StoreType.CSV},
-        {StoreType.RCFILE},
-        {StoreType.SEQUENCEFILE},
-        {StoreType.TEXTFILE}
-    });
-  }
-
-  @Test
-  public void testDeflateCodecCompressionData() throws IOException {
-    storageCompressionTest(storeType, DeflateCodec.class);
-  }
-
-  @Test
-  public void testGzipCodecCompressionData() throws IOException {
-    if (storeType == StoreType.RCFILE) {
-      if( ZlibFactory.isNativeZlibLoaded(conf)) {
-        storageCompressionTest(storeType, GzipCodec.class);
-      }
-    } else if (storeType == StoreType.SEQUENCEFILE) {
-      if( ZlibFactory.isNativeZlibLoaded(conf)) {
-        storageCompressionTest(storeType, GzipCodec.class);
-      }
-    } else {
-      storageCompressionTest(storeType, GzipCodec.class);
-    }
-  }
-
-  @Test
-  public void testSnappyCodecCompressionData() throws IOException {
-    if (SnappyCodec.isNativeCodeLoaded()) {
-      storageCompressionTest(storeType, SnappyCodec.class);
-    }
-  }
-
-  @Test
-  public void testLz4CodecCompressionData() throws IOException {
-    if(NativeCodeLoader.isNativeCodeLoaded() && Lz4Codec.isNativeCodeLoaded())
-    storageCompressionTest(storeType, Lz4Codec.class);
-  }
-
-  private void storageCompressionTest(StoreType storeType, Class<? extends CompressionCodec> codec) throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4);
-    schema.addColumn("age", Type.FLOAT4);
-    schema.addColumn("name", Type.TEXT);
-
-    TableMeta meta = CatalogUtil.newTableMeta(storeType);
-    meta.putOption("compression.codec", codec.getCanonicalName());
-    meta.putOption("compression.type", SequenceFile.CompressionType.BLOCK.name());
-    meta.putOption("rcfile.serde", TextSerializerDeserializer.class.getName());
-    meta.putOption("sequencefile.serde", TextSerializerDeserializer.class.getName());
-
-    String fileName = "Compression_" + codec.getSimpleName();
-    Path tablePath = new Path(testDir, fileName);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-
-    appender.init();
-
-    String extension = "";
-    if (appender instanceof CSVFile.CSVAppender) {
-      extension = ((CSVFile.CSVAppender) appender).getExtension();
-    } else if (appender instanceof DelimitedTextFile.DelimitedTextFileAppender) {
-      extension = ((DelimitedTextFile.DelimitedTextFileAppender) appender).getExtension();
-    }
-
-    int tupleNum = 100000;
-    VTuple vTuple;
-
-    for (int i = 0; i < tupleNum; i++) {
-      vTuple = new VTuple(3);
-      vTuple.put(0, DatumFactory.createInt4(i + 1));
-      vTuple.put(1, DatumFactory.createFloat4((float) i));
-      vTuple.put(2, DatumFactory.createText(String.valueOf(i)));
-      appender.addTuple(vTuple);
-    }
-    appender.close();
-
-    TableStats stat = appender.getStats();
-    assertEquals(tupleNum, stat.getNumRows().longValue());
-    tablePath = tablePath.suffix(extension);
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment[] tablets = new FileFragment[1];
-    tablets[0] = new FileFragment(fileName, tablePath, 0, fileLen);
-
-    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-
-    if (StoreType.CSV == storeType) {
-      if (SplittableCompressionCodec.class.isAssignableFrom(codec)) {
-        assertTrue(scanner.isSplittable());
-      } else {
-        assertFalse(scanner.isSplittable());
-      }
-    }
-    scanner.init();
-
-    if (storeType == StoreType.SEQUENCEFILE) {
-      assertTrue(scanner instanceof SequenceFileScanner);
-      Writable key = ((SequenceFileScanner) scanner).getKey();
-      assertEquals(key.getClass().getCanonicalName(), LongWritable.class.getCanonicalName());
-    }
-
-    int tupleCnt = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      tupleCnt++;
-    }
-    scanner.close();
-    assertEquals(tupleNum, tupleCnt);
-    assertNotSame(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
-    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestFileSystems.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFileSystems.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestFileSystems.java
deleted file mode 100644
index 17a8da7..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFileSystems.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3.S3FileSystem;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.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.storage.s3.InMemoryFileSystemStore;
-import org.apache.tajo.storage.s3.SmallBlockS3FileSystem;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-@RunWith(Parameterized.class)
-public class TestFileSystems {
-
-  protected byte[] data = null;
-
-  private static String TEST_PATH = "target/test-data/TestFileSystem";
-  private TajoConf conf = null;
-  private FileStorageManager sm = null;
-  private FileSystem fs = null;
-  Path testDir;
-
-  public TestFileSystems(FileSystem fs) throws IOException {
-    conf = new TajoConf();
-
-    if(fs instanceof S3FileSystem){
-      conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "10");
-      fs.initialize(URI.create(fs.getScheme() + ":///"), conf);
-    }
-    this.fs = fs;
-    sm = StorageManager.getFileStorageManager(conf);
-    testDir = getTestDir(this.fs, TEST_PATH);
-  }
-
-  public Path getTestDir(FileSystem fs, String dir) throws IOException {
-    Path path = new Path(dir);
-    if(fs.exists(path))
-      fs.delete(path, true);
-
-    fs.mkdirs(path);
-
-    return fs.makeQualified(path);
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> generateParameters() {
-    return Arrays.asList(new Object[][] {
-        {new SmallBlockS3FileSystem(new InMemoryFileSystemStore())},
-    });
-  }
-
-  @Test
-  public void testBlockSplit() 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(StoreType.CSV);
-
-    Tuple[] tuples = new Tuple[4];
-    for (int i = 0; i < tuples.length; i++) {
-      tuples[i] = new VTuple(3);
-      tuples[i]
-          .put(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());
-
-    Appender appender = sm.getAppender(meta, schema, path);
-    appender.init();
-    for (Tuple t : tuples) {
-      appender.addTuple(t);
-    }
-    appender.close();
-    FileStatus fileStatus = fs.getFileStatus(path);
-
-    List<Fragment> splits = sm.getSplits("table", meta, schema, path);
-    int splitSize = (int) Math.ceil(fileStatus.getLen() / (double) fileStatus.getBlockSize());
-    assertEquals(splitSize, splits.size());
-
-    for (Fragment fragment : splits) {
-      assertTrue(fragment.getLength() <= fileStatus.getBlockSize());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
deleted file mode 100644
index 387fed5..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestFrameTuple {
-  private Tuple tuple1;
-  private Tuple tuple2;
-
-  @Before
-  public void setUp() throws Exception {
-    tuple1 = new VTuple(11);
-    tuple1.put(new Datum[] {
-        DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
-        DatumFactory.createChar('9'),
-        DatumFactory.createInt2((short) 17),
-        DatumFactory.createInt4(59),
-        DatumFactory.createInt8(23l),
-        DatumFactory.createFloat4(77.9f),
-        DatumFactory.createFloat8(271.9f),
-        DatumFactory.createText("hyunsik"),
-        DatumFactory.createBlob("hyunsik".getBytes()),
-        DatumFactory.createInet4("192.168.0.1")
-    });
-    
-    tuple2 = new VTuple(11);
-    tuple2.put(new Datum[] {
-        DatumFactory.createBool(true),
-        DatumFactory.createBit((byte) 0x99),
-        DatumFactory.createChar('9'),
-        DatumFactory.createInt2((short) 17),
-        DatumFactory.createInt4(59),
-        DatumFactory.createInt8(23l),
-        DatumFactory.createFloat4(77.9f),
-        DatumFactory.createFloat8(271.9f),
-        DatumFactory.createText("hyunsik"),
-        DatumFactory.createBlob("hyunsik".getBytes()),
-        DatumFactory.createInet4("192.168.0.1")
-    });
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
-  @Test
-  public final void testFrameTuple() {
-    Tuple frame = new FrameTuple(tuple1, tuple2);
-    assertEquals(22, frame.size());
-    for (int i = 0; i < 22; i++) {
-      assertTrue(frame.contains(i));
-    }
-    
-    assertEquals(DatumFactory.createInt8(23l), frame.get(5));
-    assertEquals(DatumFactory.createInt8(23l), frame.get(16));
-    assertEquals(DatumFactory.createInet4("192.168.0.1"), frame.get(10));
-    assertEquals(DatumFactory.createInet4("192.168.0.1"), frame.get(21));
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
deleted file mode 100644
index c6149f7..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.util.BytesUtils;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class TestLazyTuple {
-
-  Schema schema;
-  byte[][] textRow;
-  byte[] nullbytes;
-  SerializerDeserializer serde;
-
-  @Before
-  public void setUp() {
-    nullbytes = "\\N".getBytes();
-
-    schema = new Schema();
-    schema.addColumn("col1", TajoDataTypes.Type.BOOLEAN);
-    schema.addColumn("col2", TajoDataTypes.Type.BIT);
-    schema.addColumn("col3", TajoDataTypes.Type.CHAR, 7);
-    schema.addColumn("col4", TajoDataTypes.Type.INT2);
-    schema.addColumn("col5", TajoDataTypes.Type.INT4);
-    schema.addColumn("col6", TajoDataTypes.Type.INT8);
-    schema.addColumn("col7", TajoDataTypes.Type.FLOAT4);
-    schema.addColumn("col8", TajoDataTypes.Type.FLOAT8);
-    schema.addColumn("col9", TajoDataTypes.Type.TEXT);
-    schema.addColumn("col10", TajoDataTypes.Type.BLOB);
-    schema.addColumn("col11", TajoDataTypes.Type.INET4);
-    schema.addColumn("col12", TajoDataTypes.Type.INT4);
-    schema.addColumn("col13", TajoDataTypes.Type.NULL_TYPE);
-
-    StringBuilder sb = new StringBuilder();
-    sb.append(DatumFactory.createBool(true)).append('|');
-    sb.append(new String(DatumFactory.createBit((byte) 0x99).asTextBytes())).append('|');
-    sb.append(DatumFactory.createChar("str")).append('|');
-    sb.append(DatumFactory.createInt2((short) 17)).append('|');
-    sb.append(DatumFactory.createInt4(59)).append('|');
-    sb.append(DatumFactory.createInt8(23l)).append('|');
-    sb.append(DatumFactory.createFloat4(77.9f)).append('|');
-    sb.append(DatumFactory.createFloat8(271.9f)).append('|');
-    sb.append(DatumFactory.createText("str2")).append('|');
-    sb.append(DatumFactory.createBlob("jinho".getBytes())).append('|');
-    sb.append(DatumFactory.createInet4("192.168.0.1")).append('|');
-    sb.append(new String(nullbytes)).append('|');
-    sb.append(NullDatum.get());
-    textRow = BytesUtils.splitPreserveAllTokens(sb.toString().getBytes(), '|');
-    serde = new TextSerializerDeserializer();
-  }
-
-  @Test
-  public void testGetDatum() {
-
-    LazyTuple t1 = new LazyTuple(schema, textRow, -1, nullbytes, serde);
-    assertEquals(DatumFactory.createBool(true), t1.get(0));
-    assertEquals(DatumFactory.createBit((byte) 0x99), t1.get(1));
-    assertEquals(DatumFactory.createChar("str"), t1.get(2));
-    assertEquals(DatumFactory.createInt2((short) 17), t1.get(3));
-    assertEquals(DatumFactory.createInt4(59), t1.get(4));
-    assertEquals(DatumFactory.createInt8(23l), t1.get(5));
-    assertEquals(DatumFactory.createFloat4(77.9f), t1.get(6));
-    assertEquals(DatumFactory.createFloat8(271.9f), t1.get(7));
-    assertEquals(DatumFactory.createText("str2"), t1.get(8));
-    assertEquals(DatumFactory.createBlob("jinho".getBytes()), t1.get(9));
-    assertEquals(DatumFactory.createInet4("192.168.0.1"), t1.get(10));
-    assertEquals(NullDatum.get(), t1.get(11));
-    assertEquals(NullDatum.get(), t1.get(12));
-  }
-
-  @Test
-  public void testContain() {
-    int colNum = schema.size();
-
-    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(3, DatumFactory.createInt4(1));
-    t1.put(7, DatumFactory.createInt4(1));
-
-    assertTrue(t1.contains(0));
-    assertFalse(t1.contains(1));
-    assertFalse(t1.contains(2));
-    assertTrue(t1.contains(3));
-    assertFalse(t1.contains(4));
-    assertFalse(t1.contains(5));
-    assertFalse(t1.contains(6));
-    assertTrue(t1.contains(7));
-    assertFalse(t1.contains(8));
-    assertFalse(t1.contains(9));
-    assertFalse(t1.contains(10));
-    assertFalse(t1.contains(11));
-    assertFalse(t1.contains(12));
-  }
-
-  @Test
-  public void testPut() {
-    int colNum = schema.size();
-    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
-    t1.put(0, DatumFactory.createText("str"));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(11, DatumFactory.createFloat4(0.76f));
-
-    assertTrue(t1.contains(0));
-    assertTrue(t1.contains(1));
-
-    assertEquals(t1.getText(0), "str");
-    assertEquals(t1.get(1).asInt4(), 2);
-    assertTrue(t1.get(11).asFloat4() == 0.76f);
-  }
-
-  @Test
-  public void testEquals() {
-    int colNum = schema.size();
-    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
-    LazyTuple t2 = new LazyTuple(schema, new byte[colNum][], -1);
-
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(3, DatumFactory.createInt4(2));
-
-    t2.put(0, DatumFactory.createInt4(1));
-    t2.put(1, DatumFactory.createInt4(2));
-    t2.put(3, DatumFactory.createInt4(2));
-
-    assertEquals(t1, t2);
-
-    Tuple t3 = new VTuple(colNum);
-    t3.put(0, DatumFactory.createInt4(1));
-    t3.put(1, DatumFactory.createInt4(2));
-    t3.put(3, DatumFactory.createInt4(2));
-    assertEquals(t1, t3);
-    assertEquals(t2, t3);
-
-    LazyTuple t4 = new LazyTuple(schema, new byte[colNum][], -1);
-    assertNotSame(t1, t4);
-  }
-
-  @Test
-  public void testHashCode() {
-    int colNum = schema.size();
-    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
-    LazyTuple t2 = new LazyTuple(schema, new byte[colNum][], -1);
-
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(3, DatumFactory.createInt4(2));
-    t1.put(4, DatumFactory.createText("str"));
-
-    t2.put(0, DatumFactory.createInt4(1));
-    t2.put(1, DatumFactory.createInt4(2));
-    t2.put(3, DatumFactory.createInt4(2));
-    t2.put(4, DatumFactory.createText("str"));
-
-    assertEquals(t1.hashCode(), t2.hashCode());
-
-    Tuple t3 = new VTuple(colNum);
-    t3.put(0, DatumFactory.createInt4(1));
-    t3.put(1, DatumFactory.createInt4(2));
-    t3.put(3, DatumFactory.createInt4(2));
-    t3.put(4, DatumFactory.createText("str"));
-    assertEquals(t1.hashCode(), t3.hashCode());
-    assertEquals(t2.hashCode(), t3.hashCode());
-
-    Tuple t4 = new VTuple(5);
-    t4.put(0, DatumFactory.createInt4(1));
-    t4.put(1, DatumFactory.createInt4(2));
-    t4.put(4, DatumFactory.createInt4(2));
-
-    assertNotSame(t1.hashCode(), t4.hashCode());
-  }
-
-  @Test
-  public void testPutTuple() {
-    int colNum = schema.size();
-    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
-
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(2, DatumFactory.createInt4(3));
-
-
-    Schema schema2 = new Schema();
-    schema2.addColumn("col1", TajoDataTypes.Type.INT8);
-    schema2.addColumn("col2", TajoDataTypes.Type.INT8);
-
-    LazyTuple t2 = new LazyTuple(schema2, new byte[schema2.size()][], -1);
-    t2.put(0, DatumFactory.createInt4(4));
-    t2.put(1, DatumFactory.createInt4(5));
-
-    t1.put(3, t2);
-
-    for (int i = 0; i < 5; i++) {
-      assertEquals(i + 1, t1.get(i).asInt4());
-    }
-  }
-
-  @Test
-  public void testInvalidNumber() {
-    byte[][] bytes = BytesUtils.splitPreserveAllTokens(" 1| |2 ||".getBytes(), '|');
-    Schema schema = new Schema();
-    schema.addColumn("col1", TajoDataTypes.Type.INT2);
-    schema.addColumn("col2", TajoDataTypes.Type.INT4);
-    schema.addColumn("col3", TajoDataTypes.Type.INT8);
-    schema.addColumn("col4", TajoDataTypes.Type.FLOAT4);
-    schema.addColumn("col5", TajoDataTypes.Type.FLOAT8);
-
-    LazyTuple tuple = new LazyTuple(schema, bytes, 0);
-    assertEquals(bytes.length, tuple.size());
-
-    for (int i = 0; i < tuple.size(); i++){
-      assertEquals(NullDatum.get(), tuple.get(i));
-    }
-  }
-
-  @Test
-  public void testClone() throws CloneNotSupportedException {
-    int colNum = schema.size();
-    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
-
-    t1.put(0, DatumFactory.createInt4(1));
-    t1.put(1, DatumFactory.createInt4(2));
-    t1.put(3, DatumFactory.createInt4(2));
-    t1.put(4, DatumFactory.createText("str"));
-
-    LazyTuple t2 = (LazyTuple) t1.clone();
-    assertNotSame(t1, t2);
-    assertEquals(t1, t2);
-
-    assertSame(t1.get(4), t2.get(4));
-
-    t1.clear();
-    assertFalse(t1.equals(t2));
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
deleted file mode 100644
index 1a4bdba..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestLineReader.java
+++ /dev/null
@@ -1,163 +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 io.netty.buffer.ByteBuf;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.DeflateCodec;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.text.ByteBufLineReader;
-import org.apache.tajo.storage.text.DelimitedTextFile;
-import org.apache.tajo.storage.text.DelimitedLineReader;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.*;
-
-public class TestLineReader {
-	private static String TEST_PATH = "target/test-data/TestLineReader";
-
-  @Test
-  public void testByteBufLineReader() throws IOException {
-    TajoConf conf = new TajoConf();
-    Path testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    FileSystem fs = testDir.getFileSystem(conf);
-
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4);
-    schema.addColumn("age", Type.INT8);
-    schema.addColumn("comment", Type.TEXT);
-    schema.addColumn("comment2", Type.TEXT);
-
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.TEXTFILE);
-    Path tablePath = new Path(testDir, "line.data");
-    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(meta, schema,
-        tablePath);
-    appender.enableStats();
-    appender.init();
-    int tupleNum = 10000;
-    VTuple vTuple;
-
-    for (int i = 0; i < tupleNum; i++) {
-      vTuple = new VTuple(4);
-      vTuple.put(0, DatumFactory.createInt4(i + 1));
-      vTuple.put(1, DatumFactory.createInt8(25l));
-      vTuple.put(2, DatumFactory.createText("emiya muljomdao"));
-      vTuple.put(3, NullDatum.get());
-      appender.addTuple(vTuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-
-    ByteBufInputChannel channel = new ByteBufInputChannel(fs.open(tablePath));
-    assertEquals(status.getLen(), channel.available());
-    ByteBufLineReader reader = new ByteBufLineReader(channel);
-    assertEquals(status.getLen(), reader.available());
-
-    long totalRead = 0;
-    int i = 0;
-    AtomicInteger bytes = new AtomicInteger();
-    for(;;){
-      ByteBuf buf = reader.readLineBuf(bytes);
-      if(buf == null) break;
-
-      totalRead += bytes.get();
-      i++;
-    }
-    IOUtils.cleanup(null, reader, channel, fs);
-    assertEquals(tupleNum, i);
-    assertEquals(status.getLen(), totalRead);
-    assertEquals(status.getLen(), reader.readBytes());
-  }
-
-  @Test
-  public void testLineDelimitedReader() throws IOException {
-    TajoConf conf = new TajoConf();
-    Path testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    FileSystem fs = testDir.getFileSystem(conf);
-
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4);
-    schema.addColumn("age", Type.INT8);
-    schema.addColumn("comment", Type.TEXT);
-    schema.addColumn("comment2", Type.TEXT);
-
-    TableMeta meta = CatalogUtil.newTableMeta(StoreType.TEXTFILE);
-    meta.putOption("compression.codec", DeflateCodec.class.getCanonicalName());
-
-    Path tablePath = new Path(testDir, "line1." + DeflateCodec.class.getSimpleName());
-    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(meta, schema,
-        tablePath);
-    appender.enableStats();
-    appender.init();
-    int tupleNum = 10000;
-    VTuple vTuple;
-
-    long splitOffset = 0;
-    for (int i = 0; i < tupleNum; i++) {
-      vTuple = new VTuple(4);
-      vTuple.put(0, DatumFactory.createInt4(i + 1));
-      vTuple.put(1, DatumFactory.createInt8(25l));
-      vTuple.put(2, DatumFactory.createText("emiya muljomdao"));
-      vTuple.put(3, NullDatum.get());
-      appender.addTuple(vTuple);
-
-      if(i == (tupleNum / 2)){
-        splitOffset = appender.getOffset();
-      }
-    }
-    String extension = ((DelimitedTextFile.DelimitedTextFileAppender) appender).getExtension();
-    appender.close();
-
-    tablePath = tablePath.suffix(extension);
-    FileFragment fragment = new FileFragment("table", tablePath, 0, splitOffset);
-    DelimitedLineReader reader = new DelimitedLineReader(conf, fragment); // if file is compressed, will read to EOF
-    assertTrue(reader.isCompressed());
-    assertFalse(reader.isReadable());
-    reader.init();
-    assertTrue(reader.isReadable());
-
-
-    int i = 0;
-    while(reader.isReadable()){
-      ByteBuf buf = reader.readLine();
-      if(buf == null) break;
-      i++;
-    }
-
-    IOUtils.cleanup(null, reader, fs);
-    assertEquals(tupleNum, i);
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
deleted file mode 100644
index cc4aa51..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.KeyValueSet;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.apache.tajo.util.TUtil;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-
-import static org.junit.Assert.*;
-
-@RunWith(Parameterized.class)
-public class TestMergeScanner {
-  private TajoConf conf;
-  StorageManager sm;
-  private static String TEST_PATH = "target/test-data/TestMergeScanner";
-
-  private static String TEST_MULTIPLE_FILES_AVRO_SCHEMA =
-      "{\n" +
-      "  \"type\": \"record\",\n" +
-      "  \"namespace\": \"org.apache.tajo\",\n" +
-      "  \"name\": \"testMultipleFiles\",\n" +
-      "  \"fields\": [\n" +
-      "    { \"name\": \"id\", \"type\": \"int\" },\n" +
-      "    { \"name\": \"file\", \"type\": \"string\" },\n" +
-      "    { \"name\": \"name\", \"type\": \"string\" },\n" +
-      "    { \"name\": \"age\", \"type\": \"long\" }\n" +
-      "  ]\n" +
-      "}\n";
-
-  private Path testDir;
-  private StoreType storeType;
-  private FileSystem fs;
-
-  public TestMergeScanner(StoreType storeType) {
-    this.storeType = storeType;
-  }
-
-  @Parameters
-  public static Collection<Object[]> generateParameters() {
-    return Arrays.asList(new Object[][] {
-        {StoreType.CSV},
-        {StoreType.RAW},
-        {StoreType.RCFILE},
-        {StoreType.PARQUET},
-        {StoreType.SEQUENCEFILE},
-        {StoreType.AVRO},
-        // RowFile requires Byte-buffer read support, so we omitted RowFile.
-        //{StoreType.ROWFILE},
-    });
-  }
-
-  @Before
-  public void setup() throws Exception {
-    conf = new TajoConf();
-    conf.setVar(ConfVars.ROOT_DIR, TEST_PATH);
-    conf.setStrings("tajo.storage.projectable-scanner", "rcfile", "parquet", "avro");
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-    sm = StorageManager.getFileStorageManager(conf, testDir);
-  }
-
-  @Test
-  public void testMultipleFiles() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4);
-    schema.addColumn("file", Type.TEXT);
-    schema.addColumn("name", Type.TEXT);
-    schema.addColumn("age", Type.INT8);
-
-    KeyValueSet options = new KeyValueSet();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
-    if (storeType == StoreType.AVRO) {
-      meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL,
-                     TEST_MULTIPLE_FILES_AVRO_SCHEMA);
-    }
-
-    Path table1Path = new Path(testDir, storeType + "_1.data");
-    Appender appender1 = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, table1Path);
-    appender1.enableStats();
-    appender1.init();
-    int tupleNum = 10000;
-    VTuple vTuple;
-
-    for(int i = 0; i < tupleNum; i++) {
-      vTuple = new VTuple(4);
-      vTuple.put(0, DatumFactory.createInt4(i + 1));
-      vTuple.put(1, DatumFactory.createText("hyunsik"));
-      vTuple.put(2, DatumFactory.createText("jihoon"));
-      vTuple.put(3, DatumFactory.createInt8(25l));
-      appender1.addTuple(vTuple);
-    }
-    appender1.close();
-
-    TableStats stat1 = appender1.getStats();
-    if (stat1 != null) {
-      assertEquals(tupleNum, stat1.getNumRows().longValue());
-    }
-
-    Path table2Path = new Path(testDir, storeType + "_2.data");
-    Appender appender2 = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, table2Path);
-    appender2.enableStats();
-    appender2.init();
-
-    for(int i = 0; i < tupleNum; i++) {
-      vTuple = new VTuple(4);
-      vTuple.put(0, DatumFactory.createInt4(i + 1));
-      vTuple.put(1, DatumFactory.createText("hyunsik"));
-      vTuple.put(2, DatumFactory.createText("jihoon"));
-      vTuple.put(3, DatumFactory.createInt8(25l));
-      appender2.addTuple(vTuple);
-    }
-    appender2.close();
-
-    TableStats stat2 = appender2.getStats();
-    if (stat2 != null) {
-      assertEquals(tupleNum, stat2.getNumRows().longValue());
-    }
-
-
-    FileStatus status1 = fs.getFileStatus(table1Path);
-    FileStatus status2 = fs.getFileStatus(table2Path);
-    Fragment[] fragment = new Fragment[2];
-    fragment[0] = new FileFragment("tablet1", table1Path, 0, status1.getLen());
-    fragment[1] = new FileFragment("tablet1", table2Path, 0, status2.getLen());
-
-    Schema targetSchema = new Schema();
-    targetSchema.addColumn(schema.getColumn(0));
-    targetSchema.addColumn(schema.getColumn(2));
-
-    Scanner scanner = new MergeScanner(conf, schema, meta, TUtil.newList(fragment), targetSchema);
-    assertEquals(isProjectableStorage(meta.getStoreType()), scanner.isProjectable());
-
-    scanner.init();
-    int totalCounts = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      totalCounts++;
-      if (isProjectableStorage(meta.getStoreType())) {
-        assertNotNull(tuple.get(0));
-        assertNull(tuple.get(1));
-        assertNotNull(tuple.get(2));
-        assertNull(tuple.get(3));
-      }
-    }
-    scanner.close();
-
-    assertEquals(tupleNum * 2, totalCounts);
-	}
-
-  private static boolean isProjectableStorage(StoreType type) {
-    switch (type) {
-      case RCFILE:
-      case PARQUET:
-      case SEQUENCEFILE:
-      case CSV:
-      case AVRO:
-        return true;
-      default:
-        return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
deleted file mode 100644
index 12ea551..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestSplitProcessor.java
+++ /dev/null
@@ -1,72 +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 io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.util.CharsetUtil;
-import org.apache.tajo.storage.text.FieldSplitProcessor;
-import org.apache.tajo.storage.text.LineSplitProcessor;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static io.netty.util.ReferenceCountUtil.releaseLater;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestSplitProcessor {
-
-  @Test
-  public void testFieldSplitProcessor() throws IOException {
-    String data = "abc||de";
-    final ByteBuf buf = releaseLater(
-        Unpooled.copiedBuffer(data, CharsetUtil.ISO_8859_1));
-
-    final int len = buf.readableBytes();
-    FieldSplitProcessor processor = new FieldSplitProcessor('|');
-
-    assertEquals(3, buf.forEachByte(0, len, processor));
-    assertEquals(4, buf.forEachByte(4, len - 4, processor));
-    assertEquals(-1, buf.forEachByte(5, len - 5, processor));
-
-  }
-
-  @Test
-  public void testLineSplitProcessor() throws IOException {
-    String data = "abc\r\n\n";
-    final ByteBuf buf = releaseLater(
-        Unpooled.copiedBuffer(data, CharsetUtil.ISO_8859_1));
-
-    final int len = buf.readableBytes();
-    LineSplitProcessor processor = new LineSplitProcessor();
-
-    //find CR
-    assertEquals(3, buf.forEachByte(0, len, processor));
-
-    // find CRLF
-    assertEquals(4, buf.forEachByte(4, len - 4, processor));
-    assertEquals(buf.getByte(4), '\n');
-    // need to skip LF
-    assertTrue(processor.isPrevCharCR());
-
-    // find LF
-    assertEquals(5, buf.forEachByte(5, len - 5, processor)); //line length is zero
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
deleted file mode 100644
index 13aeef6..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
+++ /dev/null
@@ -1,203 +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.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.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.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.UUID;
-
-import static org.junit.Assert.*;
-
-public class TestStorageManager {
-	private TajoConf conf;
-	private static String TEST_PATH = "target/test-data/TestStorageManager";
-  StorageManager sm = null;
-  private Path testDir;
-  private FileSystem fs;
-
-	@Before
-	public void setUp() throws Exception {
-		conf = new TajoConf();
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-    sm = StorageManager.getFileStorageManager(conf, testDir);
-	}
-
-	@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(StoreType.CSV);
-		
-		Tuple[] tuples = new Tuple[4];
-		for(int i=0; i < tuples.length; i++) {
-		  tuples[i] = new VTuple(3);
-		  tuples[i].put(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());
-		Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, path);
-    appender.init();
-		for(Tuple t : tuples) {
-		  appender.addTuple(t);
-		}
-		appender.close();
-
-		Scanner scanner = StorageManager.getFileStorageManager(conf).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();
-
-    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));
-      FileStorageManager sm = StorageManager.getFileStorageManager(new TajoConf(conf), tablePath);
-
-      Schema schema = new Schema();
-      schema.addColumn("id", Type.INT4);
-      schema.addColumn("age",Type.INT4);
-      schema.addColumn("name",Type.TEXT);
-      TableMeta meta = CatalogUtil.newTableMeta(StoreType.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();
-
-      File dir = new File(testDataPath);
-      dir.delete();
-    }
-  }
-
-  @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();
-
-    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));
-      FileStorageManager sm = StorageManager.getFileStorageManager(new TajoConf(conf), tablePath);
-
-      Schema schema = new Schema();
-      schema.addColumn("id", Type.INT4);
-      schema.addColumn("age", Type.INT4);
-      schema.addColumn("name", Type.TEXT);
-      TableMeta meta = CatalogUtil.newTableMeta(StoreType.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();
-
-      File dir = new File(testDataPath);
-      dir.delete();
-    }
-  }
-}


[10/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
new file mode 100644
index 0000000..a0ad492
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTextSerializerDeserializer.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.util.NumberUtil;
+
+import java.io.IOException;
+
+public class HBaseTextSerializerDeserializer {
+  public static Datum deserialize(Column col, byte[] bytes) throws IOException {
+    Datum datum;
+    switch (col.getDataType().getType()) {
+      case INT1:
+      case INT2:
+        datum = bytes == null  || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createInt2((short)NumberUtil.parseInt(bytes, 0, bytes.length));
+        break;
+      case INT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createInt4(NumberUtil.parseInt(bytes, 0, bytes.length));
+        break;
+      case INT8:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createInt8(new String(bytes, 0, bytes.length));
+        break;
+      case FLOAT4:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createFloat4(new String(bytes, 0, bytes.length));
+        break;
+      case FLOAT8:
+        datum = bytes == null || bytes.length == 0 ? NullDatum.get() :
+            DatumFactory.createFloat8(NumberUtil.parseDouble(bytes, 0, bytes.length));
+        break;
+      case TEXT:
+        datum = bytes == null ? NullDatum.get() : DatumFactory.createText(bytes);
+        break;
+      default:
+        datum = NullDatum.get();
+        break;
+    }
+    return datum;
+  }
+
+  public static byte[] serialize(Column col, Datum datum) throws IOException {
+    if (datum == null || datum instanceof NullDatum) {
+      return null;
+    }
+
+    return datum.asChars().getBytes();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
new file mode 100644
index 0000000..07f7988
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HFileAppender.java
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.Bytes;
+
+import java.io.IOException;
+import java.util.TreeSet;
+
+public class HFileAppender extends AbstractHBaseAppender {
+  private static final Log LOG = LogFactory.getLog(HFileAppender.class);
+
+  private RecordWriter<ImmutableBytesWritable, Cell> writer;
+  private TaskAttemptContext writerContext;
+  private Path workingFilePath;
+  private FileOutputCommitter committer;
+
+  public HFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                       Schema schema, TableMeta meta, Path stagingDir) {
+    super(conf, taskAttemptId, schema, meta, stagingDir);
+  }
+
+  @Override
+  public void init() throws IOException {
+    super.init();
+
+    Configuration taskConf = new Configuration();
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    taskConf.set(FileOutputFormat.OUTDIR, stagingResultDir.toString());
+
+    ExecutionBlockId ebId = taskAttemptId.getQueryUnitId().getExecutionBlockId();
+    writerContext = new TaskAttemptContextImpl(taskConf,
+        new TaskAttemptID(ebId.getQueryId().toString(), ebId.getId(), TaskType.MAP,
+            taskAttemptId.getQueryUnitId().getId(), taskAttemptId.getId()));
+
+    HFileOutputFormat2 hFileOutputFormat2 = new HFileOutputFormat2();
+    try {
+      writer = hFileOutputFormat2.getRecordWriter(writerContext);
+
+      committer = new FileOutputCommitter(FileOutputFormat.getOutputPath(writerContext), writerContext);
+      workingFilePath = committer.getWorkPath();
+    } catch (InterruptedException e) {
+      throw new IOException(e.getMessage(), e);
+    }
+
+    LOG.info("Created hbase file writer: " + workingFilePath);
+  }
+
+  long totalNumBytes = 0;
+  ImmutableBytesWritable keyWritable = new ImmutableBytesWritable();
+  boolean first = true;
+  TreeSet<KeyValue> kvSet = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
+
+
+  @Override
+  public void addTuple(Tuple tuple) throws IOException {
+    Datum datum;
+
+    byte[] rowkey = getRowKeyBytes(tuple);
+
+    if (!first && !Bytes.equals(keyWritable.get(), 0, keyWritable.getLength(), rowkey, 0, rowkey.length)) {
+      try {
+        for (KeyValue kv : kvSet) {
+          writer.write(keyWritable, kv);
+          totalNumBytes += keyWritable.getLength() + kv.getLength();
+        }
+        kvSet.clear();
+        // Statistical section
+        if (enabledStats) {
+          stats.incrementRow();
+        }
+      } catch (InterruptedException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+
+    first = false;
+
+    keyWritable.set(rowkey);
+
+    readKeyValues(tuple, rowkey);
+    if (keyValues != null) {
+      for (KeyValue eachKeyVal: keyValues) {
+        kvSet.add(eachKeyVal);
+      }
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+  }
+
+  @Override
+  public long getEstimatedOutputSize() throws IOException {
+    // StoreTableExec uses this value as rolling file length
+    // Not rolling
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!kvSet.isEmpty()) {
+      try {
+        for (KeyValue kv : kvSet) {
+          writer.write(keyWritable, kv);
+          totalNumBytes += keyWritable.getLength() + keyWritable.getLength();
+        }
+        kvSet.clear();
+        // Statistical section
+        if (enabledStats) {
+          stats.incrementRow();
+        }
+      } catch (InterruptedException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+
+    if (enabledStats) {
+      stats.setNumBytes(totalNumBytes);
+    }
+    if (writer != null) {
+      try {
+        writer.close(writerContext);
+        committer.commitTask(writerContext);
+      } catch (InterruptedException e) {
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
new file mode 100644
index 0000000..3a58e50
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/IndexPredication.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+
+public class IndexPredication {
+  private Column column;
+  private int columnId;
+  private Datum startValue;
+  private Datum stopValue;
+
+  public Column getColumn() {
+    return column;
+  }
+
+  public void setColumn(Column column) {
+    this.column = column;
+  }
+
+  public int getColumnId() {
+    return columnId;
+  }
+
+  public void setColumnId(int columnId) {
+    this.columnId = columnId;
+  }
+
+  public Datum getStartValue() {
+    return startValue;
+  }
+
+  public void setStartValue(Datum startValue) {
+    this.startValue = startValue;
+  }
+
+  public Datum getStopValue() {
+    return stopValue;
+  }
+
+  public void setStopValue(Datum stopValue) {
+    this.stopValue = stopValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
new file mode 100644
index 0000000..4577703
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/RowKeyMapping.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+public class RowKeyMapping {
+  private boolean isBinary;
+  private int keyFieldIndex;
+
+  public boolean isBinary() {
+    return isBinary;
+  }
+
+  public void setBinary(boolean isBinary) {
+    this.isBinary = isBinary;
+  }
+
+  public int getKeyFieldIndex() {
+    return keyFieldIndex;
+  }
+
+  public void setKeyFieldIndex(int keyFieldIndex) {
+    this.keyFieldIndex = keyFieldIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..668b116
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/proto/StorageFragmentProtos.proto
@@ -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.
+ */
+
+option java_package = "org.apache.tajo.storage.hbase";
+option java_outer_classname = "StorageFragmentProtos";
+option optimize_for = SPEED;
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
+
+import "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;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..68939d6
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestColumnMapping.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.util.KeyValueSet;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class TestColumnMapping {
+  @Test
+  public void testColumnKeyValueMapping() throws Exception {
+    KeyValueSet keyValueSet = new KeyValueSet();
+    keyValueSet.set(HBaseStorageConstants.META_TABLE_KEY, "test");
+    keyValueSet.set(HBaseStorageConstants.META_COLUMNS_KEY, ":key,col2:key:,col2:value:#b,col3:");
+
+    Schema schema = new Schema();
+    schema.addColumn("c1", Type.TEXT);
+    schema.addColumn("c2", Type.TEXT);
+    schema.addColumn("c3", Type.TEXT);
+    schema.addColumn("c4", Type.TEXT);
+
+    TableMeta tableMeta = new TableMeta(StoreType.HBASE, keyValueSet);
+
+    ColumnMapping columnMapping = new ColumnMapping(schema, tableMeta);
+
+    List<String> cfNames = columnMapping.getColumnFamilyNames();
+    assertEquals(2, cfNames.size());
+    assertEquals("col2", cfNames.get(0));
+    assertEquals("col3", cfNames.get(1));
+
+    for (int i = 0; i < columnMapping.getIsBinaryColumns().length; i++) {
+      if (i == 2) {
+        assertTrue(columnMapping.getIsBinaryColumns()[i]);
+      } else {
+        assertFalse(columnMapping.getIsBinaryColumns()[i]);
+      }
+    }
+
+    for (int i = 0; i < columnMapping.getIsRowKeyMappings().length; i++) {
+      if (i == 0) {
+        assertTrue(columnMapping.getIsRowKeyMappings()[i]);
+      } else {
+        assertFalse(columnMapping.getIsRowKeyMappings()[i]);
+      }
+    }
+
+    String[] expectedColumnNames = { null, null, null, null};
+    for (int i = 0; i < schema.size(); i++) {
+      String columnName = columnMapping.getMappingColumns()[i][1] == null ? null :
+          new String(columnMapping.getMappingColumns()[i][1]);
+      assertEquals(expectedColumnNames[i], columnName);
+    }
+
+    for (int i = 0; i < schema.size(); i++) {
+      if (i == 1) {
+        assertTrue(columnMapping.getIsColumnKeys()[i]);
+      } else {
+        assertFalse(columnMapping.getIsColumnKeys()[i]);
+      }
+    }
+
+    for (int i = 0; i < schema.size(); i++) {
+      if (i == 2) {
+        assertTrue(columnMapping.getIsColumnValues()[i]);
+      } else {
+        assertFalse(columnMapping.getIsColumnValues()[i]);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..1fc4065
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/test/java/org/apache/tajo/storage/hbase/TestHBaseStorageManager.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.plan.expr.*;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.util.Pair;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestHBaseStorageManager {
+  @Test
+  public void testGetIndexPredications() throws Exception {
+    Column rowkeyColumn = new Column("rk", Type.TEXT);
+    // where rk >= '020' and rk <= '055'
+    ScanNode scanNode = new ScanNode(1);
+    EvalNode evalNode1 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("020")));
+    EvalNode evalNode2 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("055")));
+    EvalNode evalNodeA = new BinaryEval(EvalType.AND, evalNode1, evalNode2);
+    scanNode.setQual(evalNodeA);
+
+    HBaseStorageManager storageManager =
+        (HBaseStorageManager) StorageManager.getStorageManager(new TajoConf(), StoreType.HBASE);
+    List<Set<EvalNode>> indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertNotNull(indexEvals);
+    assertEquals(1, indexEvals.size());
+    Pair<Datum, Datum> indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    // where (rk >= '020' and rk <= '055') or rk = '075'
+    EvalNode evalNode3 = new BinaryEval(EvalType.EQUAL, new FieldEval(rowkeyColumn),new ConstEval(new TextDatum("075")));
+    EvalNode evalNodeB = new BinaryEval(EvalType.OR, evalNodeA, evalNode3);
+    scanNode.setQual(evalNodeB);
+    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertEquals(2, indexEvals.size());
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
+    assertEquals("075", indexPredicateValue.getFirst().asChars());
+    assertEquals("075", indexPredicateValue.getSecond().asChars());
+
+    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078')
+    EvalNode evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("072")));
+    EvalNode evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("078")));
+    EvalNode evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
+    EvalNode evalNodeD = new BinaryEval(EvalType.OR, evalNodeA, evalNodeC);
+    scanNode.setQual(evalNodeD);
+    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertEquals(2, indexEvals.size());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
+    assertEquals("072", indexPredicateValue.getFirst().asChars());
+    assertEquals("078", indexPredicateValue.getSecond().asChars());
+
+    // where (rk >= '020' and rk <= '055') or (rk >= '072' and rk <= '078' and rk >= '073')
+    evalNode4 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("072")));
+    evalNode5 = new BinaryEval(EvalType.LEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("078")));
+    evalNodeC = new BinaryEval(EvalType.AND, evalNode4, evalNode5);
+    EvalNode evalNode6 = new BinaryEval(EvalType.GEQ, new FieldEval(rowkeyColumn), new ConstEval(new TextDatum("073")));
+    evalNodeD = new BinaryEval(EvalType.AND, evalNodeC, evalNode6);
+    EvalNode evalNodeE = new BinaryEval(EvalType.OR, evalNodeA, evalNodeD);
+    scanNode.setQual(evalNodeE);
+    indexEvals = storageManager.findIndexablePredicateSet(scanNode, new Column[]{rowkeyColumn});
+    assertEquals(2, indexEvals.size());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(0));
+    assertEquals("020", indexPredicateValue.getFirst().asChars());
+    assertEquals("055", indexPredicateValue.getSecond().asChars());
+
+    indexPredicateValue = storageManager.getIndexablePredicateValue(null, indexEvals.get(1));
+    assertEquals("073", indexPredicateValue.getFirst().asChars());
+    assertEquals("078", indexPredicateValue.getSecond().asChars());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..5105ac5
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/pom.xml
@@ -0,0 +1,380 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Copyright 2012 Database Lab., Korea Univ.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.9.1-SNAPSHOT</version>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tajo-storage-hdfs</artifactId>
+  <packaging>jar</packaging>
+  <name>Tajo HDFS Storage</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    <parquet.version>1.5.0</parquet.version>
+    <parquet.format.version>2.1.0</parquet.format.version>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <excludes>
+            <exclude>src/test/resources/testVariousTypes.avsc</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/StorageFragmentProtos.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-plan</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>trevni-core</artifactId>
+      <version>1.7.3</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>trevni-avro</artifactId>
+      <version>1.7.3</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>slf4j-api</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jersey-json</artifactId>
+          <groupId>com.sun.jersey</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-server-tests</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-hs</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-format</artifactId>
+      <version>${parquet.format.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-buffer</artifactId>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
new file mode 100644
index 0000000..4bf4c99
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -0,0 +1,587 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang.StringUtils;
+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.io.IOUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.*;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.storage.compress.CodecPool;
+import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+import org.apache.tajo.util.BytesUtils;
+
+import java.io.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+public class CSVFile {
+
+  public static final byte LF = '\n';
+  public static int EOF = -1;
+
+  private static final Log LOG = LogFactory.getLog(CSVFile.class);
+
+  public static class CSVAppender extends FileAppender {
+    private final TableMeta meta;
+    private final Schema schema;
+    private final int columnNum;
+    private final FileSystem fs;
+    private FSDataOutputStream fos;
+    private DataOutputStream outputStream;
+    private CompressionOutputStream deflateFilter;
+    private char delimiter;
+    private TableStatistics stats = null;
+    private Compressor compressor;
+    private CompressionCodecFactory codecFactory;
+    private CompressionCodec codec;
+    private Path compressedPath;
+    private byte[] nullChars;
+    private int BUFFER_SIZE = 128 * 1024;
+    private int bufferedBytes = 0;
+    private long pos = 0;
+    private boolean isShuffle;
+
+    private NonSyncByteArrayOutputStream os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
+    private SerializerDeserializer serde;
+
+    public CSVAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
+                       final Schema schema, final TableMeta meta, final Path workDir) throws IOException {
+      super(conf, taskAttemptId, schema, meta, workDir);
+      this.fs = workDir.getFileSystem(conf);
+      this.meta = meta;
+      this.schema = schema;
+      this.delimiter = StringEscapeUtils.unescapeJava(
+          this.meta.getOption(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
+
+      this.columnNum = schema.size();
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(
+          this.meta.getOption(StorageConstants.TEXT_NULL, NullDatum.DEFAULT_TEXT));
+
+      if (StringUtils.isEmpty(nullCharacters)) {
+        nullChars = NullDatum.get().asTextBytes();
+      } else {
+        nullChars = nullCharacters.getBytes();
+      }
+    }
+
+    @Override
+    public void init() throws IOException {
+      if (!fs.exists(path.getParent())) {
+        throw new FileNotFoundException(path.getParent().toString());
+      }
+
+      //determine the intermediate file type
+      String store = conf.get(TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.varname,
+          TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.defaultVal);
+      if (enabledStats && CatalogProtos.StoreType.CSV == CatalogProtos.StoreType.valueOf(store.toUpperCase())) {
+        isShuffle = true;
+      } else {
+        isShuffle = false;
+      }
+
+      if(this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
+        String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
+        codecFactory = new CompressionCodecFactory(conf);
+        codec = codecFactory.getCodecByClassName(codecName);
+        compressor =  CodecPool.getCompressor(codec);
+        if(compressor != null) compressor.reset();  //builtin gzip is null
+
+        String extension = codec.getDefaultExtension();
+        compressedPath = path.suffix(extension);
+
+        if (fs.exists(compressedPath)) {
+          throw new AlreadyExistsStorageException(compressedPath);
+        }
+
+        fos = fs.create(compressedPath);
+        deflateFilter = codec.createOutputStream(fos, compressor);
+        outputStream = new DataOutputStream(deflateFilter);
+
+      } else {
+        if (fs.exists(path)) {
+          throw new AlreadyExistsStorageException(path);
+        }
+        fos = fs.create(path);
+        outputStream = new DataOutputStream(new BufferedOutputStream(fos));
+      }
+
+      if (enabledStats) {
+        this.stats = new TableStatistics(this.schema);
+      }
+
+      try {
+        //It will be remove, because we will add custom serde in textfile
+        String serdeClass = this.meta.getOption(StorageConstants.CSVFILE_SERDE,
+            TextSerializerDeserializer.class.getName());
+        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new IOException(e);
+      }
+
+      os.reset();
+      pos = fos.getPos();
+      bufferedBytes = 0;
+      super.init();
+    }
+
+
+    @Override
+    public void addTuple(Tuple tuple) throws IOException {
+      Datum datum;
+      int rowBytes = 0;
+
+      for (int i = 0; i < columnNum; i++) {
+        datum = tuple.get(i);
+        rowBytes += serde.serialize(schema.getColumn(i), datum, os, nullChars);
+
+        if(columnNum - 1 > i){
+          os.write((byte) delimiter);
+          rowBytes += 1;
+        }
+        if (isShuffle) {
+          // it is to calculate min/max values, and it is only used for the intermediate file.
+          stats.analyzeField(i, datum);
+        }
+      }
+      os.write(LF);
+      rowBytes += 1;
+
+      pos += rowBytes;
+      bufferedBytes += rowBytes;
+      if(bufferedBytes > BUFFER_SIZE){
+        flushBuffer();
+      }
+      // Statistical section
+      if (enabledStats) {
+        stats.incrementRow();
+      }
+    }
+
+    private void flushBuffer() throws IOException {
+      if(os.getLength() > 0) {
+        os.writeTo(outputStream);
+        os.reset();
+        bufferedBytes = 0;
+      }
+    }
+    @Override
+    public long getOffset() throws IOException {
+      return pos;
+    }
+
+    @Override
+    public void flush() throws IOException {
+      flushBuffer();
+      outputStream.flush();
+    }
+
+    @Override
+    public void close() throws IOException {
+
+      try {
+        flush();
+
+        // Statistical section
+        if (enabledStats) {
+          stats.setNumBytes(getOffset());
+        }
+
+        if(deflateFilter != null) {
+          deflateFilter.finish();
+          deflateFilter.resetState();
+          deflateFilter = null;
+        }
+
+        os.close();
+      } finally {
+        IOUtils.cleanup(LOG, fos);
+        if (compressor != null) {
+          CodecPool.returnCompressor(compressor);
+          compressor = null;
+        }
+      }
+    }
+
+    @Override
+    public TableStats getStats() {
+      if (enabledStats) {
+        return stats.getTableStat();
+      } else {
+        return null;
+      }
+    }
+
+    public boolean isCompress() {
+      return compressor != null;
+    }
+
+    public String getExtension() {
+      return codec != null ? codec.getDefaultExtension() : "";
+    }
+  }
+
+  public static class CSVScanner extends FileScanner implements SeekableScanner {
+    public CSVScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment)
+        throws IOException {
+      super(conf, schema, meta, fragment);
+      factory = new CompressionCodecFactory(conf);
+      codec = factory.getCodec(this.fragment.getPath());
+      if (codec == null || codec instanceof SplittableCompressionCodec) {
+        splittable = true;
+      }
+
+      //Delimiter
+      this.delimiter = StringEscapeUtils.unescapeJava(
+          meta.getOption(StorageConstants.TEXT_DELIMITER,
+          meta.getOption(StorageConstants.CSVFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER))).charAt(0);
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(
+          meta.getOption(StorageConstants.TEXT_NULL,
+          meta.getOption(StorageConstants.CSVFILE_NULL, NullDatum.DEFAULT_TEXT)));
+
+      if (StringUtils.isEmpty(nullCharacters)) {
+        nullChars = NullDatum.get().asTextBytes();
+      } else {
+        nullChars = nullCharacters.getBytes();
+      }
+    }
+
+    private final static int DEFAULT_PAGE_SIZE = 256 * 1024;
+    private char delimiter;
+    private FileSystem fs;
+    private FSDataInputStream fis;
+    private InputStream is; //decompressd stream
+    private CompressionCodecFactory factory;
+    private CompressionCodec codec;
+    private Decompressor decompressor;
+    private Seekable filePosition;
+    private boolean splittable = false;
+    private long startOffset, end, pos;
+    private int currentIdx = 0, validIdx = 0, recordCount = 0;
+    private int[] targetColumnIndexes;
+    private boolean eof = false;
+    private final byte[] nullChars;
+    private SplitLineReader reader;
+    private ArrayList<Long> fileOffsets;
+    private ArrayList<Integer> rowLengthList;
+    private ArrayList<Integer> startOffsets;
+    private NonSyncByteArrayOutputStream buffer;
+    private SerializerDeserializer serde;
+
+    @Override
+    public void init() throws IOException {
+      fileOffsets = new ArrayList<Long>();
+      rowLengthList = new ArrayList<Integer>();
+      startOffsets = new ArrayList<Integer>();
+      buffer = new NonSyncByteArrayOutputStream(DEFAULT_PAGE_SIZE);
+
+      // FileFragment information
+      if(fs == null) {
+        fs = FileScanner.getFileSystem((TajoConf)conf, fragment.getPath());
+      }
+      if(fis == null) fis = fs.open(fragment.getPath());
+
+      recordCount = 0;
+      pos = startOffset = fragment.getStartKey();
+      end = startOffset + fragment.getLength();
+
+      if (codec != null) {
+        decompressor = CodecPool.getDecompressor(codec);
+        if (codec instanceof SplittableCompressionCodec) {
+          SplitCompressionInputStream cIn = ((SplittableCompressionCodec) codec).createInputStream(
+              fis, decompressor, startOffset, end,
+              SplittableCompressionCodec.READ_MODE.BYBLOCK);
+
+          reader = new CompressedSplitLineReader(cIn, conf, null);
+          startOffset = cIn.getAdjustedStart();
+          end = cIn.getAdjustedEnd();
+          filePosition = cIn;
+          is = cIn;
+        } else {
+          is = new DataInputStream(codec.createInputStream(fis, decompressor));
+          reader = new SplitLineReader(is, null);
+          filePosition = fis;
+        }
+      } else {
+        fis.seek(startOffset);
+        filePosition = fis;
+        is = fis;
+        reader = new SplitLineReader(is, null);
+      }
+
+      if (targets == null) {
+        targets = schema.toArray();
+      }
+
+      targetColumnIndexes = new int[targets.length];
+      for (int i = 0; i < targets.length; i++) {
+        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
+      }
+
+      try {
+        //FIXME
+        String serdeClass = this.meta.getOption(StorageConstants.CSVFILE_SERDE,
+            TextSerializerDeserializer.class.getName());
+        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new IOException(e);
+      }
+
+      super.init();
+      Arrays.sort(targetColumnIndexes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("CSVScanner open:" + fragment.getPath() + "," + startOffset + "," + end +
+            "," + fs.getFileStatus(fragment.getPath()).getLen());
+      }
+
+      if (startOffset != 0) {
+        pos += reader.readLine(new Text(), 0, maxBytesToConsume(pos));
+      }
+      eof = false;
+      page();
+    }
+
+    private int maxBytesToConsume(long pos) {
+      return isCompress() ? Integer.MAX_VALUE : (int) Math.min(Integer.MAX_VALUE, end - pos);
+    }
+
+    private long fragmentable() throws IOException {
+      return end - getFilePosition();
+    }
+
+    private long getFilePosition() throws IOException {
+      long retVal;
+      if (isCompress()) {
+        retVal = filePosition.getPos();
+      } else {
+        retVal = pos;
+      }
+      return retVal;
+    }
+
+    private void page() throws IOException {
+//      // Index initialization
+      currentIdx = 0;
+      validIdx = 0;
+      int currentBufferPos = 0;
+      int bufferedSize = 0;
+
+      buffer.reset();
+      startOffsets.clear();
+      rowLengthList.clear();
+      fileOffsets.clear();
+
+      if(eof) {
+        return;
+      }
+
+      while (DEFAULT_PAGE_SIZE >= bufferedSize){
+
+        int ret = reader.readDefaultLine(buffer, rowLengthList, Integer.MAX_VALUE, Integer.MAX_VALUE);
+
+        if(ret == 0){
+          break;
+        } else {
+          fileOffsets.add(pos);
+          pos += ret;
+          startOffsets.add(currentBufferPos);
+          currentBufferPos += rowLengthList.get(rowLengthList.size() - 1);
+          bufferedSize += ret;
+          validIdx++;
+          recordCount++;
+        }
+
+        if(getFilePosition() > end && !reader.needAdditionalRecordAfterSplit()){
+          eof = true;
+          break;
+        }
+      }
+      if (tableStats != null) {
+        tableStats.setReadBytes(pos - startOffset);
+        tableStats.setNumRows(recordCount);
+      }
+    }
+
+    @Override
+    public float getProgress() {
+      try {
+        if(eof) {
+          return 1.0f;
+        }
+        long filePos = getFilePosition();
+        if (startOffset == filePos) {
+          return 0.0f;
+        } else {
+          long readBytes = filePos - startOffset;
+          long remainingBytes = Math.max(end - filePos, 0);
+          return Math.min(1.0f, (float)(readBytes) / (float)(readBytes + remainingBytes));
+        }
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+        return 0.0f;
+      }
+    }
+
+    @Override
+    public Tuple next() throws IOException {
+      try {
+        if (currentIdx == validIdx) {
+          if (eof) {
+            return null;
+          } else {
+            page();
+
+            if(currentIdx == validIdx){
+              return null;
+            }
+          }
+        }
+
+        long offset = -1;
+        if(!isCompress()){
+          offset = fileOffsets.get(currentIdx);
+        }
+
+        byte[][] cells = BytesUtils.splitPreserveAllTokens(buffer.getData(), startOffsets.get(currentIdx),
+            rowLengthList.get(currentIdx), delimiter, targetColumnIndexes);
+        currentIdx++;
+        return new LazyTuple(schema, cells, offset, nullChars, serde);
+      } catch (Throwable t) {
+        LOG.error("Tuple list length: " + (fileOffsets != null ? fileOffsets.size() : 0), t);
+        LOG.error("Tuple list current index: " + currentIdx, t);
+        throw new IOException(t);
+      }
+    }
+
+    private boolean isCompress() {
+      return codec != null;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      if (decompressor != null) {
+        CodecPool.returnDecompressor(decompressor);
+        decompressor = null;
+      }
+
+      init();
+    }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        if (tableStats != null) {
+          tableStats.setReadBytes(pos - startOffset);  //Actual Processed Bytes. (decompressed bytes + overhead)
+          tableStats.setNumRows(recordCount);
+        }
+
+        IOUtils.cleanup(LOG, reader, is, fis);
+        fs = null;
+        is = null;
+        fis = null;
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("CSVScanner processed record:" + recordCount);
+        }
+      } finally {
+        if (decompressor != null) {
+          CodecPool.returnDecompressor(decompressor);
+          decompressor = null;
+        }
+      }
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return true;
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public void setSearchCondition(Object expr) {
+    }
+
+    @Override
+    public void seek(long offset) throws IOException {
+      if(isCompress()) throw new UnsupportedException();
+
+      int tupleIndex = Arrays.binarySearch(fileOffsets.toArray(), offset);
+
+      if (tupleIndex > -1) {
+        this.currentIdx = tupleIndex;
+      } else if (isSplittable() && end >= offset || startOffset <= offset) {
+        eof = false;
+        fis.seek(offset);
+        pos = offset;
+        reader.reset();
+        this.currentIdx = 0;
+        this.validIdx = 0;
+        // pageBuffer();
+      } else {
+        throw new IOException("invalid offset " +
+            " < start : " +  startOffset + " , " +
+            "  end : " + end + " , " +
+            "  filePos : " + filePosition.getPos() + " , " +
+            "  input offset : " + offset + " >");
+      }
+    }
+
+    @Override
+    public long getNextOffset() throws IOException {
+      if(isCompress()) throw new UnsupportedException();
+
+      if (this.currentIdx == this.validIdx) {
+        if (fragmentable() <= 0) {
+          return -1;
+        } else {
+          page();
+          if(currentIdx == validIdx) return -1;
+        }
+      }
+      return fileOffsets.get(currentIdx);
+    }
+
+    @Override
+    public boolean isSplittable(){
+      return splittable;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
new file mode 100644
index 0000000..4f58e68
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.SplitCompressionInputStream;
+import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+
+/**
+ * Line reader for compressed splits
+ *
+ * Reading records from a compressed split is tricky, as the
+ * LineRecordReader is using the reported compressed input stream
+ * position directly to determine when a split has ended.  In addition the
+ * compressed input stream is usually faking the actual byte position, often
+ * updating it only after the first compressed block after the split is
+ * accessed.
+ *
+ * Depending upon where the last compressed block of the split ends relative
+ * to the record delimiters it can be easy to accidentally drop the last
+ * record or duplicate the last record between this split and the next.
+ *
+ * Split end scenarios:
+ *
+ * 1) Last block of split ends in the middle of a record
+ *      Nothing special that needs to be done here, since the compressed input
+ *      stream will report a position after the split end once the record
+ *      is fully read.  The consumer of the next split will discard the
+ *      partial record at the start of the split normally, and no data is lost
+ *      or duplicated between the splits.
+ *
+ * 2) Last block of split ends in the middle of a delimiter
+ *      The line reader will continue to consume bytes into the next block to
+ *      locate the end of the delimiter.  If a custom delimiter is being used
+ *      then the next record must be read by this split or it will be dropped.
+ *      The consumer of the next split will not recognize the partial
+ *      delimiter at the beginning of its split and will discard it along with
+ *      the next record.
+ *
+ *      However for the default delimiter processing there is a special case
+ *      because CR, LF, and CRLF are all valid record delimiters.  If the
+ *      block ends with a CR then the reader must peek at the next byte to see
+ *      if it is an LF and therefore part of the same record delimiter.
+ *      Peeking at the next byte is an access to the next block and triggers
+ *      the stream to report the end of the split.  There are two cases based
+ *      on the next byte:
+ *
+ *      A) The next byte is LF
+ *           The split needs to end after the current record is returned.  The
+ *           consumer of the next split will discard the first record, which
+ *           is degenerate since LF is itself a delimiter, and start consuming
+ *           records after that byte.  If the current split tries to read
+ *           another record then the record will be duplicated between splits.
+ *
+ *      B) The next byte is not LF
+ *           The current record will be returned but the stream will report
+ *           the split has ended due to the peek into the next block.  If the
+ *           next record is not read then it will be lost, as the consumer of
+ *           the next split will discard it before processing subsequent
+ *           records.  Therefore the next record beyond the reported split end
+ *           must be consumed by this split to avoid data loss.
+ *
+ * 3) Last block of split ends at the beginning of a delimiter
+ *      This is equivalent to case 1, as the reader will consume bytes into
+ *      the next block and trigger the end of the split.  No further records
+ *      should be read as the consumer of the next split will discard the
+ *      (degenerate) record at the beginning of its split.
+ *
+ * 4) Last block of split ends at the end of a delimiter
+ *      Nothing special needs to be done here. The reader will not start
+ *      examining the bytes into the next block until the next record is read,
+ *      so the stream will not report the end of the split just yet.  Once the
+ *      next record is read then the next block will be accessed and the
+ *      stream will indicate the end of the split.  The consumer of the next
+ *      split will correctly discard the first record of its split, and no
+ *      data is lost or duplicated.
+ *
+ *      If the default delimiter is used and the block ends at a CR then this
+ *      is treated as case 2 since the reader does not yet know without
+ *      looking at subsequent bytes whether the delimiter has ended.
+ *
+ * NOTE: It is assumed that compressed input streams *never* return bytes from
+ *       multiple compressed blocks from a single read.  Failure to do so will
+ *       violate the buffering performed by this class, as it will access
+ *       bytes into the next block after the split before returning all of the
+ *       records from the previous block.
+ */
+
+public class CompressedSplitLineReader extends SplitLineReader {
+  SplitCompressionInputStream scin;
+  private boolean usingCRLF;
+  private boolean needAdditionalRecord = false;
+  private boolean finished = false;
+
+  public CompressedSplitLineReader(SplitCompressionInputStream in,
+                                   Configuration conf,
+                                   byte[] recordDelimiterBytes)
+      throws IOException {
+    super(in, conf, recordDelimiterBytes);
+    scin = in;
+    usingCRLF = (recordDelimiterBytes == null);
+  }
+
+  @Override
+  protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter)
+      throws IOException {
+    int bytesRead = in.read(buffer);
+
+    // If the split ended in the middle of a record delimiter then we need
+    // to read one additional record, as the consumer of the next split will
+    // not recognize the partial delimiter as a record.
+    // However if using the default delimiter and the next character is a
+    // linefeed then next split will treat it as a delimiter all by itself
+    // and the additional record read should not be performed.
+    if (inDelimiter && bytesRead > 0) {
+      if (usingCRLF) {
+        needAdditionalRecord = (buffer[0] != '\n');
+      } else {
+        needAdditionalRecord = true;
+      }
+    }
+    return bytesRead;
+  }
+
+  @Override
+  public int readLine(Text str, int maxLineLength, int maxBytesToConsume)
+      throws IOException {
+    int bytesRead = 0;
+    if (!finished) {
+      // only allow at most one more record to be read after the stream
+      // reports the split ended
+      if (scin.getPos() > scin.getAdjustedEnd()) {
+        finished = true;
+      }
+
+      bytesRead = super.readLine(str, maxLineLength, maxBytesToConsume);
+    }
+    return bytesRead;
+  }
+
+  @Override
+  public int readDefaultLine(NonSyncByteArrayOutputStream str, ArrayList<Integer> offsets, int maxLineLength
+      , int maxBytesToConsume) throws IOException {
+    int bytesRead = 0;
+    if (!finished) {
+      // only allow at most one more record to be read after the stream
+      // reports the split ended
+      if (scin.getPos() > scin.getAdjustedEnd()) {
+        finished = true;
+      }
+
+      bytesRead = super.readDefaultLine(str, offsets, maxLineLength, maxBytesToConsume);
+    }
+    return bytesRead;
+  }
+
+  @Override
+  public boolean needAdditionalRecordAfterSplit() {
+    return !finished && needAdditionalRecord;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..47f67c6
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileAppender.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.conf.TajoConf;
+
+import java.io.IOException;
+
+public abstract class FileAppender implements Appender {
+  private static final Log LOG = LogFactory.getLog(FileAppender.class);
+
+  protected boolean inited = false;
+
+  protected final Configuration conf;
+  protected final TableMeta meta;
+  protected final Schema schema;
+  protected final Path workDir;
+  protected final QueryUnitAttemptId taskAttemptId;
+
+  protected boolean enabledStats;
+  protected Path path;
+
+  public FileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId, Schema schema,
+                      TableMeta meta, Path workDir) {
+    this.conf = conf;
+    this.meta = meta;
+    this.schema = schema;
+    this.workDir = workDir;
+    this.taskAttemptId = taskAttemptId;
+
+    try {
+      if (taskAttemptId != null) {
+        this.path = ((FileStorageManager)StorageManager.getFileStorageManager((TajoConf) conf))
+            .getAppenderFilePath(taskAttemptId, workDir);
+      } else {
+        this.path = workDir;
+      }
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+      throw new IllegalStateException("Error while opeining FileAppender: " + e.getMessage(), e);
+    }
+  }
+
+  public void init() throws IOException {
+    if (inited) {
+     throw new IllegalStateException("FileAppender is already initialized.");
+    }
+    inited = true;
+  }
+
+  public void enableStats() {
+    if (inited) {
+      throw new IllegalStateException("Should enable this option before init()");
+    }
+
+    this.enabledStats = true;
+  }
+
+  public long getEstimatedOutputSize() throws IOException {
+    return getOffset();
+  }
+
+  public abstract long getOffset() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java
new file mode 100644
index 0000000..038f0f4
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.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.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+
+public abstract class FileScanner implements Scanner {
+  private static final Log LOG = LogFactory.getLog(FileScanner.class);
+
+  protected boolean inited = false;
+  protected final Configuration conf;
+  protected final TableMeta meta;
+  protected final Schema schema;
+  protected final FileFragment fragment;
+  protected final int columnNum;
+
+  protected Column [] targets;
+
+  protected float progress;
+
+  protected TableStats tableStats;
+
+  public FileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment) {
+    this.conf = conf;
+    this.meta = meta;
+    this.schema = schema;
+    this.fragment = (FileFragment)fragment;
+    this.tableStats = new TableStats();
+    this.columnNum = this.schema.size();
+  }
+
+  public void init() throws IOException {
+    inited = true;
+    progress = 0.0f;
+
+    if (fragment != null) {
+      tableStats.setNumBytes(fragment.getLength());
+      tableStats.setNumBlocks(1);
+    }
+
+    if (schema != null) {
+      for(Column eachColumn: schema.getColumns()) {
+        ColumnStats columnStats = new ColumnStats(eachColumn);
+        tableStats.addColumnStat(columnStats);
+      }
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public void setTarget(Column[] targets) {
+    if (inited) {
+      throw new IllegalStateException("Should be called before init()");
+    }
+    this.targets = targets;
+  }
+
+  public void setSearchCondition(Object expr) {
+    if (inited) {
+      throw new IllegalStateException("Should be called before init()");
+    }
+  }
+
+  public static FileSystem getFileSystem(TajoConf tajoConf, Path path) throws IOException {
+    String tajoUser = tajoConf.getVar(TajoConf.ConfVars.USERNAME);
+    FileSystem fs;
+    if(tajoUser != null) {
+      try {
+        fs = FileSystem.get(path.toUri(), tajoConf, tajoUser);
+      } catch (InterruptedException e) {
+        LOG.warn("Occur InterruptedException while FileSystem initiating with user[" + tajoUser + "]");
+        fs = FileSystem.get(path.toUri(), tajoConf);
+      }
+    } else {
+      fs = FileSystem.get(path.toUri(), tajoConf);
+    }
+
+    return fs;
+  }
+
+  @Override
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    return tableStats;
+  }
+}


[23/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java b/tajo-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
deleted file mode 100644
index f093f9d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
+++ /dev/null
@@ -1,623 +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.index.bst;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.RowStoreUtil.RowStoreDecoder;
-import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
-import org.apache.tajo.storage.index.IndexMethod;
-import org.apache.tajo.storage.index.IndexWriter;
-import org.apache.tajo.storage.index.OrderIndexReader;
-
-import java.io.Closeable;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.Set;
-import java.util.TreeMap;
-
-import static org.apache.tajo.index.IndexProtos.TupleComparatorProto;
-
-/**
- * This is two-level binary search tree index. This is one of the value-list 
- * index structure. Thus, it is inefficient in the case where 
- * the many of the values are same. Also, the BST shows the fast performance 
- * when the selectivity of rows to be retrieved is less than 5%.
- * BSTIndexWriter is not thread-safe, whereas BSTIndexReader is thread-safe.
- */
-public class BSTIndex implements IndexMethod {
-  private static final Log LOG = LogFactory.getLog(BSTIndex.class);
-
-  public static final int ONE_LEVEL_INDEX = 1;
-  public static final int TWO_LEVEL_INDEX = 2;
-
-  private final Configuration conf;
-
-  public BSTIndex(final Configuration conf) {
-    this.conf = conf;
-  }
-  
-  @Override
-  public BSTIndexWriter getIndexWriter(final Path fileName, int level, Schema keySchema,
-      TupleComparator comparator) throws IOException {
-    return new BSTIndexWriter(fileName, level, keySchema, comparator);
-  }
-
-  @Override
-  public BSTIndexReader getIndexReader(Path fileName, Schema keySchema, TupleComparator comparator) throws IOException {
-    return new BSTIndexReader(fileName, keySchema, comparator);
-  }
-
-  public BSTIndexReader getIndexReader(Path fileName) throws IOException {
-    return new BSTIndexReader(fileName);
-  }
-
-  public class BSTIndexWriter extends IndexWriter implements Closeable {
-    private FSDataOutputStream out;
-    private FileSystem fs;
-    private int level;
-    private int loadNum = 4096;
-    private Path fileName;
-
-    private final Schema keySchema;
-    private final TupleComparator compartor;
-    private final KeyOffsetCollector collector;
-    private KeyOffsetCollector rootCollector;
-
-    private Tuple firstKey;
-    private Tuple lastKey;
-
-    private RowStoreEncoder rowStoreEncoder;
-
-    // private Tuple lastestKey = null;
-
-    /**
-     * constructor
-     *
-     * @param level
-     *          : IndexCreater.ONE_LEVEL_INDEX or IndexCreater.TWO_LEVEL_INDEX
-     * @throws IOException
-     */
-    public BSTIndexWriter(final Path fileName, int level, Schema keySchema,
-        TupleComparator comparator) throws IOException {
-      this.fileName = fileName;
-      this.level = level;
-      this.keySchema = keySchema;
-      this.compartor = comparator;
-      this.collector = new KeyOffsetCollector(comparator);
-      this.rowStoreEncoder = RowStoreUtil.createEncoder(keySchema);
-    }
-
-   public void setLoadNum(int loadNum) {
-      this.loadNum = loadNum;
-    }
-
-    public void open() throws IOException {
-      fs = fileName.getFileSystem(conf);
-      if (fs.exists(fileName)) {
-        throw new IOException("ERROR: index file (" + fileName + " already exists");
-      }
-      out = fs.create(fileName);
-    }
-
-    @Override
-    public void write(Tuple key, long offset) throws IOException {
-      if (firstKey == null || compartor.compare(key, firstKey) < 0) {
-        firstKey = key;
-      }
-      if (lastKey == null || compartor.compare(lastKey, key) < 0) {
-        lastKey = key;
-      }
-
-      collector.put(key, offset);
-    }
-
-    public TupleComparator getComparator() {
-      return this.compartor;
-    }
-
-    public void flush() throws IOException {
-      out.flush();
-    }
-
-    public void writeHeader(int entryNum) throws IOException {
-      // schema
-      byte [] schemaBytes = keySchema.getProto().toByteArray();
-      out.writeInt(schemaBytes.length);
-      out.write(schemaBytes);
-
-      // comparator
-      byte [] comparatorBytes = compartor.getProto().toByteArray();
-      out.writeInt(comparatorBytes.length);
-      out.write(comparatorBytes);
-
-      // level
-      out.writeInt(this.level);
-      // entry
-      out.writeInt(entryNum);
-      if (entryNum > 0) {
-        byte [] minBytes = rowStoreEncoder.toBytes(firstKey);
-        out.writeInt(minBytes.length);
-        out.write(minBytes);
-        byte [] maxBytes = rowStoreEncoder.toBytes(lastKey);
-        out.writeInt(maxBytes.length);
-        out.write(maxBytes);
-      }
-      out.flush();
-    }
-
-    public void close() throws IOException {
-      /* two level initialize */
-      if (this.level == TWO_LEVEL_INDEX) {
-        rootCollector = new KeyOffsetCollector(this.compartor);
-      }
-
-      /* data writing phase */
-      TreeMap<Tuple, LinkedList<Long>> keyOffsetMap = collector.getMap();
-      Set<Tuple> keySet = keyOffsetMap.keySet();
-
-      int entryNum = keySet.size();
-      writeHeader(entryNum);
-      
-      int loadCount = this.loadNum - 1;
-      for (Tuple key : keySet) {
-
-        if (this.level == TWO_LEVEL_INDEX) {
-          loadCount++;
-          if (loadCount == this.loadNum) {
-            rootCollector.put(key, out.getPos());
-            loadCount = 0;
-          }
-        }
-        /* key writing */
-        byte[] buf = rowStoreEncoder.toBytes(key);
-        out.writeInt(buf.length);
-        out.write(buf);
-        
-        /**/
-        LinkedList<Long> offsetList = keyOffsetMap.get(key);
-        /* offset num writing */
-        int offsetSize = offsetList.size();
-        out.writeInt(offsetSize);
-        /* offset writing */
-        for (Long offset : offsetList) {
-          out.writeLong(offset);
-        }
-      }
-
-      out.flush();
-      out.close();
-      keySet.clear();
-      collector.clear();
-
-      FSDataOutputStream rootOut = null;
-      /* root index creating phase */
-      if (this.level == TWO_LEVEL_INDEX) {
-        TreeMap<Tuple, LinkedList<Long>> rootMap = rootCollector.getMap();
-        keySet = rootMap.keySet();
-
-        rootOut = fs.create(new Path(fileName + ".root"));
-        rootOut.writeInt(this.loadNum);
-        rootOut.writeInt(keySet.size());
-
-        /* root key writing */
-        for (Tuple key : keySet) {
-          byte[] buf = rowStoreEncoder.toBytes(key);
-          rootOut.writeInt(buf.length);
-          rootOut.write(buf);
-
-          LinkedList<Long> offsetList = rootMap.get(key);
-          if (offsetList.size() > 1 || offsetList.size() == 0) {
-            throw new IOException("Why root index doen't have one offset?");
-          }
-          rootOut.writeLong(offsetList.getFirst());
-
-        }
-        rootOut.flush();
-        rootOut.close();
-
-        keySet.clear();
-        rootCollector.clear();
-      }
-    }
-
-    private class KeyOffsetCollector {
-      private TreeMap<Tuple, LinkedList<Long>> map;
-
-      public KeyOffsetCollector(TupleComparator comparator) {
-        map = new TreeMap<Tuple, LinkedList<Long>>(comparator);
-      }
-
-      public void put(Tuple key, long offset) {
-        if (map.containsKey(key)) {
-          map.get(key).add(offset);
-        } else {
-          LinkedList<Long> list = new LinkedList<Long>();
-          list.add(offset);
-          map.put(key, list);
-        }
-      }
-
-      public TreeMap<Tuple, LinkedList<Long>> getMap() {
-        return this.map;
-      }
-
-      public void clear() {
-        this.map.clear();
-      }
-    }
-  }
-
-  /**
-   * BSTIndexReader is thread-safe.
-   */
-  public class BSTIndexReader implements OrderIndexReader , Closeable{
-    private Path fileName;
-    private Schema keySchema;
-    private TupleComparator comparator;
-
-    private FileSystem fs;
-    private FSDataInputStream indexIn;
-    private FSDataInputStream subIn;
-
-    private int level;
-    private int entryNum;
-    private int loadNum = -1;
-    private Tuple firstKey;
-    private Tuple lastKey;
-
-    // the cursors of BST
-    private int rootCursor;
-    private int keyCursor;
-    private int offsetCursor;
-
-    // mutex
-    private final Object mutex = new Object();
-
-    private RowStoreDecoder rowStoreDecoder;
-
-    /**
-     *
-     * @param fileName
-     * @param keySchema
-     * @param comparator
-     * @throws IOException
-     */
-    public BSTIndexReader(final Path fileName, Schema keySchema, TupleComparator comparator) throws IOException {
-      this.fileName = fileName;
-      this.keySchema = keySchema;
-      this.comparator = comparator;
-      this.rowStoreDecoder = RowStoreUtil.createDecoder(keySchema);
-    }
-
-    public BSTIndexReader(final Path fileName) throws IOException {
-      this.fileName = fileName;
-    }
-
-    public Schema getKeySchema() {
-      return this.keySchema;
-    }
-
-    public TupleComparator getComparator() {
-      return this.comparator;
-    }
-
-    private void readHeader() throws IOException {
-      // schema
-      int schemaByteSize = indexIn.readInt();
-      byte [] schemaBytes = new byte[schemaByteSize];
-      StorageUtil.readFully(indexIn, schemaBytes, 0, schemaByteSize);
-
-      SchemaProto.Builder builder = SchemaProto.newBuilder();
-      builder.mergeFrom(schemaBytes);
-      SchemaProto proto = builder.build();
-      this.keySchema = new Schema(proto);
-      this.rowStoreDecoder = RowStoreUtil.createDecoder(keySchema);
-
-      // comparator
-      int compByteSize = indexIn.readInt();
-      byte [] compBytes = new byte[compByteSize];
-      StorageUtil.readFully(indexIn, compBytes, 0, compByteSize);
-
-      TupleComparatorProto.Builder compProto = TupleComparatorProto.newBuilder();
-      compProto.mergeFrom(compBytes);
-      this.comparator = new BaseTupleComparator(compProto.build());
-
-      // level
-      this.level = indexIn.readInt();
-      // entry
-      this.entryNum = indexIn.readInt();
-      if (entryNum > 0) { // if there is no any entry, do not read firstKey/lastKey values
-        byte [] minBytes = new byte[indexIn.readInt()];
-        StorageUtil.readFully(indexIn, minBytes, 0, minBytes.length);
-        this.firstKey = rowStoreDecoder.toTuple(minBytes);
-
-        byte [] maxBytes = new byte[indexIn.readInt()];
-        StorageUtil.readFully(indexIn, maxBytes, 0, maxBytes.length);
-        this.lastKey = rowStoreDecoder.toTuple(maxBytes);
-      }
-    }
-
-    public void open()
-        throws IOException {
-      /* init the index file */
-      fs = fileName.getFileSystem(conf);
-      if (!fs.exists(fileName)) {
-        throw new FileNotFoundException("ERROR: does not exist " + fileName.toString());
-      }
-
-      indexIn = fs.open(this.fileName);
-      readHeader();
-      fillData();
-    }
-
-    private void fillData() throws IOException {
-      /* load on memory */
-      if (this.level == TWO_LEVEL_INDEX) {
-
-        Path rootPath = new Path(this.fileName + ".root");
-        if (!fs.exists(rootPath)) {
-          throw new FileNotFoundException("root index did not created");
-        }
-
-        subIn = indexIn;
-        indexIn = fs.open(rootPath);
-        /* root index header reading : type => loadNum => indexSize */
-        this.loadNum = indexIn.readInt();
-        this.entryNum = indexIn.readInt();
-        /**/
-        fillRootIndex(entryNum, indexIn);
-
-      } else {
-        fillLeafIndex(entryNum, indexIn, -1);
-      }
-    }
-
-    /**
-     *
-     * @return
-     * @throws IOException
-     */
-    public long find(Tuple key) throws IOException {
-      return find(key, false);
-    }
-
-    @Override
-    public long find(Tuple key, boolean nextKey) throws IOException {
-      synchronized (mutex) {
-        int pos = -1;
-        if (this.level == ONE_LEVEL_INDEX) {
-            pos = oneLevBS(key);
-        } else if (this.level == TWO_LEVEL_INDEX) {
-            pos = twoLevBS(key, this.loadNum + 1);
-        } else {
-          throw new IOException("More than TWL_LEVEL_INDEX is not supported.");
-        }
-
-        if (nextKey) {
-          if (pos + 1 >= this.offsetSubIndex.length) {
-            return -1;
-          }
-          keyCursor = pos + 1;
-          offsetCursor = 0;
-        } else {
-          if (correctable) {
-            keyCursor = pos;
-            offsetCursor = 0;
-          } else {
-            return -1;
-          }
-        }
-
-        return this.offsetSubIndex[keyCursor][offsetCursor];
-      }
-    }
-
-    public long next() throws IOException {
-      synchronized (mutex) {
-        if (offsetSubIndex[keyCursor].length - 1 > offsetCursor) {
-          offsetCursor++;
-        } else {
-          if (offsetSubIndex.length - 1 > keyCursor) {
-            keyCursor++;
-            offsetCursor = 0;
-          } else {
-            if (offsetIndex.length -1 > rootCursor) {
-              rootCursor++;
-              fillLeafIndex(loadNum + 1, subIn, this.offsetIndex[rootCursor]);
-              keyCursor = 1;
-              offsetCursor = 0;
-            } else {
-              return -1;
-            }
-          }
-        }
-
-        return this.offsetSubIndex[keyCursor][offsetCursor];
-      }
-    }
-    
-    public boolean isCurInMemory() {
-      return (offsetSubIndex[keyCursor].length - 1 >= offsetCursor);
-    }
-
-    private void fillLeafIndex(int entryNum, FSDataInputStream in, long pos)
-        throws IOException {
-      int counter = 0;
-      try {
-        if (pos != -1) {
-          in.seek(pos);
-        }
-        this.dataSubIndex = new Tuple[entryNum];
-        this.offsetSubIndex = new long[entryNum][];
-
-        byte[] buf;
-        for (int i = 0; i < entryNum; i++) {
-          counter++;
-          buf = new byte[in.readInt()];
-          StorageUtil.readFully(in, buf, 0, buf.length);
-          dataSubIndex[i] = rowStoreDecoder.toTuple(buf);
-
-          int offsetNum = in.readInt();
-          this.offsetSubIndex[i] = new long[offsetNum];
-          for (int j = 0; j < offsetNum; j++) {
-            this.offsetSubIndex[i][j] = in.readLong();
-          }
-
-        }
-
-      } catch (IOException e) {
-        counter--;
-        if (pos != -1) {
-          in.seek(pos);
-        }
-        this.dataSubIndex = new Tuple[counter];
-        this.offsetSubIndex = new long[counter][];
-
-        byte[] buf;
-        for (int i = 0; i < counter; i++) {
-          buf = new byte[in.readInt()];
-          StorageUtil.readFully(in, buf, 0, buf.length);
-          dataSubIndex[i] = rowStoreDecoder.toTuple(buf);
-
-          int offsetNum = in.readInt();
-          this.offsetSubIndex[i] = new long[offsetNum];
-          for (int j = 0; j < offsetNum; j++) {
-            this.offsetSubIndex[i][j] = in.readLong();
-          }
-
-        }
-      }
-    }
-
-    public Tuple getFirstKey() {
-      return this.firstKey;
-    }
-
-    public Tuple getLastKey() {
-      return this.lastKey;
-    }
-
-    private void fillRootIndex(int entryNum, FSDataInputStream in)
-        throws IOException {
-      this.dataIndex = new Tuple[entryNum];
-      this.offsetIndex = new long[entryNum];
-      Tuple keyTuple;
-      byte[] buf;
-      for (int i = 0; i < entryNum; i++) {
-        buf = new byte[in.readInt()];
-        StorageUtil.readFully(in, buf, 0, buf.length);
-        keyTuple = rowStoreDecoder.toTuple(buf);
-        dataIndex[i] = keyTuple;
-        this.offsetIndex[i] = in.readLong();
-      }
-    }
-
-    /* memory index, only one is used. */
-    private Tuple[] dataIndex = null;
-    private Tuple[] dataSubIndex = null;
-
-    /* offset index */
-    private long[] offsetIndex = null;
-    private long[][] offsetSubIndex = null;
-
-    private boolean correctable = true;
-
-    private int oneLevBS(Tuple key) throws IOException {
-      correctable = true;
-      int pos = binarySearch(this.dataSubIndex, key, 0, this.dataSubIndex.length);
-      return pos;
-    }
-
-    private int twoLevBS(Tuple key, int loadNum) throws IOException {
-      int pos = binarySearch(this.dataIndex, key, 0, this.dataIndex.length);
-      if(pos > 0) {
-        rootCursor = pos;
-      } else {
-        rootCursor = 0;
-      }
-      fillLeafIndex(loadNum, subIn, this.offsetIndex[rootCursor]);
-      pos = binarySearch(this.dataSubIndex, key, 0, this.dataSubIndex.length);
-       
-      return pos;
-    }
-
-    private int binarySearch(Tuple[] arr, Tuple key, int startPos, int endPos) {
-      int offset = -1;
-      int start = startPos;
-      int end = endPos;
-
-      //http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6412541
-      int centerPos = (start + end) >>> 1;
-      while (true) {
-        if (comparator.compare(arr[centerPos], key) > 0) {
-          if (centerPos == 0) {
-            correctable = false;
-            break;
-          } else if (comparator.compare(arr[centerPos - 1], key) < 0) {
-            correctable = false;
-            offset = centerPos - 1;
-            break;
-          } else {
-            end = centerPos;
-            centerPos = (start + end) / 2;
-          }
-        } else if (comparator.compare(arr[centerPos], key) < 0) {
-          if (centerPos == arr.length - 1) {
-            correctable = false;
-            offset = centerPos;
-            break;
-          } else if (comparator.compare(arr[centerPos + 1], key) > 0) {
-            correctable = false;
-            offset = centerPos;
-            break;
-          } else {
-            start = centerPos + 1;
-            centerPos = (start + end) / 2;
-          }
-        } else {
-          correctable = true;
-          offset = centerPos;
-          break;
-        }
-      }
-      return offset;
-    }
-
-    @Override
-    public void close() throws IOException {
-      this.indexIn.close();
-      this.subIn.close();
-    }
-
-    @Override
-    public String toString() {
-      return "BSTIndex (" + firstKey + ", " + lastKey + ") " + fileName;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
deleted file mode 100644
index b10d423..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetAppender.java
+++ /dev/null
@@ -1,151 +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.parquet;
-
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.storage.StorageConstants;
-import parquet.hadoop.ParquetOutputFormat;
-import parquet.hadoop.metadata.CompressionCodecName;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.storage.FileAppender;
-import org.apache.tajo.storage.TableStatistics;
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-
-/**
- * FileAppender for writing to Parquet files.
- */
-public class ParquetAppender extends FileAppender {
-  private TajoParquetWriter writer;
-  private int blockSize;
-  private int pageSize;
-  private CompressionCodecName compressionCodecName;
-  private boolean enableDictionary;
-  private boolean validating;
-  private TableStatistics stats;
-
-  /**
-   * Creates a new ParquetAppender.
-   *
-   * @param conf Configuration properties.
-   * @param schema The table schema.
-   * @param meta The table metadata.
-   * @param workDir The path of the Parquet file to write to.
-   */
-  public ParquetAppender(Configuration conf, QueryUnitAttemptId taskAttemptId, Schema schema, TableMeta meta,
-                         Path workDir) throws IOException {
-    super(conf, taskAttemptId, schema, meta, workDir);
-    this.blockSize = Integer.parseInt(
-        meta.getOption(ParquetOutputFormat.BLOCK_SIZE, StorageConstants.PARQUET_DEFAULT_BLOCK_SIZE));
-    this.pageSize = Integer.parseInt(
-        meta.getOption(ParquetOutputFormat.PAGE_SIZE, StorageConstants.PARQUET_DEFAULT_PAGE_SIZE));
-    this.compressionCodecName = CompressionCodecName.fromConf(
-        meta.getOption(ParquetOutputFormat.COMPRESSION, StorageConstants.PARQUET_DEFAULT_COMPRESSION_CODEC_NAME));
-    this.enableDictionary = Boolean.parseBoolean(
-        meta.getOption(ParquetOutputFormat.ENABLE_DICTIONARY, StorageConstants.PARQUET_DEFAULT_IS_DICTIONARY_ENABLED));
-    this.validating = Boolean.parseBoolean(
-        meta.getOption(ParquetOutputFormat.VALIDATION, StorageConstants.PARQUET_DEFAULT_IS_VALIDATION_ENABLED));
-  }
-
-  /**
-   * Initializes the Appender. This method creates a new TajoParquetWriter
-   * and initializes the table statistics if enabled.
-   */
-  public void init() throws IOException {
-    writer = new TajoParquetWriter(path,
-                                   schema,
-                                   compressionCodecName,
-                                   blockSize,
-                                   pageSize,
-                                   enableDictionary,
-                                   validating);
-    if (enabledStats) {
-      this.stats = new TableStatistics(schema);
-    }
-    super.init();
-  }
-
-  /**
-   * Gets the current offset. Tracking offsets is currenly not implemented, so
-   * this method always returns 0.
-   *
-   * @return 0
-   */
-  @Override
-  public long getOffset() throws IOException {
-    return 0;
-  }
-
-  /**
-   * Write a Tuple to the Parquet file.
-   *
-   * @param tuple The Tuple to write.
-   */
-  @Override
-  public void addTuple(Tuple tuple) throws IOException {
-    if (enabledStats) {
-      for (int i = 0; i < schema.size(); ++i) {
-        stats.analyzeField(i, tuple.get(i));
-      }
-    }
-    writer.write(tuple);
-    if (enabledStats) {
-      stats.incrementRow();
-    }
-  }
-
-  /**
-   * The ParquetWriter does not need to be flushed, so this is a no-op.
-   */
-  @Override
-  public void flush() throws IOException {
-  }
-
-  /**
-   * Closes the Appender.
-   */
-  @Override
-  public void close() throws IOException {
-    writer.close();
-  }
-
-  public long getEstimatedOutputSize() throws IOException {
-    return writer.getEstimatedWrittenSize();
-  }
-
-  /**
-   * If table statistics is enabled, retrieve the table statistics.
-   *
-   * @return Table statistics if enabled or null otherwise.
-   */
-  @Override
-  public TableStats getStats() {
-    if (enabledStats) {
-      return stats.getTableStat();
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
deleted file mode 100644
index 2f8efcf..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/ParquetScanner.java
+++ /dev/null
@@ -1,119 +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.parquet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-
-/**
- * FileScanner for reading Parquet files
- */
-public class ParquetScanner extends FileScanner {
-  private TajoParquetReader reader;
-
-  /**
-   * Creates a new ParquetScanner.
-   *
-   * @param conf
-   * @param schema
-   * @param meta
-   * @param fragment
-   */
-  public ParquetScanner(Configuration conf, final Schema schema,
-                        final TableMeta meta, final Fragment fragment) {
-    super(conf, schema, meta, fragment);
-  }
-
-  /**
-   * Initializes the ParquetScanner. This method initializes the
-   * TajoParquetReader.
-   */
-  @Override
-  public void init() throws IOException {
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-    reader = new TajoParquetReader(fragment.getPath(), schema, new Schema(targets));
-    super.init();
-  }
-
-  /**
-   * Reads the next Tuple from the Parquet file.
-   *
-   * @return The next Tuple from the Parquet file or null if end of file is
-   *         reached.
-   */
-  @Override
-  public Tuple next() throws IOException {
-    return reader.read();
-  }
-
-  /**
-   * Resets the scanner
-   */
-  @Override
-  public void reset() throws IOException {
-  }
-
-  /**
-   * Closes the scanner.
-   */
-  @Override
-  public void close() throws IOException {
-    if (reader != null) {
-      reader.close();
-    }
-  }
-
-  /**
-   * Returns whether this scanner is projectable.
-   *
-   * @return true
-   */
-  @Override
-  public boolean isProjectable() {
-    return true;
-  }
-
-  /**
-   * Returns whether this scanner is selectable.
-   *
-   * @return false
-   */
-  @Override
-  public boolean isSelectable() {
-    return false;
-  }
-
-  /**
-   * Returns whether this scanner is splittable.
-   *
-   * @return false
-   */
-  @Override
-  public boolean isSplittable() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
deleted file mode 100644
index a765f48..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetReader.java
+++ /dev/null
@@ -1,85 +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.parquet;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.thirdparty.parquet.ParquetReader;
-import parquet.filter.UnboundRecordFilter;
-
-import java.io.IOException;
-
-/**
- * Tajo implementation of {@link ParquetReader} to read Tajo records from a
- * Parquet file. Users should use {@link ParquetScanner} and not this class
- * directly.
- */
-public class TajoParquetReader extends ParquetReader<Tuple> {
-  /**
-   * Creates a new TajoParquetReader.
-   *
-   * @param file The file to read from.
-   * @param readSchema Tajo schema of the table.
-   */
-  public TajoParquetReader(Path file, Schema readSchema) throws IOException {
-    super(file, new TajoReadSupport(readSchema));
-  }
-
-  /**
-   * Creates a new TajoParquetReader.
-   *
-   * @param file The file to read from.
-   * @param readSchema Tajo schema of the table.
-   * @param requestedSchema Tajo schema of the projection.
-   */
-  public TajoParquetReader(Path file, Schema readSchema,
-                           Schema requestedSchema) throws IOException {
-    super(file, new TajoReadSupport(readSchema, requestedSchema));
-  }
-
-  /**
-   * Creates a new TajoParquetReader.
-   *
-   * @param file The file to read from.
-   * @param readSchema Tajo schema of the table.
-   * @param recordFilter Record filter.
-   */
-  public TajoParquetReader(Path file, Schema readSchema,
-                           UnboundRecordFilter recordFilter)
-      throws IOException {
-    super(file, new TajoReadSupport(readSchema), recordFilter);
-  }
-
-  /**
-   * Creates a new TajoParquetReader.
-   *
-   * @param file The file to read from.
-   * @param readSchema Tajo schema of the table.
-   * @param requestedSchema Tajo schema of the projection.
-   * @param recordFilter Record filter.
-   */
-  public TajoParquetReader(Path file, Schema readSchema,
-                           Schema requestedSchema,
-                           UnboundRecordFilter recordFilter)
-      throws IOException {
-    super(file, new TajoReadSupport(readSchema, requestedSchema),
-          recordFilter);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
deleted file mode 100644
index 69b76c4..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoParquetWriter.java
+++ /dev/null
@@ -1,104 +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.parquet;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.thirdparty.parquet.ParquetWriter;
-import parquet.hadoop.metadata.CompressionCodecName;
-
-import java.io.IOException;
-
-/**
- * Tajo implementation of {@link ParquetWriter} to write Tajo records to a
- * Parquet file. Users should use {@link ParquetAppender} and not this class
- * directly.
- */
-public class TajoParquetWriter extends ParquetWriter<Tuple> {
-  /**
-   * Create a new TajoParquetWriter
-   *
-   * @param file The file name to write to.
-   * @param schema The Tajo schema of the table.
-   * @param compressionCodecName Compression codec to use, or
-   *                             CompressionCodecName.UNCOMPRESSED.
-   * @param blockSize The block size threshold.
-   * @param pageSize See parquet write up. Blocks are subdivided into pages
-   *                 for alignment.
-   * @throws IOException
-   */
-  public TajoParquetWriter(Path file,
-                           Schema schema,
-                           CompressionCodecName compressionCodecName,
-                           int blockSize,
-                           int pageSize) throws IOException {
-    super(file,
-          new TajoWriteSupport(schema),
-          compressionCodecName,
-          blockSize,
-          pageSize);
-  }
-
-  /**
-   * Create a new TajoParquetWriter.
-   *
-   * @param file The file name to write to.
-   * @param schema The Tajo schema of the table.
-   * @param compressionCodecName Compression codec to use, or
-   *                             CompressionCodecName.UNCOMPRESSED.
-   * @param blockSize The block size threshold.
-   * @param pageSize See parquet write up. Blocks are subdivided into pages
-   *                 for alignment.
-   * @param enableDictionary Whether to use a dictionary to compress columns.
-   * @param validating Whether to turn on validation.
-   * @throws IOException
-   */
-  public TajoParquetWriter(Path file,
-                           Schema schema,
-                           CompressionCodecName compressionCodecName,
-                           int blockSize,
-                           int pageSize,
-                           boolean enableDictionary,
-                           boolean validating) throws IOException {
-    super(file,
-          new TajoWriteSupport(schema),
-          compressionCodecName,
-          blockSize,
-          pageSize,
-          enableDictionary,
-          validating);
-  }
-
-  /**
-   * Creates a new TajoParquetWriter. The default block size is 128 MB.
-   * The default page size is 1 MB. Default compression is no compression.
-   *
-   * @param file The Path of the file to write to.
-   * @param schema The Tajo schema of the table.
-   * @throws IOException
-   */
-  public TajoParquetWriter(Path file, Schema schema) throws IOException {
-    this(file,
-         schema,
-         CompressionCodecName.UNCOMPRESSED,
-         DEFAULT_BLOCK_SIZE,
-         DEFAULT_PAGE_SIZE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
deleted file mode 100644
index 269f782..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoReadSupport.java
+++ /dev/null
@@ -1,101 +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.parquet;
-
-import java.util.Map;
-
-import parquet.Log;
-import parquet.hadoop.api.InitContext;
-import parquet.hadoop.api.ReadSupport;
-import parquet.io.api.RecordMaterializer;
-import parquet.schema.MessageType;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.json.CatalogGsonHelper;
-import org.apache.tajo.storage.Tuple;
-
-/**
- * Tajo implementation of {@link ReadSupport} for {@link Tuple}s.
- * Users should use {@link ParquetScanner} and not this class directly.
- */
-public class TajoReadSupport extends ReadSupport<Tuple> {
-  private static final Log LOG = Log.getLog(TajoReadSupport.class);
-
-  private Schema readSchema;
-  private Schema requestedSchema;
-
-  /**
-   * Creates a new TajoReadSupport.
-   *
-   * @param requestedSchema The Tajo schema of the requested projection passed
-   *        down by ParquetScanner.
-   */
-  public TajoReadSupport(Schema readSchema, Schema requestedSchema) {
-    super();
-    this.readSchema = readSchema;
-    this.requestedSchema = requestedSchema;
-  }
-
-  /**
-   * Creates a new TajoReadSupport.
-   *
-   * @param readSchema The schema of the table.
-   */
-  public TajoReadSupport(Schema readSchema) {
-    super();
-    this.readSchema = readSchema;
-    this.requestedSchema = readSchema;
-  }
-
-  /**
-   * Initializes the ReadSupport.
-   *
-   * @param context The InitContext.
-   * @return A ReadContext that defines how to read the file.
-   */
-  @Override
-  public ReadContext init(InitContext context) {
-    if (requestedSchema == null) {
-      throw new RuntimeException("requestedSchema is null.");
-    }
-    MessageType requestedParquetSchema =
-      new TajoSchemaConverter().convert(requestedSchema);
-    LOG.debug("Reading data with projection:\n" + requestedParquetSchema);
-    return new ReadContext(requestedParquetSchema);
-  }
-
-  /**
-   * Prepares for read.
-   *
-   * @param configuration The job configuration.
-   * @param keyValueMetaData App-specific metadata from the file.
-   * @param fileSchema The schema of the Parquet file.
-   * @param readContext Returned by the init method.
-   */
-  @Override
-  public RecordMaterializer<Tuple> prepareForRead(
-      Configuration configuration,
-      Map<String, String> keyValueMetaData,
-      MessageType fileSchema,
-      ReadContext readContext) {
-    MessageType parquetRequestedSchema = readContext.getRequestedSchema();
-    return new TajoRecordMaterializer(parquetRequestedSchema, requestedSchema, readSchema);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
deleted file mode 100644
index 7c3d79d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java
+++ /dev/null
@@ -1,386 +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.parquet;
-
-import com.google.protobuf.Message;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-import java.nio.ByteBuffer;
-
-import parquet.io.api.GroupConverter;
-import parquet.io.api.Converter;
-import parquet.io.api.PrimitiveConverter;
-import parquet.io.api.Binary;
-import parquet.schema.Type;
-import parquet.schema.GroupType;
-
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.BlobDatum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.ProtobufDatumFactory;
-
-/**
- * Converter to convert a Parquet record into a Tajo Tuple.
- */
-public class TajoRecordConverter extends GroupConverter {
-  private final GroupType parquetSchema;
-  private final Schema tajoReadSchema;
-  private final int[] projectionMap;
-  private final int tupleSize;
-
-  private final Converter[] converters;
-
-  private Tuple currentTuple;
-
-  /**
-   * Creates a new TajoRecordConverter.
-   *
-   * @param parquetSchema The Parquet schema of the projection.
-   * @param tajoReadSchema The Tajo schema of the table.
-   * @param projectionMap An array mapping the projection column to the column
-   *                      index in the table.
-   */
-  public TajoRecordConverter(GroupType parquetSchema, Schema tajoReadSchema,
-                             int[] projectionMap) {
-    this.parquetSchema = parquetSchema;
-    this.tajoReadSchema = tajoReadSchema;
-    this.projectionMap = projectionMap;
-    this.tupleSize = tajoReadSchema.size();
-
-    // The projectionMap.length does not match parquetSchema.getFieldCount()
-    // when the projection contains NULL_TYPE columns. We will skip over the
-    // NULL_TYPE columns when we construct the converters and populate the
-    // NULL_TYPE columns with NullDatums in start().
-    int index = 0;
-    this.converters = new Converter[parquetSchema.getFieldCount()];
-    for (int i = 0; i < projectionMap.length; ++i) {
-      final int projectionIndex = projectionMap[i];
-      Column column = tajoReadSchema.getColumn(projectionIndex);
-      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE) {
-        continue;
-      }
-      Type type = parquetSchema.getType(index);
-      converters[index] = newConverter(column, type, new ParentValueContainer() {
-        @Override
-        void add(Object value) {
-          TajoRecordConverter.this.set(projectionIndex, value);
-        }
-      });
-      ++index;
-    }
-  }
-
-  private void set(int index, Object value) {
-    currentTuple.put(index, (Datum)value);
-  }
-
-  private Converter newConverter(Column column, Type type,
-                                 ParentValueContainer parent) {
-    DataType dataType = column.getDataType();
-    switch (dataType.getType()) {
-      case BOOLEAN:
-        return new FieldBooleanConverter(parent);
-      case BIT:
-        return new FieldBitConverter(parent);
-      case CHAR:
-        return new FieldCharConverter(parent);
-      case INT2:
-        return new FieldInt2Converter(parent);
-      case INT4:
-        return new FieldInt4Converter(parent);
-      case INT8:
-        return new FieldInt8Converter(parent);
-      case FLOAT4:
-        return new FieldFloat4Converter(parent);
-      case FLOAT8:
-        return new FieldFloat8Converter(parent);
-      case INET4:
-        return new FieldInet4Converter(parent);
-      case INET6:
-        throw new RuntimeException("No converter for INET6");
-      case TEXT:
-        return new FieldTextConverter(parent);
-      case PROTOBUF:
-        return new FieldProtobufConverter(parent, dataType);
-      case BLOB:
-        return new FieldBlobConverter(parent);
-      case NULL_TYPE:
-        throw new RuntimeException("No converter for NULL_TYPE.");
-      default:
-        throw new RuntimeException("Unsupported data type");
-    }
-  }
-
-  /**
-   * Gets the converter for a specific field.
-   *
-   * @param fieldIndex Index of the field in the projection.
-   * @return The converter for the field.
-   */
-  @Override
-  public Converter getConverter(int fieldIndex) {
-    return converters[fieldIndex];
-  }
-
-  /**
-   * Called before processing fields. This method fills any fields that have
-   * NULL values or have type NULL_TYPE with a NullDatum.
-   */
-  @Override
-  public void start() {
-    currentTuple = new VTuple(tupleSize);
-  }
-
-  /**
-   * Called after all fields have been processed.
-   */
-  @Override
-  public void end() {
-    for (int i = 0; i < projectionMap.length; ++i) {
-      final int projectionIndex = projectionMap[i];
-      Column column = tajoReadSchema.getColumn(projectionIndex);
-      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE
-          || currentTuple.get(projectionIndex) == null) {
-        set(projectionIndex, NullDatum.get());
-      }
-    }
-  }
-
-  /**
-   * Returns the current record converted by this converter.
-   *
-   * @return The current record.
-   */
-  public Tuple getCurrentRecord() {
-    return currentTuple;
-  }
-
-  static abstract class ParentValueContainer {
-    /**
-     * Adds the value to the parent.
-     *
-     * @param value The value to add.
-     */
-    abstract void add(Object value);
-  }
-
-  static final class FieldBooleanConverter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldBooleanConverter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addBoolean(boolean value) {
-      parent.add(DatumFactory.createBool(value));
-    }
-  }
-
-  static final class FieldBitConverter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldBitConverter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addInt(int value) {
-      parent.add(DatumFactory.createBit((byte)(value & 0xff)));
-    }
-  }
-
-  static final class FieldCharConverter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldCharConverter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addBinary(Binary value) {
-      parent.add(DatumFactory.createChar(value.toStringUsingUTF8()));
-    }
-  }
-
-  static final class FieldInt2Converter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldInt2Converter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addInt(int value) {
-      parent.add(DatumFactory.createInt2((short)value));
-    }
-  }
-
-  static final class FieldInt4Converter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldInt4Converter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addInt(int value) {
-      parent.add(DatumFactory.createInt4(value));
-    }
-  }
-
-  static final class FieldInt8Converter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldInt8Converter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addLong(long value) {
-      parent.add(DatumFactory.createInt8(value));
-    }
-
-    @Override
-    final public void addInt(int value) {
-      parent.add(DatumFactory.createInt8(Long.valueOf(value)));
-    }
-  }
-
-  static final class FieldFloat4Converter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldFloat4Converter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addInt(int value) {
-      parent.add(DatumFactory.createFloat4(Float.valueOf(value)));
-    }
-
-    @Override
-    final public void addLong(long value) {
-      parent.add(DatumFactory.createFloat4(Float.valueOf(value)));
-    }
-
-    @Override
-    final public void addFloat(float value) {
-      parent.add(DatumFactory.createFloat4(value));
-    }
-  }
-
-  static final class FieldFloat8Converter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldFloat8Converter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addInt(int value) {
-      parent.add(DatumFactory.createFloat8(Double.valueOf(value)));
-    }
-
-    @Override
-    final public void addLong(long value) {
-      parent.add(DatumFactory.createFloat8(Double.valueOf(value)));
-    }
-
-    @Override
-    final public void addFloat(float value) {
-      parent.add(DatumFactory.createFloat8(Double.valueOf(value)));
-    }
-
-    @Override
-    final public void addDouble(double value) {
-      parent.add(DatumFactory.createFloat8(value));
-    }
-  }
-
-  static final class FieldInet4Converter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldInet4Converter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addBinary(Binary value) {
-      parent.add(DatumFactory.createInet4(value.getBytes()));
-    }
-  }
-
-  static final class FieldTextConverter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldTextConverter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addBinary(Binary value) {
-      parent.add(DatumFactory.createText(value.toStringUsingUTF8()));
-    }
-  }
-
-  static final class FieldBlobConverter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-
-    public FieldBlobConverter(ParentValueContainer parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    final public void addBinary(Binary value) {
-      parent.add(new BlobDatum(ByteBuffer.wrap(value.getBytes())));
-    }
-  }
-
-  static final class FieldProtobufConverter extends PrimitiveConverter {
-    private final ParentValueContainer parent;
-    private final DataType dataType;
-
-    public FieldProtobufConverter(ParentValueContainer parent,
-                                  DataType dataType) {
-      this.parent = parent;
-      this.dataType = dataType;
-    }
-
-    @Override
-    final public void addBinary(Binary value) {
-      try {
-        ProtobufDatumFactory factory =
-            ProtobufDatumFactory.get(dataType.getCode());
-        Message.Builder builder = factory.newBuilder();
-        builder.mergeFrom(value.getBytes());
-        parent.add(factory.createDatum(builder));
-      } catch (InvalidProtocolBufferException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
deleted file mode 100644
index e31828c..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoRecordMaterializer.java
+++ /dev/null
@@ -1,78 +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.parquet;
-
-import parquet.io.api.GroupConverter;
-import parquet.io.api.RecordMaterializer;
-import parquet.schema.MessageType;
-
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.storage.Tuple;
-
-/**
- * Materializes a Tajo Tuple from a stream of Parquet data.
- */
-class TajoRecordMaterializer extends RecordMaterializer<Tuple> {
-  private final TajoRecordConverter root;
-
-  /**
-   * Creates a new TajoRecordMaterializer.
-   *
-   * @param parquetSchema The Parquet schema of the projection.
-   * @param tajoSchema The Tajo schema of the projection.
-   * @param tajoReadSchema The Tajo schema of the table.
-   */
-  public TajoRecordMaterializer(MessageType parquetSchema, Schema tajoSchema,
-                                Schema tajoReadSchema) {
-    int[] projectionMap = getProjectionMap(tajoReadSchema, tajoSchema);
-    this.root = new TajoRecordConverter(parquetSchema, tajoReadSchema,
-                                        projectionMap);
-  }
-
-  private int[] getProjectionMap(Schema schema, Schema projection) {
-    Column[] targets = projection.toArray();
-    int[] projectionMap = new int[targets.length];
-    for (int i = 0; i < targets.length; ++i) {
-      int tid = schema.getColumnId(targets[i].getQualifiedName());
-      projectionMap[i] = tid;
-    }
-    return projectionMap;
-  }
-
-  /**
-   * Returns the current record being materialized.
-   *
-   * @return The record being materialized.
-   */
-  @Override
-  public Tuple getCurrentRecord() {
-    return root.getCurrentRecord();
-  }
-
-  /**
-   * Returns the root converter.
-   *
-   * @return The root converter
-   */
-  @Override
-  public GroupConverter getRootConverter() {
-    return root;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
deleted file mode 100644
index 2592231..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java
+++ /dev/null
@@ -1,206 +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.parquet;
-
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import parquet.schema.MessageType;
-import parquet.schema.OriginalType;
-import parquet.schema.PrimitiveType;
-import parquet.schema.PrimitiveType.PrimitiveTypeName;
-import parquet.schema.Type;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Converts between Parquet and Tajo schemas. See package documentation for
- * details on the mapping.
- */
-public class TajoSchemaConverter {
-  private static final String TABLE_SCHEMA = "table_schema";
-
-  /**
-   * Creates a new TajoSchemaConverter.
-   */
-  public TajoSchemaConverter() {
-  }
-
-  /**
-   * Converts a Parquet schema to a Tajo schema.
-   *
-   * @param parquetSchema The Parquet schema to convert.
-   * @return The resulting Tajo schema.
-   */
-  public Schema convert(MessageType parquetSchema) {
-    return convertFields(parquetSchema.getFields());
-  }
-
-  private Schema convertFields(List<Type> parquetFields) {
-    List<Column> columns = new ArrayList<Column>();
-    for (int i = 0; i < parquetFields.size(); ++i) {
-      Type fieldType = parquetFields.get(i);
-      if (fieldType.isRepetition(Type.Repetition.REPEATED)) {
-        throw new RuntimeException("REPEATED not supported outside LIST or" +
-            " MAP. Type: " + fieldType);
-      }
-      columns.add(convertField(fieldType));
-    }
-    Column[] columnsArray = new Column[columns.size()];
-    columnsArray = columns.toArray(columnsArray);
-    return new Schema(columnsArray);
-  }
-
-  private Column convertField(final Type fieldType) {
-    if (fieldType.isPrimitive()) {
-      return convertPrimitiveField(fieldType);
-    } else {
-      return convertComplexField(fieldType);
-    }
-  }
-
-  private Column convertPrimitiveField(final Type fieldType) {
-    final String fieldName = fieldType.getName();
-    final PrimitiveTypeName parquetPrimitiveTypeName =
-        fieldType.asPrimitiveType().getPrimitiveTypeName();
-    final OriginalType originalType = fieldType.getOriginalType();
-    return parquetPrimitiveTypeName.convert(
-        new PrimitiveType.PrimitiveTypeNameConverter<Column, RuntimeException>() {
-      @Override
-      public Column convertBOOLEAN(PrimitiveTypeName primitiveTypeName) {
-        return new Column(fieldName, TajoDataTypes.Type.BOOLEAN);
-      }
-
-      @Override
-      public Column convertINT32(PrimitiveTypeName primitiveTypeName) {
-        return new Column(fieldName, TajoDataTypes.Type.INT4);
-      }
-
-      @Override
-      public Column convertINT64(PrimitiveTypeName primitiveTypeName) {
-        return new Column(fieldName, TajoDataTypes.Type.INT8);
-      }
-
-      @Override
-      public Column convertFLOAT(PrimitiveTypeName primitiveTypeName) {
-        return new Column(fieldName, TajoDataTypes.Type.FLOAT4);
-      }
-
-      @Override
-      public Column convertDOUBLE(PrimitiveTypeName primitiveTypeName) {
-        return new Column(fieldName, TajoDataTypes.Type.FLOAT8);
-      }
-
-      @Override
-      public Column convertFIXED_LEN_BYTE_ARRAY(
-          PrimitiveTypeName primitiveTypeName) {
-        return new Column(fieldName, TajoDataTypes.Type.BLOB);
-      }
-
-      @Override
-      public Column convertBINARY(PrimitiveTypeName primitiveTypeName) {
-        if (originalType == OriginalType.UTF8) {
-          return new Column(fieldName, TajoDataTypes.Type.TEXT);
-        } else {
-          return new Column(fieldName, TajoDataTypes.Type.BLOB);
-        }
-      }
-
-      @Override
-      public Column convertINT96(PrimitiveTypeName primitiveTypeName) {
-        throw new RuntimeException("Converting from INT96 not supported.");
-      }
-    });
-  }
-
-  private Column convertComplexField(final Type fieldType) {
-    throw new RuntimeException("Complex types not supported.");
-  }
-
-  /**
-   * Converts a Tajo schema to a Parquet schema.
-   *
-   * @param tajoSchema The Tajo schema to convert.
-   * @return The resulting Parquet schema.
-   */
-  public MessageType convert(Schema tajoSchema) {
-    List<Type> types = new ArrayList<Type>();
-    for (int i = 0; i < tajoSchema.size(); ++i) {
-      Column column = tajoSchema.getColumn(i);
-      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE) {
-        continue;
-      }
-      types.add(convertColumn(column));
-    }
-    return new MessageType(TABLE_SCHEMA, types);
-  }
-
-  private Type convertColumn(Column column) {
-    TajoDataTypes.Type type = column.getDataType().getType();
-    switch (type) {
-      case BOOLEAN:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BOOLEAN);
-      case BIT:
-      case INT2:
-      case INT4:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.INT32);
-      case INT8:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.INT64);
-      case FLOAT4:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.FLOAT);
-      case FLOAT8:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.DOUBLE);
-      case CHAR:
-      case TEXT:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY,
-                         OriginalType.UTF8);
-      case PROTOBUF:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY);
-      case BLOB:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY);
-      case INET4:
-      case INET6:
-        return primitive(column.getSimpleName(),
-                         PrimitiveType.PrimitiveTypeName.BINARY);
-      default:
-        throw new RuntimeException("Cannot convert Tajo type: " + type);
-    }
-  }
-
-  private PrimitiveType primitive(String name,
-                                  PrimitiveType.PrimitiveTypeName primitive) {
-    return new PrimitiveType(Type.Repetition.OPTIONAL, primitive, name, null);
-  }
-
-  private PrimitiveType primitive(String name,
-                                  PrimitiveType.PrimitiveTypeName primitive,
-                                  OriginalType originalType) {
-    return new PrimitiveType(Type.Repetition.OPTIONAL, primitive, name,
-                             originalType);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
deleted file mode 100644
index 35165de..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java
+++ /dev/null
@@ -1,149 +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.parquet;
-
-import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-
-import parquet.hadoop.api.WriteSupport;
-import parquet.io.api.Binary;
-import parquet.io.api.RecordConsumer;
-import parquet.schema.GroupType;
-import parquet.schema.MessageType;
-import parquet.schema.Type;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.datum.Datum;
-
-/**
- * Tajo implementation of {@link WriteSupport} for {@link Tuple}s.
- * Users should use {@link ParquetAppender} and not this class directly.
- */
-public class TajoWriteSupport extends WriteSupport<Tuple> {
-  private RecordConsumer recordConsumer;
-  private MessageType rootSchema;
-  private Schema rootTajoSchema;
-
-  /**
-   * Creates a new TajoWriteSupport.
-   *
-   * @param tajoSchema The Tajo schema for the table.
-   */
-  public TajoWriteSupport(Schema tajoSchema) {
-    this.rootSchema = new TajoSchemaConverter().convert(tajoSchema);
-    this.rootTajoSchema = tajoSchema;
-  }
-
-  /**
-   * Initializes the WriteSupport.
-   *
-   * @param configuration The job's configuration.
-   * @return A WriteContext that describes how to write the file.
-   */
-  @Override
-  public WriteContext init(Configuration configuration) {
-    Map<String, String> extraMetaData = new HashMap<String, String>();
-    return new WriteContext(rootSchema, extraMetaData);
-  }
-
-  /**
-   * Called once per row group.
-   *
-   * @param recordConsumer The {@link RecordConsumer} to write to.
-   */
-  @Override
-  public void prepareForWrite(RecordConsumer recordConsumer) {
-    this.recordConsumer = recordConsumer;
-  }
-
-  /**
-   * Writes a Tuple to the file.
-   *
-   * @param tuple The Tuple to write to the file.
-   */
-  @Override
-  public void write(Tuple tuple) {
-    recordConsumer.startMessage();
-    writeRecordFields(rootSchema, rootTajoSchema, tuple);
-    recordConsumer.endMessage();
-  }
-
-  private void writeRecordFields(GroupType schema, Schema tajoSchema,
-                                 Tuple tuple) {
-    List<Type> fields = schema.getFields();
-    // Parquet ignores Tajo NULL_TYPE columns, so the index may differ.
-    int index = 0;
-    for (int tajoIndex = 0; tajoIndex < tajoSchema.size(); ++tajoIndex) {
-      Column column = tajoSchema.getColumn(tajoIndex);
-      if (column.getDataType().getType() == TajoDataTypes.Type.NULL_TYPE) {
-        continue;
-      }
-      Datum datum = tuple.get(tajoIndex);
-      Type fieldType = fields.get(index);
-      if (!tuple.isNull(tajoIndex)) {
-        recordConsumer.startField(fieldType.getName(), index);
-        writeValue(fieldType, column, datum);
-        recordConsumer.endField(fieldType.getName(), index);
-      } else if (fieldType.isRepetition(Type.Repetition.REQUIRED)) {
-        throw new RuntimeException("Null-value for required field: " +
-            column.getSimpleName());
-      }
-      ++index;
-    }
-  }
-
-  private void writeValue(Type fieldType, Column column, Datum datum) {
-    switch (column.getDataType().getType()) {
-      case BOOLEAN:
-        recordConsumer.addBoolean((Boolean) datum.asBool());
-        break;
-      case BIT:
-      case INT2:
-      case INT4:
-        recordConsumer.addInteger(datum.asInt4());
-        break;
-      case INT8:
-        recordConsumer.addLong(datum.asInt8());
-        break;
-      case FLOAT4:
-        recordConsumer.addFloat(datum.asFloat4());
-        break;
-      case FLOAT8:
-        recordConsumer.addDouble(datum.asFloat8());
-        break;
-      case CHAR:
-      case TEXT:
-        recordConsumer.addBinary(Binary.fromString(datum.asChars()));
-        break;
-      case PROTOBUF:
-      case BLOB:
-      case INET4:
-      case INET6:
-        recordConsumer.addBinary(Binary.fromByteArray(datum.asByteArray()));
-        break;
-      default:
-        break;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/package-info.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/package-info.java b/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/package-info.java
deleted file mode 100644
index d7d16b7..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/parquet/package-info.java
+++ /dev/null
@@ -1,96 +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.
- */
-/**
- * <p>
- * Provides read and write support for Parquet files. Tajo schemas are
- * converted to Parquet schemas according to the following mapping of Tajo
- * and Parquet types:
- * </p>
- *
- * <table>
- *   <tr>
- *     <th>Tajo type</th>
- *     <th>Parquet type</th>
- *   </tr>
- *   <tr>
- *     <td>NULL_TYPE</td>
- *     <td>No type. The field is not encoded in Parquet.</td>
- *   </tr>
- *   <tr>
- *     <td>BOOLEAN</td>
- *     <td>BOOLEAN</td>
- *   </tr>
- *   <tr>
- *     <td>BIT</td>
- *     <td>INT32</td>
- *   </tr>
- *   <tr>
- *     <td>INT2</td>
- *     <td>INT32</td>
- *   </tr>
- *   <tr>
- *     <td>INT4</td>
- *     <td>INT32</td>
- *   </tr>
- *   <tr>
- *     <td>INT8</td>
- *     <td>INT64</td>
- *   </tr>
- *   <tr>
- *     <td>FLOAT4</td>
- *     <td>FLOAT</td>
- *   </tr>
- *   <tr>
- *     <td>FLOAT8</td>
- *     <td>DOUBLE</td>
- *   </tr>
- *   <tr>
- *     <td>CHAR</td>
- *     <td>BINARY (with OriginalType UTF8)</td>
- *   </tr>
- *   <tr>
- *     <td>TEXT</td>
- *     <td>BINARY (with OriginalType UTF8)</td>
- *   </tr>
- *   <tr>
- *     <td>PROTOBUF</td>
- *     <td>BINARY</td>
- *   </tr>
- *   <tr>
- *     <td>BLOB</td>
- *     <td>BINARY</td>
- *   </tr>
- *   <tr>
- *     <td>INET4</td>
- *     <td>BINARY</td>
- *   </tr>
- * </table>
- *
- * <p>
- * Because Tajo fields can be NULL, all Parquet fields are marked as optional.
- * </p>
- *
- * <p>
- * The conversion from Tajo to Parquet is lossy without the original Tajo
- * schema. As a result, Parquet files are read using the Tajo schema saved in
- * the Tajo catalog for the table the Parquet files belong to, which was
- * defined when the table was created.
- * </p>
- */
-
-package org.apache.tajo.storage.parquet;

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
deleted file mode 100644
index 5e200a0..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
+++ /dev/null
@@ -1,261 +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.rcfile;
-
-import com.google.common.base.Objects;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-
-/**
- * <tt>BytesRefArrayWritable</tt> holds an array reference to BytesRefWritable,
- * and is able to resize without recreating new array if not necessary.
- * <p>
- *
- * Each <tt>BytesRefArrayWritable holds</tt> instance has a <i>valid</i> field,
- * which is the desired valid number of <tt>BytesRefWritable</tt> it holds.
- * <tt>resetValid</tt> can reset the valid, but it will not care the underlying
- * BytesRefWritable.
- */
-
-public class BytesRefArrayWritable implements Writable,
-    Comparable<BytesRefArrayWritable> {
-
-  private BytesRefWritable[] bytesRefWritables = null;
-
-  private int valid = 0;
-
-  /**
-   * Constructs an empty array with the specified capacity.
-   *
-   * @param capacity
-   *          initial capacity
-   * @exception IllegalArgumentException
-   *              if the specified initial capacity is negative
-   */
-  public BytesRefArrayWritable(int capacity) {
-    if (capacity < 0) {
-      throw new IllegalArgumentException("Capacity can not be negative.");
-    }
-    bytesRefWritables = new BytesRefWritable[0];
-    ensureCapacity(capacity);
-  }
-
-  /**
-   * Constructs an empty array with a capacity of ten.
-   */
-  public BytesRefArrayWritable() {
-    this(10);
-  }
-
-  /**
-   * Returns the number of valid elements.
-   *
-   * @return the number of valid elements
-   */
-  public int size() {
-    return valid;
-  }
-
-  /**
-   * Gets the BytesRefWritable at the specified position. Make sure the position
-   * is valid by first call resetValid.
-   *
-   * @param index
-   *          the position index, starting from zero
-   * @throws IndexOutOfBoundsException
-   */
-  public BytesRefWritable get(int index) {
-    if (index >= valid) {
-      throw new IndexOutOfBoundsException(
-          "This BytesRefArrayWritable only has " + valid + " valid values.");
-    }
-    return bytesRefWritables[index];
-  }
-
-  /**
-   * Gets the BytesRefWritable at the specified position without checking.
-   *
-   * @param index
-   *          the position index, starting from zero
-   * @throws IndexOutOfBoundsException
-   */
-  public BytesRefWritable unCheckedGet(int index) {
-    return bytesRefWritables[index];
-  }
-
-  /**
-   * Set the BytesRefWritable at the specified position with the specified
-   * BytesRefWritable.
-   *
-   * @param index
-   *          index position
-   * @param bytesRefWritable
-   *          the new element
-   * @throws IllegalArgumentException
-   *           if the specified new element is null
-   */
-  public void set(int index, BytesRefWritable bytesRefWritable) {
-    if (bytesRefWritable == null) {
-      throw new IllegalArgumentException("Can not assign null.");
-    }
-    ensureCapacity(index + 1);
-    bytesRefWritables[index] = bytesRefWritable;
-    if (valid <= index) {
-      valid = index + 1;
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public int compareTo(BytesRefArrayWritable other) {
-    if (other == null) {
-      throw new IllegalArgumentException("Argument can not be null.");
-    }
-    if (this == other) {
-      return 0;
-    }
-    int sizeDiff = valid - other.valid;
-    if (sizeDiff != 0) {
-      return sizeDiff;
-    }
-    for (int i = 0; i < valid; i++) {
-      if (other.contains(bytesRefWritables[i])) {
-        continue;
-      } else {
-        return 1;
-      }
-    }
-    return 0;
-  }
-
-  @Override
-  public int hashCode(){
-    return Objects.hashCode(bytesRefWritables);
-  }
-  /**
-   * Returns <tt>true</tt> if this instance contains one or more the specified
-   * BytesRefWritable.
-   *
-   * @param bytesRefWritable
-   *          BytesRefWritable element to be tested
-   * @return <tt>true</tt> if contains the specified element
-   * @throws IllegalArgumentException
-   *           if the specified element is null
-   */
-  public boolean contains(BytesRefWritable bytesRefWritable) {
-    if (bytesRefWritable == null) {
-      throw new IllegalArgumentException("Argument can not be null.");
-    }
-    for (int i = 0; i < valid; i++) {
-      if (bytesRefWritables[i].equals(bytesRefWritable)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public boolean equals(Object o) {
-    if (o == null || !(o instanceof BytesRefArrayWritable)) {
-      return false;
-    }
-    return compareTo((BytesRefArrayWritable) o) == 0;
-  }
-
-  /**
-   * Removes all elements.
-   */
-  public void clear() {
-    valid = 0;
-  }
-
-  /**
-   * enlarge the capacity if necessary, to ensure that it can hold the number of
-   * elements specified by newValidCapacity argument. It will also narrow the
-   * valid capacity when needed. Notice: it only enlarge or narrow the valid
-   * capacity with no care of the already stored invalid BytesRefWritable.
-   *
-   * @param newValidCapacity
-   *          the desired capacity
-   */
-  public void resetValid(int newValidCapacity) {
-    ensureCapacity(newValidCapacity);
-    valid = newValidCapacity;
-  }
-
-  protected void ensureCapacity(int newCapacity) {
-    int size = bytesRefWritables.length;
-    if (size < newCapacity) {
-      bytesRefWritables = Arrays.copyOf(bytesRefWritables, newCapacity);
-      while (size < newCapacity) {
-        bytesRefWritables[size] = new BytesRefWritable();
-        size++;
-      }
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int count = in.readInt();
-    ensureCapacity(count);
-    for (int i = 0; i < count; i++) {
-      bytesRefWritables[i].readFields(in);
-    }
-    valid = count;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(valid);
-
-    for (int i = 0; i < valid; i++) {
-      BytesRefWritable cu = bytesRefWritables[i];
-      cu.write(out);
-    }
-  }
-
-  static {
-    WritableFactories.setFactory(BytesRefArrayWritable.class,
-        new WritableFactory() {
-
-          @Override
-          public Writable newInstance() {
-            return new BytesRefArrayWritable();
-          }
-
-        });
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
deleted file mode 100644
index c83b505..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
+++ /dev/null
@@ -1,248 +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.rcfile;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-/**
- * <tt>BytesRefWritable</tt> referenced a section of byte array. It can be used
- * to avoid unnecessary byte copy.
- */
-public class BytesRefWritable implements Writable, Comparable<BytesRefWritable> {
-
-  private static final byte[] EMPTY_BYTES = new byte[0];
-  public static final BytesRefWritable ZeroBytesRefWritable = new BytesRefWritable();
-
-  int start = 0;
-  int length = 0;
-  byte[] bytes = null;
-
-  LazyDecompressionCallback lazyDecompressObj;
-
-  /**
-   * Create a zero-size bytes.
-   */
-  public BytesRefWritable() {
-    this(EMPTY_BYTES);
-  }
-
-  /**
-   * Create a BytesRefWritable with <tt>length</tt> bytes.
-   */
-  public BytesRefWritable(int length) {
-    assert length > 0;
-    this.length = length;
-    bytes = new byte[this.length];
-    start = 0;
-  }
-
-  /**
-   * Create a BytesRefWritable referenced to the given bytes.
-   */
-  public BytesRefWritable(byte[] bytes) {
-    this.bytes = bytes;
-    length = bytes.length;
-    start = 0;
-  }
-
-  /**
-   * Create a BytesRefWritable referenced to one section of the given bytes. The
-   * section is determined by argument <tt>offset</tt> and <tt>len</tt>.
-   */
-  public BytesRefWritable(byte[] data, int offset, int len) {
-    bytes = data;
-    start = offset;
-    length = len;
-  }
-
-  /**
-   * Create a BytesRefWritable referenced to one section of the given bytes. The
-   * argument <tt>lazyDecompressData</tt> refers to a LazyDecompressionCallback
-   * object. The arguments <tt>offset</tt> and <tt>len</tt> are referred to
-   * uncompressed bytes of <tt>lazyDecompressData</tt>. Use <tt>offset</tt> and
-   * <tt>len</tt> after uncompressing the data.
-   */
-  public BytesRefWritable(LazyDecompressionCallback lazyDecompressData,
-                          int offset, int len) {
-    lazyDecompressObj = lazyDecompressData;
-    start = offset;
-    length = len;
-  }
-
-  private void lazyDecompress() throws IOException {
-    if (bytes == null && lazyDecompressObj != null) {
-      bytes = lazyDecompressObj.decompress();
-    }
-  }
-
-  /**
-   * Returns a copy of the underlying bytes referenced by this instance.
-   *
-   * @return a new copied byte array
-   * @throws IOException
-   */
-  public byte[] getBytesCopy() throws IOException {
-    lazyDecompress();
-    byte[] bb = new byte[length];
-    System.arraycopy(bytes, start, bb, 0, length);
-    return bb;
-  }
-
-  /**
-   * Returns the underlying bytes.
-   *
-   * @throws IOException
-   */
-  public byte[] getData() throws IOException {
-    lazyDecompress();
-    return bytes;
-  }
-
-  /**
-   * readFields() will corrupt the array. So use the set method whenever
-   * possible.
-   *
-   * @see #readFields(DataInput)
-   */
-  public void set(byte[] newData, int offset, int len) {
-    bytes = newData;
-    start = offset;
-    length = len;
-    lazyDecompressObj = null;
-  }
-
-  /**
-   * readFields() will corrupt the array. So use the set method whenever
-   * possible.
-   *
-   * @see #readFields(DataInput)
-   */
-  public void set(LazyDecompressionCallback newData, int offset, int len) {
-    bytes = null;
-    start = offset;
-    length = len;
-    lazyDecompressObj = newData;
-  }
-
-  public void writeDataTo(DataOutput out) throws IOException {
-    lazyDecompress();
-    out.write(bytes, start, length);
-  }
-
-  /**
-   * Always reuse the bytes array if length of bytes array is equal or greater
-   * to the current record, otherwise create a new one. readFields will corrupt
-   * the array. Please use set() whenever possible.
-   *
-   * @see #set(byte[], int, int)
-   */
-  public void readFields(DataInput in) throws IOException {
-    int len = in.readInt();
-    if (len > bytes.length) {
-      bytes = new byte[len];
-    }
-    start = 0;
-    length = len;
-    in.readFully(bytes, start, length);
-  }
-
-  /** {@inheritDoc} */
-  public void write(DataOutput out) throws IOException {
-    lazyDecompress();
-    out.writeInt(length);
-    out.write(bytes, start, length);
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public int hashCode() {
-    return super.hashCode();
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(3 * length);
-    for (int idx = start; idx < length; idx++) {
-      // if not the first, put a blank separator in
-      if (idx != 0) {
-        sb.append(' ');
-      }
-      String num = Integer.toHexString(0xff & bytes[idx]);
-      // if it is only one digit, add a leading 0.
-      if (num.length() < 2) {
-        sb.append('0');
-      }
-      sb.append(num);
-    }
-    return sb.toString();
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public int compareTo(BytesRefWritable other) {
-    if (other == null) {
-      throw new IllegalArgumentException("Argument can not be null.");
-    }
-    if (this == other) {
-      return 0;
-    }
-    try {
-      return WritableComparator.compareBytes(getData(), start, getLength(),
-          other.getData(), other.start, other.getLength());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public boolean equals(Object right_obj) {
-    if (right_obj == null || !(right_obj instanceof BytesRefWritable)) {
-      return false;
-    }
-    return compareTo((BytesRefWritable) right_obj) == 0;
-  }
-
-  static {
-    WritableFactories.setFactory(BytesRefWritable.class, new WritableFactory() {
-
-      @Override
-      public Writable newInstance() {
-        return new BytesRefWritable();
-      }
-
-    });
-  }
-
-  public int getLength() {
-    return length;
-  }
-
-  public int getStart() {
-    return start;
-  }
-}


[09/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
new file mode 100644
index 0000000..df73448
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileStorageManager.java
@@ -0,0 +1,854 @@
+/**
+ * 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.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.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.tajo.OverridableConf;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.Bytes;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class FileStorageManager extends StorageManager {
+  private final Log LOG = LogFactory.getLog(FileStorageManager.class);
+
+  static final String OUTPUT_FILE_PREFIX="part-";
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SUBQUERY =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(2);
+          return fmt;
+        }
+      };
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_TASK =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(6);
+          return fmt;
+        }
+      };
+
+  static final ThreadLocal<NumberFormat> OUTPUT_FILE_FORMAT_SEQ =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(3);
+          return fmt;
+        }
+      };
+
+  protected FileSystem fs;
+  protected Path tableBaseDir;
+  protected boolean blocksMetadataEnabled;
+  private static final HdfsVolumeId zeroVolumeId = new HdfsVolumeId(Bytes.toBytes(0));
+
+  public FileStorageManager(StoreType storeType) {
+    super(storeType);
+  }
+
+  @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)
+      LOG.warn("does not support block metadata. ('dfs.datanode.hdfs-blocks-metadata.enabled')");
+  }
+
+  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);
+  }
+
+  public Scanner getFileScanner(TableMeta meta, Schema schema, Path path, FileStatus status)
+      throws IOException {
+    Fragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
+    return getScanner(meta, schema, fragment);
+  }
+
+  public FileSystem getFileSystem() {
+    return this.fs;
+  }
+
+  public Path getWarehouseDir() {
+    return this.tableBaseDir;
+  }
+
+  public void delete(Path tablePath) throws IOException {
+    FileSystem fs = tablePath.getFileSystem(conf);
+    fs.delete(tablePath, true);
+  }
+
+  public boolean exists(Path path) throws IOException {
+    FileSystem fileSystem = path.getFileSystem(conf);
+    return fileSystem.exists(path);
+  }
+
+  /**
+   * This method deletes only data contained in the given path.
+   *
+   * @param path The path in which data are deleted.
+   * @throws IOException
+   */
+  public void deleteData(Path path) throws IOException {
+    FileSystem fileSystem = path.getFileSystem(conf);
+    FileStatus[] fileLists = fileSystem.listStatus(path);
+    for (FileStatus status : fileLists) {
+      fileSystem.delete(status.getPath(), true);
+    }
+  }
+
+  public Path getTablePath(String tableName) {
+    return new Path(tableBaseDir, tableName);
+  }
+
+  @VisibleForTesting
+  public Appender getAppender(TableMeta meta, Schema schema, Path filePath)
+      throws IOException {
+    return getAppender(null, null, meta, schema, filePath);
+  }
+
+  public FileFragment[] split(String tableName) throws IOException {
+    Path tablePath = new Path(tableBaseDir, tableName);
+    return split(tableName, tablePath, fs.getDefaultBlockSize());
+  }
+
+  public FileFragment[] split(String tableName, long fragmentSize) throws IOException {
+    Path tablePath = new Path(tableBaseDir, tableName);
+    return split(tableName, tablePath, fragmentSize);
+  }
+
+  public FileFragment[] splitBroadcastTable(Path tablePath) throws IOException {
+    FileSystem fs = tablePath.getFileSystem(conf);
+    List<FileFragment> listTablets = new ArrayList<FileFragment>();
+    FileFragment tablet;
+
+    FileStatus[] fileLists = fs.listStatus(tablePath);
+    for (FileStatus file : fileLists) {
+      tablet = new FileFragment(tablePath.getName(), file.getPath(), 0, file.getLen());
+      listTablets.add(tablet);
+    }
+
+    FileFragment[] tablets = new FileFragment[listTablets.size()];
+    listTablets.toArray(tablets);
+
+    return tablets;
+  }
+
+  public FileFragment[] split(Path tablePath) throws IOException {
+    FileSystem fs = tablePath.getFileSystem(conf);
+    return split(tablePath.getName(), tablePath, fs.getDefaultBlockSize());
+  }
+
+  public FileFragment[] split(String tableName, Path tablePath) throws IOException {
+    return split(tableName, tablePath, fs.getDefaultBlockSize());
+  }
+
+  private FileFragment[] split(String tableName, Path tablePath, long size)
+      throws IOException {
+    FileSystem fs = tablePath.getFileSystem(conf);
+
+    long defaultBlockSize = size;
+    List<FileFragment> listTablets = new ArrayList<FileFragment>();
+    FileFragment tablet;
+
+    FileStatus[] fileLists = fs.listStatus(tablePath);
+    for (FileStatus file : fileLists) {
+      long remainFileSize = file.getLen();
+      long start = 0;
+      if (remainFileSize > defaultBlockSize) {
+        while (remainFileSize > defaultBlockSize) {
+          tablet = new FileFragment(tableName, file.getPath(), start, defaultBlockSize);
+          listTablets.add(tablet);
+          start += defaultBlockSize;
+          remainFileSize -= defaultBlockSize;
+        }
+        listTablets.add(new FileFragment(tableName, file.getPath(), start, remainFileSize));
+      } else {
+        listTablets.add(new FileFragment(tableName, file.getPath(), 0, remainFileSize));
+      }
+    }
+
+    FileFragment[] tablets = new FileFragment[listTablets.size()];
+    listTablets.toArray(tablets);
+
+    return tablets;
+  }
+
+  public static FileFragment[] splitNG(Configuration conf, String tableName, TableMeta meta,
+                                       Path tablePath, long size)
+      throws IOException {
+    FileSystem fs = tablePath.getFileSystem(conf);
+
+    long defaultBlockSize = size;
+    List<FileFragment> listTablets = new ArrayList<FileFragment>();
+    FileFragment tablet;
+
+    FileStatus[] fileLists = fs.listStatus(tablePath);
+    for (FileStatus file : fileLists) {
+      long remainFileSize = file.getLen();
+      long start = 0;
+      if (remainFileSize > defaultBlockSize) {
+        while (remainFileSize > defaultBlockSize) {
+          tablet = new FileFragment(tableName, file.getPath(), start, defaultBlockSize);
+          listTablets.add(tablet);
+          start += defaultBlockSize;
+          remainFileSize -= defaultBlockSize;
+        }
+        listTablets.add(new FileFragment(tableName, file.getPath(), start, remainFileSize));
+      } else {
+        listTablets.add(new FileFragment(tableName, file.getPath(), 0, remainFileSize));
+      }
+    }
+
+    FileFragment[] tablets = new FileFragment[listTablets.size()];
+    listTablets.toArray(tablets);
+
+    return tablets;
+  }
+
+  public long calculateSize(Path tablePath) throws IOException {
+    FileSystem fs = tablePath.getFileSystem(conf);
+    long totalSize = 0;
+
+    if (fs.exists(tablePath)) {
+      totalSize = fs.getContentSummary(tablePath).getLength();
+    }
+
+    return totalSize;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // FileInputFormat Area
+  /////////////////////////////////////////////////////////////////////////////
+
+  public static final PathFilter hiddenFileFilter = new PathFilter() {
+    public boolean accept(Path p) {
+      String name = p.getName();
+      return !name.startsWith("_") && !name.startsWith(".");
+    }
+  };
+
+  public Path getAppenderFilePath(QueryUnitAttemptId taskAttemptId, Path workDir) {
+    if (taskAttemptId == null) {
+      // For testcase
+      return workDir;
+    }
+    // The final result of a task will be written in a file named part-ss-nnnnnnn,
+    // where ss is the subquery id associated with this task, and nnnnnn is the task id.
+    Path outFilePath = StorageUtil.concatPath(workDir, TajoConstants.RESULT_DIR_NAME,
+        OUTPUT_FILE_PREFIX +
+            OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskAttemptId.getQueryUnitId().getExecutionBlockId().getId()) + "-" +
+            OUTPUT_FILE_FORMAT_TASK.get().format(taskAttemptId.getQueryUnitId().getId()) + "-" +
+            OUTPUT_FILE_FORMAT_SEQ.get().format(0));
+    LOG.info("Output File Path: " + outFilePath);
+
+    return outFilePath;
+  }
+
+  /**
+   * Proxy PathFilter that accepts a path only if all filters given in the
+   * constructor do. Used by the listPaths() to apply the built-in
+   * hiddenFileFilter together with a user provided one (if any).
+   */
+  private static class MultiPathFilter implements PathFilter {
+    private List<PathFilter> filters;
+
+    public MultiPathFilter(List<PathFilter> filters) {
+      this.filters = filters;
+    }
+
+    public boolean accept(Path path) {
+      for (PathFilter filter : filters) {
+        if (!filter.accept(path)) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
+  /**
+   * List input directories.
+   * Subclasses may override to, e.g., select only files matching a regular
+   * expression.
+   *
+   * @return array of FileStatus objects
+   * @throws IOException if zero items.
+   */
+  protected List<FileStatus> listStatus(Path... dirs) throws IOException {
+    List<FileStatus> result = new ArrayList<FileStatus>();
+    if (dirs.length == 0) {
+      throw new IOException("No input paths specified in job");
+    }
+
+    List<IOException> errors = new ArrayList<IOException>();
+
+    // creates a MultiPathFilter with the hiddenFileFilter and the
+    // user provided one (if any).
+    List<PathFilter> filters = new ArrayList<PathFilter>();
+    filters.add(hiddenFileFilter);
+
+    PathFilter inputFilter = new MultiPathFilter(filters);
+
+    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));
+      } else if (matches.length == 0) {
+        errors.add(new IOException("Input Pattern " + p + " matches 0 files"));
+      } else {
+        for (FileStatus globStat : matches) {
+          if (globStat.isDirectory()) {
+            for (FileStatus stat : fs.listStatus(globStat.getPath(),
+                inputFilter)) {
+              result.add(stat);
+            }
+          } else {
+            result.add(globStat);
+          }
+        }
+      }
+    }
+
+    if (!errors.isEmpty()) {
+      throw new InvalidInputException(errors);
+    }
+    LOG.info("Total input paths to process : " + result.size());
+    return result;
+  }
+
+  /**
+   * Is the given filename splitable? Usually, true, but if the file is
+   * stream compressed, it will not be.
+   * <p/>
+   * <code>FileInputFormat</code> implementations can override this and return
+   * <code>false</code> to ensure that individual input files are never split-up
+   * so that Mappers process entire files.
+   *
+   *
+   * @param path the file name to check
+   * @param status get the file length
+   * @return is this file isSplittable?
+   */
+  protected boolean isSplittable(TableMeta meta, Schema schema, Path path, FileStatus status) throws IOException {
+    Scanner scanner = getFileScanner(meta, schema, path, status);
+    boolean split = scanner.isSplittable();
+    scanner.close();
+    return split;
+  }
+
+  private static final double SPLIT_SLOP = 1.1;   // 10% slop
+
+  protected int getBlockIndex(BlockLocation[] blkLocations,
+                              long offset) {
+    for (int i = 0; i < blkLocations.length; i++) {
+      // is the offset inside this block?
+      if ((blkLocations[i].getOffset() <= offset) &&
+          (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())) {
+        return i;
+      }
+    }
+    BlockLocation last = blkLocations[blkLocations.length - 1];
+    long fileLength = last.getOffset() + last.getLength() - 1;
+    throw new IllegalArgumentException("Offset " + offset +
+        " is outside of file (0.." +
+        fileLength + ")");
+  }
+
+  /**
+   * A factory that makes the split for this class. It can be overridden
+   * by sub-classes to make sub-types
+   */
+  protected FileFragment makeSplit(String fragmentId, Path file, long start, long length) {
+    return new FileFragment(fragmentId, file, start, length);
+  }
+
+  protected FileFragment makeSplit(String fragmentId, Path file, long start, long length,
+                                   String[] hosts) {
+    return new FileFragment(fragmentId, file, start, length, hosts);
+  }
+
+  protected FileFragment makeSplit(String fragmentId, Path file, BlockLocation blockLocation)
+      throws IOException {
+    return new FileFragment(fragmentId, file, blockLocation);
+  }
+
+  // for Non Splittable. eg, compressed gzip TextFile
+  protected FileFragment makeNonSplit(String fragmentId, Path file, long start, long length,
+                                      BlockLocation[] blkLocations) throws IOException {
+
+    Map<String, Integer> hostsBlockMap = new HashMap<String, Integer>();
+    for (BlockLocation blockLocation : blkLocations) {
+      for (String host : blockLocation.getHosts()) {
+        if (hostsBlockMap.containsKey(host)) {
+          hostsBlockMap.put(host, hostsBlockMap.get(host) + 1);
+        } else {
+          hostsBlockMap.put(host, 1);
+        }
+      }
+    }
+
+    List<Map.Entry<String, Integer>> entries = new ArrayList<Map.Entry<String, Integer>>(hostsBlockMap.entrySet());
+    Collections.sort(entries, new Comparator<Map.Entry<String, Integer>>() {
+
+      @Override
+      public int compare(Map.Entry<String, Integer> v1, Map.Entry<String, Integer> v2) {
+        return v1.getValue().compareTo(v2.getValue());
+      }
+    });
+
+    String[] hosts = new String[blkLocations[0].getHosts().length];
+
+    for (int i = 0; i < hosts.length; i++) {
+      Map.Entry<String, Integer> entry = entries.get((entries.size() - 1) - i);
+      hosts[i] = entry.getKey();
+    }
+    return new FileFragment(fragmentId, file, start, length, hosts);
+  }
+
+  /**
+   * Get the minimum split size
+   *
+   * @return the minimum number of bytes that can be in a split
+   */
+  public long getMinSplitSize() {
+    return conf.getLongVar(TajoConf.ConfVars.MINIMUM_SPLIT_SIZE);
+  }
+
+  /**
+   * Get Disk Ids by Volume Bytes
+   */
+  private int[] getDiskIds(VolumeId[] volumeIds) {
+    int[] diskIds = new int[volumeIds.length];
+    for (int i = 0; i < volumeIds.length; i++) {
+      int diskId = -1;
+      if (volumeIds[i] != null && volumeIds[i].hashCode() > 0) {
+        diskId = volumeIds[i].hashCode() - zeroVolumeId.hashCode();
+      }
+      diskIds[i] = diskId;
+    }
+    return diskIds;
+  }
+
+  /**
+   * 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
+   */
+  public List<Fragment> getSplits(String tableName, TableMeta meta, Schema schema, Path... inputs)
+      throws IOException {
+    // generate splits'
+
+    List<Fragment> splits = Lists.newArrayList();
+    List<Fragment> volumeSplits = Lists.newArrayList();
+    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)));
+      } else {
+        files.addAll(listStatus(p));
+      }
+
+      int previousSplitSize = splits.size();
+      for (FileStatus file : files) {
+        Path path = file.getPath();
+        long length = file.getLen();
+        if (length > 0) {
+          // Get locations of blocks of file
+          BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
+          boolean splittable = isSplittable(meta, schema, path, file);
+          if (blocksMetadataEnabled && fs instanceof DistributedFileSystem) {
+
+            if (splittable) {
+              for (BlockLocation blockLocation : blkLocations) {
+                volumeSplits.add(makeSplit(tableName, path, blockLocation));
+              }
+              blockLocations.addAll(Arrays.asList(blkLocations));
+
+            } else { // Non splittable
+              long blockSize = blkLocations[0].getLength();
+              if (blockSize >= length) {
+                blockLocations.addAll(Arrays.asList(blkLocations));
+                for (BlockLocation blockLocation : blkLocations) {
+                  volumeSplits.add(makeSplit(tableName, path, blockLocation));
+                }
+              } else {
+                splits.add(makeNonSplit(tableName, path, 0, length, blkLocations));
+              }
+            }
+
+          } else {
+            if (splittable) {
+
+              long minSize = Math.max(getMinSplitSize(), 1);
+
+              long blockSize = file.getBlockSize(); // s3n rest api contained block size but blockLocations is one
+              long splitSize = Math.max(minSize, blockSize);
+              long bytesRemaining = length;
+
+              // for s3
+              while (((double) bytesRemaining) / splitSize > SPLIT_SLOP) {
+                int blkIndex = getBlockIndex(blkLocations, length - bytesRemaining);
+                splits.add(makeSplit(tableName, path, length - bytesRemaining, splitSize,
+                    blkLocations[blkIndex].getHosts()));
+                bytesRemaining -= splitSize;
+              }
+              if (bytesRemaining > 0) {
+                int blkIndex = getBlockIndex(blkLocations, length - bytesRemaining);
+                splits.add(makeSplit(tableName, path, length - bytesRemaining, bytesRemaining,
+                    blkLocations[blkIndex].getHosts()));
+              }
+            } else { // Non splittable
+              splits.add(makeNonSplit(tableName, path, 0, length, blkLocations));
+            }
+          }
+        } else {
+          //for zero length files
+          splits.add(makeSplit(tableName, path, 0, length));
+        }
+      }
+      if(LOG.isDebugEnabled()){
+        LOG.debug("# of splits per partition: " + (splits.size() - previousSplitSize));
+      }
+    }
+
+    // Combine original fileFragments with new VolumeId information
+    setVolumeMeta(volumeSplits, blockLocations);
+    splits.addAll(volumeSplits);
+    LOG.info("Total # of splits: " + splits.size());
+    return splits;
+  }
+
+  private void setVolumeMeta(List<Fragment> splits, final List<BlockLocation> blockLocations)
+      throws IOException {
+
+    int locationSize = blockLocations.size();
+    int splitSize = splits.size();
+    if (locationSize == 0 || splitSize == 0) return;
+
+    if (locationSize != splitSize) {
+      // splits and locations don't match up
+      LOG.warn("Number of block locations not equal to number of splits: "
+          + "#locations=" + locationSize
+          + " #splits=" + splitSize);
+      return;
+    }
+
+    DistributedFileSystem fs = (DistributedFileSystem)DistributedFileSystem.get(conf);
+    int lsLimit = conf.getInt(DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
+    int blockLocationIdx = 0;
+
+    Iterator<Fragment> iter = splits.iterator();
+    while (locationSize > blockLocationIdx) {
+
+      int subSize = Math.min(locationSize - blockLocationIdx, lsLimit);
+      List<BlockLocation> locations = blockLocations.subList(blockLocationIdx, blockLocationIdx + subSize);
+      //BlockStorageLocation containing additional volume location information for each replica of each block.
+      BlockStorageLocation[] blockStorageLocations = fs.getFileBlockStorageLocations(locations);
+
+      for (BlockStorageLocation blockStorageLocation : blockStorageLocations) {
+        ((FileFragment)iter.next()).setDiskIds(getDiskIds(blockStorageLocation.getVolumeIds()));
+        blockLocationIdx++;
+      }
+    }
+    LOG.info("# of splits with volumeId " + splitSize);
+  }
+
+  private static class InvalidInputException extends IOException {
+    List<IOException> errors;
+    public InvalidInputException(List<IOException> errors) {
+      this.errors = errors;
+    }
+
+    @Override
+    public String getMessage(){
+      StringBuffer sb = new StringBuffer();
+      int messageLimit = Math.min(errors.size(), 10);
+      for (int i = 0; i < messageLimit ; i ++) {
+        sb.append(errors.get(i).getMessage()).append("\n");
+      }
+
+      if(messageLimit < errors.size())
+        sb.append("skipped .....").append("\n");
+
+      return sb.toString();
+    }
+  }
+
+  @Override
+  public List<Fragment> getSplits(String tableName, TableDesc table, ScanNode scanNode) throws IOException {
+    return getSplits(tableName, table.getMeta(), table.getSchema(), new Path(table.getPath()));
+  }
+
+  @Override
+  public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException {
+    if (!tableDesc.isExternal()) {
+      String [] splitted = CatalogUtil.splitFQTableName(tableDesc.getName());
+      String databaseName = splitted[0];
+      String simpleTableName = splitted[1];
+
+      // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} )
+      Path tablePath = StorageUtil.concatPath(tableBaseDir, databaseName, simpleTableName);
+      tableDesc.setPath(tablePath.toUri());
+    } else {
+      Preconditions.checkState(tableDesc.getPath() != null, "ERROR: LOCATION must be given.");
+    }
+
+    Path path = new Path(tableDesc.getPath());
+
+    FileSystem fs = path.getFileSystem(conf);
+    TableStats stats = new TableStats();
+    if (tableDesc.isExternal()) {
+      if (!fs.exists(path)) {
+        LOG.error(path.toUri() + " does not exist");
+        throw new IOException("ERROR: " + path.toUri() + " does not exist");
+      }
+    } else {
+      fs.mkdirs(path);
+    }
+
+    long totalSize = 0;
+
+    try {
+      totalSize = calculateSize(path);
+    } catch (IOException e) {
+      LOG.warn("Cannot calculate the size of the relation", e);
+    }
+
+    stats.setNumBytes(totalSize);
+
+    if (tableDesc.isExternal()) { // if it is an external table, there is no way to know the exact row number without processing.
+      stats.setNumRows(TajoConstants.UNKNOWN_ROW_NUMBER);
+    }
+
+    tableDesc.setStats(stats);
+  }
+
+  @Override
+  public void purgeTable(TableDesc tableDesc) throws IOException {
+    try {
+      Path path = new Path(tableDesc.getPath());
+      FileSystem fs = path.getFileSystem(conf);
+      LOG.info("Delete table data dir: " + path);
+      fs.delete(path, true);
+    } catch (IOException e) {
+      throw new InternalError(e.getMessage());
+    }
+  }
+
+  @Override
+  public List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments) throws IOException {
+    // Listing table data file which is not empty.
+    // If the table is a partitioned table, return file list which has same partition key.
+    Path tablePath = new Path(tableDesc.getPath());
+    FileSystem fs = tablePath.getFileSystem(conf);
+
+    List<FileStatus> nonZeroLengthFiles = new ArrayList<FileStatus>();
+    if (fs.exists(tablePath)) {
+      getNonZeroLengthDataFiles(fs, tablePath, nonZeroLengthFiles, currentPage, numFragments,
+          new AtomicInteger(0));
+    }
+
+    List<Fragment> fragments = new ArrayList<Fragment>();
+
+    //In the case of partitioned table, return same partition key data files.
+    int numPartitionColumns = 0;
+    if (tableDesc.hasPartition()) {
+      numPartitionColumns = tableDesc.getPartitionMethod().getExpressionSchema().getColumns().size();
+    }
+    String[] previousPartitionPathNames = null;
+    for (FileStatus eachFile: nonZeroLengthFiles) {
+      FileFragment fileFragment = new FileFragment(tableDesc.getName(), eachFile.getPath(), 0, eachFile.getLen(), null);
+
+      if (numPartitionColumns > 0) {
+        // finding partition key;
+        Path filePath = fileFragment.getPath();
+        Path parentPath = filePath;
+        String[] parentPathNames = new String[numPartitionColumns];
+        for (int i = 0; i < numPartitionColumns; i++) {
+          parentPath = parentPath.getParent();
+          parentPathNames[numPartitionColumns - i - 1] = parentPath.getName();
+        }
+
+        // If current partitionKey == previousPartitionKey, add to result.
+        if (previousPartitionPathNames == null) {
+          fragments.add(fileFragment);
+        } else if (previousPartitionPathNames != null && Arrays.equals(previousPartitionPathNames, parentPathNames)) {
+          fragments.add(fileFragment);
+        } else {
+          break;
+        }
+        previousPartitionPathNames = parentPathNames;
+      } else {
+        fragments.add(fileFragment);
+      }
+    }
+
+    return fragments;
+  }
+
+  private void getNonZeroLengthDataFiles(FileSystem fs, Path path, List<FileStatus> result,
+                                                int startFileIndex, int numResultFiles,
+                                                AtomicInteger currentFileIndex) throws IOException {
+    if (fs.isDirectory(path)) {
+      FileStatus[] files = fs.listStatus(path, FileStorageManager.hiddenFileFilter);
+      if (files != null && files.length > 0) {
+        for (FileStatus eachFile : files) {
+          if (result.size() >= numResultFiles) {
+            return;
+          }
+          if (eachFile.isDirectory()) {
+            getNonZeroLengthDataFiles(fs, eachFile.getPath(), result, startFileIndex, numResultFiles,
+                currentFileIndex);
+          } else if (eachFile.isFile() && eachFile.getLen() > 0) {
+            if (currentFileIndex.get() >= startFileIndex) {
+              result.add(eachFile);
+            }
+            currentFileIndex.incrementAndGet();
+          }
+        }
+      }
+    } else {
+      FileStatus fileStatus = fs.getFileStatus(path);
+      if (fileStatus != null && fileStatus.getLen() > 0) {
+        if (currentFileIndex.get() >= startFileIndex) {
+          result.add(fileStatus);
+        }
+        currentFileIndex.incrementAndGet();
+        if (result.size() >= numResultFiles) {
+          return;
+        }
+      }
+    }
+  }
+
+  @Override
+  public StorageProperty getStorageProperty() {
+    StorageProperty storageProperty = new StorageProperty();
+    storageProperty.setSortedInsert(false);
+    if (storeType == StoreType.RAW) {
+      storageProperty.setSupportsInsertInto(false);
+    } else {
+      storageProperty.setSupportsInsertInto(true);
+    }
+
+    return storageProperty;
+  }
+
+  @Override
+  public void closeStorageManager() {
+  }
+
+  @Override
+  public void beforeInsertOrCATS(LogicalNode node) throws IOException {
+  }
+
+  @Override
+  public void rollbackOutputCommit(LogicalNode node) throws IOException {
+  }
+
+  @Override
+  public TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
+                                          Schema inputSchema, SortSpec[] sortSpecs, TupleRange dataRange)
+      throws IOException {
+    return null;
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param conf The system property
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param path The data file path
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public static synchronized SeekableScanner getSeekableScanner(
+      TajoConf conf, TableMeta meta, Schema schema, Path path) throws IOException {
+
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus status = fs.getFileStatus(path);
+    FileFragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
+
+    return getSeekableScanner(conf, meta, schema, fragment, schema);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..33b2750
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppender.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.util.Pair;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class HashShuffleAppender implements Appender {
+  private static Log LOG = LogFactory.getLog(HashShuffleAppender.class);
+
+  private FileAppender appender;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+  private int partId;
+
+  private TableStats tableStats;
+
+  //<taskId,<page start offset,<task start, task end>>>
+  private Map<QueryUnitAttemptId, List<Pair<Long, Pair<Integer, Integer>>>> taskTupleIndexes;
+
+  //page start offset, length
+  private List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+
+  private Pair<Long, Integer> currentPage;
+
+  private int pageSize; //MB
+
+  private int rowNumInPage;
+
+  private int totalRows;
+
+  private long offset;
+
+  private ExecutionBlockId ebId;
+
+  public HashShuffleAppender(ExecutionBlockId ebId, int partId, int pageSize, FileAppender appender) {
+    this.ebId = ebId;
+    this.partId = partId;
+    this.appender = appender;
+    this.pageSize = pageSize;
+  }
+
+  @Override
+  public void init() throws IOException {
+    currentPage = new Pair(0L, 0);
+    taskTupleIndexes = new HashMap<QueryUnitAttemptId, List<Pair<Long, Pair<Integer, Integer>>>>();
+    rowNumInPage = 0;
+  }
+
+  /**
+   * Write multiple tuples. Each tuple is written by a FileAppender which is responsible specified partition.
+   * After writing if a current page exceeds pageSize, pageOffset will be added.
+   * @param taskId
+   * @param tuples
+   * @return written bytes
+   * @throws java.io.IOException
+   */
+  public int addTuples(QueryUnitAttemptId taskId, List<Tuple> tuples) throws IOException {
+    synchronized(appender) {
+      if (closed.get()) {
+        return 0;
+      }
+      long currentPos = appender.getOffset();
+
+      for (Tuple eachTuple: tuples) {
+        appender.addTuple(eachTuple);
+      }
+      long posAfterWritten = appender.getOffset();
+
+      int writtenBytes = (int)(posAfterWritten - currentPos);
+
+      int nextRowNum = rowNumInPage + tuples.size();
+      List<Pair<Long, Pair<Integer, Integer>>> taskIndexes = taskTupleIndexes.get(taskId);
+      if (taskIndexes == null) {
+        taskIndexes = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
+        taskTupleIndexes.put(taskId, taskIndexes);
+      }
+      taskIndexes.add(
+          new Pair<Long, Pair<Integer, Integer>>(currentPage.getFirst(), new Pair(rowNumInPage, nextRowNum)));
+      rowNumInPage = nextRowNum;
+
+      if (posAfterWritten - currentPage.getFirst() > pageSize) {
+        nextPage(posAfterWritten);
+        rowNumInPage = 0;
+      }
+
+      totalRows += tuples.size();
+      return writtenBytes;
+    }
+  }
+
+  public long getOffset() throws IOException {
+    if (closed.get()) {
+      return offset;
+    } else {
+      return appender.getOffset();
+    }
+  }
+
+  private void nextPage(long pos) {
+    currentPage.setSecond((int) (pos - currentPage.getFirst()));
+    pages.add(currentPage);
+    currentPage = new Pair(pos, 0);
+  }
+
+  @Override
+  public void addTuple(Tuple t) throws IOException {
+    throw new IOException("Not support addTuple, use addTuples()");
+  }
+
+  @Override
+  public void flush() throws IOException {
+    synchronized(appender) {
+      if (closed.get()) {
+        return;
+      }
+      appender.flush();
+    }
+  }
+
+  @Override
+  public long getEstimatedOutputSize() throws IOException {
+    return pageSize * pages.size();
+  }
+
+  @Override
+  public void close() throws IOException {
+    synchronized(appender) {
+      if (closed.get()) {
+        return;
+      }
+      appender.flush();
+      offset = appender.getOffset();
+      if (offset > currentPage.getFirst()) {
+        nextPage(offset);
+      }
+      appender.close();
+      if (LOG.isDebugEnabled()) {
+        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=" + pages.size());
+        }
+      }
+      closed.set(true);
+      tableStats = appender.getStats();
+    }
+  }
+
+  @Override
+  public void enableStats() {
+  }
+
+  @Override
+  public TableStats getStats() {
+    synchronized(appender) {
+      return appender.getStats();
+    }
+  }
+
+  public List<Pair<Long, Integer>> getPages() {
+    return pages;
+  }
+
+  public Map<QueryUnitAttemptId, List<Pair<Long, Pair<Integer, Integer>>>> getTaskTupleIndexes() {
+    return taskTupleIndexes;
+  }
+
+  public List<Pair<Long, Pair<Integer, Integer>>> getMergedTupleIndexes() {
+    List<Pair<Long, Pair<Integer, Integer>>> merged = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
+
+    for (List<Pair<Long, Pair<Integer, Integer>>> eachFailureIndex: taskTupleIndexes.values()) {
+      merged.addAll(eachFailureIndex);
+    }
+
+    return merged;
+  }
+
+  public void taskFinished(QueryUnitAttemptId taskId) {
+    taskTupleIndexes.remove(taskId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..636ae0f
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/HashShuffleAppenderManager.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryUnitAttemptId;
+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;
+
+public class HashShuffleAppenderManager {
+  private static final Log LOG = LogFactory.getLog(HashShuffleAppenderManager.class);
+
+  private Map<ExecutionBlockId, Map<Integer, PartitionAppenderMeta>> appenderMap =
+      new ConcurrentHashMap<ExecutionBlockId, Map<Integer, PartitionAppenderMeta>>();
+  private TajoConf systemConf;
+  private FileSystem defaultFS;
+  private FileSystem localFS;
+  private LocalDirAllocator lDirAllocator;
+  private int pageSize;
+
+  public HashShuffleAppenderManager(TajoConf systemConf) throws IOException {
+    this.systemConf = systemConf;
+
+    // initialize LocalDirAllocator
+    lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname);
+
+    // initialize DFS and LocalFileSystems
+    defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf);
+    localFS = FileSystem.getLocal(systemConf);
+    pageSize = systemConf.getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME) * 1024 * 1024;
+  }
+
+  public HashShuffleAppender getAppender(TajoConf tajoConf, ExecutionBlockId ebId, int partId,
+                              TableMeta meta, Schema outSchema) throws IOException {
+    synchronized (appenderMap) {
+      Map<Integer, PartitionAppenderMeta> partitionAppenderMap = appenderMap.get(ebId);
+
+      if (partitionAppenderMap == null) {
+        partitionAppenderMap = new ConcurrentHashMap<Integer, PartitionAppenderMeta>();
+        appenderMap.put(ebId, partitionAppenderMap);
+      }
+
+      PartitionAppenderMeta partitionAppenderMeta = partitionAppenderMap.get(partId);
+      if (partitionAppenderMeta == null) {
+        Path dataFile = getDataFile(ebId, partId);
+        FileSystem fs = dataFile.getFileSystem(systemConf);
+        if (fs.exists(dataFile)) {
+          FileStatus status = fs.getFileStatus(dataFile);
+          LOG.info("File " + dataFile + " already exists, size=" + status.getLen());
+        }
+
+        if (!fs.exists(dataFile.getParent())) {
+          fs.mkdirs(dataFile.getParent());
+        }
+        FileAppender appender = (FileAppender)((FileStorageManager)StorageManager.getFileStorageManager(
+            tajoConf, null)).getAppender(meta, outSchema, dataFile);
+        appender.enableStats();
+        appender.init();
+
+        partitionAppenderMeta = new PartitionAppenderMeta();
+        partitionAppenderMeta.partId = partId;
+        partitionAppenderMeta.dataFile = dataFile;
+        partitionAppenderMeta.appender = new HashShuffleAppender(ebId, partId, pageSize, appender);
+        partitionAppenderMeta.appender.init();
+        partitionAppenderMap.put(partId, partitionAppenderMeta);
+
+        LOG.info("Create Hash shuffle file(partId=" + partId + "): " + dataFile);
+      }
+
+      return partitionAppenderMeta.appender;
+    }
+  }
+
+  public static int getPartParentId(int partId, TajoConf tajoConf) {
+    return partId % tajoConf.getIntVar(TajoConf.ConfVars.HASH_SHUFFLE_PARENT_DIRS);
+  }
+
+  private Path getDataFile(ExecutionBlockId ebId, int partId) throws IOException {
+    try {
+      // the base dir for an output dir
+      String executionBlockBaseDir = ebId.getQueryId().toString() + "/output" + "/" + ebId.getId() + "/hash-shuffle";
+      Path baseDirPath = localFS.makeQualified(lDirAllocator.getLocalPathForWrite(executionBlockBaseDir, systemConf));
+      //LOG.info(ebId + "'s basedir is created (" + baseDirPath + ")");
+
+      // If EB has many partition, too many shuffle file are in single directory.
+      return StorageUtil.concatPath(baseDirPath, "" + getPartParentId(partId, systemConf), "" + partId);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new IOException(e);
+    }
+  }
+
+  public List<HashShuffleIntermediate> close(ExecutionBlockId ebId) throws IOException {
+    Map<Integer, PartitionAppenderMeta> partitionAppenderMap = null;
+    synchronized (appenderMap) {
+      partitionAppenderMap = appenderMap.remove(ebId);
+    }
+
+    if (partitionAppenderMap == null) {
+      LOG.info("Close HashShuffleAppender:" + ebId + ", not a hash shuffle");
+      return null;
+    }
+
+    // Send Intermediate data to QueryMaster.
+    List<HashShuffleIntermediate> intermEntries = new ArrayList<HashShuffleIntermediate>();
+    for (PartitionAppenderMeta eachMeta : partitionAppenderMap.values()) {
+      try {
+        eachMeta.appender.close();
+        HashShuffleIntermediate intermediate =
+            new HashShuffleIntermediate(eachMeta.partId, eachMeta.appender.getOffset(),
+                eachMeta.appender.getPages(),
+                eachMeta.appender.getMergedTupleIndexes());
+        intermEntries.add(intermediate);
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+        throw e;
+      }
+    }
+
+    LOG.info("Close HashShuffleAppender:" + ebId + ", intermediates=" + intermEntries.size());
+
+    return intermEntries;
+  }
+
+  public void finalizeTask(QueryUnitAttemptId taskId) {
+    synchronized (appenderMap) {
+      Map<Integer, PartitionAppenderMeta> partitionAppenderMap =
+        appenderMap.get(taskId.getQueryUnitId().getExecutionBlockId());
+      if (partitionAppenderMap == null) {
+        return;
+      }
+
+      for (PartitionAppenderMeta eachAppender: partitionAppenderMap.values()) {
+        eachAppender.appender.taskFinished(taskId);
+      }
+    }
+  }
+
+  public static class HashShuffleIntermediate {
+    private int partId;
+
+    private long volume;
+
+    //[<page start offset,<task start, task end>>]
+    private Collection<Pair<Long, Pair<Integer, Integer>>> failureTskTupleIndexes;
+
+    //[<page start offset, length>]
+    private List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+
+    public HashShuffleIntermediate(int partId, long volume,
+                                   List<Pair<Long, Integer>> pages,
+                                   Collection<Pair<Long, Pair<Integer, Integer>>> failureTskTupleIndexes) {
+      this.partId = partId;
+      this.volume = volume;
+      this.failureTskTupleIndexes = failureTskTupleIndexes;
+      this.pages = pages;
+    }
+
+    public int getPartId() {
+      return partId;
+    }
+
+    public long getVolume() {
+      return volume;
+    }
+
+    public Collection<Pair<Long, Pair<Integer, Integer>>> getFailureTskTupleIndexes() {
+      return failureTskTupleIndexes;
+    }
+
+    public List<Pair<Long, Integer>> getPages() {
+      return pages;
+    }
+  }
+
+  static class PartitionAppenderMeta {
+    int partId;
+    HashShuffleAppender appender;
+    Path dataFile;
+
+    public int getPartId() {
+      return partId;
+    }
+
+    public HashShuffleAppender getAppender() {
+      return appender;
+    }
+
+    public Path getDataFile() {
+      return dataFile;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/LineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/LineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/LineReader.java
new file mode 100644
index 0000000..0f31baf
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/LineReader.java
@@ -0,0 +1,559 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+
+/**
+ * A class that provides a line reader from an input stream.
+ * Depending on the constructor used, lines will either be terminated by:
+ * <ul>
+ * <li>one of the following: '\n' (LF) , '\r' (CR),
+ * or '\r\n' (CR+LF).</li>
+ * <li><em>or</em>, a custom byte sequence delimiter</li>
+ * </ul>
+ * In both cases, EOF also terminates an otherwise unterminated
+ * line.
+ */
+
+public class LineReader implements Closeable {
+  private static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
+  private int bufferSize = DEFAULT_BUFFER_SIZE;
+  private InputStream in;
+  private byte[] buffer;
+  // the number of bytes of real data in the buffer
+  private int bufferLength = 0;
+  // the current position in the buffer
+  private int bufferPosn = 0;
+
+  private static final byte CR = '\r';
+  private static final byte LF = '\n';
+
+  // The line delimiter
+  private final byte[] recordDelimiterBytes;
+
+  /**
+   * Create a line reader that reads from the given stream using the
+   * default buffer-size (64k).
+   *
+   * @param in The input stream
+   * @throws java.io.IOException
+   */
+  public LineReader(InputStream in) {
+    this(in, DEFAULT_BUFFER_SIZE);
+  }
+
+  /**
+   * Create a line reader that reads from the given stream using the
+   * given buffer-size.
+   *
+   * @param in         The input stream
+   * @param bufferSize Size of the read buffer
+   * @throws java.io.IOException
+   */
+  public LineReader(InputStream in, int bufferSize) {
+    this.in = in;
+    this.bufferSize = bufferSize;
+    this.buffer = new byte[this.bufferSize];
+    this.recordDelimiterBytes = null;
+  }
+
+  /**
+   * Create a line reader that reads from the given stream using the
+   * <code>io.file.buffer.size</code> specified in the given
+   * <code>Configuration</code>.
+   *
+   * @param in   input stream
+   * @param conf configuration
+   * @throws java.io.IOException
+   */
+  public LineReader(InputStream in, Configuration conf) throws IOException {
+    this(in, conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE));
+  }
+
+  /**
+   * Create a line reader that reads from the given stream using the
+   * default buffer-size, and using a custom delimiter of array of
+   * bytes.
+   *
+   * @param in                   The input stream
+   * @param recordDelimiterBytes The delimiter
+   */
+  public LineReader(InputStream in, byte[] recordDelimiterBytes) {
+    this.in = in;
+    this.bufferSize = DEFAULT_BUFFER_SIZE;
+    this.buffer = new byte[this.bufferSize];
+    this.recordDelimiterBytes = recordDelimiterBytes;
+  }
+
+  /**
+   * Create a line reader that reads from the given stream using the
+   * given buffer-size, and using a custom delimiter of array of
+   * bytes.
+   *
+   * @param in                   The input stream
+   * @param bufferSize           Size of the read buffer
+   * @param recordDelimiterBytes The delimiter
+   * @throws java.io.IOException
+   */
+  public LineReader(InputStream in, int bufferSize,
+                    byte[] recordDelimiterBytes) {
+    this.in = in;
+    this.bufferSize = bufferSize;
+    this.buffer = new byte[this.bufferSize];
+    this.recordDelimiterBytes = recordDelimiterBytes;
+  }
+
+  /**
+   * Create a line reader that reads from the given stream using the
+   * <code>io.file.buffer.size</code> specified in the given
+   * <code>Configuration</code>, and using a custom delimiter of array of
+   * bytes.
+   *
+   * @param in                   input stream
+   * @param conf                 configuration
+   * @param recordDelimiterBytes The delimiter
+   * @throws java.io.IOException
+   */
+  public LineReader(InputStream in, Configuration conf,
+                    byte[] recordDelimiterBytes) throws IOException {
+    this.in = in;
+    this.bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE);
+    this.buffer = new byte[this.bufferSize];
+    this.recordDelimiterBytes = recordDelimiterBytes;
+  }
+
+
+  /**
+   * Close the underlying stream.
+   *
+   * @throws java.io.IOException
+   */
+  public void close() throws IOException {
+    in.close();
+  }
+
+  public void reset() {
+    bufferLength = 0;
+    bufferPosn = 0;
+
+  }
+
+  /**
+   * Read one line from the InputStream into the given Text.
+   *
+   * @param str               the object to store the given line (without newline)
+   * @param maxLineLength     the maximum number of bytes to store into str;
+   *                          the rest of the line is silently discarded.
+   * @param maxBytesToConsume the maximum number of bytes to consume
+   *                          in this call.  This is only a hint, because if the line cross
+   *                          this threshold, we allow it to happen.  It can overshoot
+   *                          potentially by as much as one buffer length.
+   * @return the number of bytes read including the (longest) newline
+   *         found.
+   * @throws java.io.IOException if the underlying stream throws
+   */
+  public int readLine(Text str, int maxLineLength,
+                      int maxBytesToConsume) throws IOException {
+    if (this.recordDelimiterBytes != null) {
+      return readCustomLine(str, maxLineLength, maxBytesToConsume);
+    } else {
+      return readDefaultLine(str, maxLineLength, maxBytesToConsume);
+    }
+  }
+
+  protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter)
+      throws IOException {
+    return in.read(buffer);
+  }
+  /**
+   * Read a line terminated by one of CR, LF, or CRLF.
+   */
+  private int readDefaultLine(Text str, int maxLineLength, int maxBytesToConsume)
+      throws IOException {
+    /* We're reading data from in, but the head of the stream may be
+     * already buffered in buffer, so we have several cases:
+     * 1. No newline characters are in the buffer, so we need to copy
+     *    everything and read another buffer from the stream.
+     * 2. An unambiguously terminated line is in buffer, so we just
+     *    copy to str.
+     * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
+     *    in CR.  In this case we copy everything up to CR to str, but
+     *    we also need to see what follows CR: if it's LF, then we
+     *    need consume LF as well, so next call to readLine will read
+     *    from after that.
+     * We use a flag prevCharCR to signal if previous character was CR
+     * and, if it happens to be at the end of the buffer, delay
+     * consuming it until we have a chance to look at the char that
+     * follows.
+     */
+    str.clear();
+    int txtLength = 0; //tracks str.getLength(), as an optimization
+    int newlineLength = 0; //length of terminating newline
+    boolean prevCharCR = false; //true of prev char was CR
+    long bytesConsumed = 0;
+    do {
+      int startPosn = bufferPosn; //starting from where we left off the last time
+      if (bufferPosn >= bufferLength) {
+        startPosn = bufferPosn = 0;
+        if (prevCharCR) {
+          ++bytesConsumed; //account for CR from previous read
+        }
+        bufferLength = fillBuffer(in, buffer, prevCharCR);
+        if (bufferLength <= 0) {
+          break; // EOF
+        }
+      }
+      for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
+        if (buffer[bufferPosn] == LF) {
+          newlineLength = (prevCharCR) ? 2 : 1;
+          ++bufferPosn; // at next invocation proceed from following byte
+          break;
+        }
+        if (prevCharCR) { //CR + notLF, we are at notLF
+          newlineLength = 1;
+          break;
+        }
+        prevCharCR = (buffer[bufferPosn] == CR);
+      }
+      int readLength = bufferPosn - startPosn;
+      if (prevCharCR && newlineLength == 0) {
+        --readLength; //CR at the end of the buffer
+      }
+      bytesConsumed += readLength;
+      int appendLength = readLength - newlineLength;
+      if (appendLength > maxLineLength - txtLength) {
+        appendLength = maxLineLength - txtLength;
+      }
+      if (appendLength > 0) {
+        str.append(buffer, startPosn, appendLength);
+        txtLength += appendLength;
+      }
+    } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
+
+    if (bytesConsumed > (long) Integer.MAX_VALUE) {
+      throw new IOException("Too many bytes before newline: " + bytesConsumed);
+    }
+    return (int) bytesConsumed;
+  }
+
+  /**
+   * Read a line terminated by one of CR, LF, or CRLF.
+   */
+  public int readDefaultLine(NonSyncByteArrayOutputStream str, ArrayList<Integer> offsets, int maxLineLength
+      , int maxBytesToConsume)
+      throws IOException {
+    /* We're reading data from in, but the head of the stream may be
+     * already buffered in buffer, so we have several cases:
+     * 1. No newline characters are in the buffer, so we need to copy
+     *    everything and read another buffer from the stream.
+     * 2. An unambiguously terminated line is in buffer, so we just
+     *    copy to str.
+     * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
+     *    in CR.  In this case we copy everything up to CR to str, but
+     *    we also need to see what follows CR: if it's LF, then we
+     *    need consume LF as well, so next call to readLine will read
+     *    from after that.
+     * We use a flag prevCharCR to signal if previous character was CR
+     * and, if it happens to be at the end of the buffer, delay
+     * consuming it until we have a chance to look at the char that
+     * follows.
+     */
+
+    int txtLength = 0; //tracks str.getLength(), as an optimization
+    int newlineLength = 0; //length of terminating newline
+    boolean prevCharCR = false; //true of prev char was CR
+    long bytesConsumed = 0;
+    do {
+      int startPosn = bufferPosn; //starting from where we left off the last time
+      if (bufferPosn >= bufferLength) {
+        startPosn = bufferPosn = 0;
+        if (prevCharCR) {
+          ++bytesConsumed; //account for CR from previous read
+        }
+        bufferLength = fillBuffer(in, buffer, prevCharCR);
+        if (bufferLength <= 0) {
+          break; // EOF
+        }
+      }
+      for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
+        if (buffer[bufferPosn] == LF) {
+          newlineLength = (prevCharCR) ? 2 : 1;
+          ++bufferPosn; // at next invocation proceed from following byte
+          break;
+        }
+        if (prevCharCR) { //CR + notLF, we are at notLF
+          newlineLength = 1;
+          break;
+        }
+        prevCharCR = (buffer[bufferPosn] == CR);
+      }
+      int readLength = bufferPosn - startPosn;
+      if (prevCharCR && newlineLength == 0) {
+        --readLength; //CR at the end of the buffer
+      }
+      bytesConsumed += readLength;
+      int appendLength = readLength - newlineLength;
+      if (appendLength > maxLineLength - txtLength) {
+        appendLength = maxLineLength - txtLength;
+      }
+      if (appendLength > 0) {
+        str.write(buffer, startPosn, appendLength);
+        txtLength += appendLength;
+      }
+    } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
+
+    if (bytesConsumed > (long) Integer.MAX_VALUE) {
+      throw new IOException("Too many bytes before newline: " + bytesConsumed);
+    }
+
+    if (bytesConsumed > 0) offsets.add(txtLength);
+    return (int) bytesConsumed;
+  }
+
+  /**
+   * Read a line terminated by one of CR, LF, or CRLF.
+   */
+
+/*  int validIdx = 0;
+  public int readDefaultLines(NonSyncByteArrayOutputStream str, ArrayList<Integer> offsets, ArrayList<Long> foffsets,
+                             long pos, int maxLineLength, int maxBytesToConsume)
+      throws IOException {
+    *//* We're reading data from in, but the head of the stream may be
+     * already buffered in buffer, so we have several cases:
+     * 1. No newline characters are in the buffer, so we need to copy
+     *    everything and read another buffer from the stream.
+     * 2. An unambiguously terminated line is in buffer, so we just
+     *    copy to str.
+     * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
+     *    in CR.  In this case we copy everything up to CR to str, but
+     *    we also need to see what follows CR: if it's LF, then we
+     *    need consume LF as well, so next call to readLine will read
+     *    from after that.
+     * We use a flag prevCharCR to signal if previous character was CR
+     * and, if it happens to be at the end of the buffer, delay
+     * consuming it until we have a chance to look at the char that
+     * follows.
+     *//*
+    //str.clear();
+    str.reset();
+    offsets.clear();
+    foffsets.clear();
+
+    validIdx = 0;
+    long bufferBytesConsumed = 0;
+
+    int txtLength = 0; //tracks str.getLength(), as an optimization
+    int newlineLength = 0; //length of terminating newline
+    boolean prevCharCR = false; //true of prev char was CR
+    long bytesConsumed = 0;
+    do {
+
+      int startPosn = bufferPosn; //starting from where we left off the last time
+      if (bufferPosn >= bufferLength) {
+        startPosn = bufferPosn = 0;
+        if (prevCharCR) {
+          ++bytesConsumed; //account for CR from previous read
+        }
+        bufferLength = in.read(buffer);
+        if (bufferLength <= 0) {
+          break; // EOF
+        }
+      }
+      for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
+        if (buffer[bufferPosn] == LF) {
+          newlineLength = (prevCharCR) ? 2 : 1;
+          ++bufferPosn; // at next invocation proceed from following byte
+          break;
+        }
+        if (prevCharCR) { //CR + notLF, we are at notLF
+          newlineLength = 1;
+          break;
+        }
+        prevCharCR = (buffer[bufferPosn] == CR);
+      }
+      int readLength = bufferPosn - startPosn;
+      if (prevCharCR && newlineLength == 0) {
+        --readLength; //CR at the end of the buffer
+      }
+      bytesConsumed += readLength;
+      int appendLength = readLength - newlineLength;
+      if (appendLength > maxLineLength - txtLength) {
+        appendLength = maxLineLength - txtLength;
+      }
+
+      if (appendLength > 0) {
+        str.write(buffer, startPosn, appendLength);
+        //System.out.println(startPosn + "," + appendLength);
+        //str.append(buffer, startPosn, appendLength);
+        txtLength += appendLength;
+      }
+
+      if(newlineLength > 0){
+        validIdx++;
+
+        if (bytesConsumed > (long)Integer.MAX_VALUE) {
+          throw new IOException("Too many bytes before newline: " + bytesConsumed);
+        }
+        offsets.add(txtLength);
+        foffsets.add(pos);
+        pos+= bytesConsumed;
+        bufferBytesConsumed += bytesConsumed;
+
+        txtLength = 0;
+        newlineLength = 0;
+        prevCharCR = false; //true of prev char was CR
+        bytesConsumed = 0;
+      } else {
+        bufferBytesConsumed += bytesConsumed;
+        bytesConsumed = 0;
+      }
+    } while ((bufferBytesConsumed < 256 * 1024));
+
+    return (int)bufferBytesConsumed;
+  }*/
+
+  /**
+   * Read a line terminated by a custom delimiter.
+   */
+  private int readCustomLine(Text str, int maxLineLength, int maxBytesToConsume)
+      throws IOException {
+   /* We're reading data from inputStream, but the head of the stream may be
+    *  already captured in the previous buffer, so we have several cases:
+    *
+    * 1. The buffer tail does not contain any character sequence which
+    *    matches with the head of delimiter. We count it as a
+    *    ambiguous byte count = 0
+    *
+    * 2. The buffer tail contains a X number of characters,
+    *    that forms a sequence, which matches with the
+    *    head of delimiter. We count ambiguous byte count = X
+    *
+    *    // ***  eg: A segment of input file is as follows
+    *
+    *    " record 1792: I found this bug very interesting and
+    *     I have completely read about it. record 1793: This bug
+    *     can be solved easily record 1794: This ."
+    *
+    *    delimiter = "record";
+    *
+    *    supposing:- String at the end of buffer =
+    *    "I found this bug very interesting and I have completely re"
+    *    There for next buffer = "ad about it. record 179       ...."
+    *
+    *     The matching characters in the input
+    *     buffer tail and delimiter head = "re"
+    *     Therefore, ambiguous byte count = 2 ****   //
+    *
+    *     2.1 If the following bytes are the remaining characters of
+    *         the delimiter, then we have to capture only up to the starting
+    *         position of delimiter. That means, we need not include the
+    *         ambiguous characters in str.
+    *
+    *     2.2 If the following bytes are not the remaining characters of
+    *         the delimiter ( as mentioned in the example ),
+    *         then we have to include the ambiguous characters in str.
+    */
+    str.clear();
+    int txtLength = 0; // tracks str.getLength(), as an optimization
+    long bytesConsumed = 0;
+    int delPosn = 0;
+    int ambiguousByteCount = 0; // To capture the ambiguous characters count
+    do {
+      int startPosn = bufferPosn; // Start from previous end position
+      if (bufferPosn >= bufferLength) {
+        startPosn = bufferPosn = 0;
+        bufferLength = fillBuffer(in, buffer, ambiguousByteCount > 0);
+        if (bufferLength <= 0) {
+          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
+          break; // EOF
+        }
+      }
+      for (; bufferPosn < bufferLength; ++bufferPosn) {
+        if (buffer[bufferPosn] == recordDelimiterBytes[delPosn]) {
+          delPosn++;
+          if (delPosn >= recordDelimiterBytes.length) {
+            bufferPosn++;
+            break;
+          }
+        } else if (delPosn != 0) {
+          bufferPosn--;
+          delPosn = 0;
+        }
+      }
+      int readLength = bufferPosn - startPosn;
+      bytesConsumed += readLength;
+      int appendLength = readLength - delPosn;
+      if (appendLength > maxLineLength - txtLength) {
+        appendLength = maxLineLength - txtLength;
+      }
+      if (appendLength > 0) {
+        if (ambiguousByteCount > 0) {
+          str.append(recordDelimiterBytes, 0, ambiguousByteCount);
+          //appending the ambiguous characters (refer case 2.2)
+          bytesConsumed += ambiguousByteCount;
+          ambiguousByteCount = 0;
+        }
+        str.append(buffer, startPosn, appendLength);
+        txtLength += appendLength;
+      }
+      if (bufferPosn >= bufferLength) {
+        if (delPosn > 0 && delPosn < recordDelimiterBytes.length) {
+          ambiguousByteCount = delPosn;
+          bytesConsumed -= ambiguousByteCount; //to be consumed in next
+        }
+      }
+    } while (delPosn < recordDelimiterBytes.length
+        && bytesConsumed < maxBytesToConsume);
+    if (bytesConsumed > (long) Integer.MAX_VALUE) {
+      throw new IOException("Too many bytes before delimiter: " + bytesConsumed);
+    }
+    return (int) bytesConsumed;
+  }
+
+  /**
+   * Read from the InputStream into the given Text.
+   *
+   * @param str           the object to store the given line
+   * @param maxLineLength the maximum number of bytes to store into str.
+   * @return the number of bytes read including the newline
+   * @throws java.io.IOException if the underlying stream throws
+   */
+  public int readLine(Text str, int maxLineLength) throws IOException {
+    return readLine(str, maxLineLength, Integer.MAX_VALUE);
+  }
+
+  /**
+   * Read from the InputStream into the given Text.
+   *
+   * @param str the object to store the given line
+   * @return the number of bytes read including the newline
+   * @throws java.io.IOException if the underlying stream throws
+   */
+  public int readLine(Text str) throws IOException {
+    return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
+  }
+}


[12/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
new file mode 100644
index 0000000..14e67b2
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ResizableLimitSpec.java
@@ -0,0 +1,142 @@
+/*
+ * 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.tuple.offheap;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.util.FileUtil;
+
+/**
+ * It specifies the maximum size or increasing ratio. In addition,
+ * it guarantees that all numbers are less than or equal to Integer.MAX_VALUE 2^31
+ * due to ByteBuffer.
+ */
+public class ResizableLimitSpec {
+  private final Log LOG = LogFactory.getLog(ResizableLimitSpec.class);
+
+  public static final int MAX_SIZE_BYTES = Integer.MAX_VALUE;
+  public static final ResizableLimitSpec DEFAULT_LIMIT = new ResizableLimitSpec(Integer.MAX_VALUE);
+
+  private final long initSize;
+  private final long limitBytes;
+  private final float incRatio;
+  private final float allowedOVerflowRatio;
+  private final static float DEFAULT_ALLOWED_OVERFLOW_RATIO = 0.1f;
+  private final static float DEFAULT_INCREASE_RATIO = 1.0f;
+
+  public ResizableLimitSpec(long initSize) {
+    this(initSize, MAX_SIZE_BYTES, DEFAULT_ALLOWED_OVERFLOW_RATIO);
+  }
+
+  public ResizableLimitSpec(long initSize, long limitBytes) {
+    this(initSize, limitBytes, DEFAULT_ALLOWED_OVERFLOW_RATIO);
+  }
+
+  public ResizableLimitSpec(long initSize, long limitBytes, float allowedOverflow) {
+    this(initSize, limitBytes, allowedOverflow, DEFAULT_INCREASE_RATIO);
+  }
+
+  public ResizableLimitSpec(long initSize, long limitBytes, float allowedOverflowRatio, float incRatio) {
+    Preconditions.checkArgument(initSize > 0, "initial size must be greater than 0 bytes.");
+    Preconditions.checkArgument(initSize <= MAX_SIZE_BYTES, "The maximum initial size is 2GB.");
+    Preconditions.checkArgument(limitBytes > 0, "The limit size must be greater than 0 bytes.");
+    Preconditions.checkArgument(limitBytes <= MAX_SIZE_BYTES, "The maximum limit size is 2GB.");
+    Preconditions.checkArgument(incRatio > 0.0f, "Increase Ratio must be greater than 0.");
+
+    if (initSize == limitBytes) {
+      long overflowedSize = (long) (initSize + (initSize * allowedOverflowRatio));
+
+      if (overflowedSize > Integer.MAX_VALUE) {
+        overflowedSize = Integer.MAX_VALUE;
+      }
+
+      this.initSize = overflowedSize;
+      this.limitBytes = overflowedSize;
+    } else {
+      this.initSize = initSize;
+      limitBytes = (long) (limitBytes + (limitBytes * allowedOverflowRatio));
+
+      if (limitBytes > Integer.MAX_VALUE) {
+        this.limitBytes = Integer.MAX_VALUE;
+      } else {
+        this.limitBytes = limitBytes;
+      }
+    }
+
+    this.allowedOVerflowRatio = allowedOverflowRatio;
+    this.incRatio = incRatio;
+  }
+
+  public long initialSize() {
+    return initSize;
+  }
+
+  public long limit() {
+    return limitBytes;
+  }
+
+  public float remainRatio(long currentSize) {
+    Preconditions.checkArgument(currentSize > 0, "Size must be greater than 0 bytes.");
+    if (currentSize > Integer.MAX_VALUE) {
+      currentSize = Integer.MAX_VALUE;
+    }
+    return (float)currentSize / (float)limitBytes;
+  }
+
+  public boolean canIncrease(long currentSize) {
+    return remain(currentSize) > 0;
+  }
+
+  public long remain(long currentSize) {
+    Preconditions.checkArgument(currentSize > 0, "Size must be greater than 0 bytes.");
+    return limitBytes > Integer.MAX_VALUE ? Integer.MAX_VALUE - currentSize : limitBytes - currentSize;
+  }
+
+  public int increasedSize(int currentSize) {
+    if (currentSize < initSize) {
+      return (int) initSize;
+    }
+
+    if (currentSize > Integer.MAX_VALUE) {
+      LOG.warn("Current size already exceeds the maximum size (" + Integer.MAX_VALUE + " bytes)");
+      return Integer.MAX_VALUE;
+    }
+    long nextSize = (long) (currentSize + ((float) currentSize * incRatio));
+
+    if (nextSize > limitBytes) {
+      LOG.info("Increasing reaches size limit (" + FileUtil.humanReadableByteCount(limitBytes, false) + ")");
+      nextSize = limitBytes;
+    }
+
+    if (nextSize > Integer.MAX_VALUE) {
+      LOG.info("Increasing reaches maximum size (" + FileUtil.humanReadableByteCount(Integer.MAX_VALUE, false) + ")");
+      nextSize = Integer.MAX_VALUE;
+    }
+
+    return (int) nextSize;
+  }
+
+  @Override
+  public String toString() {
+    return "init=" + FileUtil.humanReadableByteCount(initSize, false) + ",limit="
+        + FileUtil.humanReadableByteCount(limitBytes, false) + ",overflow_ratio=" + allowedOVerflowRatio
+        + ",inc_ratio=" + incRatio;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
new file mode 100644
index 0000000..a2b2561
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/RowWriter.java
@@ -0,0 +1,73 @@
+/***
+ * 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.tuple.offheap;
+
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.IntervalDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+
+/**
+ * The call sequence should be as follows:
+ *
+ * <pre>
+ *   startRow() -->  skipField() or putXXX --> endRow()
+ * </pre>
+ *
+ * The total number of skipField and putXXX invocations must be equivalent to the number of fields.
+ */
+public interface RowWriter {
+
+  public TajoDataTypes.DataType [] dataTypes();
+
+  public boolean startRow();
+
+  public void endRow();
+
+  public void skipField();
+
+  public void putBool(boolean val);
+
+  public void putInt2(short val);
+
+  public void putInt4(int val);
+
+  public void putInt8(long val);
+
+  public void putFloat4(float val);
+
+  public void putFloat8(double val);
+
+  public void putText(String val);
+
+  public void putText(byte[] val);
+
+  public void putBlob(byte[] val);
+
+  public void putTimestamp(long val);
+
+  public void putTime(long val);
+
+  public void putDate(int val);
+
+  public void putInterval(IntervalDatum val);
+
+  public void putInet4(int val);
+
+  public void putProtoDatum(ProtobufDatum datum);
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
new file mode 100644
index 0000000..b742e6d
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTuple.java
@@ -0,0 +1,311 @@
+/***
+ * 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.tuple.offheap;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+
+import org.apache.tajo.datum.*;
+import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.util.SizeOf;
+import org.apache.tajo.util.StringUtils;
+import org.apache.tajo.util.UnsafeUtil;
+
+import sun.misc.Unsafe;
+import sun.nio.ch.DirectBuffer;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.Charset;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public abstract class UnSafeTuple implements Tuple {
+  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
+
+  private DirectBuffer bb;
+  private int relativePos;
+  private int length;
+  private DataType [] types;
+
+  protected void set(ByteBuffer bb, int relativePos, int length, DataType [] types) {
+    this.bb = (DirectBuffer) bb;
+    this.relativePos = relativePos;
+    this.length = length;
+    this.types = types;
+  }
+
+  void set(ByteBuffer bb, DataType [] types) {
+    set(bb, 0, bb.limit(), types);
+  }
+
+  @Override
+  public int size() {
+    return types.length;
+  }
+
+  public ByteBuffer nioBuffer() {
+    return ((ByteBuffer)((ByteBuffer)bb).duplicate().position(relativePos).limit(relativePos + length)).slice();
+  }
+
+  public HeapTuple toHeapTuple() {
+    byte [] bytes = new byte[length];
+    UNSAFE.copyMemory(null, bb.address() + relativePos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, length);
+    return new HeapTuple(bytes, types);
+  }
+
+  public void copyFrom(UnSafeTuple tuple) {
+    Preconditions.checkNotNull(tuple);
+
+    ((ByteBuffer) bb).clear();
+    if (length < tuple.length) {
+      UnsafeUtil.free((ByteBuffer) bb);
+      bb = (DirectBuffer) ByteBuffer.allocateDirect(tuple.length).order(ByteOrder.nativeOrder());
+      this.relativePos = 0;
+      this.length = tuple.length;
+    }
+
+    ((ByteBuffer) bb).put(tuple.nioBuffer());
+  }
+
+  private int getFieldOffset(int fieldId) {
+    return UNSAFE.getInt(bb.address() + relativePos + SizeOf.SIZE_OF_INT + (fieldId * SizeOf.SIZE_OF_INT));
+  }
+
+  public long getFieldAddr(int fieldId) {
+    int fieldOffset = getFieldOffset(fieldId);
+    if (fieldOffset == -1) {
+      throw new RuntimeException("Invalid Field Access: " + fieldId);
+    }
+    return bb.address() + relativePos + fieldOffset;
+  }
+
+  @Override
+  public boolean contains(int fieldid) {
+    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
+  }
+
+  @Override
+  public boolean isNull(int fieldid) {
+    return getFieldOffset(fieldid) == OffHeapRowBlock.NULL_FIELD_OFFSET;
+  }
+
+  @Override
+  public boolean isNotNull(int fieldid) {
+    return getFieldOffset(fieldid) > OffHeapRowBlock.NULL_FIELD_OFFSET;
+  }
+
+  @Override
+  public void clear() {
+    // nothing to do
+  }
+
+  @Override
+  public void put(int fieldId, Datum value) {
+    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum).");
+  }
+
+  @Override
+  public void put(int fieldId, Datum[] values) {
+    throw new UnsupportedException("UnSafeTuple does not support put(int, Datum []).");
+  }
+
+  @Override
+  public void put(int fieldId, Tuple tuple) {
+    throw new UnsupportedException("UnSafeTuple does not support put(int, Tuple).");
+  }
+
+  @Override
+  public void put(Datum[] values) {
+    throw new UnsupportedException("UnSafeTuple does not support put(Datum []).");
+  }
+
+  @Override
+  public Datum get(int fieldId) {
+    if (isNull(fieldId)) {
+      return NullDatum.get();
+    }
+
+    switch (types[fieldId].getType()) {
+    case BOOLEAN:
+      return DatumFactory.createBool(getBool(fieldId));
+    case INT1:
+    case INT2:
+      return DatumFactory.createInt2(getInt2(fieldId));
+    case INT4:
+      return DatumFactory.createInt4(getInt4(fieldId));
+    case INT8:
+      return DatumFactory.createInt8(getInt4(fieldId));
+    case FLOAT4:
+      return DatumFactory.createFloat4(getFloat4(fieldId));
+    case FLOAT8:
+      return DatumFactory.createFloat8(getFloat8(fieldId));
+    case TEXT:
+      return DatumFactory.createText(getText(fieldId));
+    case TIMESTAMP:
+      return DatumFactory.createTimestamp(getInt8(fieldId));
+    case DATE:
+      return DatumFactory.createDate(getInt4(fieldId));
+    case TIME:
+      return DatumFactory.createTime(getInt8(fieldId));
+    case INTERVAL:
+      return getInterval(fieldId);
+    case INET4:
+      return DatumFactory.createInet4(getInt4(fieldId));
+    case PROTOBUF:
+      return getProtobufDatum(fieldId);
+    default:
+      throw new UnsupportedException("Unknown type: " + types[fieldId]);
+    }
+  }
+
+  @Override
+  public void setOffset(long offset) {
+  }
+
+  @Override
+  public long getOffset() {
+    return 0;
+  }
+
+  @Override
+  public boolean getBool(int fieldId) {
+    return UNSAFE.getByte(getFieldAddr(fieldId)) == 0x01;
+  }
+
+  @Override
+  public byte getByte(int fieldId) {
+    return UNSAFE.getByte(getFieldAddr(fieldId));
+  }
+
+  @Override
+  public char getChar(int fieldId) {
+    return UNSAFE.getChar(getFieldAddr(fieldId));
+  }
+
+  @Override
+  public byte[] getBytes(int fieldId) {
+    long pos = getFieldAddr(fieldId);
+    int len = UNSAFE.getInt(pos);
+    pos += SizeOf.SIZE_OF_INT;
+
+    byte [] bytes = new byte[len];
+    UNSAFE.copyMemory(null, pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
+    return bytes;
+  }
+
+  @Override
+  public short getInt2(int fieldId) {
+    long addr = getFieldAddr(fieldId);
+    return UNSAFE.getShort(addr);
+  }
+
+  @Override
+  public int getInt4(int fieldId) {
+    return UNSAFE.getInt(getFieldAddr(fieldId));
+  }
+
+  @Override
+  public long getInt8(int fieldId) {
+    return UNSAFE.getLong(getFieldAddr(fieldId));
+  }
+
+  @Override
+  public float getFloat4(int fieldId) {
+    return UNSAFE.getFloat(getFieldAddr(fieldId));
+  }
+
+  @Override
+  public double getFloat8(int fieldId) {
+    return UNSAFE.getDouble(getFieldAddr(fieldId));
+  }
+
+  @Override
+  public String getText(int fieldId) {
+    long pos = getFieldAddr(fieldId);
+    int len = UNSAFE.getInt(pos);
+    pos += SizeOf.SIZE_OF_INT;
+
+    byte [] bytes = new byte[len];
+    UNSAFE.copyMemory(null, pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
+    return new String(bytes);
+  }
+
+  public IntervalDatum getInterval(int fieldId) {
+    long pos = getFieldAddr(fieldId);
+    int months = UNSAFE.getInt(pos);
+    pos += SizeOf.SIZE_OF_INT;
+    long millisecs = UNSAFE.getLong(pos);
+    return new IntervalDatum(months, millisecs);
+  }
+
+  @Override
+  public Datum getProtobufDatum(int fieldId) {
+    byte [] bytes = getBytes(fieldId);
+
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(types[fieldId].getCode());
+    Message.Builder builder = factory.newBuilder();
+    try {
+      builder.mergeFrom(bytes);
+    } catch (InvalidProtocolBufferException e) {
+      return NullDatum.get();
+    }
+
+    return new ProtobufDatum(builder.build());
+  }
+
+  @Override
+  public char[] getUnicodeChars(int fieldId) {
+    long pos = getFieldAddr(fieldId);
+    int len = UNSAFE.getInt(pos);
+    pos += SizeOf.SIZE_OF_INT;
+
+    byte [] bytes = new byte[len];
+    UNSAFE.copyMemory(null, pos, bytes, UnsafeUtil.ARRAY_BYTE_BASE_OFFSET, len);
+    return StringUtils.convertBytesToChars(bytes, Charset.forName("UTF-8"));
+  }
+
+  @Override
+  public Tuple clone() throws CloneNotSupportedException {
+    return toHeapTuple();
+  }
+
+  @Override
+  public Datum[] getValues() {
+    Datum [] datums = new Datum[size()];
+    for (int i = 0; i < size(); i++) {
+      if (contains(i)) {
+        datums[i] = get(i);
+      } else {
+        datums[i] = NullDatum.get();
+      }
+    }
+    return datums;
+  }
+
+  @Override
+  public String toString() {
+    return VTuple.toDisplayString(getValues());
+  }
+
+  public abstract void release();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
new file mode 100644
index 0000000..73e1e2f
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/UnSafeTupleBytesComparator.java
@@ -0,0 +1,99 @@
+/***
+ * 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.tuple.offheap;
+
+import com.google.common.primitives.Longs;
+import com.google.common.primitives.UnsignedLongs;
+import org.apache.tajo.util.SizeOf;
+import org.apache.tajo.util.UnsafeUtil;
+import sun.misc.Unsafe;
+
+import java.nio.ByteOrder;
+
+/**
+ * It directly access UTF bytes in UnSafeTuple without any copy. It is used by compiled TupleComparator.
+ */
+public class UnSafeTupleBytesComparator {
+  private static final Unsafe UNSAFE = UnsafeUtil.unsafe;
+
+  static final boolean littleEndian =
+      ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN);
+
+  public static int compare(long ptr1, long ptr2) {
+    int lstrLen = UNSAFE.getInt(ptr1);
+    int rstrLen = UNSAFE.getInt(ptr2);
+
+    ptr1 += SizeOf.SIZE_OF_INT;
+    ptr2 += SizeOf.SIZE_OF_INT;
+
+    int minLength = Math.min(lstrLen, rstrLen);
+    int minWords = minLength / Longs.BYTES;
+
+        /*
+         * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a
+         * time is no slower than comparing 4 bytes at a time even on 32-bit.
+         * On the other hand, it is substantially faster on 64-bit.
+         */
+    for (int i = 0; i < minWords * Longs.BYTES; i += Longs.BYTES) {
+      long lw = UNSAFE.getLong(ptr1);
+      long rw = UNSAFE.getLong(ptr2);
+      long diff = lw ^ rw;
+
+      if (diff != 0) {
+        if (!littleEndian) {
+          return UnsignedLongs.compare(lw, rw);
+        }
+
+        // Use binary search
+        int n = 0;
+        int y;
+        int x = (int) diff;
+        if (x == 0) {
+          x = (int) (diff >>> 32);
+          n = 32;
+        }
+
+        y = x << 16;
+        if (y == 0) {
+          n += 16;
+        } else {
+          x = y;
+        }
+
+        y = x << 8;
+        if (y == 0) {
+          n += 8;
+        }
+        return (int) (((lw >>> n) & 0xFFL) - ((rw >>> n) & 0xFFL));
+      }
+
+      ptr1 += SizeOf.SIZE_OF_LONG;
+      ptr2 += SizeOf.SIZE_OF_LONG;
+    }
+
+    // The epilogue to cover the last (minLength % 8) elements.
+    for (int i = minWords * Longs.BYTES; i < minLength; i++) {
+      int result = UNSAFE.getByte(ptr1++) - UNSAFE.getByte(ptr2++);
+      if (result != 0) {
+        return result;
+      }
+    }
+    return lstrLen - rstrLen;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
new file mode 100644
index 0000000..51dbb29
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/tuple/offheap/ZeroCopyTuple.java
@@ -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.
+ */
+
+package org.apache.tajo.tuple.offheap;
+
+import java.nio.ByteBuffer;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class ZeroCopyTuple extends UnSafeTuple {
+
+  public void set(ByteBuffer bb, int relativePos, int length, DataType [] types) {
+    super.set(bb, relativePos, length, types);
+  }
+
+  @Override
+  public void release() {
+    // nothing to do
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/proto/IndexProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/proto/IndexProtos.proto b/tajo-storage/tajo-storage-common/src/main/proto/IndexProtos.proto
new file mode 100644
index 0000000..f5c8a08
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/proto/IndexProtos.proto
@@ -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.
+ */
+
+option java_package = "org.apache.tajo.index";
+option java_outer_classname = "IndexProtos";
+option optimize_for = SPEED;
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
+
+import "CatalogProtos.proto";
+
+message TupleComparatorProto {
+  required SchemaProto schema = 1;
+  repeated SortSpecProto sortSpecs = 2;
+  repeated TupleComparatorSpecProto compSpecs = 3;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
new file mode 100644
index 0000000..47d11c7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
@@ -0,0 +1,184 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+  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.
+  -->
+
+<configuration>
+  <!-- Storage Manager Configuration -->
+  <property>
+    <name>tajo.storage.manager.hdfs.class</name>
+    <value>org.apache.tajo.storage.FileStorageManager</value>
+  </property>
+  <property>
+    <name>tajo.storage.manager.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.manager.concurrency.perDisk</name>
+    <value>1</value>
+    <description></description>
+  </property>
+
+  <!--- Registered Scanner Handler -->
+  <property>
+    <name>tajo.storage.scanner-handler</name>
+    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
+  </property>
+
+  <!--- Fragment Class Configurations -->
+  <property>
+    <name>tajo.storage.fragment.textfile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.csv.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.raw.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.rcfile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.row.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.parquet.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.sequencefile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.avro.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseFragment</value>
+  </property>
+
+  <!--- Scanner Handler -->
+  <property>
+    <name>tajo.storage.scanner-handler.textfile.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.csv.class</name>
+    <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.raw.class</name>
+    <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.rcfile.class</name>
+    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.rowfile.class</name>
+    <value>org.apache.tajo.storage.RowFile$RowFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.parquet.class</name>
+    <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.sequencefile.class</name>
+    <value>org.apache.tajo.storage.sequencefile.SequenceFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.avro.class</name>
+    <value>org.apache.tajo.storage.avro.AvroScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseScanner</value>
+  </property>
+  
+  <!--- Appender Handler -->
+  <property>
+    <name>tajo.storage.appender-handler</name>
+    <value>textfile,csv,raw,rcfile,row,parquet,sequencefile,avro,hbase</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.textfile.class</name>
+    <value>org.apache.tajo.storage.text.DelimitedTextFile$DelimitedTextFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.csv.class</name>
+    <value>org.apache.tajo.storage.CSVFile$CSVAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.raw.class</name>
+    <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.rcfile.class</name>
+    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.rowfile.class</name>
+    <value>org.apache.tajo.storage.RowFile$RowFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.parquet.class</name>
+    <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.sequencefile.class</name>
+    <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.avro.class</name>
+    <value>org.apache.tajo.storage.avro.AvroAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.hfile.class</name>
+    <value>org.apache.tajo.storage.hbase.HFileAppender</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
new file mode 100644
index 0000000..0251dc7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestFrameTuple {
+  private Tuple tuple1;
+  private Tuple tuple2;
+
+  @Before
+  public void setUp() throws Exception {
+    tuple1 = new VTuple(11);
+    tuple1.put(new Datum[] {
+        DatumFactory.createBool(true),
+        DatumFactory.createBit((byte) 0x99),
+        DatumFactory.createChar('9'),
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("hyunsik"),
+        DatumFactory.createBlob("hyunsik".getBytes()),
+        DatumFactory.createInet4("192.168.0.1")
+    });
+    
+    tuple2 = new VTuple(11);
+    tuple2.put(new Datum[] {
+        DatumFactory.createBool(true),
+        DatumFactory.createBit((byte) 0x99),
+        DatumFactory.createChar('9'),
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("hyunsik"),
+        DatumFactory.createBlob("hyunsik".getBytes()),
+        DatumFactory.createInet4("192.168.0.1")
+    });
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public final void testFrameTuple() {
+    Tuple frame = new FrameTuple(tuple1, tuple2);
+    assertEquals(22, frame.size());
+    for (int i = 0; i < 22; i++) {
+      assertTrue(frame.contains(i));
+    }
+    
+    assertEquals(DatumFactory.createInt8(23l), frame.get(5));
+    assertEquals(DatumFactory.createInt8(23l), frame.get(16));
+    assertEquals(DatumFactory.createInet4("192.168.0.1"), frame.get(10));
+    assertEquals(DatumFactory.createInet4("192.168.0.1"), frame.get(21));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
new file mode 100644
index 0000000..c6149f7
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
@@ -0,0 +1,258 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.util.BytesUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestLazyTuple {
+
+  Schema schema;
+  byte[][] textRow;
+  byte[] nullbytes;
+  SerializerDeserializer serde;
+
+  @Before
+  public void setUp() {
+    nullbytes = "\\N".getBytes();
+
+    schema = new Schema();
+    schema.addColumn("col1", TajoDataTypes.Type.BOOLEAN);
+    schema.addColumn("col2", TajoDataTypes.Type.BIT);
+    schema.addColumn("col3", TajoDataTypes.Type.CHAR, 7);
+    schema.addColumn("col4", TajoDataTypes.Type.INT2);
+    schema.addColumn("col5", TajoDataTypes.Type.INT4);
+    schema.addColumn("col6", TajoDataTypes.Type.INT8);
+    schema.addColumn("col7", TajoDataTypes.Type.FLOAT4);
+    schema.addColumn("col8", TajoDataTypes.Type.FLOAT8);
+    schema.addColumn("col9", TajoDataTypes.Type.TEXT);
+    schema.addColumn("col10", TajoDataTypes.Type.BLOB);
+    schema.addColumn("col11", TajoDataTypes.Type.INET4);
+    schema.addColumn("col12", TajoDataTypes.Type.INT4);
+    schema.addColumn("col13", TajoDataTypes.Type.NULL_TYPE);
+
+    StringBuilder sb = new StringBuilder();
+    sb.append(DatumFactory.createBool(true)).append('|');
+    sb.append(new String(DatumFactory.createBit((byte) 0x99).asTextBytes())).append('|');
+    sb.append(DatumFactory.createChar("str")).append('|');
+    sb.append(DatumFactory.createInt2((short) 17)).append('|');
+    sb.append(DatumFactory.createInt4(59)).append('|');
+    sb.append(DatumFactory.createInt8(23l)).append('|');
+    sb.append(DatumFactory.createFloat4(77.9f)).append('|');
+    sb.append(DatumFactory.createFloat8(271.9f)).append('|');
+    sb.append(DatumFactory.createText("str2")).append('|');
+    sb.append(DatumFactory.createBlob("jinho".getBytes())).append('|');
+    sb.append(DatumFactory.createInet4("192.168.0.1")).append('|');
+    sb.append(new String(nullbytes)).append('|');
+    sb.append(NullDatum.get());
+    textRow = BytesUtils.splitPreserveAllTokens(sb.toString().getBytes(), '|');
+    serde = new TextSerializerDeserializer();
+  }
+
+  @Test
+  public void testGetDatum() {
+
+    LazyTuple t1 = new LazyTuple(schema, textRow, -1, nullbytes, serde);
+    assertEquals(DatumFactory.createBool(true), t1.get(0));
+    assertEquals(DatumFactory.createBit((byte) 0x99), t1.get(1));
+    assertEquals(DatumFactory.createChar("str"), t1.get(2));
+    assertEquals(DatumFactory.createInt2((short) 17), t1.get(3));
+    assertEquals(DatumFactory.createInt4(59), t1.get(4));
+    assertEquals(DatumFactory.createInt8(23l), t1.get(5));
+    assertEquals(DatumFactory.createFloat4(77.9f), t1.get(6));
+    assertEquals(DatumFactory.createFloat8(271.9f), t1.get(7));
+    assertEquals(DatumFactory.createText("str2"), t1.get(8));
+    assertEquals(DatumFactory.createBlob("jinho".getBytes()), t1.get(9));
+    assertEquals(DatumFactory.createInet4("192.168.0.1"), t1.get(10));
+    assertEquals(NullDatum.get(), t1.get(11));
+    assertEquals(NullDatum.get(), t1.get(12));
+  }
+
+  @Test
+  public void testContain() {
+    int colNum = schema.size();
+
+    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(3, DatumFactory.createInt4(1));
+    t1.put(7, DatumFactory.createInt4(1));
+
+    assertTrue(t1.contains(0));
+    assertFalse(t1.contains(1));
+    assertFalse(t1.contains(2));
+    assertTrue(t1.contains(3));
+    assertFalse(t1.contains(4));
+    assertFalse(t1.contains(5));
+    assertFalse(t1.contains(6));
+    assertTrue(t1.contains(7));
+    assertFalse(t1.contains(8));
+    assertFalse(t1.contains(9));
+    assertFalse(t1.contains(10));
+    assertFalse(t1.contains(11));
+    assertFalse(t1.contains(12));
+  }
+
+  @Test
+  public void testPut() {
+    int colNum = schema.size();
+    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
+    t1.put(0, DatumFactory.createText("str"));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(11, DatumFactory.createFloat4(0.76f));
+
+    assertTrue(t1.contains(0));
+    assertTrue(t1.contains(1));
+
+    assertEquals(t1.getText(0), "str");
+    assertEquals(t1.get(1).asInt4(), 2);
+    assertTrue(t1.get(11).asFloat4() == 0.76f);
+  }
+
+  @Test
+  public void testEquals() {
+    int colNum = schema.size();
+    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
+    LazyTuple t2 = new LazyTuple(schema, new byte[colNum][], -1);
+
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(3, DatumFactory.createInt4(2));
+
+    t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(3, DatumFactory.createInt4(2));
+
+    assertEquals(t1, t2);
+
+    Tuple t3 = new VTuple(colNum);
+    t3.put(0, DatumFactory.createInt4(1));
+    t3.put(1, DatumFactory.createInt4(2));
+    t3.put(3, DatumFactory.createInt4(2));
+    assertEquals(t1, t3);
+    assertEquals(t2, t3);
+
+    LazyTuple t4 = new LazyTuple(schema, new byte[colNum][], -1);
+    assertNotSame(t1, t4);
+  }
+
+  @Test
+  public void testHashCode() {
+    int colNum = schema.size();
+    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
+    LazyTuple t2 = new LazyTuple(schema, new byte[colNum][], -1);
+
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(3, DatumFactory.createInt4(2));
+    t1.put(4, DatumFactory.createText("str"));
+
+    t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(3, DatumFactory.createInt4(2));
+    t2.put(4, DatumFactory.createText("str"));
+
+    assertEquals(t1.hashCode(), t2.hashCode());
+
+    Tuple t3 = new VTuple(colNum);
+    t3.put(0, DatumFactory.createInt4(1));
+    t3.put(1, DatumFactory.createInt4(2));
+    t3.put(3, DatumFactory.createInt4(2));
+    t3.put(4, DatumFactory.createText("str"));
+    assertEquals(t1.hashCode(), t3.hashCode());
+    assertEquals(t2.hashCode(), t3.hashCode());
+
+    Tuple t4 = new VTuple(5);
+    t4.put(0, DatumFactory.createInt4(1));
+    t4.put(1, DatumFactory.createInt4(2));
+    t4.put(4, DatumFactory.createInt4(2));
+
+    assertNotSame(t1.hashCode(), t4.hashCode());
+  }
+
+  @Test
+  public void testPutTuple() {
+    int colNum = schema.size();
+    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
+
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(2, DatumFactory.createInt4(3));
+
+
+    Schema schema2 = new Schema();
+    schema2.addColumn("col1", TajoDataTypes.Type.INT8);
+    schema2.addColumn("col2", TajoDataTypes.Type.INT8);
+
+    LazyTuple t2 = new LazyTuple(schema2, new byte[schema2.size()][], -1);
+    t2.put(0, DatumFactory.createInt4(4));
+    t2.put(1, DatumFactory.createInt4(5));
+
+    t1.put(3, t2);
+
+    for (int i = 0; i < 5; i++) {
+      assertEquals(i + 1, t1.get(i).asInt4());
+    }
+  }
+
+  @Test
+  public void testInvalidNumber() {
+    byte[][] bytes = BytesUtils.splitPreserveAllTokens(" 1| |2 ||".getBytes(), '|');
+    Schema schema = new Schema();
+    schema.addColumn("col1", TajoDataTypes.Type.INT2);
+    schema.addColumn("col2", TajoDataTypes.Type.INT4);
+    schema.addColumn("col3", TajoDataTypes.Type.INT8);
+    schema.addColumn("col4", TajoDataTypes.Type.FLOAT4);
+    schema.addColumn("col5", TajoDataTypes.Type.FLOAT8);
+
+    LazyTuple tuple = new LazyTuple(schema, bytes, 0);
+    assertEquals(bytes.length, tuple.size());
+
+    for (int i = 0; i < tuple.size(); i++){
+      assertEquals(NullDatum.get(), tuple.get(i));
+    }
+  }
+
+  @Test
+  public void testClone() throws CloneNotSupportedException {
+    int colNum = schema.size();
+    LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1);
+
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(3, DatumFactory.createInt4(2));
+    t1.put(4, DatumFactory.createText("str"));
+
+    LazyTuple t2 = (LazyTuple) t1.clone();
+    assertNotSame(t1, t2);
+    assertEquals(t1, t2);
+
+    assertSame(t1.get(4), t2.get(4));
+
+    t1.clear();
+    assertFalse(t1.equals(t2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestTupleComparator.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
new file mode 100644
index 0000000..639ca04
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTupleComparator {
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public final void testCompare() {
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.INT4);
+    schema.addColumn("col2", Type.INT4);
+    schema.addColumn("col3", Type.INT4);
+    schema.addColumn("col4", Type.INT4);
+    schema.addColumn("col5", Type.TEXT);
+    
+    Tuple tuple1 = new VTuple(5);
+    Tuple tuple2 = new VTuple(5);
+
+    tuple1.put(
+        new Datum[] {
+        DatumFactory.createInt4(9),
+        DatumFactory.createInt4(3),
+        DatumFactory.createInt4(33),
+        DatumFactory.createInt4(4),
+        DatumFactory.createText("abc")});
+    tuple2.put(
+        new Datum[] {
+        DatumFactory.createInt4(1),
+        DatumFactory.createInt4(25),
+        DatumFactory.createInt4(109),
+        DatumFactory.createInt4(4),
+        DatumFactory.createText("abd")});
+
+    SortSpec sortKey1 = new SortSpec(schema.getColumn("col4"), true, false);
+    SortSpec sortKey2 = new SortSpec(schema.getColumn("col5"), true, false);
+
+    BaseTupleComparator tc = new BaseTupleComparator(schema,
+        new SortSpec[] {sortKey1, sortKey2});
+    assertEquals(-1, tc.compare(tuple1, tuple2));
+    assertEquals(1, tc.compare(tuple2, tuple1));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestVTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestVTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestVTuple.java
new file mode 100644
index 0000000..1bbd9ec
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestVTuple.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+
+import org.apache.tajo.datum.DatumFactory;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestVTuple {
+
+	/**
+	 * @throws Exception
+	 */
+	@Before
+	public void setUp() throws Exception {
+		
+	}
+	
+	@Test
+	public void testContain() {
+		VTuple t1 = new VTuple(260);
+		t1.put(0, DatumFactory.createInt4(1));
+		t1.put(1, DatumFactory.createInt4(1));
+		t1.put(27, DatumFactory.createInt4(1));
+		t1.put(96, DatumFactory.createInt4(1));
+		t1.put(257, DatumFactory.createInt4(1));
+		
+		assertTrue(t1.contains(0));
+		assertTrue(t1.contains(1));
+		assertFalse(t1.contains(2));
+		assertFalse(t1.contains(3));
+		assertFalse(t1.contains(4));
+		assertTrue(t1.contains(27));
+		assertFalse(t1.contains(28));
+		assertFalse(t1.contains(95));
+		assertTrue(t1.contains(96));
+		assertFalse(t1.contains(97));
+		assertTrue(t1.contains(257));
+	}
+	
+	@Test
+	public void testPut() {
+		VTuple t1 = new VTuple(260);
+		t1.put(0, DatumFactory.createText("str"));
+		t1.put(1, DatumFactory.createInt4(2));
+		t1.put(257, DatumFactory.createFloat4(0.76f));
+		
+		assertTrue(t1.contains(0));
+		assertTrue(t1.contains(1));
+		
+		assertEquals(t1.getText(0),"str");
+		assertEquals(t1.get(1).asInt4(),2);
+		assertTrue(t1.get(257).asFloat4() == 0.76f);
+	}
+
+  @Test
+	public void testEquals() {
+	  Tuple t1 = new VTuple(5);
+	  Tuple t2 = new VTuple(5);
+	  
+	  t1.put(0, DatumFactory.createInt4(1));
+	  t1.put(1, DatumFactory.createInt4(2));
+	  t1.put(3, DatumFactory.createInt4(2));
+	  
+	  t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(3, DatumFactory.createInt4(2));
+    
+    assertEquals(t1,t2);
+    
+    Tuple t3 = new VTuple(5);
+    t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(4, DatumFactory.createInt4(2));
+    
+    assertNotSame(t1,t3);
+	}
+	
+	@Test
+	public void testHashCode() {
+	  Tuple t1 = new VTuple(5);
+    Tuple t2 = new VTuple(5);
+    
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(3, DatumFactory.createInt4(2));
+    t1.put(4, DatumFactory.createText("hyunsik"));
+    
+    t2.put(0, DatumFactory.createInt4(1));
+    t2.put(1, DatumFactory.createInt4(2));
+    t2.put(3, DatumFactory.createInt4(2));
+    t2.put(4, DatumFactory.createText("hyunsik"));
+    
+    assertEquals(t1.hashCode(),t2.hashCode());
+    
+    Tuple t3 = new VTuple(5);
+    t3.put(0, DatumFactory.createInt4(1));
+    t3.put(1, DatumFactory.createInt4(2));
+    t3.put(4, DatumFactory.createInt4(2));
+    
+    assertNotSame(t1.hashCode(),t3.hashCode());
+	}
+
+  @Test
+  public void testPutTuple() {
+    Tuple t1 = new VTuple(5);
+
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(2, DatumFactory.createInt4(3));
+
+    Tuple t2 = new VTuple(2);
+    t2.put(0, DatumFactory.createInt4(4));
+    t2.put(1, DatumFactory.createInt4(5));
+
+    t1.put(3, t2);
+
+    for (int i = 0; i < 5; i++) {
+      assertEquals(i+1, t1.get(i).asInt4());
+    }
+  }
+
+  @Test
+  public void testClone() throws CloneNotSupportedException {
+    Tuple t1 = new VTuple(5);
+
+    t1.put(0, DatumFactory.createInt4(1));
+    t1.put(1, DatumFactory.createInt4(2));
+    t1.put(3, DatumFactory.createInt4(2));
+    t1.put(4, DatumFactory.createText("str"));
+
+    VTuple t2 = (VTuple) t1.clone();
+    assertNotSame(t1, t2);
+    assertEquals(t1, t2);
+
+    assertSame(t1.get(4), t2.get(4));
+
+    t1.clear();
+    assertFalse(t1.equals(t2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
new file mode 100644
index 0000000..d1c561b
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
@@ -0,0 +1,164 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+  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.
+  -->
+
+<configuration>
+  <property>
+    <name>fs.s3.impl</name>
+    <value>org.apache.tajo.storage.s3.SmallBlockS3FileSystem</value>
+  </property>
+
+  <!-- Storage Manager Configuration -->
+  <property>
+    <name>tajo.storage.manager.hdfs.class</name>
+    <value>org.apache.tajo.storage.FileStorageManager</value>
+  </property>
+  <property>
+    <name>tajo.storage.manager.hbase.class</name>
+    <value>org.apache.tajo.storage.hbase.HBaseStorageManager</value>
+  </property>
+
+  <!--- Registered Scanner Handler -->
+  <property>
+    <name>tajo.storage.scanner-handler</name>
+    <value>csv,raw,rcfile,row,trevni,parquet,sequencefile,avro</value>
+  </property>
+
+  <!--- Fragment Class Configurations -->
+  <property>
+    <name>tajo.storage.fragment.csv.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.raw.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.rcfile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.row.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.trevni.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.parquet.class</name>
+    <value>org.apache.tajo.storage.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.sequencefile.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+  <property>
+    <name>tajo.storage.fragment.avro.class</name>
+    <value>org.apache.tajo.storage.fragment.FileFragment</value>
+  </property>
+
+  <!--- Scanner Handler -->
+  <property>
+    <name>tajo.storage.scanner-handler.csv.class</name>
+    <value>org.apache.tajo.storage.CSVFile$CSVScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.raw.class</name>
+    <value>org.apache.tajo.storage.RawFile$RawFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.rcfile.class</name>
+    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.rowfile.class</name>
+    <value>org.apache.tajo.storage.RowFile$RowFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.trevni.class</name>
+    <value>org.apache.tajo.storage.trevni.TrevniScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.parquet.class</name>
+    <value>org.apache.tajo.storage.parquet.ParquetScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.sequencefile.class</name>
+    <value>org.apache.tajo.storage.sequencefile.SequenceFileScanner</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.scanner-handler.avro.class</name>
+    <value>org.apache.tajo.storage.avro.AvroScanner</value>
+  </property>
+
+  <!--- Appender Handler -->
+  <property>
+    <name>tajo.storage.appender-handler</name>
+    <value>csv,raw,rcfile,row,trevni,parquet,sequencefile,avro</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.csv.class</name>
+    <value>org.apache.tajo.storage.CSVFile$CSVAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.raw.class</name>
+    <value>org.apache.tajo.storage.RawFile$RawFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.rcfile.class</name>
+    <value>org.apache.tajo.storage.rcfile.RCFile$RCFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.rowfile.class</name>
+    <value>org.apache.tajo.storage.RowFile$RowFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.trevni.class</name>
+    <value>org.apache.tajo.storage.trevni.TrevniAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.parquet.class</name>
+    <value>org.apache.tajo.storage.parquet.ParquetAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.sequencefile.class</name>
+    <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
+  </property>
+
+  <property>
+    <name>tajo.storage.appender-handler.avro.class</name>
+    <value>org.apache.tajo.storage.avro.AvroAppender</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..e37149d
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/pom.xml
@@ -0,0 +1,349 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Copyright 2012 Database Lab., Korea Univ.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.9.1-SNAPSHOT</version>
+    <relativePath>../../tajo-project</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>tajo-storage-hbase</artifactId>
+  <packaging>jar</packaging>
+  <name>Tajo HBase Storage</name>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+  </properties>
+
+  <repositories>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>https://repository.jboss.org/nexus/content/repositories/releases/
+      </url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+          <encoding>${project.build.sourceEncoding}</encoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <tajo.test>TRUE</tajo.test>
+          </systemProperties>
+          <argLine>-Xms512m -Xmx1024m -XX:MaxPermSize=128m -Dfile.encoding=UTF-8</argLine>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/StorageFragmentProtos.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-plan</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>slf4j-api</artifactId>
+          <groupId>org.slf4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>jersey-json</artifactId>
+          <groupId>com.sun.jersey</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-2.1-jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-server-tests</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-app</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-yarn-api</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-hs</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+      <version>${hbase.version}</version>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-report-plugin</artifactId>
+        <version>2.15</version>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..8615235
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AbstractHBaseAppender.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Appender;
+import org.apache.tajo.storage.TableStatistics;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.TUtil;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An abstract class for HBase appender.
+ */
+public abstract class AbstractHBaseAppender implements Appender {
+  protected Configuration conf;
+  protected Schema schema;
+  protected TableMeta meta;
+  protected QueryUnitAttemptId taskAttemptId;
+  protected Path stagingDir;
+  protected boolean inited = false;
+
+  protected ColumnMapping columnMapping;
+  protected TableStatistics stats;
+  protected boolean enabledStats;
+
+  protected int columnNum;
+
+  protected byte[][][] mappingColumnFamilies;
+  protected boolean[] isBinaryColumns;
+  protected boolean[] isRowKeyMappings;
+  protected boolean[] isColumnKeys;
+  protected boolean[] isColumnValues;
+  protected int[] rowKeyFieldIndexes;
+  protected int[] rowkeyColumnIndexes;
+  protected char rowKeyDelimiter;
+
+  // the following four variables are used for '<cfname>:key:' or '<cfname>:value:' mapping
+  protected int[] columnKeyValueDataIndexes;
+  protected byte[][] columnKeyDatas;
+  protected byte[][] columnValueDatas;
+  protected byte[][] columnKeyCfNames;
+
+  protected KeyValue[] keyValues;
+
+  public AbstractHBaseAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
+                       Schema schema, TableMeta meta, Path stagingDir) {
+    this.conf = conf;
+    this.schema = schema;
+    this.meta = meta;
+    this.stagingDir = stagingDir;
+    this.taskAttemptId = taskAttemptId;
+  }
+
+  @Override
+  public void init() throws IOException {
+    if (inited) {
+      throw new IllegalStateException("FileAppender is already initialized.");
+    }
+    inited = true;
+    if (enabledStats) {
+      stats = new TableStatistics(this.schema);
+    }
+    columnMapping = new ColumnMapping(schema, meta);
+
+    mappingColumnFamilies = columnMapping.getMappingColumns();
+
+    isRowKeyMappings = columnMapping.getIsRowKeyMappings();
+    List<Integer> rowkeyColumnIndexList = new ArrayList<Integer>();
+    for (int i = 0; i < isRowKeyMappings.length; i++) {
+      if (isRowKeyMappings[i]) {
+        rowkeyColumnIndexList.add(i);
+      }
+    }
+    rowkeyColumnIndexes = TUtil.toArray(rowkeyColumnIndexList);
+
+    isBinaryColumns = columnMapping.getIsBinaryColumns();
+    isColumnKeys = columnMapping.getIsColumnKeys();
+    isColumnValues = columnMapping.getIsColumnValues();
+    rowKeyDelimiter = columnMapping.getRowKeyDelimiter();
+    rowKeyFieldIndexes = columnMapping.getRowKeyFieldIndexes();
+
+    this.columnNum = schema.size();
+
+    // In the case of '<cfname>:key:' or '<cfname>:value:' KeyValue object should be set with the qualifier and value
+    // which are mapped to the same column family.
+    columnKeyValueDataIndexes = new int[isColumnKeys.length];
+    int index = 0;
+    int numKeyValues = 0;
+    Map<String, Integer> cfNameIndexMap = new HashMap<String, Integer>();
+    for (int i = 0; i < isColumnKeys.length; i++) {
+      if (isRowKeyMappings[i]) {
+        continue;
+      }
+      if (isColumnKeys[i] || isColumnValues[i]) {
+        String cfName = new String(mappingColumnFamilies[i][0]);
+        if (!cfNameIndexMap.containsKey(cfName)) {
+          cfNameIndexMap.put(cfName, index);
+          columnKeyValueDataIndexes[i] = index;
+          index++;
+          numKeyValues++;
+        } else {
+          columnKeyValueDataIndexes[i] = cfNameIndexMap.get(cfName);
+        }
+      } else {
+        numKeyValues++;
+      }
+    }
+    columnKeyCfNames = new byte[cfNameIndexMap.size()][];
+    for (Map.Entry<String, Integer> entry: cfNameIndexMap.entrySet()) {
+      columnKeyCfNames[entry.getValue()] = entry.getKey().getBytes();
+    }
+    columnKeyDatas = new byte[cfNameIndexMap.size()][];
+    columnValueDatas = new byte[cfNameIndexMap.size()][];
+
+    keyValues = new KeyValue[numKeyValues];
+  }
+
+  private ByteArrayOutputStream bout = new ByteArrayOutputStream();
+
+  protected byte[] getRowKeyBytes(Tuple tuple) throws IOException {
+    Datum datum;
+    byte[] rowkey;
+    if (rowkeyColumnIndexes.length > 1) {
+      bout.reset();
+      for (int i = 0; i < rowkeyColumnIndexes.length; i++) {
+        datum = tuple.get(rowkeyColumnIndexes[i]);
+        if (isBinaryColumns[rowkeyColumnIndexes[i]]) {
+          rowkey = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(rowkeyColumnIndexes[i]), datum);
+        } else {
+          rowkey = HBaseTextSerializerDeserializer.serialize(schema.getColumn(rowkeyColumnIndexes[i]), datum);
+        }
+        bout.write(rowkey);
+        if (i < rowkeyColumnIndexes.length - 1) {
+          bout.write(rowKeyDelimiter);
+        }
+      }
+      rowkey = bout.toByteArray();
+    } else {
+      int index = rowkeyColumnIndexes[0];
+      datum = tuple.get(index);
+      if (isBinaryColumns[index]) {
+        rowkey = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(index), datum);
+      } else {
+        rowkey = HBaseTextSerializerDeserializer.serialize(schema.getColumn(index), datum);
+      }
+    }
+
+    return rowkey;
+  }
+
+  protected void readKeyValues(Tuple tuple, byte[] rowkey) throws IOException {
+    int keyValIndex = 0;
+    for (int i = 0; i < columnNum; i++) {
+      if (isRowKeyMappings[i]) {
+        continue;
+      }
+      Datum datum = tuple.get(i);
+      byte[] value;
+      if (isBinaryColumns[i]) {
+        value = HBaseBinarySerializerDeserializer.serialize(schema.getColumn(i), datum);
+      } else {
+        value = HBaseTextSerializerDeserializer.serialize(schema.getColumn(i), datum);
+      }
+
+      if (isColumnKeys[i]) {
+        columnKeyDatas[columnKeyValueDataIndexes[i]] = value;
+      } else if (isColumnValues[i]) {
+        columnValueDatas[columnKeyValueDataIndexes[i]] = value;
+      } else {
+        keyValues[keyValIndex] = new KeyValue(rowkey, mappingColumnFamilies[i][0], mappingColumnFamilies[i][1], value);
+        keyValIndex++;
+      }
+    }
+
+    for (int i = 0; i < columnKeyDatas.length; i++) {
+      keyValues[keyValIndex++] = new KeyValue(rowkey, columnKeyCfNames[i], columnKeyDatas[i], columnValueDatas[i]);
+    }
+  }
+
+  @Override
+  public void enableStats() {
+    enabledStats = true;
+  }
+
+  @Override
+  public TableStats getStats() {
+    if (enabledStats) {
+      return stats.getTableStat();
+    } else {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..79161cc
--- /dev/null
+++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/AddSortForInsertRewriter.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.hbase;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.plan.logical.SortNode;
+import org.apache.tajo.plan.logical.SortNode.SortPurpose;
+import org.apache.tajo.plan.logical.UnaryNode;
+import org.apache.tajo.plan.rewrite.RewriteRule;
+import org.apache.tajo.plan.util.PlannerUtil;
+
+public class AddSortForInsertRewriter implements RewriteRule {
+  private int[] sortColumnIndexes;
+  private Column[] sortColumns;
+  public AddSortForInsertRewriter(TableDesc tableDesc, Column[] sortColumns) {
+    this.sortColumns = sortColumns;
+    this.sortColumnIndexes = new int[sortColumns.length];
+
+    Schema tableSchema = tableDesc.getSchema();
+    for (int i = 0; i < sortColumns.length; i++) {
+      sortColumnIndexes[i] = tableSchema.getColumnId(sortColumns[i].getQualifiedName());
+    }
+  }
+
+  @Override
+  public String getName() {
+    return "AddSortForInsertRewriter";
+  }
+
+  @Override
+  public boolean isEligible(LogicalPlan plan) {
+    StoreType storeType = PlannerUtil.getStoreType(plan);
+    return storeType != null;
+  }
+
+  @Override
+  public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
+    LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+    UnaryNode insertNode = rootNode.getChild();
+    LogicalNode childNode = insertNode.getChild();
+
+    Schema sortSchema = childNode.getOutSchema();
+    SortNode sortNode = plan.createNode(SortNode.class);
+    sortNode.setSortPurpose(SortPurpose.STORAGE_SPECIFIED);
+    sortNode.setInSchema(sortSchema);
+    sortNode.setOutSchema(sortSchema);
+
+    SortSpec[] sortSpecs = new SortSpec[sortColumns.length];
+    int index = 0;
+
+    for (int i = 0; i < sortColumnIndexes.length; i++) {
+      Column sortColumn = sortSchema.getColumn(sortColumnIndexes[i]);
+      if (sortColumn == null) {
+        throw new PlanningException("Can't fine proper sort column:" + sortColumns[i]);
+      }
+      sortSpecs[index++] = new SortSpec(sortColumn, true, true);
+    }
+    sortNode.setSortSpecs(sortSpecs);
+
+    sortNode.setChild(insertNode.getChild());
+    insertNode.setChild(sortNode);
+    plan.getRootBlock().registerNode(sortNode);
+
+    return plan;
+  }
+}


[02/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..4081a80
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -0,0 +1,867 @@
+/**
+ * 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.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.ProtobufDatumFactory;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.rcfile.RCFile;
+import org.apache.tajo.storage.sequencefile.SequenceFileScanner;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.KeyValueSet;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class TestStorages {
+	private TajoConf conf;
+	private static String TEST_PATH = "target/test-data/TestStorages";
+
+  private static String TEST_PROJECTION_AVRO_SCHEMA =
+      "{\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"namespace\": \"org.apache.tajo\",\n" +
+      "  \"name\": \"testProjection\",\n" +
+      "  \"fields\": [\n" +
+      "    { \"name\": \"id\", \"type\": \"int\" },\n" +
+      "    { \"name\": \"age\", \"type\": \"long\" },\n" +
+      "    { \"name\": \"score\", \"type\": \"float\" }\n" +
+      "  ]\n" +
+      "}\n";
+
+  private static String TEST_NULL_HANDLING_TYPES_AVRO_SCHEMA =
+      "{\n" +
+      "  \"type\": \"record\",\n" +
+      "  \"namespace\": \"org.apache.tajo\",\n" +
+      "  \"name\": \"testNullHandlingTypes\",\n" +
+      "  \"fields\": [\n" +
+      "    { \"name\": \"col1\", \"type\": [\"null\", \"boolean\"] },\n" +
+      "    { \"name\": \"col2\", \"type\": [\"null\", \"int\"] },\n" +
+      "    { \"name\": \"col3\", \"type\": [\"null\", \"string\"] },\n" +
+      "    { \"name\": \"col4\", \"type\": [\"null\", \"int\"] },\n" +
+      "    { \"name\": \"col5\", \"type\": [\"null\", \"int\"] },\n" +
+      "    { \"name\": \"col6\", \"type\": [\"null\", \"long\"] },\n" +
+      "    { \"name\": \"col7\", \"type\": [\"null\", \"float\"] },\n" +
+      "    { \"name\": \"col8\", \"type\": [\"null\", \"double\"] },\n" +
+      "    { \"name\": \"col9\", \"type\": [\"null\", \"string\"] },\n" +
+      "    { \"name\": \"col10\", \"type\": [\"null\", \"bytes\"] },\n" +
+      "    { \"name\": \"col11\", \"type\": [\"null\", \"bytes\"] },\n" +
+      "    { \"name\": \"col12\", \"type\": \"null\" },\n" +
+      "    { \"name\": \"col13\", \"type\": [\"null\", \"bytes\"] }\n" +
+      "  ]\n" +
+      "}\n";
+
+  private StoreType storeType;
+  private boolean splitable;
+  private boolean statsable;
+  private boolean seekable;
+  private Path testDir;
+  private FileSystem fs;
+
+  public TestStorages(StoreType type, boolean splitable, boolean statsable, boolean seekable) throws IOException {
+    this.storeType = type;
+    this.splitable = splitable;
+    this.statsable = statsable;
+    this.seekable = seekable;
+
+    conf = new TajoConf();
+
+    if (storeType == StoreType.RCFILE) {
+      conf.setInt(RCFile.RECORD_INTERVAL_CONF_STR, 100);
+    }
+
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return Arrays.asList(new Object[][] {
+        //type, splitable, statsable, seekable
+        {StoreType.CSV, true, true, true},
+        {StoreType.RAW, false, true, true},
+        {StoreType.RCFILE, true, true, false},
+        {StoreType.PARQUET, false, false, false},
+        {StoreType.SEQUENCEFILE, true, true, false},
+        {StoreType.AVRO, false, false, false},
+        {StoreType.TEXTFILE, true, true, false},
+    });
+  }
+
+	@Test
+  public void testSplitable() throws IOException {
+    if (splitable) {
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age", Type.INT8);
+
+      TableMeta meta = CatalogUtil.newTableMeta(storeType);
+      Path tablePath = new Path(testDir, "Splitable.data");
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+      Appender appender = sm.getAppender(meta, schema, tablePath);
+      appender.enableStats();
+      appender.init();
+      int tupleNum = 10000;
+      VTuple vTuple;
+
+      for (int i = 0; i < tupleNum; i++) {
+        vTuple = new VTuple(2);
+        vTuple.put(0, DatumFactory.createInt4(i + 1));
+        vTuple.put(1, DatumFactory.createInt8(25l));
+        appender.addTuple(vTuple);
+      }
+      appender.close();
+      TableStats stat = appender.getStats();
+      assertEquals(tupleNum, stat.getNumRows().longValue());
+
+      FileStatus status = fs.getFileStatus(tablePath);
+      long fileLen = status.getLen();
+      long randomNum = (long) (Math.random() * fileLen) + 1;
+
+      FileFragment[] tablets = new FileFragment[2];
+      tablets[0] = new FileFragment("Splitable", tablePath, 0, randomNum);
+      tablets[1] = new FileFragment("Splitable", tablePath, randomNum, (fileLen - randomNum));
+
+      Scanner scanner = sm.getScanner(meta, schema, tablets[0], schema);
+      assertTrue(scanner.isSplittable());
+      scanner.init();
+      int tupleCnt = 0;
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      scanner = sm.getScanner(meta, schema, tablets[1], schema);
+      assertTrue(scanner.isSplittable());
+      scanner.init();
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      assertEquals(tupleNum, tupleCnt);
+    }
+	}
+
+  @Test
+  public void testRCFileSplitable() throws IOException {
+    if (storeType == StoreType.RCFILE) {
+      Schema schema = new Schema();
+      schema.addColumn("id", Type.INT4);
+      schema.addColumn("age", Type.INT8);
+
+      TableMeta meta = CatalogUtil.newTableMeta(storeType);
+      Path tablePath = new Path(testDir, "Splitable.data");
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+      Appender appender = sm.getAppender(meta, schema, tablePath);
+      appender.enableStats();
+      appender.init();
+      int tupleNum = 10000;
+      VTuple vTuple;
+
+      for (int i = 0; i < tupleNum; i++) {
+        vTuple = new VTuple(2);
+        vTuple.put(0, DatumFactory.createInt4(i + 1));
+        vTuple.put(1, DatumFactory.createInt8(25l));
+        appender.addTuple(vTuple);
+      }
+      appender.close();
+      TableStats stat = appender.getStats();
+      assertEquals(tupleNum, stat.getNumRows().longValue());
+
+      FileStatus status = fs.getFileStatus(tablePath);
+      long fileLen = status.getLen();
+      long randomNum = 122; // header size
+
+      FileFragment[] tablets = new FileFragment[2];
+      tablets[0] = new FileFragment("Splitable", tablePath, 0, randomNum);
+      tablets[1] = new FileFragment("Splitable", tablePath, randomNum, (fileLen - randomNum));
+
+      Scanner scanner = sm.getScanner(meta, schema, tablets[0], schema);
+      assertTrue(scanner.isSplittable());
+      scanner.init();
+      int tupleCnt = 0;
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      scanner = sm.getScanner(meta, schema, tablets[1], schema);
+      assertTrue(scanner.isSplittable());
+      scanner.init();
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      assertEquals(tupleNum, tupleCnt);
+    }
+  }
+
+  @Test
+  public void testProjection() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.INT8);
+    schema.addColumn("score", Type.FLOAT4);
+
+    TableMeta meta = CatalogUtil.newTableMeta(storeType);
+    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
+    if (storeType == StoreType.AVRO) {
+      meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL,
+                     TEST_PROJECTION_AVRO_SCHEMA);
+    }
+
+    Path tablePath = new Path(testDir, "testProjection.data");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+    appender.init();
+    int tupleNum = 10000;
+    VTuple vTuple;
+
+    for (int i = 0; i < tupleNum; i++) {
+      vTuple = new VTuple(3);
+      vTuple.put(0, DatumFactory.createInt4(i + 1));
+      vTuple.put(1, DatumFactory.createInt8(i + 2));
+      vTuple.put(2, DatumFactory.createFloat4(i + 3));
+      appender.addTuple(vTuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    FileFragment fragment = new FileFragment("testReadAndWrite", tablePath, 0, status.getLen());
+
+    Schema target = new Schema();
+    target.addColumn("age", Type.INT8);
+    target.addColumn("score", Type.FLOAT4);
+    Scanner scanner = sm.getScanner(meta, schema, fragment, target);
+    scanner.init();
+    int tupleCnt = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      if (storeType == StoreType.RCFILE
+          || storeType == StoreType.CSV
+          || storeType == StoreType.PARQUET
+          || storeType == StoreType.SEQUENCEFILE
+          || storeType == StoreType.AVRO) {
+        assertTrue(tuple.get(0) == null);
+      }
+      assertTrue(tupleCnt + 2 == tuple.get(1).asInt8());
+      assertTrue(tupleCnt + 3 == tuple.get(2).asFloat4());
+      tupleCnt++;
+    }
+    scanner.close();
+
+    assertEquals(tupleNum, tupleCnt);
+  }
+
+  @Test
+  public void testVariousTypes() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR, 7);
+    schema.addColumn("col4", Type.INT2);
+    schema.addColumn("col5", Type.INT4);
+    schema.addColumn("col6", Type.INT8);
+    schema.addColumn("col7", Type.FLOAT4);
+    schema.addColumn("col8", Type.FLOAT8);
+    schema.addColumn("col9", Type.TEXT);
+    schema.addColumn("col10", Type.BLOB);
+    schema.addColumn("col11", Type.INET4);
+    schema.addColumn("col12", Type.NULL_TYPE);
+    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
+    if (storeType == StoreType.AVRO) {
+      String path = FileUtil.getResourcePath("testVariousTypes.avsc").toString();
+      meta.putOption(StorageConstants.AVRO_SCHEMA_URL, path);
+    }
+
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+    appender.init();
+
+    QueryId queryid = new QueryId("12345", 5);
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
+
+    Tuple tuple = new VTuple(13);
+    tuple.put(new Datum[] {
+        DatumFactory.createBool(true),
+        DatumFactory.createBit((byte) 0x99),
+        DatumFactory.createChar("hyunsik"),
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("hyunsik"),
+        DatumFactory.createBlob("hyunsik".getBytes()),
+        DatumFactory.createInet4("192.168.0.1"),
+        NullDatum.get(),
+        factory.createDatum(queryid.getProto())
+    });
+    appender.addTuple(tuple);
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  sm.getScanner(meta, schema, fragment);
+    scanner.init();
+
+    Tuple retrieved;
+    while ((retrieved = scanner.next()) != null) {
+      for (int i = 0; i < tuple.size(); i++) {
+        assertEquals(tuple.get(i), retrieved.get(i));
+      }
+    }
+    scanner.close();
+  }
+
+  @Test
+  public void testNullHandlingTypes() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR, 7);
+    schema.addColumn("col4", Type.INT2);
+    schema.addColumn("col5", Type.INT4);
+    schema.addColumn("col6", Type.INT8);
+    schema.addColumn("col7", Type.FLOAT4);
+    schema.addColumn("col8", Type.FLOAT8);
+    schema.addColumn("col9", Type.TEXT);
+    schema.addColumn("col10", Type.BLOB);
+    schema.addColumn("col11", Type.INET4);
+    schema.addColumn("col12", Type.NULL_TYPE);
+    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.setOptions(CatalogUtil.newPhysicalProperties(storeType));
+    meta.putOption(StorageConstants.TEXT_NULL, "\\\\N");
+    meta.putOption(StorageConstants.RCFILE_NULL, "\\\\N");
+    meta.putOption(StorageConstants.RCFILE_SERDE, TextSerializerDeserializer.class.getName());
+    meta.putOption(StorageConstants.SEQUENCEFILE_NULL, "\\");
+    if (storeType == StoreType.AVRO) {
+      meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL,
+                     TEST_NULL_HANDLING_TYPES_AVRO_SCHEMA);
+    }
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+    appender.init();
+
+    QueryId queryid = new QueryId("12345", 5);
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
+
+    Tuple seedTuple = new VTuple(13);
+    seedTuple.put(new Datum[]{
+        DatumFactory.createBool(true),                // 0
+        DatumFactory.createBit((byte) 0x99),          // 1
+        DatumFactory.createChar("hyunsik"),           // 2
+        DatumFactory.createInt2((short) 17),          // 3
+        DatumFactory.createInt4(59),                  // 4
+        DatumFactory.createInt8(23l),                 // 5
+        DatumFactory.createFloat4(77.9f),             // 6
+        DatumFactory.createFloat8(271.9f),            // 7
+        DatumFactory.createText("hyunsik"),           // 8
+        DatumFactory.createBlob("hyunsik".getBytes()),// 9
+        DatumFactory.createInet4("192.168.0.1"),      // 10
+        NullDatum.get(),                              // 11
+        factory.createDatum(queryid.getProto())       // 12
+    });
+
+    // Making tuples with different null column positions
+    Tuple tuple;
+    for (int i = 0; i < 13; i++) {
+      tuple = new VTuple(13);
+      for (int j = 0; j < 13; j++) {
+        if (i == j) { // i'th column will have NULL value
+          tuple.put(j, NullDatum.get());
+        } else {
+          tuple.put(j, seedTuple.get(j));
+        }
+      }
+      appender.addTuple(tuple);
+    }
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    Tuple retrieved;
+    int i = 0;
+    while ((retrieved = scanner.next()) != null) {
+      assertEquals(13, retrieved.size());
+      for (int j = 0; j < 13; j++) {
+        if (i == j) {
+          assertEquals(NullDatum.get(), retrieved.get(j));
+        } else {
+          assertEquals(seedTuple.get(j), retrieved.get(j));
+        }
+      }
+
+      i++;
+    }
+    scanner.close();
+  }
+
+  @Test
+  public void testRCFileTextSerializeDeserialize() throws IOException {
+    if(storeType != StoreType.RCFILE) return;
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR, 7);
+    schema.addColumn("col4", Type.INT2);
+    schema.addColumn("col5", Type.INT4);
+    schema.addColumn("col6", Type.INT8);
+    schema.addColumn("col7", Type.FLOAT4);
+    schema.addColumn("col8", Type.FLOAT8);
+    schema.addColumn("col9", Type.TEXT);
+    schema.addColumn("col10", Type.BLOB);
+    schema.addColumn("col11", Type.INET4);
+    schema.addColumn("col12", Type.NULL_TYPE);
+    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.putOption(StorageConstants.CSVFILE_SERDE, TextSerializerDeserializer.class.getName());
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+    appender.enableStats();
+    appender.init();
+
+    QueryId queryid = new QueryId("12345", 5);
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
+
+    Tuple tuple = new VTuple(13);
+    tuple.put(new Datum[] {
+        DatumFactory.createBool(true),
+        DatumFactory.createBit((byte) 0x99),
+        DatumFactory.createChar("jinho"),
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("jinho"),
+        DatumFactory.createBlob("hyunsik babo".getBytes()),
+        DatumFactory.createInet4("192.168.0.1"),
+        NullDatum.get(),
+        factory.createDatum(queryid.getProto())
+    });
+    appender.addTuple(tuple);
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
+
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    Tuple retrieved;
+    while ((retrieved=scanner.next()) != null) {
+      for (int i = 0; i < tuple.size(); i++) {
+        assertEquals(tuple.get(i), retrieved.get(i));
+      }
+    }
+    scanner.close();
+    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
+    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
+  }
+
+  @Test
+  public void testRCFileBinarySerializeDeserialize() throws IOException {
+    if(storeType != StoreType.RCFILE) return;
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR, 7);
+    schema.addColumn("col4", Type.INT2);
+    schema.addColumn("col5", Type.INT4);
+    schema.addColumn("col6", Type.INT8);
+    schema.addColumn("col7", Type.FLOAT4);
+    schema.addColumn("col8", Type.FLOAT8);
+    schema.addColumn("col9", Type.TEXT);
+    schema.addColumn("col10", Type.BLOB);
+    schema.addColumn("col11", Type.INET4);
+    schema.addColumn("col12", Type.NULL_TYPE);
+    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.putOption(StorageConstants.RCFILE_SERDE, BinarySerializerDeserializer.class.getName());
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+    appender.enableStats();
+    appender.init();
+
+    QueryId queryid = new QueryId("12345", 5);
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
+
+    Tuple tuple = new VTuple(13);
+    tuple.put(new Datum[] {
+        DatumFactory.createBool(true),
+        DatumFactory.createBit((byte) 0x99),
+        DatumFactory.createChar("jinho"),
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("jinho"),
+        DatumFactory.createBlob("hyunsik babo".getBytes()),
+        DatumFactory.createInet4("192.168.0.1"),
+        NullDatum.get(),
+        factory.createDatum(queryid.getProto())
+    });
+    appender.addTuple(tuple);
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
+
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    Tuple retrieved;
+    while ((retrieved=scanner.next()) != null) {
+      for (int i = 0; i < tuple.size(); i++) {
+        assertEquals(tuple.get(i), retrieved.get(i));
+      }
+    }
+    scanner.close();
+    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
+    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
+  }
+
+  @Test
+  public void testSequenceFileTextSerializeDeserialize() throws IOException {
+    if(storeType != StoreType.SEQUENCEFILE) return;
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR, 7);
+    schema.addColumn("col4", Type.INT2);
+    schema.addColumn("col5", Type.INT4);
+    schema.addColumn("col6", Type.INT8);
+    schema.addColumn("col7", Type.FLOAT4);
+    schema.addColumn("col8", Type.FLOAT8);
+    schema.addColumn("col9", Type.TEXT);
+    schema.addColumn("col10", Type.BLOB);
+    schema.addColumn("col11", Type.INET4);
+    schema.addColumn("col12", Type.NULL_TYPE);
+    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+    appender.enableStats();
+    appender.init();
+
+    QueryId queryid = new QueryId("12345", 5);
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
+
+    Tuple tuple = new VTuple(13);
+    tuple.put(new Datum[] {
+        DatumFactory.createBool(true),
+        DatumFactory.createBit((byte) 0x99),
+        DatumFactory.createChar("jinho"),
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("jinho"),
+        DatumFactory.createBlob("hyunsik babo".getBytes()),
+        DatumFactory.createInet4("192.168.0.1"),
+        NullDatum.get(),
+        factory.createDatum(queryid.getProto())
+    });
+    appender.addTuple(tuple);
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
+
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    assertTrue(scanner instanceof SequenceFileScanner);
+    Writable key = ((SequenceFileScanner) scanner).getKey();
+    assertEquals(key.getClass().getCanonicalName(), LongWritable.class.getCanonicalName());
+
+    Tuple retrieved;
+    while ((retrieved=scanner.next()) != null) {
+      for (int i = 0; i < tuple.size(); i++) {
+        assertEquals(tuple.get(i), retrieved.get(i));
+      }
+    }
+    scanner.close();
+    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
+    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
+  }
+
+  @Test
+  public void testSequenceFileBinarySerializeDeserialize() throws IOException {
+    if(storeType != StoreType.SEQUENCEFILE) return;
+
+    Schema schema = new Schema();
+    schema.addColumn("col1", Type.BOOLEAN);
+    schema.addColumn("col2", Type.BIT);
+    schema.addColumn("col3", Type.CHAR, 7);
+    schema.addColumn("col4", Type.INT2);
+    schema.addColumn("col5", Type.INT4);
+    schema.addColumn("col6", Type.INT8);
+    schema.addColumn("col7", Type.FLOAT4);
+    schema.addColumn("col8", Type.FLOAT8);
+    schema.addColumn("col9", Type.TEXT);
+    schema.addColumn("col10", Type.BLOB);
+    schema.addColumn("col11", Type.INET4);
+    schema.addColumn("col12", Type.NULL_TYPE);
+    schema.addColumn("col13", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
+
+    KeyValueSet options = new KeyValueSet();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.putOption(StorageConstants.SEQUENCEFILE_SERDE, BinarySerializerDeserializer.class.getName());
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    Appender appender = sm.getAppender(meta, schema, tablePath);
+    appender.enableStats();
+    appender.init();
+
+    QueryId queryid = new QueryId("12345", 5);
+    ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
+
+    Tuple tuple = new VTuple(13);
+    tuple.put(new Datum[] {
+        DatumFactory.createBool(true),
+        DatumFactory.createBit((byte) 0x99),
+        DatumFactory.createChar("jinho"),
+        DatumFactory.createInt2((short) 17),
+        DatumFactory.createInt4(59),
+        DatumFactory.createInt8(23l),
+        DatumFactory.createFloat4(77.9f),
+        DatumFactory.createFloat8(271.9f),
+        DatumFactory.createText("jinho"),
+        DatumFactory.createBlob("hyunsik babo".getBytes()),
+        DatumFactory.createInet4("192.168.0.1"),
+        NullDatum.get(),
+        factory.createDatum(queryid.getProto())
+    });
+    appender.addTuple(tuple);
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen());
+
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+    scanner.init();
+
+    assertTrue(scanner instanceof SequenceFileScanner);
+    Writable key = ((SequenceFileScanner) scanner).getKey();
+    assertEquals(key.getClass().getCanonicalName(), BytesWritable.class.getCanonicalName());
+
+    Tuple retrieved;
+    while ((retrieved=scanner.next()) != null) {
+      for (int i = 0; i < tuple.size(); i++) {
+        assertEquals(tuple.get(i), retrieved.get(i));
+      }
+    }
+    scanner.close();
+    assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue());
+    assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue());
+  }
+
+  @Test
+  public void testTime() throws IOException {
+    if (storeType == StoreType.CSV || storeType == StoreType.RAW) {
+      Schema schema = new Schema();
+      schema.addColumn("col1", Type.DATE);
+      schema.addColumn("col2", Type.TIME);
+      schema.addColumn("col3", Type.TIMESTAMP);
+
+      KeyValueSet options = new KeyValueSet();
+      TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+
+      Path tablePath = new Path(testDir, "testTime.data");
+      FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+      Appender appender = sm.getAppender(meta, schema, tablePath);
+      appender.init();
+
+      Tuple tuple = new VTuple(3);
+      tuple.put(new Datum[]{
+          DatumFactory.createDate("1980-04-01"),
+          DatumFactory.createTime("12:34:56"),
+          DatumFactory.createTimestmpDatumWithUnixTime((int)(System.currentTimeMillis() / 1000))
+      });
+      appender.addTuple(tuple);
+      appender.flush();
+      appender.close();
+
+      FileStatus status = fs.getFileStatus(tablePath);
+      FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+      Scanner scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema, fragment);
+      scanner.init();
+
+      Tuple retrieved;
+      while ((retrieved = scanner.next()) != null) {
+        for (int i = 0; i < tuple.size(); i++) {
+          assertEquals(tuple.get(i), retrieved.get(i));
+        }
+      }
+      scanner.close();
+    }
+  }
+
+  @Test
+  public void testSeekableScanner() throws IOException {
+    if (!seekable) {
+      return;
+    }
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.INT8);
+    schema.addColumn("comment", Type.TEXT);
+
+    TableMeta meta = CatalogUtil.newTableMeta(storeType);
+    Path tablePath = new Path(testDir, "Seekable.data");
+    FileStorageManager sm = (FileStorageManager)StorageManager.getFileStorageManager(conf);
+    FileAppender appender = (FileAppender) sm.getAppender(meta, schema, tablePath);
+    appender.enableStats();
+    appender.init();
+    int tupleNum = 100000;
+    VTuple vTuple;
+
+    List<Long> offsets = Lists.newArrayList();
+    offsets.add(0L);
+    for (int i = 0; i < tupleNum; i++) {
+      vTuple = new VTuple(3);
+      vTuple.put(0, DatumFactory.createInt4(i + 1));
+      vTuple.put(1, DatumFactory.createInt8(25l));
+      vTuple.put(2, DatumFactory.createText("test" + i));
+      appender.addTuple(vTuple);
+
+      // find a seek position
+      if (i % (tupleNum / 3) == 0) {
+        offsets.add(appender.getOffset());
+      }
+    }
+
+    // end of file
+    if (!offsets.contains(appender.getOffset())) {
+      offsets.add(appender.getOffset());
+    }
+
+    appender.close();
+    if (statsable) {
+      TableStats stat = appender.getStats();
+      assertEquals(tupleNum, stat.getNumRows().longValue());
+    }
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    assertEquals(status.getLen(), appender.getOffset());
+
+    Scanner scanner;
+    int tupleCnt = 0;
+    long prevOffset = 0;
+    long readBytes = 0;
+    long readRows = 0;
+    for (long offset : offsets) {
+      scanner = StorageManager.getFileStorageManager(conf).getScanner(meta, schema,
+	        new FileFragment("table", tablePath, prevOffset, offset - prevOffset), schema);
+      scanner.init();
+
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+
+      scanner.close();
+      if (statsable) {
+        readBytes += scanner.getInputStats().getNumBytes();
+        readRows += scanner.getInputStats().getNumRows();
+      }
+      prevOffset = offset;
+    }
+
+    assertEquals(tupleNum, tupleCnt);
+    if (statsable) {
+      assertEquals(appender.getStats().getNumBytes().longValue(), readBytes);
+      assertEquals(appender.getStats().getNumRows().longValue(), readRows);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
new file mode 100644
index 0000000..4f7ea1c
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/avro/TestAvroUtil.java
@@ -0,0 +1,106 @@
+/**
+ * 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.tajo.HttpFileServer;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.FileUtil;
+import org.apache.tajo.util.NetUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for {@link org.apache.tajo.storage.avro.AvroUtil}.
+ */
+public class TestAvroUtil {
+  private Schema expected;
+  private URL schemaUrl;
+
+  @Before
+  public void setUp() throws Exception {
+    schemaUrl = FileUtil.getResourcePath("testVariousTypes.avsc");
+    assertNotNull(schemaUrl);
+
+    File file = new File(schemaUrl.getPath());
+    assertTrue(file.exists());
+
+    expected = new Schema.Parser().parse(file);
+  }
+
+  @Test
+  public void testGetSchema() throws IOException, URISyntaxException {
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
+    meta.putOption(StorageConstants.AVRO_SCHEMA_LITERAL, FileUtil.readTextFile(new File(schemaUrl.getPath())));
+    Schema schema = AvroUtil.getAvroSchema(meta, new TajoConf());
+    assertEquals(expected, schema);
+
+    meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
+    meta.putOption(StorageConstants.AVRO_SCHEMA_URL, schemaUrl.getPath());
+    schema = AvroUtil.getAvroSchema(meta, new TajoConf());
+    assertEquals(expected, schema);
+
+    HttpFileServer server = new HttpFileServer(NetUtils.createSocketAddr("127.0.0.1:0"));
+    try {
+      server.start();
+      InetSocketAddress addr = server.getBindAddress();
+
+      String url = "http://127.0.0.1:" + addr.getPort() + schemaUrl.getPath();
+      meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.AVRO);
+      meta.putOption(StorageConstants.AVRO_SCHEMA_URL, url);
+      schema = AvroUtil.getAvroSchema(meta, new TajoConf());
+    } finally {
+      server.stop();
+    }
+    assertEquals(expected, schema);
+  }
+
+  @Test
+  public void testGetSchemaFromHttp() throws IOException, URISyntaxException {
+    HttpFileServer server = new HttpFileServer(NetUtils.createSocketAddr("127.0.0.1:0"));
+    try {
+      server.start();
+      InetSocketAddress addr = server.getBindAddress();
+
+      Schema schema = AvroUtil.getAvroSchemaFromHttp("http://127.0.0.1:" + addr.getPort() + schemaUrl.getPath());
+      assertEquals(expected, schema);
+    } finally {
+      server.stop();
+    }
+  }
+
+  @Test
+  public void testGetSchemaFromFileSystem() throws IOException, URISyntaxException {
+    Schema schema = AvroUtil.getAvroSchemaFromFileSystem(schemaUrl.toString(), new TajoConf());
+
+    assertEquals(expected, schema);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..383740d
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
@@ -0,0 +1,947 @@
+/**
+ * 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.index;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexReader;
+import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexWriter;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class TestBSTIndex {
+  private TajoConf conf;
+  private Schema schema;
+  private TableMeta meta;
+
+  private static final int TUPLE_NUM = 10000;
+  private static final int LOAD_NUM = 100;
+  private static final String TEST_PATH = "target/test-data/TestIndex";
+  private Path testDir;
+  private FileSystem fs;
+  private StoreType storeType;
+
+  public TestBSTIndex(StoreType type) {
+    this.storeType = type;
+    conf = new TajoConf();
+    conf.setVar(TajoConf.ConfVars.ROOT_DIR, TEST_PATH);
+    schema = new Schema();
+    schema.addColumn(new Column("int", Type.INT4));
+    schema.addColumn(new Column("long", Type.INT8));
+    schema.addColumn(new Column("double", Type.FLOAT8));
+    schema.addColumn(new Column("float", Type.FLOAT4));
+    schema.addColumn(new Column("string", Type.TEXT));
+  }
+
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return Arrays.asList(new Object[][]{
+        {StoreType.CSV},
+        {StoreType.RAW}
+    });
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+  }
+
+  @Test
+  public void testFindValue() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testFindValue_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+    Tuple tuple;
+    for (int i = 0; i < TUPLE_NUM; i++) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindValue_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX,
+        keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(1));
+      keyTuple.put(1, tuple.get(2));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    tuple = new VTuple(keySchema.size());
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + storeType + ".idx"), keySchema, comp);
+    reader.open();
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    for (int i = 0; i < TUPLE_NUM - 1; i++) {
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
+      long offsets = reader.find(tuple);
+      scanner.seek(offsets);
+      tuple = scanner.next();
+      assertTrue("seek check [" + (i) + " ," + (tuple.get(1).asInt8()) + "]", (i) == (tuple.get(1).asInt8()));
+      assertTrue("seek check [" + (i) + " ," + (tuple.get(2).asFloat8()) + "]", (i) == (tuple.get(2).asFloat8()));
+
+      offsets = reader.next();
+      if (offsets == -1) {
+        continue;
+      }
+      scanner.seek(offsets);
+      tuple = scanner.next();
+      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(1).asInt8()));
+    }
+    reader.close();
+    scanner.close();
+  }
+
+  @Test
+  public void testBuildIndexWithAppender() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testBuildIndexWithAppender_" + storeType);
+    FileAppender appender = (FileAppender) ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(meta, schema, tablePath);
+    appender.init();
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    Tuple tuple;
+    long offset;
+    for (int i = 0; i < TUPLE_NUM; i++) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+
+      offset = appender.getOffset();
+      appender.addTuple(tuple);
+      creater.write(tuple, offset);
+    }
+    appender.flush();
+    appender.close();
+
+    creater.flush();
+    creater.close();
+
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    tuple = new VTuple(keySchema.size());
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    for (int i = 0; i < TUPLE_NUM - 1; i++) {
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
+      long offsets = reader.find(tuple);
+      scanner.seek(offsets);
+      tuple = scanner.next();
+      assertTrue("[seek check " + (i) + " ]", (i) == (tuple.get(1).asInt8()));
+      assertTrue("[seek check " + (i) + " ]", (i) == (tuple.get(2).asFloat8()));
+
+      offsets = reader.next();
+      if (offsets == -1) {
+        continue;
+      }
+      scanner.seek(offsets);
+      tuple = scanner.next();
+      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(1).asInt8()));
+    }
+    reader.close();
+    scanner.close();
+  }
+
+  @Test
+  public void testFindOmittedValue() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = StorageUtil.concatPath(testDir, "testFindOmittedValue_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+    Tuple tuple;
+    for (int i = 0; i < TUPLE_NUM; i += 2) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, status.getLen());
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindOmittedValue_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(1));
+      keyTuple.put(1, tuple.get(2));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindOmittedValue_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    for (int i = 1; i < TUPLE_NUM - 1; i += 2) {
+      keyTuple.put(0, DatumFactory.createInt8(i));
+      keyTuple.put(1, DatumFactory.createFloat8(i));
+      long offsets = reader.find(keyTuple);
+      assertEquals(-1, offsets);
+    }
+    reader.close();
+  }
+
+  @Test
+  public void testFindNextKeyValue() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testFindNextKeyValue_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+    Tuple tuple;
+    for (int i = 0; i < TUPLE_NUM; i++) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindNextKeyValue_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(0));
+      keyTuple.put(1, tuple.get(1));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyValue_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple result;
+    for (int i = 0; i < TUPLE_NUM - 1; i++) {
+      keyTuple = new VTuple(2);
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
+      long offsets = reader.find(keyTuple, true);
+      scanner.seek(offsets);
+      result = scanner.next();
+      assertTrue("[seek check " + (i + 1) + " ]",
+          (i + 1) == (result.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (result.get(1).asInt8()));
+
+      offsets = reader.next();
+      if (offsets == -1) {
+        continue;
+      }
+      scanner.seek(offsets);
+      result = scanner.next();
+      assertTrue("[seek check " + (i + 2) + " ]", (i + 2) == (result.get(0).asInt8()));
+      assertTrue("[seek check " + (i + 2) + " ]", (i + 2) == (result.get(1).asFloat8()));
+    }
+    reader.close();
+    scanner.close();
+  }
+
+  @Test
+  public void testFindNextKeyOmittedValue() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testFindNextKeyOmittedValue_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf))
+        .getAppender(meta, schema, tablePath);
+    appender.init();
+    Tuple tuple;
+    for (int i = 0; i < TUPLE_NUM; i += 2) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindNextKeyOmittedValue_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(0));
+      keyTuple.put(1, tuple.get(1));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyOmittedValue_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple result;
+    for (int i = 1; i < TUPLE_NUM - 1; i += 2) {
+      keyTuple = new VTuple(2);
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
+      long offsets = reader.find(keyTuple, true);
+      scanner.seek(offsets);
+      result = scanner.next();
+      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (result.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (result.get(1).asInt8()));
+    }
+    scanner.close();
+  }
+
+  @Test
+  public void testFindMinValue() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testFindMinValue" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+
+    Tuple tuple;
+    for (int i = 5; i < TUPLE_NUM + 5; i++) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(1));
+      keyTuple.put(1, tuple.get(2));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    tuple = new VTuple(keySchema.size());
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    tuple.put(0, DatumFactory.createInt8(0));
+    tuple.put(1, DatumFactory.createFloat8(0));
+
+    offset = reader.find(tuple);
+    assertEquals(-1, offset);
+
+    offset = reader.find(tuple, true);
+    assertTrue(offset >= 0);
+    scanner.seek(offset);
+    tuple = scanner.next();
+    assertEquals(5, tuple.get(1).asInt4());
+    assertEquals(5l, tuple.get(2).asInt8());
+    reader.close();
+    scanner.close();
+  }
+
+  @Test
+  public void testMinMax() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testMinMax_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+    Tuple tuple;
+    for (int i = 5; i < TUPLE_NUM; i += 2) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testMinMax_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(0));
+      keyTuple.put(1, tuple.get(1));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testMinMax_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+
+    Tuple min = reader.getFirstKey();
+    assertEquals(5, min.get(0).asInt4());
+    assertEquals(5l, min.get(0).asInt8());
+
+    Tuple max = reader.getLastKey();
+    assertEquals(TUPLE_NUM - 1, max.get(0).asInt4());
+    assertEquals(TUPLE_NUM - 1, max.get(0).asInt8());
+    reader.close();
+  }
+
+  private class ConcurrentAccessor implements Runnable {
+    final BSTIndexReader reader;
+    final Random rnd = new Random(System.currentTimeMillis());
+    boolean failed = false;
+
+    ConcurrentAccessor(BSTIndexReader reader) {
+      this.reader = reader;
+    }
+
+    public boolean isFailed() {
+      return this.failed;
+    }
+
+    @Override
+    public void run() {
+      Tuple findKey = new VTuple(2);
+      int keyVal;
+      for (int i = 0; i < 10000; i++) {
+        keyVal = rnd.nextInt(10000);
+        findKey.put(0, DatumFactory.createInt4(keyVal));
+        findKey.put(1, DatumFactory.createInt8(keyVal));
+        try {
+          assertTrue(reader.find(findKey) != -1);
+        } catch (Exception e) {
+          e.printStackTrace();
+          this.failed = true;
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testConcurrentAccess() throws IOException, InterruptedException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testConcurrentAccess_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+
+    Tuple tuple;
+    for (int i = 0; i < TUPLE_NUM; i++) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testConcurrentAccess_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(0));
+      keyTuple.put(1, tuple.get(1));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testConcurrentAccess_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+
+    Thread[] threads = new Thread[5];
+    ConcurrentAccessor[] accs = new ConcurrentAccessor[5];
+    for (int i = 0; i < threads.length; i++) {
+      accs[i] = new ConcurrentAccessor(reader);
+      threads[i] = new Thread(accs[i]);
+      threads[i].start();
+    }
+
+    for (int i = 0; i < threads.length; i++) {
+      threads[i].join();
+      assertFalse(accs[i].isFailed());
+    }
+    reader.close();
+  }
+
+
+  @Test
+  public void testFindValueDescOrder() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testFindValueDescOrder_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+
+    Tuple tuple;
+    for (int i = (TUPLE_NUM - 1); i >= 0; i--) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("long"), false, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("double"), false, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(1));
+      keyTuple.put(1, tuple.get(2));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+    tuple = new VTuple(keySchema.size());
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    scanner = FileStorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    for (int i = (TUPLE_NUM - 1); i > 0; i--) {
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
+      long offsets = reader.find(tuple);
+      scanner.seek(offsets);
+      tuple = scanner.next();
+      assertTrue("seek check [" + (i) + " ," + (tuple.get(1).asInt8()) + "]", (i) == (tuple.get(1).asInt8()));
+      assertTrue("seek check [" + (i) + " ," + (tuple.get(2).asFloat8()) + "]", (i) == (tuple.get(2).asFloat8()));
+
+      offsets = reader.next();
+      if (offsets == -1) {
+        continue;
+      }
+      scanner.seek(offsets);
+      tuple = scanner.next();
+      assertTrue("[seek check " + (i - 1) + " ]", (i - 1) == (tuple.get(0).asInt4()));
+      assertTrue("[seek check " + (i - 1) + " ]", (i - 1) == (tuple.get(1).asInt8()));
+    }
+    reader.close();
+    scanner.close();
+  }
+
+  @Test
+  public void testFindNextKeyValueDescOrder() throws IOException {
+    meta = CatalogUtil.newTableMeta(storeType);
+
+    Path tablePath = new Path(testDir, "testFindNextKeyValueDescOrder_" + storeType);
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+
+    Tuple tuple;
+    for (int i = (TUPLE_NUM - 1); i >= 0; i--) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("int"), false, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("long"), false, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir,
+        "testFindNextKeyValueDescOrder_" + storeType + ".idx"), BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = scanner.getNextOffset();
+      tuple = scanner.next();
+      if (tuple == null) break;
+
+      keyTuple.put(0, tuple.get(0));
+      keyTuple.put(1, tuple.get(1));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    scanner.close();
+
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyValueDescOrder_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+
+    assertEquals(keySchema, reader.getKeySchema());
+    assertEquals(comp, reader.getComparator());
+
+    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
+    scanner.init();
+
+    Tuple result;
+    for (int i = (TUPLE_NUM - 1); i > 0; i--) {
+      keyTuple = new VTuple(2);
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
+      long offsets = reader.find(keyTuple, true);
+      scanner.seek(offsets);
+      result = scanner.next();
+      assertTrue("[seek check " + (i - 1) + " ]",
+          (i - 1) == (result.get(0).asInt4()));
+      assertTrue("[seek check " + (i - 1) + " ]", (i - 1) == (result.get(1).asInt8()));
+
+      offsets = reader.next();
+      if (offsets == -1) {
+        continue;
+      }
+      scanner.seek(offsets);
+      result = scanner.next();
+      assertTrue("[seek check " + (i - 2) + " ]", (i - 2) == (result.get(0).asInt8()));
+      assertTrue("[seek check " + (i - 2) + " ]", (i - 2) == (result.get(1).asFloat8()));
+    }
+    reader.close();
+    scanner.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..d7c9f49
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
@@ -0,0 +1,248 @@
+/**
+ * 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.index;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexReader;
+import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexWriter;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.storage.CSVFile.CSVScanner;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestSingleCSVFileBSTIndex {
+  
+  private TajoConf conf;
+  private Schema schema;
+  private TableMeta meta;
+  private FileSystem fs;
+
+  private static final int TUPLE_NUM = 10000;
+  private static final int LOAD_NUM = 100;
+  private static final String TEST_PATH = "target/test-data/TestSingleCSVFileBSTIndex";
+  private Path testDir;
+  
+  public TestSingleCSVFileBSTIndex() {
+    conf = new TajoConf();
+    conf.setVar(ConfVars.ROOT_DIR, TEST_PATH);
+    schema = new Schema();
+    schema.addColumn(new Column("int", Type.INT4));
+    schema.addColumn(new Column("long", Type.INT8));
+    schema.addColumn(new Column("double", Type.FLOAT8));
+    schema.addColumn(new Column("float", Type.FLOAT4));
+    schema.addColumn(new Column("string", Type.TEXT));
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+  }
+
+  @Test
+  public void testFindValueInSingleCSV() throws IOException {
+    meta = CatalogUtil.newTableMeta(StoreType.CSV);
+
+    Path tablePath = StorageUtil.concatPath(testDir, "testFindValueInSingleCSV", "table.csv");
+    fs.mkdirs(tablePath.getParent());
+
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+    Tuple tuple;
+    for (int i = 0; i < TUPLE_NUM; i++) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir,
+        "FindValueInCSV.idx"), BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+
+    SeekableScanner fileScanner = new CSVScanner(conf, schema, meta, tablet);
+    fileScanner.init();
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = fileScanner.getNextOffset();
+      tuple = fileScanner.next();
+      if (tuple == null)
+        break;
+
+      keyTuple.put(0, tuple.get(1));
+      keyTuple.put(1, tuple.get(2));
+      creater.write(keyTuple, offset);
+    }
+
+    creater.flush();
+    creater.close();
+    fileScanner.close();
+
+    tuple = new VTuple(keySchema.size());
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir,
+        "FindValueInCSV.idx"), keySchema, comp);
+    reader.open();
+    fileScanner = new CSVScanner(conf, schema, meta, tablet);
+    fileScanner.init();
+    for (int i = 0; i < TUPLE_NUM - 1; i++) {
+      tuple.put(0, DatumFactory.createInt8(i));
+      tuple.put(1, DatumFactory.createFloat8(i));
+      long offsets = reader.find(tuple);
+      fileScanner.seek(offsets);
+      tuple = fileScanner.next();
+      assertEquals(i,  (tuple.get(1).asInt8()));
+      assertEquals(i, (tuple.get(2).asFloat8()) , 0.01);
+
+      offsets = reader.next();
+      if (offsets == -1) {
+        continue;
+      }
+      fileScanner.seek(offsets);
+      tuple = fileScanner.next();
+      assertTrue("[seek check " + (i + 1) + " ]",
+          (i + 1) == (tuple.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]",
+          (i + 1) == (tuple.get(1).asInt8()));
+    }
+  }
+
+  @Test
+  public void testFindNextKeyValueInSingleCSV() throws IOException {
+    meta = CatalogUtil.newTableMeta(StoreType.CSV);
+
+    Path tablePath = StorageUtil.concatPath(testDir, "testFindNextKeyValueInSingleCSV",
+        "table1.csv");
+    fs.mkdirs(tablePath.getParent());
+    Appender appender = ((FileStorageManager)StorageManager.getFileStorageManager(conf)).getAppender(meta, schema, tablePath);
+    appender.init();
+    Tuple tuple;
+    for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
+      tuple = new VTuple(5);
+      tuple.put(0, DatumFactory.createInt4(i));
+      tuple.put(1, DatumFactory.createInt8(i));
+      tuple.put(2, DatumFactory.createFloat8(i));
+      tuple.put(3, DatumFactory.createFloat4(i));
+      tuple.put(4, DatumFactory.createText("field_" + i));
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    long fileLen = status.getLen();
+    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
+    
+    SortSpec [] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
+    
+    BSTIndex bst = new BSTIndex(conf);
+    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "FindNextKeyValueInCSV.idx"),
+        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
+    creater.setLoadNum(LOAD_NUM);
+    creater.open();
+    
+    SeekableScanner fileScanner  = new CSVScanner(conf, schema, meta, tablet);
+    fileScanner.init();
+    Tuple keyTuple;
+    long offset;
+    while (true) {
+      keyTuple = new VTuple(2);
+      offset = fileScanner.getNextOffset();
+      tuple = fileScanner.next();
+      if (tuple == null) break;
+      
+      keyTuple.put(0, tuple.get(0));
+      keyTuple.put(1, tuple.get(1));
+      creater.write(keyTuple, offset);
+    }
+    
+    creater.flush();
+    creater.close();
+    fileScanner.close();    
+    
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "FindNextKeyValueInCSV.idx"), keySchema, comp);
+    reader.open();
+    fileScanner  = new CSVScanner(conf, schema, meta, tablet);
+    fileScanner.init();
+    Tuple result;
+    for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
+      keyTuple = new VTuple(2);
+      keyTuple.put(0, DatumFactory.createInt4(i));
+      keyTuple.put(1, DatumFactory.createInt8(i));
+      long offsets = reader.find(keyTuple, true);
+      fileScanner.seek(offsets);
+      result = fileScanner.next();
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(0).asInt4()));
+      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asInt8()));
+      
+      offsets = reader.next();
+      if (offsets == -1) {
+        continue;
+      }
+      fileScanner.seek(offsets);
+      result = fileScanner.next();
+      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(0).asInt8()));
+      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(1).asFloat8()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
new file mode 100644
index 0000000..109fed9
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.parquet;
+
+import com.google.common.base.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class TestReadWrite {
+  private static final String HELLO = "hello";
+
+  private Path createTmpFile() throws IOException {
+    File tmp = File.createTempFile(getClass().getSimpleName(), ".tmp");
+    tmp.deleteOnExit();
+    tmp.delete();
+
+    // it prevents accessing HDFS namenode of TajoTestingCluster.
+    LocalFileSystem localFS = LocalFileSystem.getLocal(new Configuration());
+    return localFS.makeQualified(new Path(tmp.getPath()));
+  }
+
+  private Schema createAllTypesSchema() {
+    List<Column> columns = new ArrayList<Column>();
+    columns.add(new Column("myboolean", Type.BOOLEAN));
+    columns.add(new Column("mybit", Type.BIT));
+    columns.add(new Column("mychar", Type.CHAR));
+    columns.add(new Column("myint2", Type.INT2));
+    columns.add(new Column("myint4", Type.INT4));
+    columns.add(new Column("myint8", Type.INT8));
+    columns.add(new Column("myfloat4", Type.FLOAT4));
+    columns.add(new Column("myfloat8", Type.FLOAT8));
+    columns.add(new Column("mytext", Type.TEXT));
+    columns.add(new Column("myblob", Type.BLOB));
+    columns.add(new Column("mynull", Type.NULL_TYPE));
+    Column[] columnsArray = new Column[columns.size()];
+    columnsArray = columns.toArray(columnsArray);
+    return new Schema(columnsArray);
+  }
+
+  @Test
+  public void testAll() throws Exception {
+    Path file = createTmpFile();
+    Schema schema = createAllTypesSchema();
+    Tuple tuple = new VTuple(schema.size());
+    tuple.put(0, DatumFactory.createBool(true));
+    tuple.put(1, DatumFactory.createBit((byte)128));
+    tuple.put(2, DatumFactory.createChar('t'));
+    tuple.put(3, DatumFactory.createInt2((short)2048));
+    tuple.put(4, DatumFactory.createInt4(4096));
+    tuple.put(5, DatumFactory.createInt8(8192L));
+    tuple.put(6, DatumFactory.createFloat4(0.2f));
+    tuple.put(7, DatumFactory.createFloat8(4.1));
+    tuple.put(8, DatumFactory.createText(HELLO));
+    tuple.put(9, DatumFactory.createBlob(HELLO.getBytes(Charsets.UTF_8)));
+    tuple.put(10, NullDatum.get());
+
+    TajoParquetWriter writer = new TajoParquetWriter(file, schema);
+    writer.write(tuple);
+    writer.close();
+
+    TajoParquetReader reader = new TajoParquetReader(file, schema);
+    tuple = reader.read();
+
+    assertNotNull(tuple);
+    assertEquals(true, tuple.getBool(0));
+    assertEquals((byte)128, tuple.getByte(1));
+    assertTrue(String.valueOf('t').equals(String.valueOf(tuple.getChar(2))));
+    assertEquals((short)2048, tuple.getInt2(3));
+    assertEquals(4096, tuple.getInt4(4));
+    assertEquals(8192L, tuple.getInt8(5));
+    assertEquals(new Float(0.2f), new Float(tuple.getFloat4(6)));
+    assertEquals(new Double(4.1), new Double(tuple.getFloat8(7)));
+    assertTrue(HELLO.equals(tuple.getText(8)));
+    assertArrayEquals(HELLO.getBytes(Charsets.UTF_8), tuple.getBytes(9));
+    assertEquals(NullDatum.get(), tuple.get(10));
+  }
+}


[26/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java
deleted file mode 100644
index 16c4faa..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.SchemaObject;
-import org.apache.tajo.catalog.statistics.TableStats;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * Scanner Interface
- */
-public interface Scanner extends SchemaObject, Closeable {
-
-  void init() throws IOException;
-
-  /**
-   * It returns one tuple at each call. 
-   * 
-   * @return retrieve null if the scanner has no more tuples. 
-   * Otherwise it returns one tuple.
-   * 
-   * @throws IOException if internal I/O error occurs during next method
-   */
-  Tuple next() throws IOException;
-  
-  /**
-   * Reset the cursor. After executed, the scanner 
-   * will retrieve the first tuple.
-   * 
-   * @throws IOException if internal I/O error occurs during reset method
-   */
-  void reset() throws IOException;
-  
-  /**
-   * Close scanner
-   * 
-   * @throws IOException if internal I/O error occurs during close method
-   */
-  void close() throws IOException;
-
-
-  /**
-   * It returns if the projection is executed in the underlying scanner layer.
-   *
-   * @return true if this scanner can project the given columns.
-   */
-  boolean isProjectable();
-
-  /**
-   * Set target columns
-   * @param targets columns to be projected
-   */
-  void setTarget(Column [] targets);
-
-  /**
-   * It returns if the selection is executed in the underlying scanner layer.
-   *
-   * @return true if this scanner can filter tuples against a given condition.
-   */
-  boolean isSelectable();
-
-  /**
-   * Set a search condition
-   * @param expr to be searched
-   *
-   * TODO - to be changed Object type
-   */
-  void setSearchCondition(Object expr);
-
-  /**
-   * It returns if the file is splittable.
-   *
-   * @return true if this scanner can split the a file.
-   */
-  boolean isSplittable();
-
-  /**
-   * How much of the input has the Scanner consumed
-   * @return progress from <code>0.0</code> to <code>1.0</code>.
-   */
-  float getProgress();
-
-  TableStats getInputStats();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java
deleted file mode 100644
index 894e7ee..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java
+++ /dev/null
@@ -1,28 +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 java.io.IOException;
-
-public interface SeekableScanner extends Scanner {
-
-  public abstract long getNextOffset() throws IOException;
-
-  public abstract void seek(long offset) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
deleted file mode 100644
index 564a9f5..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.datum.Datum;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-@Deprecated
-public interface SerializerDeserializer {
-
-  public int serialize(Column col, Datum datum, OutputStream out, byte[] nullCharacters) throws IOException;
-
-  public Datum deserialize(Column col, byte[] bytes, int offset, int length, byte[] nullCharacters) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java b/tajo-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java
deleted file mode 100644
index 3579674..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-public class SplitLineReader extends LineReader {
-  public SplitLineReader(InputStream in, byte[] recordDelimiterBytes) {
-    super(in, recordDelimiterBytes);
-  }
-
-  public SplitLineReader(InputStream in, Configuration conf,
-                         byte[] recordDelimiterBytes) throws IOException {
-    super(in, conf, recordDelimiterBytes);
-  }
-
-  public boolean needAdditionalRecordAfterSplit() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.java b/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.java
deleted file mode 100644
index cc85c1d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.fragment.FileFragment;
-
-import java.io.IOException;
-
-public abstract class Storage {
-  protected final Configuration conf;
-  
-  public Storage(final Configuration conf) {
-    this.conf = conf;
-  }
-  
-  public Configuration getConf() {
-    return this.conf;
-  }
-  
-  public abstract Appender getAppender(TableMeta meta, Path path)
-    throws IOException;
-
-  public abstract Scanner openScanner(Schema schema, FileFragment[] tablets)
-    throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
deleted file mode 100644
index e37be58..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
+++ /dev/null
@@ -1,926 +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.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.plan.rewrite.RewriteRule;
-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.storage.hbase.HBaseStorageManager;
-import org.apache.tajo.util.TUtil;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.net.URI;
-import java.text.NumberFormat;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * StorageManager manages the functions of storing and reading data.
- * StorageManager is a abstract class.
- * For supporting such as HDFS, HBASE, a specific StorageManager should be implemented by inheriting this class.
- *
- */
-public abstract class StorageManager {
-  private final Log LOG = LogFactory.getLog(StorageManager.class);
-
-  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
-      Configuration.class,
-      Schema.class,
-      TableMeta.class,
-      Fragment.class
-  };
-
-  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
-      Configuration.class,
-      QueryUnitAttemptId.class,
-      Schema.class,
-      TableMeta.class,
-      Path.class
-  };
-
-  protected TajoConf conf;
-  protected StoreType storeType;
-
-  /**
-   * Cache of StorageManager.
-   * Key is manager key(warehouse path) + store type
-   */
-  private static final Map<String, StorageManager> storageManagers = Maps.newHashMap();
-
-  /**
-   * Cache of scanner handlers for each storage type.
-   */
-  protected static final Map<String, Class<? extends Scanner>> SCANNER_HANDLER_CACHE
-      = new ConcurrentHashMap<String, Class<? extends Scanner>>();
-
-  /**
-   * Cache of appender handlers for each storage type.
-   */
-  protected static final Map<String, Class<? extends Appender>> APPENDER_HANDLER_CACHE
-      = new ConcurrentHashMap<String, Class<? extends Appender>>();
-
-  /**
-   * Cache of constructors for each class. Pins the classes so they
-   * can't be garbage collected until ReflectionUtils can be collected.
-   */
-  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
-      new ConcurrentHashMap<Class<?>, Constructor<?>>();
-
-  public StorageManager(StoreType storeType) {
-    this.storeType = storeType;
-  }
-
-  /**
-   * Initialize storage manager.
-   * @throws IOException
-   */
-  protected abstract void storageInit() throws IOException;
-
-  /**
-   * This method is called after executing "CREATE TABLE" statement.
-   * If a storage is a file based storage, a storage manager may create directory.
-   *
-   * @param tableDesc Table description which is created.
-   * @param ifNotExists Creates the table only when the table does not exist.
-   * @throws IOException
-   */
-  public abstract void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException;
-
-  /**
-   * This method is called after executing "DROP TABLE" statement with the 'PURGE' option
-   * which is the option to delete all the data.
-   *
-   * @param tableDesc
-   * @throws IOException
-   */
-  public abstract void purgeTable(TableDesc tableDesc) throws IOException;
-
-  /**
-   * Returns the splits that will serve as input for the scan tasks. The
-   * number of splits matches the number of regions in a table.
-   * @param fragmentId The table name or previous ExecutionBlockId
-   * @param tableDesc The table description for the target data.
-   * @param scanNode The logical node for scanning.
-   * @return The list of input fragments.
-   * @throws IOException
-   */
-  public abstract List<Fragment> getSplits(String fragmentId, TableDesc tableDesc,
-                                           ScanNode scanNode) throws IOException;
-
-  /**
-   * It returns the splits that will serve as input for the non-forward query scanner such as 'select * from table1'.
-   * The result list should be small. If there is many fragments for scanning, TajoMaster uses the paging navigation.
-   * @param tableDesc The table description for the target data.
-   * @param currentPage The current page number within the entire list.
-   * @param numFragments The number of fragments in the result.
-   * @return The list of input fragments.
-   * @throws IOException
-   */
-  public abstract List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments)
-      throws IOException;
-
-  /**
-   * It returns the storage property.
-   * @return The storage property
-   */
-  public abstract StorageProperty getStorageProperty();
-
-  /**
-   * Release storage manager resource
-   */
-  public abstract void closeStorageManager();
-
-  /**
-   * It is called by a Repartitioner for range shuffling when the SortRangeType of SortNode is STORAGE_SPECIFIED.
-   * In general Repartitioner determines the partition range using previous output statistics data.
-   * In the special cases, such as HBase Repartitioner uses the result of this method.
-   *
-   * @param queryContext The current query context which contains query properties.
-   * @param tableDesc The table description for the target data.
-   * @param inputSchema The input schema
-   * @param sortSpecs The sort specification that contains the sort column and sort order.
-   * @return The list of sort ranges.
-   * @throws IOException
-   */
-  public abstract TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
-                                                   Schema inputSchema, SortSpec[] sortSpecs,
-                                                   TupleRange dataRange) throws IOException;
-
-  /**
-   * This method is called before executing 'INSERT' or 'CREATE TABLE as SELECT'.
-   * In general Tajo creates the target table after finishing the final sub-query of CATS.
-   * But In the special cases, such as HBase INSERT or CAST query uses the target table information.
-   * That kind of the storage should implements the logic related to creating table in this method.
-   *
-   * @param node The child node of the root node.
-   * @throws IOException
-   */
-  public abstract void beforeInsertOrCATS(LogicalNode node) throws IOException;
-
-  /**
-   * It is called when the query failed.
-   * Each storage manager should implement to be processed when the query fails in this method.
-   *
-   * @param node The child node of the root node.
-   * @throws IOException
-   */
-  public abstract void rollbackOutputCommit(LogicalNode node) throws IOException;
-
-  /**
-   * Returns the current storage type.
-   * @return
-   */
-  public StoreType getStoreType() {
-    return storeType;
-  }
-
-  /**
-   * Initialize StorageManager instance. It should be called before using.
-   *
-   * @param tajoConf
-   * @throws IOException
-   */
-  public void init(TajoConf tajoConf) throws IOException {
-    this.conf = tajoConf;
-    storageInit();
-  }
-
-  /**
-   * Close StorageManager
-   * @throws IOException
-   */
-  public void close() throws IOException {
-    synchronized(storageManagers) {
-      for (StorageManager eachStorageManager: storageManagers.values()) {
-        eachStorageManager.closeStorageManager();
-      }
-    }
-  }
-
-  /**
-   * Returns the splits that will serve as input for the scan tasks. The
-   * number of splits matches the number of regions in a table.
-   *
-   * @param fragmentId The table name or previous ExecutionBlockId
-   * @param tableDesc The table description for the target data.
-   * @return The list of input fragments.
-   * @throws IOException
-   */
-  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc) throws IOException {
-    return getSplits(fragmentId, tableDesc, null);
-  }
-
-  /**
-   * Returns FileStorageManager instance.
-   *
-   * @param tajoConf Tajo system property.
-   * @return
-   * @throws IOException
-   */
-  public static FileStorageManager getFileStorageManager(TajoConf tajoConf) throws IOException {
-    return getFileStorageManager(tajoConf, null);
-  }
-
-  /**
-   * Returns FileStorageManager instance and sets WAREHOUSE_DIR property in tajoConf with warehousePath parameter.
-   *
-   * @param tajoConf Tajo system property.
-   * @param warehousePath The warehouse directory to be set in the tajoConf.
-   * @return
-   * @throws IOException
-   */
-  public static FileStorageManager getFileStorageManager(TajoConf tajoConf, Path warehousePath) throws IOException {
-    URI uri;
-    TajoConf copiedConf = new TajoConf(tajoConf);
-    if (warehousePath != null) {
-      copiedConf.setVar(ConfVars.WAREHOUSE_DIR, warehousePath.toUri().toString());
-    }
-    uri = TajoConf.getWarehouseDir(copiedConf).toUri();
-    String key = "file".equals(uri.getScheme()) ? "file" : uri.toString();
-    return (FileStorageManager) getStorageManager(copiedConf, StoreType.CSV, key);
-  }
-
-  /**
-   * Returns the proper StorageManager instance according to the storeType.
-   *
-   * @param tajoConf Tajo system property.
-   * @param storeType Storage type
-   * @return
-   * @throws IOException
-   */
-  public static StorageManager getStorageManager(TajoConf tajoConf, String storeType) throws IOException {
-    if ("HBASE".equals(storeType)) {
-      return getStorageManager(tajoConf, StoreType.HBASE);
-    } else {
-      return getStorageManager(tajoConf, StoreType.CSV);
-    }
-  }
-
-  /**
-   * Returns the proper StorageManager instance according to the storeType.
-   *
-   * @param tajoConf Tajo system property.
-   * @param storeType Storage type
-   * @return
-   * @throws IOException
-   */
-  public static StorageManager getStorageManager(TajoConf tajoConf, StoreType storeType) throws IOException {
-    return getStorageManager(tajoConf, storeType, null);
-  }
-
-  /**
-   * Returns the proper StorageManager instance according to the storeType
-   *
-   * @param tajoConf Tajo system property.
-   * @param storeType Storage type
-   * @param managerKey Key that can identify each storage manager(may be a path)
-   * @return
-   * @throws IOException
-   */
-  public static synchronized StorageManager getStorageManager (
-      TajoConf tajoConf, StoreType storeType, String managerKey) throws IOException {
-    synchronized (storageManagers) {
-      String storeKey = storeType + managerKey;
-      StorageManager manager = storageManagers.get(storeKey);
-      if (manager == null) {
-        switch (storeType) {
-          case HBASE:
-            manager = new HBaseStorageManager(storeType);
-            break;
-          default:
-            manager = new FileStorageManager(storeType);
-        }
-
-        manager.init(tajoConf);
-        storageManagers.put(storeKey, manager);
-      }
-
-      return manager;
-    }
-  }
-
-  /**
-   * Returns Scanner instance.
-   *
-   * @param meta The table meta
-   * @param schema The input schema
-   * @param fragment The fragment for scanning
-   * @param target Columns which are selected.
-   * @return Scanner instance
-   * @throws IOException
-   */
-  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment, Schema target) throws IOException {
-    return getScanner(meta, schema, FragmentConvertor.convert(conf, fragment), target);
-  }
-
-  /**
-   * Returns Scanner instance.
-   *
-   * @param meta The table meta
-   * @param schema The input schema
-   * @param fragment The fragment for scanning
-   * @return Scanner instance
-   * @throws IOException
-   */
-  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment) throws IOException {
-    return getScanner(meta, schema, fragment, schema);
-  }
-
-  /**
-   * Returns Scanner instance.
-   *
-   * @param meta The table meta
-   * @param schema The input schema
-   * @param fragment The fragment for scanning
-   * @param target The output schema
-   * @return Scanner instance
-   * @throws IOException
-   */
-  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
-    if (fragment.isEmpty()) {
-      Scanner scanner = new NullScanner(conf, schema, meta, fragment);
-      scanner.setTarget(target.toArray());
-
-      return scanner;
-    }
-
-    Scanner scanner;
-
-    Class<? extends Scanner> scannerClass = getScannerClass(meta.getStoreType());
-    scanner = newScannerInstance(scannerClass, conf, schema, meta, fragment);
-    if (scanner.isProjectable()) {
-      scanner.setTarget(target.toArray());
-    }
-
-    return scanner;
-  }
-
-  /**
-   * Returns Scanner instance.
-   *
-   * @param conf The system property
-   * @param meta The table meta
-   * @param schema The input schema
-   * @param fragment The fragment for scanning
-   * @param target The output schema
-   * @return Scanner instance
-   * @throws IOException
-   */
-  public static synchronized SeekableScanner getSeekableScanner(
-      TajoConf conf, TableMeta meta, Schema schema, FileFragment fragment, Schema target) throws IOException {
-    return (SeekableScanner)getStorageManager(conf, meta.getStoreType()).getScanner(meta, schema, fragment, target);
-  }
-
-  /**
-   * Returns Scanner instance.
-   *
-   * @param conf The system property
-   * @param meta The table meta
-   * @param schema The input schema
-   * @param path The data file path
-   * @return Scanner instance
-   * @throws IOException
-   */
-  public static synchronized SeekableScanner getSeekableScanner(
-      TajoConf conf, TableMeta meta, Schema schema, Path path) throws IOException {
-
-    FileSystem fs = path.getFileSystem(conf);
-    FileStatus status = fs.getFileStatus(path);
-    FileFragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
-
-    return getSeekableScanner(conf, meta, schema, fragment, schema);
-  }
-
-  /**
-   * Returns Appender instance.
-   * @param queryContext Query property.
-   * @param taskAttemptId Task id.
-   * @param meta Table meta data.
-   * @param schema Output schema.
-   * @param workDir Working directory
-   * @return Appender instance
-   * @throws IOException
-   */
-  public Appender getAppender(OverridableConf queryContext,
-                              QueryUnitAttemptId taskAttemptId, TableMeta meta, Schema schema, Path workDir)
-      throws IOException {
-    Appender appender;
-
-    Class<? extends Appender> appenderClass;
-
-    String handlerName = meta.getStoreType().name().toLowerCase();
-    appenderClass = APPENDER_HANDLER_CACHE.get(handlerName);
-    if (appenderClass == null) {
-      appenderClass = conf.getClass(
-          String.format("tajo.storage.appender-handler.%s.class",
-              meta.getStoreType().name().toLowerCase()), null, Appender.class);
-      APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
-    }
-
-    if (appenderClass == null) {
-      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
-    }
-
-    appender = newAppenderInstance(appenderClass, conf, taskAttemptId, meta, schema, workDir);
-
-    return appender;
-  }
-
-  /**
-   * Creates a scanner instance.
-   *
-   * @param theClass Concrete class of scanner
-   * @param conf System property
-   * @param schema Input schema
-   * @param meta Table meta data
-   * @param fragment The fragment for scanning
-   * @param <T>
-   * @return The scanner instance
-   */
-  public static <T> T newScannerInstance(Class<T> theClass, Configuration conf, Schema schema, TableMeta meta,
-                                         Fragment fragment) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(DEFAULT_SCANNER_PARAMS);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(new Object[]{conf, schema, meta, fragment});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    return result;
-  }
-
-  /**
-   * Creates a scanner instance.
-   *
-   * @param theClass Concrete class of scanner
-   * @param conf System property
-   * @param taskAttemptId Task id
-   * @param meta Table meta data
-   * @param schema Input schema
-   * @param workDir Working directory
-   * @param <T>
-   * @return The scanner instance
-   */
-  public static <T> T newAppenderInstance(Class<T> theClass, Configuration conf, QueryUnitAttemptId taskAttemptId,
-                                          TableMeta meta, Schema schema, Path workDir) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(DEFAULT_APPENDER_PARAMS);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(new Object[]{conf, taskAttemptId, schema, meta, workDir});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    return result;
-  }
-
-  /**
-   * Return the Scanner class for the StoreType that is defined in storage-default.xml.
-   *
-   * @param storeType store type
-   * @return The Scanner class
-   * @throws IOException
-   */
-  public Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException {
-    String handlerName = storeType.name().toLowerCase();
-    Class<? extends Scanner> scannerClass = SCANNER_HANDLER_CACHE.get(handlerName);
-    if (scannerClass == null) {
-      scannerClass = conf.getClass(
-          String.format("tajo.storage.scanner-handler.%s.class",storeType.name().toLowerCase()), null, Scanner.class);
-      SCANNER_HANDLER_CACHE.put(handlerName, scannerClass);
-    }
-
-    if (scannerClass == null) {
-      throw new IOException("Unknown Storage Type: " + storeType.name());
-    }
-
-    return scannerClass;
-  }
-
-  /**
-   * Return length of the fragment.
-   * In the UNKNOWN_LENGTH case get FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH from the configuration.
-   *
-   * @param conf Tajo system property
-   * @param fragment Fragment
-   * @return
-   */
-  public static long getFragmentLength(TajoConf conf, Fragment fragment) {
-    if (fragment.getLength() == TajoConstants.UNKNOWN_LENGTH) {
-      return conf.getLongVar(ConfVars.FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH);
-    } else {
-      return fragment.getLength();
-    }
-  }
-
-  /**
-   * It is called after making logical plan. Storage manager should verify the schema for inserting.
-   *
-   * @param tableDesc The table description of insert target.
-   * @param outSchema  The output schema of select query for inserting.
-   * @throws IOException
-   */
-  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException {
-    // nothing to do
-  }
-
-  /**
-   * Returns the list of storage specified rewrite rules.
-   * This values are used by LogicalOptimizer.
-   *
-   * @param queryContext The query property
-   * @param tableDesc The description of the target table.
-   * @return The list of storage specified rewrite rules
-   * @throws IOException
-   */
-  public List<RewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
-    return null;
-  }
-
-  /**
-   * Finalizes result data. Tajo stores result data in the staging directory.
-   * If the query fails, clean up the staging directory.
-   * Otherwise the query is successful, move to the final directory from the staging directory.
-   *
-   * @param queryContext The query property
-   * @param finalEbId The final execution block id
-   * @param plan The query plan
-   * @param schema The final output schema
-   * @param tableDesc The description of the target table
-   * @return Saved path
-   * @throws IOException
-   */
-  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
-                               LogicalPlan plan, Schema schema,
-                               TableDesc tableDesc) throws IOException {
-    return commitOutputData(queryContext, finalEbId, plan, schema, tableDesc, true);
-  }
-
-  /**
-   * Finalizes result data. Tajo stores result data in the staging directory.
-   * If the query fails, clean up the staging directory.
-   * Otherwise the query is successful, move to the final directory from the staging directory.
-   *
-   * @param queryContext The query property
-   * @param finalEbId The final execution block id
-   * @param plan The query plan
-   * @param schema The final output schema
-   * @param tableDesc The description of the target table
-   * @param changeFileSeq If true change result file name with max sequence.
-   * @return Saved path
-   * @throws IOException
-   */
-  protected Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
-                               LogicalPlan plan, Schema schema,
-                               TableDesc tableDesc, boolean changeFileSeq) throws IOException {
-    Path stagingDir = new Path(queryContext.get(QueryVars.STAGING_DIR));
-    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-    Path finalOutputDir;
-    if (!queryContext.get(QueryVars.OUTPUT_TABLE_PATH, "").isEmpty()) {
-      finalOutputDir = new Path(queryContext.get(QueryVars.OUTPUT_TABLE_PATH));
-      FileSystem fs = stagingResultDir.getFileSystem(conf);
-
-      if (queryContext.getBool(QueryVars.OUTPUT_OVERWRITE, false)) { // INSERT OVERWRITE INTO
-
-        // It moves the original table into the temporary location.
-        // Then it moves the new result table into the original table location.
-        // Upon failed, it recovers the original table if possible.
-        boolean movedToOldTable = false;
-        boolean committed = false;
-        Path oldTableDir = new Path(stagingDir, TajoConstants.INSERT_OVERWIRTE_OLD_TABLE_NAME);
-
-        if (!queryContext.get(QueryVars.OUTPUT_PARTITIONS, "").isEmpty()) {
-          // This is a map for existing non-leaf directory to rename. A key is current directory and a value is
-          // renaming directory.
-          Map<Path, Path> renameDirs = TUtil.newHashMap();
-          // This is a map for recovering existing partition directory. A key is current directory and a value is
-          // temporary directory to back up.
-          Map<Path, Path> recoveryDirs = TUtil.newHashMap();
-
-          try {
-            if (!fs.exists(finalOutputDir)) {
-              fs.mkdirs(finalOutputDir);
-            }
-
-            visitPartitionedDirectory(fs, stagingResultDir, finalOutputDir, stagingResultDir.toString(),
-                renameDirs, oldTableDir);
-
-            // Rename target partition directories
-            for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
-              // Backup existing data files for recovering
-              if (fs.exists(entry.getValue())) {
-                String recoveryPathString = entry.getValue().toString().replaceAll(finalOutputDir.toString(),
-                    oldTableDir.toString());
-                Path recoveryPath = new Path(recoveryPathString);
-                fs.rename(entry.getValue(), recoveryPath);
-                fs.exists(recoveryPath);
-                recoveryDirs.put(entry.getValue(), recoveryPath);
-              }
-              // Delete existing directory
-              fs.delete(entry.getValue(), true);
-              // Rename staging directory to final output directory
-              fs.rename(entry.getKey(), entry.getValue());
-            }
-
-          } catch (IOException ioe) {
-            // Remove created dirs
-            for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
-              fs.delete(entry.getValue(), true);
-            }
-
-            // Recovery renamed dirs
-            for(Map.Entry<Path, Path> entry : recoveryDirs.entrySet()) {
-              fs.delete(entry.getValue(), true);
-              fs.rename(entry.getValue(), entry.getKey());
-            }
-            throw new IOException(ioe.getMessage());
-          }
-        } else {
-          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 {
-        String queryType = queryContext.get(QueryVars.COMMAND_TYPE);
-
-        if (queryType != null && queryType.equals(NodeType.INSERT.name())) { // INSERT INTO an existing table
-
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(3);
-
-          if (!queryContext.get(QueryVars.OUTPUT_PARTITIONS, "").isEmpty()) {
-            for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
-              if (eachFile.isFile()) {
-                LOG.warn("Partition table can't have file in a staging dir: " + eachFile.getPath());
-                continue;
-              }
-              moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, -1, changeFileSeq);
-            }
-          } else {
-            int maxSeq = StorageUtil.getMaxFileSequence(fs, finalOutputDir, false) + 1;
-            for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
-              if (eachFile.getPath().getName().startsWith("_")) {
-                continue;
-              }
-              moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, maxSeq++, changeFileSeq);
-            }
-          }
-          // checking all file moved and remove empty dir
-          verifyAllFileMoved(fs, stagingResultDir);
-          FileStatus[] files = fs.listStatus(stagingResultDir);
-          if (files != null && files.length != 0) {
-            for (FileStatus eachFile: files) {
-              LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
-            }
-          }
-        } else { // CREATE TABLE AS SELECT (CTAS)
-          fs.rename(stagingResultDir, finalOutputDir);
-          LOG.info("Moved from the staging dir to the output directory '" + finalOutputDir);
-        }
-      }
-    } else {
-      finalOutputDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
-    }
-
-    return finalOutputDir;
-  }
-
-  /**
-   * Attach the sequence number to the output file name and than move the file into the final result path.
-   *
-   * @param fs FileSystem
-   * @param stagingResultDir The staging result dir
-   * @param fileStatus The file status
-   * @param finalOutputPath Final output path
-   * @param nf Number format
-   * @param fileSeq The sequence number
-   * @throws IOException
-   */
-  private void moveResultFromStageToFinal(FileSystem fs, Path stagingResultDir,
-                                          FileStatus fileStatus, Path finalOutputPath,
-                                          NumberFormat nf,
-                                          int fileSeq, boolean changeFileSeq) throws IOException {
-    if (fileStatus.isDirectory()) {
-      String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
-      if (subPath != null) {
-        Path finalSubPath = new Path(finalOutputPath, subPath);
-        if (!fs.exists(finalSubPath)) {
-          fs.mkdirs(finalSubPath);
-        }
-        int maxSeq = StorageUtil.getMaxFileSequence(fs, finalSubPath, false);
-        for (FileStatus eachFile : fs.listStatus(fileStatus.getPath())) {
-          if (eachFile.getPath().getName().startsWith("_")) {
-            continue;
-          }
-          moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputPath, nf, ++maxSeq, changeFileSeq);
-        }
-      } else {
-        throw new IOException("Wrong staging dir:" + stagingResultDir + "," + fileStatus.getPath());
-      }
-    } else {
-      String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
-      if (subPath != null) {
-        Path finalSubPath = new Path(finalOutputPath, subPath);
-        if (changeFileSeq) {
-          finalSubPath = new Path(finalSubPath.getParent(), replaceFileNameSeq(finalSubPath, fileSeq, nf));
-        }
-        if (!fs.exists(finalSubPath.getParent())) {
-          fs.mkdirs(finalSubPath.getParent());
-        }
-        if (fs.exists(finalSubPath)) {
-          throw new IOException("Already exists data file:" + finalSubPath);
-        }
-        boolean success = fs.rename(fileStatus.getPath(), finalSubPath);
-        if (success) {
-          LOG.info("Moving staging file[" + fileStatus.getPath() + "] + " +
-              "to final output[" + finalSubPath + "]");
-        } else {
-          LOG.error("Can't move staging file[" + fileStatus.getPath() + "] + " +
-              "to final output[" + finalSubPath + "]");
-        }
-      }
-    }
-  }
-
-  /**
-   * Removes the path of the parent.
-   * @param parentPath
-   * @param childPath
-   * @return
-   */
-  private String extractSubPath(Path parentPath, Path childPath) {
-    String parentPathStr = parentPath.toUri().getPath();
-    String childPathStr = childPath.toUri().getPath();
-
-    if (parentPathStr.length() > childPathStr.length()) {
-      return null;
-    }
-
-    int index = childPathStr.indexOf(parentPathStr);
-    if (index != 0) {
-      return null;
-    }
-
-    return childPathStr.substring(parentPathStr.length() + 1);
-  }
-
-  /**
-   * Attach the sequence number to a path.
-   *
-   * @param path Path
-   * @param seq sequence number
-   * @param nf Number format
-   * @return New path attached with sequence number
-   * @throws IOException
-   */
-  private String replaceFileNameSeq(Path path, int seq, NumberFormat nf) throws IOException {
-    String[] tokens = path.getName().split("-");
-    if (tokens.length != 4) {
-      throw new IOException("Wrong result file name:" + path);
-    }
-    return tokens[0] + "-" + tokens[1] + "-" + tokens[2] + "-" + nf.format(seq);
-  }
-
-  /**
-   * Make sure all files are moved.
-   * @param fs FileSystem
-   * @param stagingPath The stagind directory
-   * @return
-   * @throws IOException
-   */
-  private boolean verifyAllFileMoved(FileSystem fs, Path stagingPath) throws IOException {
-    FileStatus[] files = fs.listStatus(stagingPath);
-    if (files != null && files.length != 0) {
-      for (FileStatus eachFile: files) {
-        if (eachFile.isFile()) {
-          LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
-          return false;
-        } else {
-          if (verifyAllFileMoved(fs, eachFile.getPath())) {
-            fs.delete(eachFile.getPath(), false);
-          } else {
-            return false;
-          }
-        }
-      }
-    }
-
-    return true;
-  }
-
-  /**
-   * This method sets a rename map which includes renamed staging directory to final output directory recursively.
-   * If there exists some data files, this delete it for duplicate data.
-   *
-   *
-   * @param fs
-   * @param stagingPath
-   * @param outputPath
-   * @param stagingParentPathString
-   * @throws IOException
-   */
-  private void visitPartitionedDirectory(FileSystem fs, Path stagingPath, Path outputPath,
-                                         String stagingParentPathString,
-                                         Map<Path, Path> renameDirs, Path oldTableDir) throws IOException {
-    FileStatus[] files = fs.listStatus(stagingPath);
-
-    for(FileStatus eachFile : files) {
-      if (eachFile.isDirectory()) {
-        Path oldPath = eachFile.getPath();
-
-        // Make recover directory.
-        String recoverPathString = oldPath.toString().replaceAll(stagingParentPathString,
-            oldTableDir.toString());
-        Path recoveryPath = new Path(recoverPathString);
-        if (!fs.exists(recoveryPath)) {
-          fs.mkdirs(recoveryPath);
-        }
-
-        visitPartitionedDirectory(fs, eachFile.getPath(), outputPath, stagingParentPathString,
-            renameDirs, oldTableDir);
-        // Find last order partition for renaming
-        String newPathString = oldPath.toString().replaceAll(stagingParentPathString,
-            outputPath.toString());
-        Path newPath = new Path(newPathString);
-        if (!isLeafDirectory(fs, eachFile.getPath())) {
-          renameDirs.put(eachFile.getPath(), newPath);
-        } else {
-          if (!fs.exists(newPath)) {
-            fs.mkdirs(newPath);
-          }
-        }
-      }
-    }
-  }
-
-  private boolean isLeafDirectory(FileSystem fs, Path path) throws IOException {
-    boolean retValue = false;
-
-    FileStatus[] files = fs.listStatus(path);
-    for (FileStatus file : files) {
-      if (fs.isDirectory(file.getPath())) {
-        retValue = true;
-        break;
-      }
-    }
-
-    return retValue;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/StorageProperty.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageProperty.java b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageProperty.java
deleted file mode 100644
index 6816d08..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageProperty.java
+++ /dev/null
@@ -1,40 +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;
-
-public class StorageProperty {
-  private boolean supportsInsertInto;
-  private boolean sortedInsert;
-
-  public boolean isSupportsInsertInto() {
-    return supportsInsertInto;
-  }
-
-  public void setSupportsInsertInto(boolean supportsInsertInto) {
-    this.supportsInsertInto = supportsInsertInto;
-  }
-
-  public boolean isSortedInsert() {
-    return sortedInsert;
-  }
-
-  public void setSortedInsert(boolean sortedInsert) {
-    this.sortedInsert = sortedInsert;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
deleted file mode 100644
index 4a66678..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.KeyValueSet;
-import parquet.hadoop.ParquetOutputFormat;
-import sun.nio.ch.DirectBuffer;
-
-import java.io.DataInput;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-public class StorageUtil extends StorageConstants {
-  public static int getRowByteSize(Schema schema) {
-    int sum = 0;
-    for(Column col : schema.getColumns()) {
-      sum += StorageUtil.getColByteSize(col);
-    }
-
-    return sum;
-  }
-
-  public static int getColByteSize(Column col) {
-    switch (col.getDataType().getType()) {
-      case BOOLEAN:
-        return 1;
-      case CHAR:
-        return 1;
-      case BIT:
-        return 1;
-      case INT2:
-        return 2;
-      case INT4:
-        return 4;
-      case INT8:
-        return 8;
-      case FLOAT4:
-        return 4;
-      case FLOAT8:
-        return 8;
-      case INET4:
-        return 4;
-      case INET6:
-        return 32;
-      case TEXT:
-        return 256;
-      case BLOB:
-        return 256;
-      case DATE:
-        return 4;
-      case TIME:
-        return 8;
-      case TIMESTAMP:
-        return 8;
-      default:
-        return 0;
-    }
-  }
-
-  public static void writeTableMeta(Configuration conf, Path tableroot, TableMeta meta) throws IOException {
-    FileSystem fs = tableroot.getFileSystem(conf);
-    FSDataOutputStream out = fs.create(new Path(tableroot, ".meta"));
-    FileUtil.writeProto(out, meta.getProto());
-    out.flush();
-    out.close();
-  }
-  
-  public static Path concatPath(String parent, String...childs) {
-    return concatPath(new Path(parent), childs);
-  }
-  
-  public static Path concatPath(Path parent, String...childs) {
-    StringBuilder sb = new StringBuilder();
-    
-    for(int i=0; i < childs.length; i++) {      
-      sb.append(childs[i]);
-      if(i < childs.length - 1)
-        sb.append("/");
-    }
-    
-    return new Path(parent, sb.toString());
-  }
-
-  public static KeyValueSet newPhysicalProperties(CatalogProtos.StoreType type) {
-    KeyValueSet options = new KeyValueSet();
-    if (CatalogProtos.StoreType.CSV == type) {
-      options.set(CSVFILE_DELIMITER, DEFAULT_FIELD_DELIMITER);
-    } else if (CatalogProtos.StoreType.RCFILE == type) {
-      options.set(RCFILE_SERDE, DEFAULT_BINARY_SERDE);
-    } else if (CatalogProtos.StoreType.SEQUENCEFILE == type) {
-      options.set(SEQUENCEFILE_SERDE, DEFAULT_TEXT_SERDE);
-      options.set(SEQUENCEFILE_DELIMITER, DEFAULT_FIELD_DELIMITER);
-    } else if (type == CatalogProtos.StoreType.PARQUET) {
-      options.set(ParquetOutputFormat.BLOCK_SIZE, PARQUET_DEFAULT_BLOCK_SIZE);
-      options.set(ParquetOutputFormat.PAGE_SIZE, PARQUET_DEFAULT_PAGE_SIZE);
-      options.set(ParquetOutputFormat.COMPRESSION, PARQUET_DEFAULT_COMPRESSION_CODEC_NAME);
-      options.set(ParquetOutputFormat.ENABLE_DICTIONARY, PARQUET_DEFAULT_IS_DICTIONARY_ENABLED);
-      options.set(ParquetOutputFormat.VALIDATION, PARQUET_DEFAULT_IS_VALIDATION_ENABLED);
-    }
-
-    return options;
-  }
-
-  static final String fileNamePatternV08 = "part-[0-9]*-[0-9]*";
-  static final String fileNamePatternV09 = "part-[0-9]*-[0-9]*-[0-9]*";
-
-  /**
-   * Written files can be one of two forms: "part-[0-9]*-[0-9]*" or "part-[0-9]*-[0-9]*-[0-9]*".
-   *
-   * This method finds the maximum sequence number from existing data files through the above patterns.
-   * If it cannot find any matched file or the maximum number, it will return -1.
-   *
-   * @param fs
-   * @param path
-   * @param recursive
-   * @return The maximum sequence number
-   * @throws java.io.IOException
-   */
-  public static int getMaxFileSequence(FileSystem fs, Path path, boolean recursive) throws IOException {
-    if (!fs.isDirectory(path)) {
-      return -1;
-    }
-
-    FileStatus[] files = fs.listStatus(path);
-
-    if (files == null || files.length == 0) {
-      return -1;
-    }
-
-    int maxValue = -1;
-    List<Path> fileNamePatternMatchedList = new ArrayList<Path>();
-
-    for (FileStatus eachFile: files) {
-      // In the case of partition table, return largest value within all partition dirs.
-      if (eachFile.isDirectory() && recursive) {
-        int value = getMaxFileSequence(fs, eachFile.getPath(), recursive);
-        if (value > maxValue) {
-          maxValue = value;
-        }
-      } else {
-        if (eachFile.getPath().getName().matches(fileNamePatternV08) ||
-            eachFile.getPath().getName().matches(fileNamePatternV09)) {
-          fileNamePatternMatchedList.add(eachFile.getPath());
-        }
-      }
-    }
-
-    if (fileNamePatternMatchedList.isEmpty()) {
-      return maxValue;
-    }
-    Path lastFile = fileNamePatternMatchedList.get(fileNamePatternMatchedList.size() - 1);
-    String pathName = lastFile.getName();
-
-    // 0.8: pathName = part-<ExecutionBlockId.seq>-<QueryUnitId.seq>
-    // 0.9: pathName = part-<ExecutionBlockId.seq>-<QueryUnitId.seq>-<Sequence>
-    String[] pathTokens = pathName.split("-");
-    if (pathTokens.length == 3) {
-      return -1;
-    } else if(pathTokens.length == 4) {
-      return Integer.parseInt(pathTokens[3]);
-    } else {
-      return -1;
-    }
-  }
-
-  public static void closeBuffer(ByteBuffer buffer) {
-    if (buffer != null) {
-      if (buffer.isDirect()) {
-        ((DirectBuffer) buffer).cleaner().clean();
-      } else {
-        buffer.clear();
-      }
-    }
-  }
-
-  public static int readFully(InputStream is, byte[] buffer, int offset, int length)
-      throws IOException {
-    int nread = 0;
-    while (nread < length) {
-      int nbytes = is.read(buffer, offset + nread, length - nread);
-      if (nbytes < 0) {
-        return nread > 0 ? nread : nbytes;
-      }
-      nread += nbytes;
-    }
-    return nread;
-  }
-
-  /**
-   * Similar to readFully(). Skips bytes in a loop.
-   * @param in The DataInput to skip bytes from
-   * @param len number of bytes to skip.
-   * @throws java.io.IOException if it could not skip requested number of bytes
-   * for any reason (including EOF)
-   */
-  public static void skipFully(DataInput in, int len) throws IOException {
-    int amt = len;
-    while (amt > 0) {
-      long ret = in.skipBytes(amt);
-      if (ret == 0) {
-        // skip may return 0 even if we're not at EOF.  Luckily, we can
-        // use the read() method to figure out if we're at the end.
-        int b = in.readByte();
-        if (b == -1) {
-          throw new EOFException( "Premature EOF from inputStream after " +
-              "skipping " + (len - amt) + " byte(s).");
-        }
-        ret = 1;
-      }
-      amt -= ret;
-    }
-  }
-
-  public static boolean isFileStorageType(StoreType storageType) {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java b/tajo-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java
deleted file mode 100644
index a2c08de..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.statistics.ColumnStats;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-
-/**
- * This class is not thread-safe.
- */
-public class TableStatistics {
-  private static final Log LOG = LogFactory.getLog(TableStatistics.class);
-  private Schema schema;
-  private Tuple minValues;
-  private Tuple maxValues;
-  private long [] numNulls;
-  private long numRows = 0;
-  private long numBytes = 0;
-
-  private boolean [] comparable;
-
-  public TableStatistics(Schema schema) {
-    this.schema = schema;
-    minValues = new VTuple(schema.size());
-    maxValues = new VTuple(schema.size());
-
-    numNulls = new long[schema.size()];
-    comparable = new boolean[schema.size()];
-
-    DataType type;
-    for (int i = 0; i < schema.size(); i++) {
-      type = schema.getColumn(i).getDataType();
-      if (type.getType() == Type.PROTOBUF) {
-        comparable[i] = false;
-      } else {
-        comparable[i] = true;
-      }
-    }
-  }
-
-  public Schema getSchema() {
-    return this.schema;
-  }
-
-  public void incrementRow() {
-    numRows++;
-  }
-
-  public long getNumRows() {
-    return this.numRows;
-  }
-
-  public void setNumBytes(long bytes) {
-    this.numBytes = bytes;
-  }
-
-  public long getNumBytes() {
-    return this.numBytes;
-  }
-
-  public void analyzeField(int idx, Datum datum) {
-    if (datum instanceof NullDatum) {
-      numNulls[idx]++;
-      return;
-    }
-
-    if (comparable[idx]) {
-      if (!maxValues.contains(idx) ||
-          maxValues.get(idx).compareTo(datum) < 0) {
-        maxValues.put(idx, datum);
-      }
-      if (!minValues.contains(idx) ||
-          minValues.get(idx).compareTo(datum) > 0) {
-        minValues.put(idx, datum);
-      }
-    }
-  }
-
-  public TableStats getTableStat() {
-    TableStats stat = new TableStats();
-
-    ColumnStats columnStats;
-    for (int i = 0; i < schema.size(); i++) {
-      columnStats = new ColumnStats(schema.getColumn(i));
-      columnStats.setNumNulls(numNulls[i]);
-      if (minValues.get(i) == null || schema.getColumn(i).getDataType().getType() == minValues.get(i).type()) {
-        columnStats.setMinValue(minValues.get(i));
-      } else {
-        LOG.warn("Wrong statistics column type (" + minValues.get(i).type() +
-            ", expected=" + schema.getColumn(i).getDataType().getType() + ")");
-      }
-      if (maxValues.get(i) == null || schema.getColumn(i).getDataType().getType() == maxValues.get(i).type()) {
-        columnStats.setMaxValue(maxValues.get(i));
-      } else {
-        LOG.warn("Wrong statistics column type (" + maxValues.get(i).type() +
-            ", expected=" + schema.getColumn(i).getDataType().getType() + ")");
-      }
-      stat.addColumnStat(columnStats);
-    }
-
-    stat.setNumRows(this.numRows);
-    stat.setNumBytes(this.numBytes);
-
-    return stat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java b/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
deleted file mode 100644
index 094d285..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
+++ /dev/null
@@ -1,226 +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.protobuf.Message;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.*;
-import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
-import org.apache.tajo.util.Bytes;
-import org.apache.tajo.util.NumberUtil;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-//Compatibility with Apache Hive
-@Deprecated
-public class TextSerializerDeserializer implements SerializerDeserializer {
-  public static final byte[] trueBytes = "true".getBytes();
-  public static final byte[] falseBytes = "false".getBytes();
-  private ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance();
-
-
-  @Override
-  public int serialize(Column col, Datum datum, OutputStream out, byte[] nullCharacters) throws IOException {
-
-    byte[] bytes;
-    int length = 0;
-    TajoDataTypes.DataType dataType = col.getDataType();
-
-    if (datum == null || datum instanceof NullDatum) {
-      switch (dataType.getType()) {
-        case CHAR:
-        case TEXT:
-          length = nullCharacters.length;
-          out.write(nullCharacters);
-          break;
-        default:
-          break;
-      }
-      return length;
-    }
-
-    switch (dataType.getType()) {
-      case BOOLEAN:
-        out.write(datum.asBool() ? trueBytes : falseBytes);
-        length = trueBytes.length;
-        break;
-      case CHAR:
-        byte[] pad = new byte[dataType.getLength() - datum.size()];
-        bytes = datum.asTextBytes();
-        out.write(bytes);
-        out.write(pad);
-        length = bytes.length + pad.length;
-        break;
-      case TEXT:
-      case BIT:
-      case INT2:
-      case INT4:
-      case INT8:
-      case FLOAT4:
-      case FLOAT8:
-      case INET4:
-      case DATE:
-      case INTERVAL:
-        bytes = datum.asTextBytes();
-        length = bytes.length;
-        out.write(bytes);
-        break;
-      case TIME:
-        bytes = ((TimeDatum)datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
-        length = bytes.length;
-        out.write(bytes);
-        break;
-      case TIMESTAMP:
-        bytes = ((TimestampDatum)datum).asChars(TajoConf.getCurrentTimeZone(), true).getBytes();
-        length = bytes.length;
-        out.write(bytes);
-        break;
-      case INET6:
-      case BLOB:
-        bytes = Base64.encodeBase64(datum.asByteArray(), false);
-        length = bytes.length;
-        out.write(bytes, 0, length);
-        break;
-      case PROTOBUF:
-        ProtobufDatum protobuf = (ProtobufDatum) datum;
-        byte[] protoBytes = protobufJsonFormat.printToString(protobuf.get()).getBytes();
-        length = protoBytes.length;
-        out.write(protoBytes, 0, protoBytes.length);
-        break;
-      case NULL_TYPE:
-      default:
-        break;
-    }
-    return length;
-  }
-
-  @Override
-  public Datum deserialize(Column col, byte[] bytes, int offset, int length, byte[] nullCharacters) throws IOException {
-
-    Datum datum;
-    switch (col.getDataType().getType()) {
-      case BOOLEAN:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createBool(bytes[offset] == 't' || bytes[offset] == 'T');
-        break;
-      case BIT:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createBit(Byte.parseByte(new String(bytes, offset, length)));
-        break;
-      case CHAR:
-        datum = isNullText(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createChar(new String(bytes, offset, length).trim());
-        break;
-      case INT1:
-      case INT2:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createInt2((short) NumberUtil.parseInt(bytes, offset, length));
-        break;
-      case INT4:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createInt4(NumberUtil.parseInt(bytes, offset, length));
-        break;
-      case INT8:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createInt8(new String(bytes, offset, length));
-        break;
-      case FLOAT4:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createFloat4(new String(bytes, offset, length));
-        break;
-      case FLOAT8:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createFloat8(NumberUtil.parseDouble(bytes, offset, length));
-        break;
-      case TEXT: {
-        byte[] chars = new byte[length];
-        System.arraycopy(bytes, offset, chars, 0, length);
-        datum = isNullText(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createText(chars);
-        break;
-      }
-      case DATE:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createDate(new String(bytes, offset, length));
-        break;
-      case TIME:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createTime(new String(bytes, offset, length));
-        break;
-      case TIMESTAMP:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createTimestamp(new String(bytes, offset, length));
-        break;
-      case INTERVAL:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createInterval(new String(bytes, offset, length));
-        break;
-      case PROTOBUF: {
-        if (isNull(bytes, offset, length, nullCharacters)) {
-          datum = NullDatum.get();
-        } else {
-          ProtobufDatumFactory factory = ProtobufDatumFactory.get(col.getDataType());
-          Message.Builder builder = factory.newBuilder();
-          try {
-            byte[] protoBytes = new byte[length];
-            System.arraycopy(bytes, offset, protoBytes, 0, length);
-            protobufJsonFormat.merge(protoBytes, builder);
-            datum = factory.createDatum(builder.build());
-          } catch (IOException e) {
-            e.printStackTrace();
-            throw new RuntimeException(e);
-          }
-        }
-        break;
-      }
-      case INET4:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createInet4(new String(bytes, offset, length));
-        break;
-      case BLOB: {
-        if (isNull(bytes, offset, length, nullCharacters)) {
-          datum = NullDatum.get();
-        } else {
-          byte[] blob = new byte[length];
-          System.arraycopy(bytes, offset, blob, 0, length);
-          datum = DatumFactory.createBlob(Base64.decodeBase64(blob));
-        }
-        break;
-      }
-      default:
-        datum = NullDatum.get();
-        break;
-    }
-    return datum;
-  }
-
-  private static boolean isNull(byte[] val, int offset, int length, byte[] nullBytes) {
-    return length == 0 || ((length == nullBytes.length)
-        && Bytes.equals(val, offset, length, nullBytes, 0, nullBytes.length));
-  }
-
-  private static boolean isNullText(byte[] val, int offset, int length, byte[] nullBytes) {
-    return length > 0 && length == nullBytes.length
-        && Bytes.equals(val, offset, length, nullBytes, 0, nullBytes.length);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java b/tajo-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
deleted file mode 100644
index 8dffd8d..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/***
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.tajo.common.ProtoObject;
-
-import java.util.Comparator;
-
-import static org.apache.tajo.index.IndexProtos.TupleComparatorProto;
-
-public abstract class TupleComparator implements Comparator<Tuple>, ProtoObject<TupleComparatorProto> {
-
-  public abstract int compare(Tuple o1, Tuple o2);
-
-  public abstract boolean isAscendingFirstKey();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/TupleRange.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/TupleRange.java b/tajo-storage/src/main/java/org/apache/tajo/storage/TupleRange.java
deleted file mode 100644
index e824b99..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/TupleRange.java
+++ /dev/null
@@ -1,112 +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.base.Objects;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SortSpec;
-
-import java.util.Comparator;
-
-/**
- * It represents a pair of start and end tuples.
- */
-public class TupleRange implements Comparable<TupleRange>, Cloneable {
-  private Tuple start;
-  private Tuple end;
-  private final TupleComparator comp;
-
-  public TupleRange(final SortSpec[] sortSpecs, final Tuple start, final Tuple end) {
-    this.comp = new BaseTupleComparator(sortSpecsToSchema(sortSpecs), sortSpecs);
-    // if there is only one value, start == end
-    this.start = start;
-    this.end = end;
-  }
-
-  public static Schema sortSpecsToSchema(SortSpec[] sortSpecs) {
-    Schema schema = new Schema();
-    for (SortSpec spec : sortSpecs) {
-      schema.addColumn(spec.getSortKey());
-    }
-
-    return schema;
-  }
-
-  public void setStart(Tuple tuple) {
-    this.start = tuple;
-  }
-
-  public final Tuple getStart() {
-    return this.start;
-  }
-
-  public void setEnd(Tuple tuple) {
-    this.end = tuple;
-  }
-
-  public final Tuple getEnd() {
-    return this.end;
-  }
-
-  public String toString() {
-    return "[" + this.start + ", " + this.end + ")";
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(start, end);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof TupleRange) {
-      TupleRange other = (TupleRange) obj;
-      return this.start.equals(other.start) && this.end.equals(other.end);
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int compareTo(TupleRange o) {
-    // TODO - should handle overlap
-    int cmpVal = comp.compare(this.start, o.start);
-    if (cmpVal != 0) {
-      return cmpVal;
-    } else {
-      return comp.compare(this.end, o.end);
-    }
-  }
-
-  public static class DescendingTupleRangeComparator
-      implements Comparator<TupleRange> {
-
-    @Override
-    public int compare(TupleRange left, TupleRange right) {
-      return right.compareTo(left);
-    }
-  }
-
-  public TupleRange clone() throws CloneNotSupportedException {
-    TupleRange newRange = (TupleRange) super.clone();
-    newRange.setStart(start.clone());
-    newRange.setEnd(end.clone());
-    return newRange;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java b/tajo-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
deleted file mode 100644
index ad19101..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
+++ /dev/null
@@ -1,29 +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.annotation;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface ForSplitableStore {
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
deleted file mode 100644
index 9e1e7ea..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/avro/AvroAppender.java
+++ /dev/null
@@ -1,221 +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.avro;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumWriter;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.FileAppender;
-import org.apache.tajo.storage.TableStatistics;
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-
-/**
- * FileAppender for writing to Avro files.
- */
-public class AvroAppender extends FileAppender {
-  private TableStatistics stats;
-  private Schema avroSchema;
-  private List<Schema.Field> avroFields;
-  private DataFileWriter<GenericRecord> dataFileWriter;
-
-  /**
-   * Creates a new AvroAppender.
-   *
-   * @param conf Configuration properties.
-   * @param schema The table schema.
-   * @param meta The table metadata.
-   * @param workDir The path of the Parquet file to write to.
-   */
-  public AvroAppender(Configuration conf,
-                      QueryUnitAttemptId taskAttemptId,
-                      org.apache.tajo.catalog.Schema schema,
-                      TableMeta meta, Path workDir) throws IOException {
-    super(conf, taskAttemptId, schema, meta, workDir);
-  }
-
-  /**
-   * Initializes the Appender.
-   */
-  public void init() throws IOException {
-    FileSystem fs = path.getFileSystem(conf);
-    if (!fs.exists(path.getParent())) {
-      throw new FileNotFoundException(path.toString());
-    }
-    FSDataOutputStream outputStream = fs.create(path);
-
-    avroSchema = AvroUtil.getAvroSchema(meta, conf);
-    avroFields = avroSchema.getFields();
-
-    DatumWriter<GenericRecord> datumWriter =
-        new GenericDatumWriter<GenericRecord>(avroSchema);
-    dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
-    dataFileWriter.create(avroSchema, outputStream);
-
-    if (enabledStats) {
-      this.stats = new TableStatistics(schema);
-    }
-    super.init();
-  }
-
-  /**
-   * Gets the current offset. Tracking offsets is currenly not implemented, so
-   * this method always returns 0.
-   *
-   * @return 0
-   */
-  @Override
-  public long getOffset() throws IOException {
-    return 0;
-  }
-
-  private Object getPrimitive(Tuple tuple, int i, Schema.Type avroType) {
-    if (tuple.get(i) instanceof NullDatum) {
-      return null;
-    }
-    switch (avroType) {
-      case NULL:
-        return null;
-      case BOOLEAN:
-        return tuple.getBool(i);
-      case INT:
-        return tuple.getInt4(i);
-      case LONG:
-        return tuple.getInt8(i);
-      case FLOAT:
-        return tuple.getFloat4(i);
-      case DOUBLE:
-        return tuple.getFloat8(i);
-      case BYTES:
-      case FIXED:
-        return ByteBuffer.wrap(tuple.getBytes(i));
-      case STRING:
-        return tuple.getText(i);
-      default:
-        throw new RuntimeException("Unknown primitive type.");
-    }
-  }
-
-  /**
-   * Write a Tuple to the Avro file.
-   *
-   * @param tuple The Tuple to write.
-   */
-  @Override
-  public void addTuple(Tuple tuple) throws IOException {
-    GenericRecord record = new GenericData.Record(avroSchema);
-    for (int i = 0; i < schema.size(); ++i) {
-      Column column = schema.getColumn(i);
-      if (enabledStats) {
-        stats.analyzeField(i, tuple.get(i));
-      }
-      Object value;
-      Schema.Field avroField = avroFields.get(i);
-      Schema.Type avroType = avroField.schema().getType();
-      switch (avroType) {
-        case NULL:
-        case BOOLEAN:
-        case INT:
-        case LONG:
-        case FLOAT:
-        case DOUBLE:
-        case BYTES:
-        case STRING:
-        case FIXED:
-          value = getPrimitive(tuple, i, avroType);
-          break;
-        case RECORD:
-          throw new RuntimeException("Avro RECORD not supported.");
-        case ENUM:
-          throw new RuntimeException("Avro ENUM not supported.");
-        case MAP:
-          throw new RuntimeException("Avro MAP not supported.");
-        case UNION:
-          List<Schema> schemas = avroField.schema().getTypes();
-          if (schemas.size() != 2) {
-            throw new RuntimeException("Avro UNION not supported.");
-          }
-          if (schemas.get(0).getType().equals(Schema.Type.NULL)) {
-            value = getPrimitive(tuple, i, schemas.get(1).getType());
-          } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) {
-            value = getPrimitive(tuple, i, schemas.get(0).getType());
-          } else {
-            throw new RuntimeException("Avro UNION not supported.");
-          }
-          break;
-        default:
-          throw new RuntimeException("Unknown type: " + avroType);
-      }
-      record.put(i, value);
-    }
-    dataFileWriter.append(record);
-
-    if (enabledStats) {
-      stats.incrementRow();
-    }
-  }
-
-  /**
-   * Flushes the current state of the file.
-   */
-  @Override
-  public void flush() throws IOException {
-    dataFileWriter.flush();
-  }
-
-  /**
-   * Closes the Appender.
-   */
-  @Override
-  public void close() throws IOException {
-    dataFileWriter.close();
-  }
-
-  /**
-   * If table statistics is enabled, retrieve the table statistics.
-   *
-   * @return Table statistics if enabled or null otherwise.
-   */
-  @Override
-  public TableStats getStats() {
-    if (enabledStats) {
-      return stats.getTableStat();
-    } else {
-      return null;
-    }
-  }
-}


[21/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
deleted file mode 100644
index 23815d9..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
+++ /dev/null
@@ -1,1808 +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.rcfile;
-
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.commons.lang.StringUtils;
-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.FileSystem;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.io.SequenceFile.Metadata;
-import org.apache.hadoop.io.compress.*;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.Closeable;
-import java.io.*;
-import java.rmi.server.UID;
-import java.security.MessageDigest;
-import java.util.Arrays;
-
-/**
- * <code>RCFile</code>s, short of Record Columnar File, are flat files
- * consisting of binary key/value pairs, which shares much similarity with
- * <code>SequenceFile</code>.
- * <p/>
- * RCFile stores columns of a table in a record columnar way. It first
- * partitions rows horizontally into row splits. and then it vertically
- * partitions each row split in a columnar way. RCFile first stores the meta
- * data of a row split, as the key part of a record, and all the data of a row
- * split as the value part. When writing, RCFile.Writer first holds records'
- * value bytes in memory, and determines a row split if the raw bytes size of
- * buffered records overflow a given parameter<tt>Writer.columnsBufferSize</tt>,
- * which can be set like: <code>conf.setInt(COLUMNS_BUFFER_SIZE_CONF_STR,
- * 4 * 1024 * 1024)</code> .
- * <p>
- * <code>RCFile</code> provides {@link Writer}, {@link Reader} and classes for
- * writing, reading respectively.
- * </p>
- * <p/>
- * <p>
- * RCFile stores columns of a table in a record columnar way. It first
- * partitions rows horizontally into row splits. and then it vertically
- * partitions each row split in a columnar way. RCFile first stores the meta
- * data of a row split, as the key part of a record, and all the data of a row
- * split as the value part.
- * </p>
- * <p/>
- * <p>
- * RCFile compresses values in a more fine-grained manner then record level
- * compression. However, It currently does not support compress the key part
- * yet. The actual compression algorithm used to compress key and/or values can
- * be specified by using the appropriate {@link CompressionCodec}.
- * </p>
- * <p/>
- * <p>
- * The {@link Reader} is used to read and explain the bytes of RCFile.
- * </p>
- * <p/>
- * <h4 id="Formats">RCFile Formats</h4>
- * <p/>
- * <p/>
- * <h5 id="Header">RC Header</h5>
- * <ul>
- * <li>version - 3 bytes of magic header <b>RCF</b>, followed by 1 byte of
- * actual version number (e.g. RCF1)</li>
- * <li>compression - A boolean which specifies if compression is turned on for
- * keys/values in this file.</li>
- * <li>compression codec - <code>CompressionCodec</code> class which is used
- * for compression of keys and/or values (if compression is enabled).</li>
- * <li>metadata - {@link Metadata} for this file.</li>
- * <li>sync - A sync marker to denote end of the header.</li>
- * </ul>
- * <p/>
- * <h5>RCFile Format</h5>
- * <ul>
- * <li><a href="#Header">Header</a></li>
- * <li>Record
- * <li>Key part
- * <ul>
- * <li>Record length in bytes</li>
- * <li>Key length in bytes</li>
- * <li>Number_of_rows_in_this_record(vint)</li>
- * <li>Column_1_ondisk_length(vint)</li>
- * <li>Column_1_row_1_value_plain_length</li>
- * <li>Column_1_row_2_value_plain_length</li>
- * <li>...</li>
- * <li>Column_2_ondisk_length(vint)</li>
- * <li>Column_2_row_1_value_plain_length</li>
- * <li>Column_2_row_2_value_plain_length</li>
- * <li>...</li>
- * </ul>
- * </li>
- * </li>
- * <li>Value part
- * <ul>
- * <li>Compressed or plain data of [column_1_row_1_value,
- * column_1_row_2_value,....]</li>
- * <li>Compressed or plain data of [column_2_row_1_value,
- * column_2_row_2_value,....]</li>
- * </ul>
- * </li>
- * </ul>
- * <p>
- * <pre>
- * {@code
- * The following is a pseudo-BNF grammar for RCFile. Comments are prefixed
- * with dashes:
- *
- * rcfile ::=
- *   <file-header>
- *   <rcfile-rowgroup>+
- *
- * file-header ::=
- *   <file-version-header>
- *   <file-key-class-name>              (only exists if version is seq6)
- *   <file-value-class-name>            (only exists if version is seq6)
- *   <file-is-compressed>
- *   <file-is-block-compressed>         (only exists if version is seq6)
- *   [<file-compression-codec-class>]
- *   <file-header-metadata>
- *   <file-sync-field>
- *
- * -- The normative RCFile implementation included with Hive is actually
- * -- based on a modified version of Hadoop's SequenceFile code. Some
- * -- things which should have been modified were not, including the code
- * -- that writes out the file version header. Consequently, RCFile and
- * -- SequenceFile originally shared the same version header.  A newer
- * -- release has created a unique version string.
- *
- * file-version-header ::= Byte[4] {'S', 'E', 'Q', 6}
- *                     |   Byte[4] {'R', 'C', 'F', 1}
- *
- * -- The name of the Java class responsible for reading the key buffer
- * -- component of the rowgroup.
- *
- * file-key-class-name ::=
- *   Text {"org.apache.hadoop.hive.ql.io.RCFile$KeyBuffer"}
- *
- * -- The name of the Java class responsible for reading the value buffer
- * -- component of the rowgroup.
- *
- * file-value-class-name ::=
- *   Text {"org.apache.hadoop.hive.ql.io.RCFile$ValueBuffer"}
- *
- * -- Boolean variable indicating whether or not the file uses compression
- * -- for the key and column buffer sections.
- *
- * file-is-compressed ::= Byte[1]
- *
- * -- A boolean field indicating whether or not the file is block compressed.
- * -- This field is *always* false. According to comments in the original
- * -- RCFile implementation this field was retained for backwards
- * -- compatability with the SequenceFile format.
- *
- * file-is-block-compressed ::= Byte[1] {false}
- *
- * -- The Java class name of the compression codec iff <file-is-compressed>
- * -- is true. The named class must implement
- * -- org.apache.hadoop.io.compress.CompressionCodec.
- * -- The expected value is org.apache.hadoop.io.compress.GzipCodec.
- *
- * file-compression-codec-class ::= Text
- *
- * -- A collection of key-value pairs defining metadata values for the
- * -- file. The Map is serialized using standard JDK serialization, i.e.
- * -- an Int corresponding to the number of key-value pairs, followed by
- * -- Text key and value pairs. The following metadata properties are
- * -- mandatory for all RCFiles:
- * --
- * -- hive.io.rcfile.column.number: the number of columns in the RCFile
- *
- * file-header-metadata ::= Map<Text, Text>
- *
- * -- A 16 byte marker that is generated by the writer. This marker appears
- * -- at regular intervals at the beginning of rowgroup-headers, and is
- * -- intended to enable readers to skip over corrupted rowgroups.
- *
- * file-sync-hash ::= Byte[16]
- *
- * -- Each row group is split into three sections: a header, a set of
- * -- key buffers, and a set of column buffers. The header section includes
- * -- an optional sync hash, information about the size of the row group, and
- * -- the total number of rows in the row group. Each key buffer
- * -- consists of run-length encoding data which is used to decode
- * -- the length and offsets of individual fields in the corresponding column
- * -- buffer.
- *
- * rcfile-rowgroup ::=
- *   <rowgroup-header>
- *   <rowgroup-key-data>
- *   <rowgroup-column-buffers>
- *
- * rowgroup-header ::=
- *   [<rowgroup-sync-marker>, <rowgroup-sync-hash>]
- *   <rowgroup-record-length>
- *   <rowgroup-key-length>
- *   <rowgroup-compressed-key-length>
- *
- * -- rowgroup-key-data is compressed if the column data is compressed.
- * rowgroup-key-data ::=
- *   <rowgroup-num-rows>
- *   <rowgroup-key-buffers>
- *
- * -- An integer (always -1) signaling the beginning of a sync-hash
- * -- field.
- *
- * rowgroup-sync-marker ::= Int
- *
- * -- A 16 byte sync field. This must match the <file-sync-hash> value read
- * -- in the file header.
- *
- * rowgroup-sync-hash ::= Byte[16]
- *
- * -- The record-length is the sum of the number of bytes used to store
- * -- the key and column parts, i.e. it is the total length of the current
- * -- rowgroup.
- *
- * rowgroup-record-length ::= Int
- *
- * -- Total length in bytes of the rowgroup's key sections.
- *
- * rowgroup-key-length ::= Int
- *
- * -- Total compressed length in bytes of the rowgroup's key sections.
- *
- * rowgroup-compressed-key-length ::= Int
- *
- * -- Number of rows in the current rowgroup.
- *
- * rowgroup-num-rows ::= VInt
- *
- * -- One or more column key buffers corresponding to each column
- * -- in the RCFile.
- *
- * rowgroup-key-buffers ::= <rowgroup-key-buffer>+
- *
- * -- Data in each column buffer is stored using a run-length
- * -- encoding scheme that is intended to reduce the cost of
- * -- repeated column field values. This mechanism is described
- * -- in more detail in the following entries.
- *
- * rowgroup-key-buffer ::=
- *   <column-buffer-length>
- *   <column-buffer-uncompressed-length>
- *   <column-key-buffer-length>
- *   <column-key-buffer>
- *
- * -- The serialized length on disk of the corresponding column buffer.
- *
- * column-buffer-length ::= VInt
- *
- * -- The uncompressed length of the corresponding column buffer. This
- * -- is equivalent to column-buffer-length if the RCFile is not compressed.
- *
- * column-buffer-uncompressed-length ::= VInt
- *
- * -- The length in bytes of the current column key buffer
- *
- * column-key-buffer-length ::= VInt
- *
- * -- The column-key-buffer contains a sequence of serialized VInt values
- * -- corresponding to the byte lengths of the serialized column fields
- * -- in the corresponding rowgroup-column-buffer. For example, consider
- * -- an integer column that contains the consecutive values 1, 2, 3, 44.
- * -- The RCFile format stores these values as strings in the column buffer,
- * -- e.g. "12344". The length of each column field is recorded in
- * -- the column-key-buffer as a sequence of VInts: 1,1,1,2. However,
- * -- if the same length occurs repeatedly, then we replace repeated
- * -- run lengths with the complement (i.e. negative) of the number of
- * -- repetitions, so 1,1,1,2 becomes 1,~2,2.
- *
- * column-key-buffer ::= Byte[column-key-buffer-length]
- *
- * rowgroup-column-buffers ::= <rowgroup-value-buffer>+
- *
- * -- RCFile stores all column data as strings regardless of the
- * -- underlying column type. The strings are neither length-prefixed or
- * -- null-terminated, and decoding them into individual fields requires
- * -- the use of the run-length information contained in the corresponding
- * -- column-key-buffer.
- *
- * rowgroup-column-buffer ::= Byte[column-buffer-length]
- *
- * Byte ::= An eight-bit byte
- *
- * VInt ::= Variable length integer. The high-order bit of each byte
- * indicates whether more bytes remain to be read. The low-order seven
- * bits are appended as increasingly more significant bits in the
- * resulting integer value.
- *
- * Int ::= A four-byte integer in big-endian format.
- *
- * Text ::= VInt, Chars (Length prefixed UTF-8 characters)
- * }
- * </pre>
- * </p>
- */
-public class RCFile {
-
-  private static final Log LOG = LogFactory.getLog(RCFile.class);
-
-  public static final String RECORD_INTERVAL_CONF_STR = "hive.io.rcfile.record.interval";
-  public static final String COLUMN_NUMBER_METADATA_STR = "hive.io.rcfile.column.number";
-
-  // All of the versions should be place in this list.
-  private static final int ORIGINAL_VERSION = 0;  // version with SEQ
-  private static final int NEW_MAGIC_VERSION = 1; // version with RCF
-
-  private static final int CURRENT_VERSION = NEW_MAGIC_VERSION;
-
-  // The first version of RCFile used the sequence file header.
-  private static final byte[] ORIGINAL_MAGIC = new byte[]{
-      (byte) 'S', (byte) 'E', (byte) 'Q'};
-  // the version that was included with the original magic, which is mapped
-  // into ORIGINAL_VERSION
-  private static final byte ORIGINAL_MAGIC_VERSION_WITH_METADATA = 6;
-
-  private static final byte[] ORIGINAL_MAGIC_VERSION = new byte[]{
-      (byte) 'S', (byte) 'E', (byte) 'Q', ORIGINAL_MAGIC_VERSION_WITH_METADATA
-  };
-
-  // The 'magic' bytes at the beginning of the RCFile
-  private static final byte[] MAGIC = new byte[]{
-      (byte) 'R', (byte) 'C', (byte) 'F'};
-
-  private static final int SYNC_ESCAPE = -1; // "length" of sync entries
-  private static final int SYNC_HASH_SIZE = 16; // number of bytes in hash
-  private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE; // escape + hash
-
-  /**
-   * The number of bytes between sync points.
-   */
-  public static final int SYNC_INTERVAL = 100 * SYNC_SIZE;
-  public static final String NULL = "rcfile.null";
-  public static final String SERDE = "rcfile.serde";
-
-  /**
-   * KeyBuffer is the key of each record in RCFile. Its on-disk layout is as
-   * below:
-   * <p/>
-   * <ul>
-   * <li>record length in bytes,it is the sum of bytes used to store the key
-   * part and the value part.</li>
-   * <li>Key length in bytes, it is how many bytes used by the key part.</li>
-   * <li>number_of_rows_in_this_record(vint),</li>
-   * <li>column_1_ondisk_length(vint),</li>
-   * <li>column_1_row_1_value_plain_length,</li>
-   * <li>column_1_row_2_value_plain_length,</li>
-   * <li>....</li>
-   * <li>column_2_ondisk_length(vint),</li>
-   * <li>column_2_row_1_value_plain_length,</li>
-   * <li>column_2_row_2_value_plain_length,</li>
-   * <li>.... .</li>
-   * <li>{the end of the key part}</li>
-   * </ul>
-   */
-  public static class KeyBuffer {
-    // each column's length in the value
-    private int[] eachColumnValueLen = null;
-    private int[] eachColumnUncompressedValueLen = null;
-    // stores each cell's length of a column in one DataOutputBuffer element
-    private NonSyncByteArrayOutputStream[] allCellValLenBuffer = null;
-    // how many rows in this split
-    private int numberRows = 0;
-    // how many columns
-    private int columnNumber = 0;
-
-    KeyBuffer(int columnNum) {
-      columnNumber = columnNum;
-      eachColumnValueLen = new int[columnNumber];
-      eachColumnUncompressedValueLen = new int[columnNumber];
-      allCellValLenBuffer = new NonSyncByteArrayOutputStream[columnNumber];
-    }
-
-    public void readFields(DataInput in) throws IOException {
-      eachColumnValueLen = new int[columnNumber];
-      eachColumnUncompressedValueLen = new int[columnNumber];
-      allCellValLenBuffer = new NonSyncByteArrayOutputStream[columnNumber];
-
-      numberRows = WritableUtils.readVInt(in);
-      for (int i = 0; i < columnNumber; i++) {
-        eachColumnValueLen[i] = WritableUtils.readVInt(in);
-        eachColumnUncompressedValueLen[i] = WritableUtils.readVInt(in);
-        int bufLen = WritableUtils.readVInt(in);
-        if (allCellValLenBuffer[i] == null) {
-          allCellValLenBuffer[i] = new NonSyncByteArrayOutputStream();
-        } else {
-          allCellValLenBuffer[i].reset();
-        }
-        allCellValLenBuffer[i].write(in, bufLen);
-      }
-    }
-
-    /**
-     * @return the numberRows
-     */
-    public int getNumberRows() {
-      return numberRows;
-    }
-  }
-
-  /**
-   * ValueBuffer is the value of each record in RCFile. Its on-disk layout is as
-   * below:
-   * <ul>
-   * <li>Compressed or plain data of [column_1_row_1_value,
-   * column_1_row_2_value,....]</li>
-   * <li>Compressed or plain data of [column_2_row_1_value,
-   * column_2_row_2_value,....]</li>
-   * </ul>
-   */
-  public static class ValueBuffer implements Closeable{
-
-    // used to load columns' value into memory
-    private NonSyncByteArrayOutputStream[] loadedColumnsValueBuffer = null;
-
-    boolean inited = false;
-
-    // used for readFields
-    KeyBuffer keyBuffer;
-    private int columnNumber = 0;
-
-    // set true for columns that needed to skip loading into memory.
-    boolean[] skippedColIDs = null;
-
-    CompressionCodec codec;
-    Decompressor decompressor = null;
-    NonSyncDataInputBuffer decompressBuffer = new NonSyncDataInputBuffer();
-    private long readBytes = 0;
-
-
-    public ValueBuffer(KeyBuffer currentKey, int columnNumber,
-                       int[] targets, CompressionCodec codec, boolean[] skippedIDs)
-        throws IOException {
-      keyBuffer = currentKey;
-      this.columnNumber = columnNumber;
-      this.skippedColIDs = skippedIDs;
-      this.codec = codec;
-      loadedColumnsValueBuffer = new NonSyncByteArrayOutputStream[targets.length];
-      if (codec != null) {
-        decompressor = org.apache.tajo.storage.compress.CodecPool.getDecompressor(codec);
-      }
-
-      for (int i = 0; i < targets.length; i++) {
-        loadedColumnsValueBuffer[i] = new NonSyncByteArrayOutputStream();
-      }
-    }
-
-    public void readFields(DataInput in) throws IOException {
-      int addIndex = 0;
-      int skipTotal = 0;
-
-
-      for (int i = 0; i < columnNumber; i++) {
-        int vaRowsLen = keyBuffer.eachColumnValueLen[i];
-        // skip this column
-        if (skippedColIDs[i]) {
-          skipTotal += vaRowsLen;
-          continue;
-        }
-
-        if (skipTotal != 0) {
-          StorageUtil.skipFully(in, skipTotal);
-          skipTotal = 0;
-        }
-
-        NonSyncByteArrayOutputStream valBuf;
-        if (codec != null) {
-          // load into compressed buf first
-
-          byte[] compressedBytes = new byte[vaRowsLen];
-          in.readFully(compressedBytes, 0, vaRowsLen);
-
-          decompressBuffer.reset(compressedBytes, vaRowsLen);
-          if(decompressor != null) decompressor.reset();
-
-          DataInputStream is;
-          if (codec instanceof SplittableCompressionCodec) {
-            SplitCompressionInputStream deflatFilter = ((SplittableCompressionCodec) codec).createInputStream(
-                decompressBuffer, decompressor, 0, vaRowsLen, SplittableCompressionCodec.READ_MODE.BYBLOCK);
-            is = new DataInputStream(deflatFilter);
-          } else {
-            CompressionInputStream deflatFilter = codec.createInputStream(decompressBuffer, decompressor);
-            is = new DataInputStream(deflatFilter);
-          }
-
-          valBuf = loadedColumnsValueBuffer[addIndex];
-          valBuf.reset();
-          valBuf.write(is, keyBuffer.eachColumnUncompressedValueLen[i]);
-          is.close();
-          decompressBuffer.close();
-        } else {
-          valBuf = loadedColumnsValueBuffer[addIndex];
-          valBuf.reset();
-          valBuf.write(in, vaRowsLen);
-        }
-        readBytes += keyBuffer.eachColumnUncompressedValueLen[i];
-        addIndex++;
-      }
-
-      if (skipTotal != 0) {
-        StorageUtil.skipFully(in, skipTotal);
-      }
-    }
-
-    public long getReadBytes() {
-      return readBytes;
-    }
-
-    public void clearColumnBuffer() throws IOException {
-      decompressBuffer.reset();
-      readBytes = 0;
-    }
-
-    @Override
-    public void close() {
-      for (NonSyncByteArrayOutputStream element : loadedColumnsValueBuffer) {
-        IOUtils.closeStream(element);
-      }
-      if (codec != null) {
-        IOUtils.closeStream(decompressBuffer);
-        if (decompressor != null) {
-          // Make sure we only return decompressor once.
-          org.apache.tajo.storage.compress.CodecPool.returnDecompressor(decompressor);
-          decompressor = null;
-        }
-      }
-    }
-  }
-
-  /**
-   * Create a metadata object with alternating key-value pairs.
-   * Eg. metadata(key1, value1, key2, value2)
-   */
-  public static Metadata createMetadata(Text... values) {
-    if (values.length % 2 != 0) {
-      throw new IllegalArgumentException("Must have a matched set of " +
-          "key-value pairs. " + values.length +
-          " strings supplied.");
-    }
-    Metadata result = new Metadata();
-    for (int i = 0; i < values.length; i += 2) {
-      result.set(values[i], values[i + 1]);
-    }
-    return result;
-  }
-
-  /**
-   * Write KeyBuffer/ValueBuffer pairs to a RCFile. RCFile's format is
-   * compatible with SequenceFile's.
-   */
-  public static class RCFileAppender extends FileAppender {
-    FSDataOutputStream out;
-
-    CompressionCodec codec = null;
-    Metadata metadata = null;
-    FileSystem fs = null;
-    TableStatistics stats = null;
-    int columnNumber = 0;
-
-    // how many records the writer buffers before it writes to disk
-    private int RECORD_INTERVAL = Integer.MAX_VALUE;
-    // the max size of memory for buffering records before writes them out
-    private int COLUMNS_BUFFER_SIZE = 16 * 1024 * 1024; // 16M
-    // the conf string for COLUMNS_BUFFER_SIZE
-    public static final String COLUMNS_BUFFER_SIZE_CONF_STR = "hive.io.rcfile.record.buffer.size";
-
-    // how many records already buffered
-    private int bufferedRecords = 0;
-    private ColumnBuffer[] columnBuffers = null;
-    boolean useNewMagic = true;
-    private byte[] nullChars;
-    private SerializerDeserializer serde;
-    private boolean isShuffle;
-
-    // Insert a globally unique 16-byte value every few entries, so that one
-    // can seek into the middle of a file and then synchronize with record
-    // starts and ends by scanning for this value.
-    long lastSyncPos; // position of last sync
-    byte[] sync; // 16 random bytes
-
-    {
-      try {
-        MessageDigest digester = MessageDigest.getInstance("MD5");
-        long time = System.currentTimeMillis();
-        digester.update((new UID() + "@" + time).getBytes());
-        sync = digester.digest();
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    /*
-     * used for buffering appends before flush them out
-     */
-    class ColumnBuffer {
-      // used for buffer a column's values
-      NonSyncByteArrayOutputStream columnValBuffer;
-      // used to store each value's length
-      NonSyncByteArrayOutputStream valLenBuffer;
-
-      /*
-       * use a run-length encoding. We only record run length if a same
-       * 'prevValueLen' occurs more than one time. And we negative the run
-       * length to distinguish a runLength and a normal value length. For
-       * example, if the values' lengths are 1,1,1,2, we record 1, ~2,2. And for
-       * value lengths 1,2,3 we record 1,2,3.
-       */
-      int columnValueLength = 0;
-      int uncompressedColumnValueLength = 0;
-      int columnKeyLength = 0;
-      int runLength = 0;
-      int prevValueLength = -1;
-
-      ColumnBuffer() throws IOException {
-        columnValBuffer = new NonSyncByteArrayOutputStream();
-        valLenBuffer = new NonSyncByteArrayOutputStream();
-      }
-
-      public int append(Column column, Datum datum) throws IOException {
-        int currentLen = serde.serialize(column, datum, columnValBuffer, nullChars);
-        columnValueLength += currentLen;
-        uncompressedColumnValueLength += currentLen;
-
-        if (prevValueLength < 0) {
-          startNewGroup(currentLen);
-          return currentLen;
-        }
-
-        if (currentLen != prevValueLength) {
-          flushGroup();
-          startNewGroup(currentLen);
-        } else {
-          runLength++;
-        }
-        return currentLen;
-      }
-
-      private void startNewGroup(int currentLen) {
-        prevValueLength = currentLen;
-        runLength = 0;
-      }
-
-      public void clear() {
-        valLenBuffer.reset();
-        columnValBuffer.reset();
-        prevValueLength = -1;
-        runLength = 0;
-        columnValueLength = 0;
-        columnKeyLength = 0;
-        uncompressedColumnValueLength = 0;
-      }
-
-      public int flushGroup() {
-        int len = 0;
-        if (prevValueLength >= 0) {
-          len += valLenBuffer.writeVLong(prevValueLength);
-          if (runLength > 0) {
-            len += valLenBuffer.writeVLong(~runLength);
-          }
-          columnKeyLength += len;
-          runLength = -1;
-          prevValueLength = -1;
-        }
-        return len;
-      }
-
-      public int UnFlushedGroupSize() {
-        int len = 0;
-        if (prevValueLength >= 0) {
-          len += WritableUtils.getVIntSize(prevValueLength);
-          if (runLength > 0) {
-            len += WritableUtils.getVIntSize(~runLength);
-          }
-        }
-        return len;
-      }
-    }
-
-    public long getLength() throws IOException {
-      return out.getPos();
-    }
-
-    public RCFileAppender(Configuration conf, final QueryUnitAttemptId taskAttemptId,
-                          final Schema schema, final TableMeta meta, final Path workDir) throws IOException {
-      super(conf, taskAttemptId, schema, meta, workDir);
-
-      RECORD_INTERVAL = conf.getInt(RECORD_INTERVAL_CONF_STR, RECORD_INTERVAL);
-      COLUMNS_BUFFER_SIZE = conf.getInt(COLUMNS_BUFFER_SIZE_CONF_STR, COLUMNS_BUFFER_SIZE);
-      columnNumber = schema.size();
-    }
-
-    public void init() throws IOException {
-      fs = path.getFileSystem(conf);
-
-      if (!fs.exists(path.getParent())) {
-        throw new FileNotFoundException(path.toString());
-      }
-
-      //determine the intermediate file type
-      String store = conf.get(TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.varname,
-          TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.defaultVal);
-      if (enabledStats && CatalogProtos.StoreType.RCFILE == CatalogProtos.StoreType.valueOf(store.toUpperCase())) {
-        isShuffle = true;
-      } else {
-        isShuffle = false;
-      }
-
-      if (this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
-        String codecClassname = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
-        try {
-          Class<? extends CompressionCodec> codecClass = conf.getClassByName(
-              codecClassname).asSubclass(CompressionCodec.class);
-          codec = ReflectionUtils.newInstance(codecClass, conf);
-        } catch (ClassNotFoundException cnfe) {
-          throw new IllegalArgumentException(
-              "Unknown codec: " + codecClassname, cnfe);
-        }
-      }
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.RCFILE_NULL,
-          NullDatum.DEFAULT_TEXT));
-      if (StringUtils.isEmpty(nullCharacters)) {
-        nullChars = NullDatum.get().asTextBytes();
-      } else {
-        nullChars = nullCharacters.getBytes();
-      }
-
-      if (metadata == null) {
-        metadata = new Metadata();
-      }
-
-      metadata.set(new Text(COLUMN_NUMBER_METADATA_STR), new Text("" + columnNumber));
-
-      String serdeClass = this.meta.getOption(StorageConstants.RCFILE_SERDE,
-          BinarySerializerDeserializer.class.getName());
-      try {
-        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
-        throw new IOException(e);
-      }
-      metadata.set(new Text(StorageConstants.RCFILE_SERDE), new Text(serdeClass));
-
-      columnBuffers = new ColumnBuffer[columnNumber];
-      for (int i = 0; i < columnNumber; i++) {
-        columnBuffers[i] = new ColumnBuffer();
-      }
-
-      init(conf, fs.create(path, true, 4096, (short) 3, fs.getDefaultBlockSize(), null), codec, metadata);
-      initializeFileHeader();
-      writeFileHeader();
-      finalizeFileHeader();
-
-      if (enabledStats) {
-        this.stats = new TableStatistics(this.schema);
-      }
-      super.init();
-    }
-
-    /**
-     * Write the initial part of file header.
-     */
-    void initializeFileHeader() throws IOException {
-      if (useNewMagic) {
-        out.write(MAGIC);
-        out.write(CURRENT_VERSION);
-      } else {
-        out.write(ORIGINAL_MAGIC_VERSION);
-      }
-    }
-
-    /**
-     * Write the final part of file header.
-     */
-    void finalizeFileHeader() throws IOException {
-      out.write(sync); // write the sync bytes
-      out.flush(); // flush header
-    }
-
-    boolean isCompressed() {
-      return codec != null;
-    }
-
-    /**
-     * Write and flush the file header.
-     */
-    void writeFileHeader() throws IOException {
-      if (useNewMagic) {
-        out.writeBoolean(isCompressed());
-      } else {
-        Text.writeString(out, "org.apache.hadoop.hive.ql.io.RCFile$KeyBuffer");
-        Text.writeString(out, "org.apache.hadoop.hive.ql.io.RCFile$ValueBuffer");
-        out.writeBoolean(isCompressed());
-        out.writeBoolean(false);
-      }
-
-      if (isCompressed()) {
-        Text.writeString(out, (codec.getClass()).getName());
-      }
-      metadata.write(out);
-    }
-
-    void init(Configuration conf, FSDataOutputStream out,
-              CompressionCodec codec, Metadata metadata) throws IOException {
-      this.out = out;
-      this.codec = codec;
-      this.metadata = metadata;
-      this.useNewMagic = conf.getBoolean(TajoConf.ConfVars.HIVEUSEEXPLICITRCFILEHEADER.varname, true);
-    }
-
-    /**
-     * create a sync point.
-     */
-    public void sync() throws IOException {
-      if (sync != null && lastSyncPos != out.getPos()) {
-        out.writeInt(SYNC_ESCAPE); // mark the start of the sync
-        out.write(sync); // write sync
-        lastSyncPos = out.getPos(); // update lastSyncPos
-      }
-    }
-
-    private void checkAndWriteSync() throws IOException {
-      if (sync != null && out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
-        sync();
-      }
-    }
-
-    private int columnBufferSize = 0;
-
-    @Override
-    public long getOffset() throws IOException {
-      return out.getPos();
-    }
-
-    @Override
-    public void flush() throws IOException {
-      flushRecords();
-      out.flush();
-    }
-
-    @Override
-    public void addTuple(Tuple t) throws IOException {
-      append(t);
-      // Statistical section
-
-      if (enabledStats) {
-        stats.incrementRow();
-      }
-    }
-
-    /**
-     * Append a row of values. Currently it only can accept <
-     * {@link Tuple}. If its <code>size()</code> is less than the
-     * column number in the file, zero bytes are appended for the empty columns.
-     * If its size() is greater then the column number in the file, the exceeded
-     * columns' bytes are ignored.
-     *
-     * @param tuple a Tuple with the list of serialized columns
-     * @throws IOException
-     */
-    public void append(Tuple tuple) throws IOException {
-      int size = schema.size();
-
-      for (int i = 0; i < size; i++) {
-        Datum datum = tuple.get(i);
-        int length = columnBuffers[i].append(schema.getColumn(i), datum);
-        columnBufferSize += length;
-        if (isShuffle) {
-          // it is to calculate min/max values, and it is only used for the intermediate file.
-          stats.analyzeField(i, datum);
-        }
-      }
-
-      if (size < columnNumber) {
-        for (int i = size; i < columnNumber; i++) {
-          columnBuffers[i].append(schema.getColumn(i), NullDatum.get());
-          if (isShuffle) {
-            stats.analyzeField(i, NullDatum.get());
-          }
-        }
-      }
-
-      bufferedRecords++;
-      //TODO compression rate base flush
-      if ((columnBufferSize > COLUMNS_BUFFER_SIZE)
-          || (bufferedRecords >= RECORD_INTERVAL)) {
-        flushRecords();
-      }
-    }
-
-    /**
-     * get number of bytes to store the keyBuffer.
-     *
-     * @return number of bytes used to store this KeyBuffer on disk
-     * @throws IOException
-     */
-    public int getKeyBufferSize() throws IOException {
-      int ret = 0;
-      ret += WritableUtils.getVIntSize(bufferedRecords);
-      for (int i = 0; i < columnBuffers.length; i++) {
-        ColumnBuffer currentBuf = columnBuffers[i];
-        ret += WritableUtils.getVIntSize(currentBuf.columnValueLength);
-        ret += WritableUtils.getVIntSize(currentBuf.uncompressedColumnValueLength);
-        ret += WritableUtils.getVIntSize(currentBuf.columnKeyLength);
-        ret += currentBuf.columnKeyLength;
-      }
-
-      return ret;
-    }
-
-    /**
-     * get number of bytes to store the key part.
-     *
-     * @return number of bytes used to store this Key part on disk
-     * @throws IOException
-     */
-    public int getKeyPartSize() throws IOException {
-      int ret = 12; //12 bytes |record count, key length, compressed key length|
-
-      ret += WritableUtils.getVIntSize(bufferedRecords);
-      for (int i = 0; i < columnBuffers.length; i++) {
-        ColumnBuffer currentBuf = columnBuffers[i];
-        ret += WritableUtils.getVIntSize(currentBuf.columnValueLength);
-        ret += WritableUtils.getVIntSize(currentBuf.uncompressedColumnValueLength);
-        ret += WritableUtils.getVIntSize(currentBuf.columnKeyLength);
-        ret += currentBuf.columnKeyLength;
-        ret += currentBuf.UnFlushedGroupSize();
-      }
-
-      return ret;
-    }
-
-    private void WriteKeyBuffer(DataOutputStream out) throws IOException {
-      WritableUtils.writeVLong(out, bufferedRecords);
-      for (int i = 0; i < columnBuffers.length; i++) {
-        ColumnBuffer currentBuf = columnBuffers[i];
-        WritableUtils.writeVLong(out, currentBuf.columnValueLength);
-        WritableUtils.writeVLong(out, currentBuf.uncompressedColumnValueLength);
-        WritableUtils.writeVLong(out, currentBuf.columnKeyLength);
-        currentBuf.valLenBuffer.writeTo(out);
-      }
-    }
-
-    private void flushRecords() throws IOException {
-
-      Compressor compressor = null;
-      NonSyncByteArrayOutputStream valueBuffer = null;
-      CompressionOutputStream deflateFilter = null;
-      DataOutputStream deflateOut = null;
-      boolean isCompressed = isCompressed();
-
-      int valueLength = 0;
-      if (isCompressed) {
-        compressor = org.apache.tajo.storage.compress.CodecPool.getCompressor(codec);
-        if (compressor != null) compressor.reset();  //builtin gzip is null
-
-        valueBuffer = new NonSyncByteArrayOutputStream();
-        deflateFilter = codec.createOutputStream(valueBuffer, compressor);
-        deflateOut = new DataOutputStream(deflateFilter);
-      }
-
-      try {
-        for (int columnIndex = 0; columnIndex < columnNumber; columnIndex++) {
-          ColumnBuffer currentBuf = columnBuffers[columnIndex];
-          currentBuf.flushGroup();
-
-          NonSyncByteArrayOutputStream columnValue = currentBuf.columnValBuffer;
-          int colLen;
-          int plainLen = columnValue.getLength();
-          if (isCompressed) {
-            deflateFilter.resetState();
-            deflateOut.write(columnValue.getData(), 0, columnValue.getLength());
-            deflateOut.flush();
-            deflateFilter.finish();
-            columnValue.close();
-            // find how much compressed data was added for this column
-            colLen = valueBuffer.getLength() - valueLength;
-            currentBuf.columnValueLength = colLen;
-          } else {
-            colLen = plainLen;
-          }
-          valueLength += colLen;
-        }
-      } catch (IOException e) {
-        IOUtils.cleanup(LOG, deflateOut, out);
-        throw e;
-      }
-
-      if (compressor != null) {
-        org.apache.tajo.storage.compress.CodecPool.returnCompressor(compressor);
-      }
-
-      int keyLength = getKeyBufferSize();
-      if (keyLength < 0) {
-        throw new IOException("negative length keys not allowed: " + keyLength);
-      }
-      // Write the key out
-      writeKey(keyLength + valueLength, keyLength);
-      // write the value out
-      if (isCompressed) {
-        try {
-          out.write(valueBuffer.getData(), 0, valueBuffer.getLength());
-        } finally {
-          IOUtils.cleanup(LOG, valueBuffer);
-        }
-      } else {
-        for (int columnIndex = 0; columnIndex < columnNumber; ++columnIndex) {
-          columnBuffers[columnIndex].columnValBuffer.writeTo(out);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Column#" + columnIndex + " : Plain Total Column Value Length: "
-                + columnBuffers[columnIndex].uncompressedColumnValueLength
-                + ",  Compr Total Column Value Length: " + columnBuffers[columnIndex].columnValueLength);
-          }
-        }
-      }
-      // clear the columnBuffers
-      clearColumnBuffers();
-
-      bufferedRecords = 0;
-      columnBufferSize = 0;
-    }
-
-    private void writeKey(int recordLen, int keyLength) throws IOException {
-      checkAndWriteSync(); // sync
-      out.writeInt(recordLen); // total record length
-      out.writeInt(keyLength); // key portion length
-
-      if (this.isCompressed()) {
-        Compressor compressor = org.apache.tajo.storage.compress.CodecPool.getCompressor(codec);
-        if (compressor != null) compressor.reset();  //builtin gzip is null
-
-        NonSyncByteArrayOutputStream compressionBuffer = new NonSyncByteArrayOutputStream();
-        CompressionOutputStream deflateFilter = codec.createOutputStream(compressionBuffer, compressor);
-        DataOutputStream deflateOut = new DataOutputStream(deflateFilter);
-
-        //compress key and write key out
-        compressionBuffer.reset();
-        deflateFilter.resetState();
-        WriteKeyBuffer(deflateOut);
-        deflateOut.flush();
-        deflateFilter.finish();
-        int compressedKeyLen = compressionBuffer.getLength();
-        out.writeInt(compressedKeyLen);
-        compressionBuffer.writeTo(out);
-        compressionBuffer.reset();
-        deflateOut.close();
-        org.apache.tajo.storage.compress.CodecPool.returnCompressor(compressor);
-      } else {
-        out.writeInt(keyLength);
-        WriteKeyBuffer(out);
-      }
-    }
-
-    private void clearColumnBuffers() throws IOException {
-      for (int i = 0; i < columnNumber; i++) {
-        columnBuffers[i].clear();
-      }
-    }
-
-    @Override
-    public TableStats getStats() {
-      if (enabledStats) {
-        return stats.getTableStat();
-      } else {
-        return null;
-      }
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (bufferedRecords > 0) {
-        flushRecords();
-      }
-      clearColumnBuffers();
-
-      if (out != null) {
-        // Statistical section
-        if (enabledStats) {
-          stats.setNumBytes(getOffset());
-        }
-        // Close the underlying stream if we own it...
-        out.flush();
-        IOUtils.cleanup(LOG, out);
-        out = null;
-      }
-    }
-  }
-
-  /**
-   * Read KeyBuffer/ValueBuffer pairs from a RCFile.
-   */
-  public static class RCFileScanner extends FileScanner {
-    private static class SelectedColumn {
-      public int colIndex;
-      public int rowReadIndex;
-      public int runLength;
-      public int prvLength;
-      public boolean isNulled;
-    }
-
-    private FSDataInputStream in;
-
-    private byte version;
-
-    private CompressionCodec codec = null;
-    private Metadata metadata = null;
-
-    private byte[] sync;
-    private byte[] syncCheck;
-    private boolean syncSeen;
-    private long lastSeenSyncPos = 0;
-
-    private long headerEnd;
-    private long start, end;
-    private final long startOffset, endOffset;
-    private int[] targetColumnIndexes;
-
-    private int currentKeyLength;
-    private int currentRecordLength;
-
-    private ValueBuffer currentValue;
-
-    private int readRowsIndexInBuffer = 0;
-
-    private int recordsNumInValBuffer = 0;
-
-    private int columnNumber = 0;
-
-    private boolean more = true;
-
-    private int passedRowsNum = 0;
-
-    private boolean decompress = false;
-
-    private Decompressor keyDecompressor;
-
-    private long readBytes = 0;
-
-    //Current state of each selected column - e.g. current run length, etc.
-    // The size of the array is equal to the number of selected columns
-    private SelectedColumn[] selectedColumns;
-
-    // column value lengths for each of the selected columns
-    private NonSyncDataInputBuffer[] colValLenBufferReadIn;
-
-    private LongWritable rowId;
-    private byte[] nullChars;
-    private SerializerDeserializer serde;
-
-    public RCFileScanner(Configuration conf, final Schema schema, final TableMeta meta,
-                         final Fragment fragment) throws IOException {
-      super(conf, schema, meta, fragment);
-      conf.setInt("io.file.buffer.size", 4096); //TODO remove
-
-      startOffset = this.fragment.getStartKey();
-      endOffset = startOffset + this.fragment.getLength();
-      start = 0;
-    }
-
-    @Override
-    public void init() throws IOException {
-      sync = new byte[SYNC_HASH_SIZE];
-      syncCheck = new byte[SYNC_HASH_SIZE];
-
-      more = startOffset < endOffset;
-      rowId = new LongWritable();
-      readBytes = 0;
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.RCFILE_NULL,
-          NullDatum.DEFAULT_TEXT));
-      if (StringUtils.isEmpty(nullCharacters)) {
-        nullChars = NullDatum.get().asTextBytes();
-      } else {
-        nullChars = nullCharacters.getBytes();
-      }
-
-      // projection
-      if (targets == null) {
-        targets = schema.toArray();
-      }
-
-      targetColumnIndexes = new int[targets.length];
-      for (int i = 0; i < targets.length; i++) {
-        targetColumnIndexes[i] = schema.getColumnId(targets[i].getQualifiedName());
-      }
-      Arrays.sort(targetColumnIndexes);
-
-      FileSystem fs = fragment.getPath().getFileSystem(conf);
-      end = fs.getFileStatus(fragment.getPath()).getLen();
-      in = openFile(fs, fragment.getPath(), 4096);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RCFile open:" + fragment.getPath() + "," + start + "," + (endOffset - startOffset) +
-            "," + fs.getFileStatus(fragment.getPath()).getLen());
-      }
-      //init RCFILE Header
-      boolean succeed = false;
-      try {
-        if (start > 0) {
-          seek(0);
-          initHeader();
-        } else {
-          initHeader();
-        }
-        succeed = true;
-      } finally {
-        if (!succeed) {
-          if (in != null) {
-            try {
-              in.close();
-            } catch (IOException e) {
-              if (LOG != null && LOG.isDebugEnabled()) {
-                LOG.debug("Exception in closing " + in, e);
-              }
-            }
-          }
-        }
-      }
-
-      columnNumber = Integer.parseInt(metadata.get(new Text(COLUMN_NUMBER_METADATA_STR)).toString());
-      selectedColumns = new SelectedColumn[targetColumnIndexes.length];
-      colValLenBufferReadIn = new NonSyncDataInputBuffer[targetColumnIndexes.length];
-      boolean[] skippedColIDs = new boolean[columnNumber];
-      Arrays.fill(skippedColIDs, true);
-      super.init();
-
-      for (int i = 0; i < targetColumnIndexes.length; i++) {
-        int tid = targetColumnIndexes[i];
-        if (tid < columnNumber) {
-          skippedColIDs[tid] = false;
-
-          SelectedColumn col = new SelectedColumn();
-          col.colIndex = tid;
-          col.runLength = 0;
-          col.prvLength = -1;
-          col.rowReadIndex = 0;
-          selectedColumns[i] = col;
-          colValLenBufferReadIn[i] = new NonSyncDataInputBuffer();
-        }
-      }
-
-      currentKey = createKeyBuffer();
-      currentValue = new ValueBuffer(null, columnNumber, targetColumnIndexes, codec, skippedColIDs);
-
-      if (startOffset > getPosition()) {    // TODO use sync cache
-        sync(startOffset); // sync to start
-      }
-    }
-
-    /**
-     * Return the metadata (Text to Text map) that was written into the
-     * file.
-     */
-    public Metadata getMetadata() {
-      return metadata;
-    }
-
-    /**
-     * Return the metadata value associated with the given key.
-     *
-     * @param key the metadata key to retrieve
-     */
-    public Text getMetadataValueOf(Text key) {
-      return metadata.get(key);
-    }
-
-    /**
-     * Override this method to specialize the type of
-     * {@link FSDataInputStream} returned.
-     */
-    protected FSDataInputStream openFile(FileSystem fs, Path file, int bufferSize) throws IOException {
-      return fs.open(file, bufferSize);
-    }
-
-    private void initHeader() throws IOException {
-      byte[] magic = new byte[MAGIC.length];
-      in.readFully(magic);
-
-      if (Arrays.equals(magic, ORIGINAL_MAGIC)) {
-        byte vers = in.readByte();
-        if (vers != ORIGINAL_MAGIC_VERSION_WITH_METADATA) {
-          throw new IOException(fragment.getPath() + " is a version " + vers +
-              " SequenceFile instead of an RCFile.");
-        }
-        version = ORIGINAL_VERSION;
-      } else {
-        if (!Arrays.equals(magic, MAGIC)) {
-          throw new IOException(fragment.getPath() + " not a RCFile and has magic of " +
-              new String(magic));
-        }
-
-        // Set 'version'
-        version = in.readByte();
-        if (version > CURRENT_VERSION) {
-          throw new VersionMismatchException((byte) CURRENT_VERSION, version);
-        }
-      }
-
-      if (version == ORIGINAL_VERSION) {
-        try {
-          Class<?> keyCls = conf.getClassByName(Text.readString(in));
-          Class<?> valCls = conf.getClassByName(Text.readString(in));
-          if (!keyCls.equals(KeyBuffer.class)
-              || !valCls.equals(ValueBuffer.class)) {
-            throw new IOException(fragment.getPath() + " not a RCFile");
-          }
-        } catch (ClassNotFoundException e) {
-          throw new IOException(fragment.getPath() + " not a RCFile", e);
-        }
-      }
-
-      decompress = in.readBoolean(); // is compressed?
-
-      if (version == ORIGINAL_VERSION) {
-        // is block-compressed? it should be always false.
-        boolean blkCompressed = in.readBoolean();
-        if (blkCompressed) {
-          throw new IOException(fragment.getPath() + " not a RCFile.");
-        }
-      }
-
-      // setup the compression codec
-      if (decompress) {
-        String codecClassname = Text.readString(in);
-        try {
-          Class<? extends CompressionCodec> codecClass = conf.getClassByName(
-              codecClassname).asSubclass(CompressionCodec.class);
-          codec = ReflectionUtils.newInstance(codecClass, conf);
-        } catch (ClassNotFoundException cnfe) {
-          throw new IllegalArgumentException(
-              "Unknown codec: " + codecClassname, cnfe);
-        }
-
-        keyDecompressor = org.apache.tajo.storage.compress.CodecPool.getDecompressor(codec);
-      }
-
-      metadata = new Metadata();
-      metadata.readFields(in);
-
-      Text text = metadata.get(new Text(StorageConstants.RCFILE_SERDE));
-
-      try {
-        String serdeClass;
-        if(text != null && !text.toString().isEmpty()){
-          serdeClass = text.toString();
-        } else{
-          serdeClass = this.meta.getOption(StorageConstants.RCFILE_SERDE, BinarySerializerDeserializer.class.getName());
-        }
-        serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
-        throw new IOException(e);
-      }
-
-      in.readFully(sync); // read sync bytes
-      headerEnd = in.getPos();
-      lastSeenSyncPos = headerEnd; //initial sync position
-      readBytes += headerEnd;
-    }
-
-    /**
-     * Return the current byte position in the input file.
-     */
-    public long getPosition() throws IOException {
-      return in.getPos();
-    }
-
-    /**
-     * Set the current byte position in the input file.
-     * <p/>
-     * <p/>
-     * The position passed must be a position returned by
-     * {@link RCFile.RCFileAppender#getLength()} when writing this file. To seek to an
-     * arbitrary position, use {@link RCFile.RCFileScanner#sync(long)}. In another
-     * words, the current seek can only seek to the end of the file. For other
-     * positions, use {@link RCFile.RCFileScanner#sync(long)}.
-     */
-    public void seek(long position) throws IOException {
-      in.seek(position);
-    }
-
-    /**
-     * Resets the values which determine if there are more rows in the buffer
-     * <p/>
-     * This can be used after one calls seek or sync, if one called next before that.
-     * Otherwise, the seek or sync will have no effect, it will continue to get rows from the
-     * buffer built up from the call to next.
-     */
-    public void resetBuffer() {
-      readRowsIndexInBuffer = 0;
-      recordsNumInValBuffer = 0;
-    }
-
-    /**
-     * Seek to the next sync mark past a given position.
-     */
-    public void sync(long position) throws IOException {
-      if (position + SYNC_SIZE >= end) {
-        seek(end);
-        return;
-      }
-
-      //this is to handle syn(pos) where pos < headerEnd.
-      if (position < headerEnd) {
-        // seek directly to first record
-        in.seek(headerEnd);
-        // note the sync marker "seen" in the header
-        syncSeen = true;
-        return;
-      }
-
-      try {
-        seek(position + 4); // skip escape
-
-        int prefix = sync.length;
-        int n = conf.getInt("io.bytes.per.checksum", 512);
-        byte[] buffer = new byte[prefix + n];
-        n = (int) Math.min(n, end - in.getPos());
-        /* fill array with a pattern that will never match sync */
-        Arrays.fill(buffer, (byte) (~sync[0]));
-        while (n > 0 && (in.getPos() + n) <= end) {
-          position = in.getPos();
-          in.readFully(buffer, prefix, n);
-          readBytes += n;
-          /* the buffer has n+sync bytes */
-          for (int i = 0; i < n; i++) {
-            int j;
-            for (j = 0; j < sync.length && sync[j] == buffer[i + j]; j++) {
-              /* nothing */
-            }
-            if (j == sync.length) {
-              /* simplified from (position + (i - prefix) + sync.length) - SYNC_SIZE */
-              in.seek(position + i - SYNC_SIZE);
-              return;
-            }
-          }
-          /* move the last 16 bytes to the prefix area */
-          System.arraycopy(buffer, buffer.length - prefix, buffer, 0, prefix);
-          n = (int) Math.min(n, end - in.getPos());
-        }
-      } catch (ChecksumException e) { // checksum failure
-        handleChecksumException(e);
-      }
-    }
-
-    private void handleChecksumException(ChecksumException e) throws IOException {
-      if (conf.getBoolean("io.skip.checksum.errors", false)) {
-        LOG.warn("Bad checksum at " + getPosition() + ". Skipping entries.");
-        sync(getPosition() + conf.getInt("io.bytes.per.checksum", 512));
-      } else {
-        throw e;
-      }
-    }
-
-    private KeyBuffer createKeyBuffer() {
-      return new KeyBuffer(columnNumber);
-    }
-
-    /**
-     * Read and return the next record length, potentially skipping over a sync
-     * block.
-     *
-     * @return the length of the next record or -1 if there is no next record
-     * @throws IOException
-     */
-    private int readRecordLength() throws IOException {
-      if (in.getPos() >= end) {
-        return -1;
-      }
-      int length = in.readInt();
-      readBytes += 4;
-      if (sync != null && length == SYNC_ESCAPE) { // process
-        // a
-        // sync entry
-        lastSeenSyncPos = in.getPos() - 4; // minus SYNC_ESCAPE's length
-        in.readFully(syncCheck); // read syncCheck
-        readBytes += SYNC_HASH_SIZE;
-        if (!Arrays.equals(sync, syncCheck)) {
-          throw new IOException("File is corrupt!");
-        }
-        syncSeen = true;
-        if (in.getPos() >= end) {
-          return -1;
-        }
-        length = in.readInt(); // re-read length
-        readBytes += 4;
-      } else {
-        syncSeen = false;
-      }
-      return length;
-    }
-
-    private void seekToNextKeyBuffer() throws IOException {
-      if (!keyInit) {
-        return;
-      }
-      if (!currentValue.inited) {
-        IOUtils.skipFully(in, currentRecordLength - currentKeyLength);
-      }
-    }
-
-    private int compressedKeyLen = 0;
-    NonSyncDataInputBuffer keyDataIn = new NonSyncDataInputBuffer();
-    NonSyncDataInputBuffer keyDecompressBuffer = new NonSyncDataInputBuffer();
-
-    KeyBuffer currentKey = null;
-    boolean keyInit = false;
-
-    protected int nextKeyBuffer() throws IOException {
-      seekToNextKeyBuffer();
-      currentRecordLength = readRecordLength();
-      if (currentRecordLength == -1) {
-        keyInit = false;
-        return -1;
-      }
-      currentKeyLength = in.readInt();
-      compressedKeyLen = in.readInt();
-      readBytes += 8;
-      if (decompress) {
-
-        byte[] compressedBytes = new byte[compressedKeyLen];
-        in.readFully(compressedBytes, 0, compressedKeyLen);
-
-        if (keyDecompressor != null) keyDecompressor.reset();
-        keyDecompressBuffer.reset(compressedBytes, compressedKeyLen);
-
-        DataInputStream is;
-        if (codec instanceof SplittableCompressionCodec) {
-          SplitCompressionInputStream deflatFilter = ((SplittableCompressionCodec) codec).createInputStream(
-              keyDecompressBuffer, keyDecompressor, 0, compressedKeyLen, SplittableCompressionCodec.READ_MODE.BYBLOCK);
-
-          keyDecompressBuffer.seek(deflatFilter.getAdjustedStart());
-          is = new DataInputStream(deflatFilter);
-        } else {
-          CompressionInputStream deflatFilter = codec.createInputStream(keyDecompressBuffer, keyDecompressor);
-          is = new DataInputStream(deflatFilter);
-        }
-
-        byte[] deCompressedBytes = new byte[currentKeyLength];
-
-        is.readFully(deCompressedBytes, 0, currentKeyLength);
-        keyDataIn.reset(deCompressedBytes, currentKeyLength);
-        currentKey.readFields(keyDataIn);
-        is.close();
-      } else {
-        currentKey.readFields(in);
-      }
-      readBytes += currentKeyLength;
-      keyInit = true;
-      currentValue.inited = false;
-
-      readRowsIndexInBuffer = 0;
-      recordsNumInValBuffer = currentKey.numberRows;
-
-      for (int selIx = 0; selIx < selectedColumns.length; selIx++) {
-        SelectedColumn col = selectedColumns[selIx];
-        if (col == null) {
-          col = new SelectedColumn();
-          col.isNulled = true;
-          selectedColumns[selIx] = col;
-          continue;
-        }
-
-        int colIx = col.colIndex;
-        NonSyncByteArrayOutputStream buf = currentKey.allCellValLenBuffer[colIx];
-        colValLenBufferReadIn[selIx].reset(buf.getData(), buf.getLength());
-        col.rowReadIndex = 0;
-        col.runLength = 0;
-        col.prvLength = -1;
-        col.isNulled = buf.getLength() == 0;
-      }
-
-      return currentKeyLength;
-    }
-
-    protected void currentValueBuffer() throws IOException {
-      if (!keyInit) {
-        nextKeyBuffer();
-      }
-      currentValue.keyBuffer = currentKey;
-      currentValue.clearColumnBuffer();
-      currentValue.readFields(in);
-      currentValue.inited = true;
-      readBytes += currentValue.getReadBytes();
-
-      if (tableStats != null) {
-        tableStats.setReadBytes(readBytes);
-        tableStats.setNumRows(passedRowsNum);
-      }
-    }
-
-    private boolean rowFetched = false;
-
-    @Override
-    public Tuple next() throws IOException {
-      if (!more) {
-        return null;
-      }
-
-      more = nextBuffer(rowId);
-      long lastSeenSyncPos = lastSeenSyncPos();
-      if (lastSeenSyncPos >= endOffset) {
-        more = false;
-        return null;
-      }
-
-      if (!more) {
-        return null;
-      }
-
-      Tuple tuple = new VTuple(schema.size());
-      getCurrentRow(tuple);
-      return tuple;
-    }
-
-    @Override
-    public float getProgress() {
-      try {
-        if(!more) {
-          return 1.0f;
-        }
-        long filePos = getPosition();
-        if (startOffset == filePos) {
-          return 0.0f;
-        } else {
-          //if scanner read the header, filePos moved to zero
-          return Math.min(1.0f, (float)(Math.max(filePos - startOffset, 0)) / (float)(fragment.getLength()));
-        }
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-        return 0.0f;
-      }
-    }
-
-    /**
-     * Returns how many rows we fetched with nextBuffer(). It only means how many rows
-     * are read by nextBuffer(). The returned result may be smaller than actual number
-     * of rows passed by, because {@link #seek(long)} can change the underlying key buffer and
-     * value buffer.
-     *
-     * @return next row number
-     * @throws IOException
-     */
-    public boolean nextBuffer(LongWritable readRows) throws IOException {
-      if (readRowsIndexInBuffer < recordsNumInValBuffer) {
-        readRows.set(passedRowsNum);
-        readRowsIndexInBuffer++;
-        passedRowsNum++;
-        rowFetched = false;
-        return true;
-      } else {
-        keyInit = false;
-      }
-
-      int ret = -1;
-      try {
-        ret = nextKeyBuffer();
-      } catch (EOFException eof) {
-        eof.printStackTrace();
-      }
-      return (ret > 0) && nextBuffer(readRows);
-    }
-
-    /**
-     * get the current row used,make sure called {@link #next()}
-     * first.
-     *
-     * @throws IOException
-     */
-    public void getCurrentRow(Tuple tuple) throws IOException {
-      if (!keyInit || rowFetched) {
-        return;
-      }
-
-      if (!currentValue.inited) {
-        currentValueBuffer();
-      }
-
-      for (int j = 0; j < selectedColumns.length; ++j) {
-        SelectedColumn col = selectedColumns[j];
-        int i = col.colIndex;
-
-        if (col.isNulled) {
-          tuple.put(i, NullDatum.get());
-        } else {
-          colAdvanceRow(j, col);
-
-          Datum datum = serde.deserialize(schema.getColumn(i),
-              currentValue.loadedColumnsValueBuffer[j].getData(), col.rowReadIndex, col.prvLength, nullChars);
-          tuple.put(i, datum);
-          col.rowReadIndex += col.prvLength;
-        }
-      }
-      rowFetched = true;
-    }
-
-    /**
-     * Advance column state to the next now: update offsets, run lengths etc
-     *
-     * @param selCol - index among selectedColumns
-     * @param col    - column object to update the state of.  prvLength will be
-     *               set to the new read position
-     * @throws IOException
-     */
-    private void colAdvanceRow(int selCol, SelectedColumn col) throws IOException {
-      if (col.runLength > 0) {
-        --col.runLength;
-      } else {
-        int length = (int) WritableUtils.readVLong(colValLenBufferReadIn[selCol]);
-        if (length < 0) {
-          // we reach a runlength here, use the previous length and reset
-          // runlength
-          col.runLength = (~length) - 1;
-        } else {
-          col.prvLength = length;
-          col.runLength = 0;
-        }
-      }
-    }
-
-    /**
-     * Returns true if the previous call to next passed a sync mark.
-     */
-    public boolean syncSeen() {
-      return syncSeen;
-    }
-
-    /**
-     * Returns the last seen sync position.
-     */
-    public long lastSeenSyncPos() {
-      return lastSeenSyncPos;
-    }
-
-    /**
-     * Returns the name of the file.
-     */
-    @Override
-    public String toString() {
-      return fragment.getPath().toString();
-    }
-
-    @Override
-    public void reset() throws IOException {
-      seek(startOffset);
-    }
-
-    @Override
-    public boolean isProjectable() {
-      return true;
-    }
-
-    @Override
-    public boolean isSelectable() {
-      return false;
-    }
-
-    @Override
-    public boolean isSplittable() {
-      return true;
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (tableStats != null) {
-        tableStats.setReadBytes(readBytes);  //Actual Processed Bytes. (decompressed bytes + header - seek)
-        tableStats.setNumRows(passedRowsNum);
-      }
-
-      IOUtils.cleanup(LOG, in, currentValue);
-      if (keyDecompressor != null) {
-        // Make sure we only return decompressor once.
-        org.apache.tajo.storage.compress.CodecPool.returnDecompressor(keyDecompressor);
-        keyDecompressor = null;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
deleted file mode 100644
index 60f1b06..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
+++ /dev/null
@@ -1,43 +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.rcfile;
-
-import org.apache.hadoop.io.compress.CompressionInputStream;
-
-import java.io.InputStream;
-
-/**
- *
- * SchemaAwareCompressionInputStream adds the ability to inform the compression
- * stream what column is being read.
- *
- */
-public abstract class SchemaAwareCompressionInputStream extends CompressionInputStream {
-
-  protected SchemaAwareCompressionInputStream(InputStream in) throws java.io.IOException {
-    super(in);
-  }
-
-  /**
-   * The column being read
-   *
-   * @param columnIndex the index of the column. Use -1 for non-column data
-   */
-  public abstract void setColumnIndex(int columnIndex);
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
deleted file mode 100644
index c0ce8b3..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
+++ /dev/null
@@ -1,44 +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.rcfile;
-
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-
-import java.io.OutputStream;
-
-/**
- *
- * SchemaAwareCompressionOutputStream adds the ability to inform the comression stream
- * the current column being compressed.
- *
- */
-public abstract class SchemaAwareCompressionOutputStream extends CompressionOutputStream {
-
-  protected SchemaAwareCompressionOutputStream(OutputStream out) {
-    super(out);
-  }
-
-  /**
-   *
-   * The column being output
-   *
-   * @param columnIndex the index of the column. Use -1 for non-column data
-   */
-  public abstract void setColumnIndex(int columnIndex);
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
deleted file mode 100644
index 14e0f26..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java
+++ /dev/null
@@ -1,274 +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.sequencefile;
-
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.ProtobufDatum;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.exception.AlreadyExistsStorageException;
-import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
-import org.apache.tajo.util.BytesUtils;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-public class SequenceFileAppender extends FileAppender {
-  private static final Log LOG = LogFactory.getLog(SequenceFileScanner.class);
-
-  private SequenceFile.Writer writer;
-
-  private TableMeta meta;
-  private Schema schema;
-  private TableStatistics stats = null;
-
-  private int columnNum;
-  private FileSystem fs;
-  private char delimiter;
-  private byte[] nullChars;
-
-  private final static int BUFFER_SIZE = 128 * 1024;
-  private long pos = 0;
-
-  private CompressionCodecFactory codecFactory;
-  private CompressionCodec codec;
-
-  private NonSyncByteArrayOutputStream os;
-  private SerializerDeserializer serde;
-
-  long rowCount;
-  private boolean isShuffle;
-
-  private Writable EMPTY_KEY;
-
-  public SequenceFileAppender(Configuration conf, QueryUnitAttemptId taskAttemptId,
-                              Schema schema, TableMeta meta, Path workDir) throws IOException {
-    super(conf, taskAttemptId, schema, meta, workDir);
-    this.meta = meta;
-    this.schema = schema;
-  }
-
-  @Override
-  public void init() throws IOException {
-    os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
-
-    this.fs = path.getFileSystem(conf);
-
-    //determine the intermediate file type
-    String store = conf.get(TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.varname,
-        TajoConf.ConfVars.SHUFFLE_FILE_FORMAT.defaultVal);
-    if (enabledStats && CatalogProtos.StoreType.SEQUENCEFILE == CatalogProtos.StoreType.valueOf(store.toUpperCase())) {
-      isShuffle = true;
-    } else {
-      isShuffle = false;
-    }
-
-    this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_DELIMITER,
-        StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0);
-    this.columnNum = schema.size();
-    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL,
-        NullDatum.DEFAULT_TEXT));
-    if (StringUtils.isEmpty(nullCharacters)) {
-      nullChars = NullDatum.get().asTextBytes();
-    } else {
-      nullChars = nullCharacters.getBytes();
-    }
-
-    if (!fs.exists(path.getParent())) {
-      throw new FileNotFoundException(path.toString());
-    }
-
-    if(this.meta.containsOption(StorageConstants.COMPRESSION_CODEC)) {
-      String codecName = this.meta.getOption(StorageConstants.COMPRESSION_CODEC);
-      codecFactory = new CompressionCodecFactory(conf);
-      codec = codecFactory.getCodecByClassName(codecName);
-    } else {
-      if (fs.exists(path)) {
-        throw new AlreadyExistsStorageException(path);
-      }
-    }
-
-    try {
-      String serdeClass = this.meta.getOption(StorageConstants.SEQUENCEFILE_SERDE,
-          TextSerializerDeserializer.class.getName());
-      serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
-    } catch (Exception e) {
-      LOG.error(e.getMessage(), e);
-      throw new IOException(e);
-    }
-
-    Class<? extends Writable>  keyClass, valueClass;
-    if (serde instanceof BinarySerializerDeserializer) {
-      keyClass = BytesWritable.class;
-      EMPTY_KEY = new BytesWritable();
-      valueClass = BytesWritable.class;
-    } else {
-      keyClass = LongWritable.class;
-      EMPTY_KEY = new LongWritable();
-      valueClass = Text.class;
-    }
-
-    String type = this.meta.getOption(StorageConstants.COMPRESSION_TYPE, CompressionType.NONE.name());
-    if (type.equals(CompressionType.BLOCK.name())) {
-      writer = SequenceFile.createWriter(fs, conf, path, keyClass, valueClass, CompressionType.BLOCK, codec);
-    } else if (type.equals(CompressionType.RECORD.name())) {
-      writer = SequenceFile.createWriter(fs, conf, path, keyClass, valueClass, CompressionType.RECORD, codec);
-    } else {
-      writer = SequenceFile.createWriter(fs, conf, path, keyClass, valueClass, CompressionType.NONE, codec);
-    }
-
-    if (enabledStats) {
-      this.stats = new TableStatistics(this.schema);
-    }
-
-    super.init();
-  }
-
-  @Override
-  public void addTuple(Tuple tuple) throws IOException {
-    Datum datum;
-
-    if (serde instanceof BinarySerializerDeserializer) {
-      byte nullByte = 0;
-      int lasti = 0;
-      for (int i = 0; i < columnNum; i++) {
-        datum = tuple.get(i);
-
-        // set bit to 1 if a field is not null
-        if (null != datum) {
-          nullByte |= 1 << (i % 8);
-        }
-
-        // write the null byte every eight elements or
-        // if this is the last element and serialize the
-        // corresponding 8 struct fields at the same time
-        if (7 == i % 8 || i == columnNum - 1) {
-          os.write(nullByte);
-
-          for (int j = lasti; j <= i; j++) {
-            datum = tuple.get(j);
-
-            switch (schema.getColumn(j).getDataType().getType()) {
-              case TEXT:
-                BytesUtils.writeVLong(os, datum.asTextBytes().length);
-                break;
-              case PROTOBUF:
-                ProtobufDatum protobufDatum = (ProtobufDatum) datum;
-                BytesUtils.writeVLong(os, protobufDatum.asByteArray().length);
-                break;
-              case CHAR:
-              case INET4:
-              case BLOB:
-                BytesUtils.writeVLong(os, datum.asByteArray().length);
-                break;
-              default:
-            }
-
-            serde.serialize(schema.getColumn(j), datum, os, nullChars);
-
-            if (isShuffle) {
-              // it is to calculate min/max values, and it is only used for the intermediate file.
-              stats.analyzeField(j, datum);
-            }
-          }
-          lasti = i + 1;
-          nullByte = 0;
-        }
-      }
-
-      BytesWritable b = new BytesWritable();
-      b.set(os.getData(), 0, os.getLength());
-      writer.append(EMPTY_KEY, b);
-
-    } else {
-      for (int i = 0; i < columnNum; i++) {
-        datum = tuple.get(i);
-        serde.serialize(schema.getColumn(i), datum, os, nullChars);
-
-        if (columnNum -1 > i) {
-          os.write((byte) delimiter);
-        }
-
-        if (isShuffle) {
-          // it is to calculate min/max values, and it is only used for the intermediate file.
-          stats.analyzeField(i, datum);
-        }
-
-      }
-      writer.append(EMPTY_KEY, new Text(os.toByteArray()));
-    }
-
-    os.reset();
-    pos += writer.getLength();
-    rowCount++;
-
-    if (enabledStats) {
-      stats.incrementRow();
-    }
-  }
-
-  @Override
-  public long getOffset() throws IOException {
-    return pos;
-  }
-
-  @Override
-  public void flush() throws IOException {
-    os.flush();
-    writer.close();
-  }
-
-  @Override
-  public void close() throws IOException {
-    // Statistical section
-    if (enabledStats) {
-      stats.setNumBytes(getOffset());
-    }
-
-    os.close();
-    writer.close();
-  }
-
-  @Override
-  public TableStats getStats() {
-    if (enabledStats) {
-      return stats.getTableStat();
-    } else {
-      return null;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
deleted file mode 100644
index 74563ff..0000000
--- a/tajo-storage/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java
+++ /dev/null
@@ -1,336 +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.sequencefile;
-
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.commons.lang.StringUtils;
-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.io.*;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.BytesUtils;
-
-import java.io.IOException;
-
-public class SequenceFileScanner extends FileScanner {
-  private static final Log LOG = LogFactory.getLog(SequenceFileScanner.class);
-
-  private FileSystem fs;
-  private SequenceFile.Reader reader;
-  private SerializerDeserializer serde;
-  private byte[] nullChars;
-  private char delimiter;
-
-  private int currentIdx = 0;
-  private int[] projectionMap;
-
-  private boolean hasBinarySerDe = false;
-  private long totalBytes = 0L;
-
-  private long start, end;
-  private boolean  more = true;
-
-  /**
-   * Whether a field is null or not. Because length is 0 does not means the
-   * field is null. In particular, a 0-length string is not null.
-   */
-  private boolean[] fieldIsNull;
-
-  /**
-   * The start positions and lengths of fields. Only valid when the data is parsed.
-   */
-  private int[] fieldStart;
-  private int[] fieldLength;
-
-  private int elementOffset, elementSize;
-
-  private Writable EMPTY_KEY;
-
-  public SequenceFileScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
-    super(conf, schema, meta, fragment);
-  }
-
-  @Override
-  public void init() throws IOException {
-    // FileFragment information
-    if(fs == null) {
-      fs = FileScanner.getFileSystem((TajoConf)conf, fragment.getPath());
-    }
-
-    reader = new SequenceFile.Reader(fs, fragment.getPath(), conf);
-
-    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(StorageConstants.SEQUENCEFILE_NULL,
-        NullDatum.DEFAULT_TEXT));
-    if (StringUtils.isEmpty(nullCharacters)) {
-      nullChars = NullDatum.get().asTextBytes();
-    } else {
-      nullChars = nullCharacters.getBytes();
-    }
-
-    String delim  = meta.getOption(StorageConstants.SEQUENCEFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
-    this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0);
-
-    this.start = fragment.getStartKey();
-    this.end = start + fragment.getLength();
-
-    if (fragment.getStartKey() > reader.getPosition())
-      reader.sync(this.start);
-
-    more = start < end;
-
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-
-
-    fieldIsNull = new boolean[schema.getColumns().size()];
-    fieldStart = new int[schema.getColumns().size()];
-    fieldLength = new int[schema.getColumns().size()];
-
-    prepareProjection(targets);
-
-    try {
-      String serdeClass = this.meta.getOption(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName());
-      serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
-
-      if (serde instanceof BinarySerializerDeserializer) {
-        hasBinarySerDe = true;
-      }
-
-      Class<? extends Writable> keyClass = (Class<? extends Writable>)Class.forName(reader.getKeyClassName());
-      EMPTY_KEY = keyClass.newInstance();
-
-    } catch (Exception e) {
-      LOG.error(e.getMessage(), e);
-      throw new IOException(e);
-    }
-    super.init();
-  }
-
-  public Writable getKey() {
-    return EMPTY_KEY;
-  }
-
-  private void prepareProjection(Column [] targets) {
-    projectionMap = new int[targets.length];
-
-    int tid;
-    for (int i = 0; i < targets.length; i++) {
-      tid = schema.getColumnId(targets[i].getQualifiedName());
-      projectionMap[i] = tid;
-    }
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    if (!more) return null;
-
-    long pos = reader.getPosition();
-    boolean remaining = reader.next(EMPTY_KEY);
-
-    if (pos >= end && reader.syncSeen()) {
-      more = false;
-    } else {
-      more = remaining;
-    }
-
-    if (more) {
-      Tuple tuple = null;
-      byte[][] cells;
-
-      if (hasBinarySerDe) {
-        BytesWritable bytesWritable = new BytesWritable();
-        reader.getCurrentValue(bytesWritable);
-        tuple = makeTuple(bytesWritable);
-        totalBytes += (long)bytesWritable.getBytes().length;
-      } else {
-        Text text = new Text();
-        reader.getCurrentValue(text);
-        cells = BytesUtils.splitPreserveAllTokens(text.getBytes(), delimiter, projectionMap);
-        totalBytes += (long)text.getBytes().length;
-        tuple = new LazyTuple(schema, cells, 0, nullChars, serde);
-      }
-      currentIdx++;
-      return tuple;
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * In hive, LazyBinarySerDe is serialized as follows: start A B A B A B end bytes[] ->
-   * |-----|---------|--- ... ---|-----|---------|
-   *
-   * Section A is one null-byte, corresponding to eight struct fields in Section
-   * B. Each bit indicates whether the corresponding field is null (0) or not null
-   * (1). Each field is a LazyBinaryObject.
-   *
-   * Following B, there is another section A and B. This pattern repeats until the
-   * all struct fields are serialized.
-   *
-   * So, tajo must make a tuple after parsing hive style BinarySerDe.
-   */
-  private Tuple makeTuple(BytesWritable value) throws IOException{
-    Tuple tuple = new VTuple(schema.getColumns().size());
-
-    int start = 0;
-    int length = value.getLength();
-
-    /**
-     * Please note that one null byte is followed by eight fields, then more
-     * null byte and fields.
-     */
-    int structByteEnd = start + length;
-    byte[] bytes = value.getBytes();
-
-    byte nullByte = bytes[start];
-    int lastFieldByteEnd = start + 1;
-
-    // Go through all bytes in the byte[]
-    for (int i = 0; i < schema.getColumns().size(); i++) {
-      fieldIsNull[i] = true;
-      if ((nullByte & (1 << (i % 8))) != 0) {
-        fieldIsNull[i] = false;
-        parse(schema.getColumn(i), bytes, lastFieldByteEnd);
-
-        fieldStart[i] = lastFieldByteEnd + elementOffset;
-        fieldLength[i] = elementSize;
-        lastFieldByteEnd = fieldStart[i] + fieldLength[i];
-
-        for (int j = 0; j < projectionMap.length; j++) {
-          if (projectionMap[j] == i) {
-            Datum datum = serde.deserialize(schema.getColumn(i), bytes, fieldStart[i], fieldLength[i], nullChars);
-            tuple.put(i, datum);
-          }
-        }
-      }
-
-      // next byte is a null byte if there are more bytes to go
-      if (7 == (i % 8)) {
-        if (lastFieldByteEnd < structByteEnd) {
-          nullByte = bytes[lastFieldByteEnd];
-          lastFieldByteEnd++;
-        } else {
-          // otherwise all null afterwards
-          nullByte = 0;
-          lastFieldByteEnd++;
-        }
-      }
-    }
-
-    return tuple;
-  }
-
-  /**
-   * Check a particular field and set its size and offset in bytes based on the
-   * field type and the bytes arrays.
-   *
-   * For void, boolean, byte, short, int, long, float and double, there is no
-   * offset and the size is fixed. For string, the first four bytes are used to store the size.
-   * So the offset is 4 and the size is computed by concating the first four bytes together.
-   * The first four bytes are defined with respect to the offset in the bytes arrays.
-   *
-   * @param col
-   *          catalog column information
-   * @param bytes
-   *          bytes arrays store the table row
-   * @param offset
-   *          offset of this field
-   */
-  private void parse(Column col, byte[] bytes, int offset) throws
-      IOException {
-    switch (col.getDataType().getType()) {
-      case BOOLEAN:
-      case BIT:
-        elementOffset = 0;
-        elementSize = 1;
-        break;
-      case INT2:
-        elementOffset = 0;
-        elementSize = 2;
-        break;
-      case INT4:
-      case INT8:
-        elementOffset = 0;
-        elementSize = WritableUtils.decodeVIntSize(bytes[offset]);
-        break;
-      case FLOAT4:
-        elementOffset = 0;
-        elementSize = 4;
-        break;
-      case FLOAT8:
-        elementOffset = 0;
-        elementSize = 8;
-        break;
-      case BLOB:
-      case PROTOBUF:
-      case INET4:
-      case CHAR:
-      case TEXT:
-        elementOffset = 1;
-        elementSize = bytes[offset];
-        break;
-      default:
-        elementOffset = 0;
-        elementSize = 0;
-    }
-  }
-
-  @Override
-  public void reset() throws IOException {
-    if (reader != null) {
-      reader.sync(0);
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (reader != null)
-      reader.close();
-
-    if (tableStats != null) {
-      tableStats.setReadBytes(totalBytes);
-      tableStats.setNumRows(currentIdx);
-    }
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return true;
-  }
-
-  @Override
-  public boolean isSelectable() {
-    return true;
-  }
-
-  @Override
-  public boolean isSplittable(){
-    return true;
-  }
-}


[16/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java b/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
deleted file mode 100644
index d2cfd82..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
+++ /dev/null
@@ -1,946 +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.index;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.index.bst.BSTIndex;
-import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexReader;
-import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexWriter;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Random;
-
-import static org.junit.Assert.*;
-
-@RunWith(Parameterized.class)
-public class TestBSTIndex {
-  private TajoConf conf;
-  private Schema schema;
-  private TableMeta meta;
-
-  private static final int TUPLE_NUM = 10000;
-  private static final int LOAD_NUM = 100;
-  private static final String TEST_PATH = "target/test-data/TestIndex";
-  private Path testDir;
-  private FileSystem fs;
-  private StoreType storeType;
-
-  public TestBSTIndex(StoreType type) {
-    this.storeType = type;
-    conf = new TajoConf();
-    conf.setVar(TajoConf.ConfVars.ROOT_DIR, TEST_PATH);
-    schema = new Schema();
-    schema.addColumn(new Column("int", Type.INT4));
-    schema.addColumn(new Column("long", Type.INT8));
-    schema.addColumn(new Column("double", Type.FLOAT8));
-    schema.addColumn(new Column("float", Type.FLOAT4));
-    schema.addColumn(new Column("string", Type.TEXT));
-  }
-
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> generateParameters() {
-    return Arrays.asList(new Object[][]{
-        {StoreType.CSV},
-        {StoreType.RAW}
-    });
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-  }
-
-  @Test
-  public void testFindValue() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testFindValue_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    Tuple tuple;
-    for (int i = 0; i < TUPLE_NUM; i++) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindValue_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX,
-        keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(1));
-      keyTuple.put(1, tuple.get(2));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    tuple = new VTuple(keySchema.size());
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + storeType + ".idx"), keySchema, comp);
-    reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    for (int i = 0; i < TUPLE_NUM - 1; i++) {
-      tuple.put(0, DatumFactory.createInt8(i));
-      tuple.put(1, DatumFactory.createFloat8(i));
-      long offsets = reader.find(tuple);
-      scanner.seek(offsets);
-      tuple = scanner.next();
-      assertTrue("seek check [" + (i) + " ," + (tuple.get(1).asInt8()) + "]", (i) == (tuple.get(1).asInt8()));
-      assertTrue("seek check [" + (i) + " ," + (tuple.get(2).asFloat8()) + "]", (i) == (tuple.get(2).asFloat8()));
-
-      offsets = reader.next();
-      if (offsets == -1) {
-        continue;
-      }
-      scanner.seek(offsets);
-      tuple = scanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(0).asInt4()));
-      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(1).asInt8()));
-    }
-    reader.close();
-    scanner.close();
-  }
-
-  @Test
-  public void testBuildIndexWithAppender() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testBuildIndexWithAppender_" + storeType);
-    FileAppender appender = (FileAppender) StorageManager.getFileStorageManager(conf).getAppender(meta, schema,
-        tablePath);
-    appender.init();
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    Tuple tuple;
-    long offset;
-    for (int i = 0; i < TUPLE_NUM; i++) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-
-      offset = appender.getOffset();
-      appender.addTuple(tuple);
-      creater.write(tuple, offset);
-    }
-    appender.flush();
-    appender.close();
-
-    creater.flush();
-    creater.close();
-
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    tuple = new VTuple(keySchema.size());
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    for (int i = 0; i < TUPLE_NUM - 1; i++) {
-      tuple.put(0, DatumFactory.createInt8(i));
-      tuple.put(1, DatumFactory.createFloat8(i));
-      long offsets = reader.find(tuple);
-      scanner.seek(offsets);
-      tuple = scanner.next();
-      assertTrue("[seek check " + (i) + " ]", (i) == (tuple.get(1).asInt8()));
-      assertTrue("[seek check " + (i) + " ]", (i) == (tuple.get(2).asFloat8()));
-
-      offsets = reader.next();
-      if (offsets == -1) {
-        continue;
-      }
-      scanner.seek(offsets);
-      tuple = scanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(0).asInt4()));
-      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (tuple.get(1).asInt8()));
-    }
-    reader.close();
-    scanner.close();
-  }
-
-  @Test
-  public void testFindOmittedValue() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = StorageUtil.concatPath(testDir, "testFindOmittedValue_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    Tuple tuple;
-    for (int i = 0; i < TUPLE_NUM; i += 2) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, status.getLen());
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindOmittedValue_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(1));
-      keyTuple.put(1, tuple.get(2));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindOmittedValue_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    for (int i = 1; i < TUPLE_NUM - 1; i += 2) {
-      keyTuple.put(0, DatumFactory.createInt8(i));
-      keyTuple.put(1, DatumFactory.createFloat8(i));
-      long offsets = reader.find(keyTuple);
-      assertEquals(-1, offsets);
-    }
-    reader.close();
-  }
-
-  @Test
-  public void testFindNextKeyValue() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testFindNextKeyValue_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    Tuple tuple;
-    for (int i = 0; i < TUPLE_NUM; i++) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindNextKeyValue_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(0));
-      keyTuple.put(1, tuple.get(1));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyValue_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple result;
-    for (int i = 0; i < TUPLE_NUM - 1; i++) {
-      keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt4(i));
-      keyTuple.put(1, DatumFactory.createInt8(i));
-      long offsets = reader.find(keyTuple, true);
-      scanner.seek(offsets);
-      result = scanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]",
-          (i + 1) == (result.get(0).asInt4()));
-      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (result.get(1).asInt8()));
-
-      offsets = reader.next();
-      if (offsets == -1) {
-        continue;
-      }
-      scanner.seek(offsets);
-      result = scanner.next();
-      assertTrue("[seek check " + (i + 2) + " ]", (i + 2) == (result.get(0).asInt8()));
-      assertTrue("[seek check " + (i + 2) + " ]", (i + 2) == (result.get(1).asFloat8()));
-    }
-    reader.close();
-    scanner.close();
-  }
-
-  @Test
-  public void testFindNextKeyOmittedValue() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testFindNextKeyOmittedValue_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    Tuple tuple;
-    for (int i = 0; i < TUPLE_NUM; i += 2) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindNextKeyOmittedValue_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(0));
-      keyTuple.put(1, tuple.get(1));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyOmittedValue_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple result;
-    for (int i = 1; i < TUPLE_NUM - 1; i += 2) {
-      keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt4(i));
-      keyTuple.put(1, DatumFactory.createInt8(i));
-      long offsets = reader.find(keyTuple, true);
-      scanner.seek(offsets);
-      result = scanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (result.get(0).asInt4()));
-      assertTrue("[seek check " + (i + 1) + " ]", (i + 1) == (result.get(1).asInt8()));
-    }
-    scanner.close();
-  }
-
-  @Test
-  public void testFindMinValue() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testFindMinValue" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-
-    Tuple tuple;
-    for (int i = 5; i < TUPLE_NUM + 5; i++) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(1));
-      keyTuple.put(1, tuple.get(2));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    tuple = new VTuple(keySchema.size());
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    tuple.put(0, DatumFactory.createInt8(0));
-    tuple.put(1, DatumFactory.createFloat8(0));
-
-    offset = reader.find(tuple);
-    assertEquals(-1, offset);
-
-    offset = reader.find(tuple, true);
-    assertTrue(offset >= 0);
-    scanner.seek(offset);
-    tuple = scanner.next();
-    assertEquals(5, tuple.get(1).asInt4());
-    assertEquals(5l, tuple.get(2).asInt8());
-    reader.close();
-    scanner.close();
-  }
-
-  @Test
-  public void testMinMax() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testMinMax_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    Tuple tuple;
-    for (int i = 5; i < TUPLE_NUM; i += 2) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testMinMax_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(0));
-      keyTuple.put(1, tuple.get(1));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testMinMax_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-
-    Tuple min = reader.getFirstKey();
-    assertEquals(5, min.get(0).asInt4());
-    assertEquals(5l, min.get(0).asInt8());
-
-    Tuple max = reader.getLastKey();
-    assertEquals(TUPLE_NUM - 1, max.get(0).asInt4());
-    assertEquals(TUPLE_NUM - 1, max.get(0).asInt8());
-    reader.close();
-  }
-
-  private class ConcurrentAccessor implements Runnable {
-    final BSTIndexReader reader;
-    final Random rnd = new Random(System.currentTimeMillis());
-    boolean failed = false;
-
-    ConcurrentAccessor(BSTIndexReader reader) {
-      this.reader = reader;
-    }
-
-    public boolean isFailed() {
-      return this.failed;
-    }
-
-    @Override
-    public void run() {
-      Tuple findKey = new VTuple(2);
-      int keyVal;
-      for (int i = 0; i < 10000; i++) {
-        keyVal = rnd.nextInt(10000);
-        findKey.put(0, DatumFactory.createInt4(keyVal));
-        findKey.put(1, DatumFactory.createInt8(keyVal));
-        try {
-          assertTrue(reader.find(findKey) != -1);
-        } catch (Exception e) {
-          e.printStackTrace();
-          this.failed = true;
-        }
-      }
-    }
-  }
-
-  @Test
-  public void testConcurrentAccess() throws IOException, InterruptedException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testConcurrentAccess_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-
-    Tuple tuple;
-    for (int i = 0; i < TUPLE_NUM; i++) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testConcurrentAccess_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(0));
-      keyTuple.put(1, tuple.get(1));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testConcurrentAccess_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-
-    Thread[] threads = new Thread[5];
-    ConcurrentAccessor[] accs = new ConcurrentAccessor[5];
-    for (int i = 0; i < threads.length; i++) {
-      accs[i] = new ConcurrentAccessor(reader);
-      threads[i] = new Thread(accs[i]);
-      threads[i].start();
-    }
-
-    for (int i = 0; i < threads.length; i++) {
-      threads[i].join();
-      assertFalse(accs[i].isFailed());
-    }
-    reader.close();
-  }
-
-
-  @Test
-  public void testFindValueDescOrder() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testFindValueDescOrder_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-
-    Tuple tuple;
-    for (int i = (TUPLE_NUM - 1); i >= 0; i--) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("long"), false, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("double"), false, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(1));
-      keyTuple.put(1, tuple.get(2));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-    tuple = new VTuple(keySchema.size());
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    for (int i = (TUPLE_NUM - 1); i > 0; i--) {
-      tuple.put(0, DatumFactory.createInt8(i));
-      tuple.put(1, DatumFactory.createFloat8(i));
-      long offsets = reader.find(tuple);
-      scanner.seek(offsets);
-      tuple = scanner.next();
-      assertTrue("seek check [" + (i) + " ," + (tuple.get(1).asInt8()) + "]", (i) == (tuple.get(1).asInt8()));
-      assertTrue("seek check [" + (i) + " ," + (tuple.get(2).asFloat8()) + "]", (i) == (tuple.get(2).asFloat8()));
-
-      offsets = reader.next();
-      if (offsets == -1) {
-        continue;
-      }
-      scanner.seek(offsets);
-      tuple = scanner.next();
-      assertTrue("[seek check " + (i - 1) + " ]", (i - 1) == (tuple.get(0).asInt4()));
-      assertTrue("[seek check " + (i - 1) + " ]", (i - 1) == (tuple.get(1).asInt8()));
-    }
-    reader.close();
-    scanner.close();
-  }
-
-  @Test
-  public void testFindNextKeyValueDescOrder() throws IOException {
-    meta = CatalogUtil.newTableMeta(storeType);
-
-    Path tablePath = new Path(testDir, "testFindNextKeyValueDescOrder_" + storeType);
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-
-    Tuple tuple;
-    for (int i = (TUPLE_NUM - 1); i >= 0; i--) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("int"), false, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("long"), false, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir,
-        "testFindNextKeyValueDescOrder_" + storeType + ".idx"), BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = scanner.getNextOffset();
-      tuple = scanner.next();
-      if (tuple == null) break;
-
-      keyTuple.put(0, tuple.get(0));
-      keyTuple.put(1, tuple.get(1));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    scanner.close();
-
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindNextKeyValueDescOrder_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-
-    assertEquals(keySchema, reader.getKeySchema());
-    assertEquals(comp, reader.getComparator());
-
-    scanner = StorageManager.getSeekableScanner(conf, meta, schema, tablet, schema);
-    scanner.init();
-
-    Tuple result;
-    for (int i = (TUPLE_NUM - 1); i > 0; i--) {
-      keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt4(i));
-      keyTuple.put(1, DatumFactory.createInt8(i));
-      long offsets = reader.find(keyTuple, true);
-      scanner.seek(offsets);
-      result = scanner.next();
-      assertTrue("[seek check " + (i - 1) + " ]",
-          (i - 1) == (result.get(0).asInt4()));
-      assertTrue("[seek check " + (i - 1) + " ]", (i - 1) == (result.get(1).asInt8()));
-
-      offsets = reader.next();
-      if (offsets == -1) {
-        continue;
-      }
-      scanner.seek(offsets);
-      result = scanner.next();
-      assertTrue("[seek check " + (i - 2) + " ]", (i - 2) == (result.get(0).asInt8()));
-      assertTrue("[seek check " + (i - 2) + " ]", (i - 2) == (result.get(1).asFloat8()));
-    }
-    reader.close();
-    scanner.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java b/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
deleted file mode 100644
index 1081ae9..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
+++ /dev/null
@@ -1,248 +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.index;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.index.bst.BSTIndex;
-import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexReader;
-import org.apache.tajo.storage.index.bst.BSTIndex.BSTIndexWriter;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.tajo.storage.CSVFile.CSVScanner;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestSingleCSVFileBSTIndex {
-  
-  private TajoConf conf;
-  private Schema schema;
-  private TableMeta meta;
-  private FileSystem fs;
-
-  private static final int TUPLE_NUM = 10000;
-  private static final int LOAD_NUM = 100;
-  private static final String TEST_PATH = "target/test-data/TestSingleCSVFileBSTIndex";
-  private Path testDir;
-  
-  public TestSingleCSVFileBSTIndex() {
-    conf = new TajoConf();
-    conf.setVar(ConfVars.ROOT_DIR, TEST_PATH);
-    schema = new Schema();
-    schema.addColumn(new Column("int", Type.INT4));
-    schema.addColumn(new Column("long", Type.INT8));
-    schema.addColumn(new Column("double", Type.FLOAT8));
-    schema.addColumn(new Column("float", Type.FLOAT4));
-    schema.addColumn(new Column("string", Type.TEXT));
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-  }
-
-  @Test
-  public void testFindValueInSingleCSV() throws IOException {
-    meta = CatalogUtil.newTableMeta(StoreType.CSV);
-
-    Path tablePath = StorageUtil.concatPath(testDir, "testFindValueInSingleCSV", "table.csv");
-    fs.mkdirs(tablePath.getParent());
-
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    Tuple tuple;
-    for (int i = 0; i < TUPLE_NUM; i++) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir,
-        "FindValueInCSV.idx"), BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-
-    SeekableScanner fileScanner = new CSVScanner(conf, schema, meta, tablet);
-    fileScanner.init();
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = fileScanner.getNextOffset();
-      tuple = fileScanner.next();
-      if (tuple == null)
-        break;
-
-      keyTuple.put(0, tuple.get(1));
-      keyTuple.put(1, tuple.get(2));
-      creater.write(keyTuple, offset);
-    }
-
-    creater.flush();
-    creater.close();
-    fileScanner.close();
-
-    tuple = new VTuple(keySchema.size());
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir,
-        "FindValueInCSV.idx"), keySchema, comp);
-    reader.open();
-    fileScanner = new CSVScanner(conf, schema, meta, tablet);
-    fileScanner.init();
-    for (int i = 0; i < TUPLE_NUM - 1; i++) {
-      tuple.put(0, DatumFactory.createInt8(i));
-      tuple.put(1, DatumFactory.createFloat8(i));
-      long offsets = reader.find(tuple);
-      fileScanner.seek(offsets);
-      tuple = fileScanner.next();
-      assertEquals(i,  (tuple.get(1).asInt8()));
-      assertEquals(i, (tuple.get(2).asFloat8()) , 0.01);
-
-      offsets = reader.next();
-      if (offsets == -1) {
-        continue;
-      }
-      fileScanner.seek(offsets);
-      tuple = fileScanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]",
-          (i + 1) == (tuple.get(0).asInt4()));
-      assertTrue("[seek check " + (i + 1) + " ]",
-          (i + 1) == (tuple.get(1).asInt8()));
-    }
-  }
-
-  @Test
-  public void testFindNextKeyValueInSingleCSV() throws IOException {
-    meta = CatalogUtil.newTableMeta(StoreType.CSV);
-
-    Path tablePath = StorageUtil.concatPath(testDir, "testFindNextKeyValueInSingleCSV",
-        "table1.csv");
-    fs.mkdirs(tablePath.getParent());
-    Appender appender = StorageManager.getFileStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-    Tuple tuple;
-    for(int i = 0 ; i < TUPLE_NUM; i ++ ) {
-      tuple = new VTuple(5);
-      tuple.put(0, DatumFactory.createInt4(i));
-      tuple.put(1, DatumFactory.createInt8(i));
-      tuple.put(2, DatumFactory.createFloat8(i));
-      tuple.put(3, DatumFactory.createFloat4(i));
-      tuple.put(4, DatumFactory.createText("field_" + i));
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    long fileLen = status.getLen();
-    FileFragment tablet = new FileFragment("table1_1", status.getPath(), 0, fileLen);
-    
-    SortSpec [] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    BaseTupleComparator comp = new BaseTupleComparator(keySchema, sortKeys);
-    
-    BSTIndex bst = new BSTIndex(conf);
-    BSTIndexWriter creater = bst.getIndexWriter(new Path(testDir, "FindNextKeyValueInCSV.idx"),
-        BSTIndex.TWO_LEVEL_INDEX, keySchema, comp);
-    creater.setLoadNum(LOAD_NUM);
-    creater.open();
-    
-    SeekableScanner fileScanner  = new CSVScanner(conf, schema, meta, tablet);
-    fileScanner.init();
-    Tuple keyTuple;
-    long offset;
-    while (true) {
-      keyTuple = new VTuple(2);
-      offset = fileScanner.getNextOffset();
-      tuple = fileScanner.next();
-      if (tuple == null) break;
-      
-      keyTuple.put(0, tuple.get(0));
-      keyTuple.put(1, tuple.get(1));
-      creater.write(keyTuple, offset);
-    }
-    
-    creater.flush();
-    creater.close();
-    fileScanner.close();    
-    
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "FindNextKeyValueInCSV.idx"), keySchema, comp);
-    reader.open();
-    fileScanner  = new CSVScanner(conf, schema, meta, tablet);
-    fileScanner.init();
-    Tuple result;
-    for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) {
-      keyTuple = new VTuple(2);
-      keyTuple.put(0, DatumFactory.createInt4(i));
-      keyTuple.put(1, DatumFactory.createInt8(i));
-      long offsets = reader.find(keyTuple, true);
-      fileScanner.seek(offsets);
-      result = fileScanner.next();
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(0).asInt4()));
-      assertTrue("[seek check " + (i + 1) + " ]" , (i + 1) == (result.get(1).asInt8()));
-      
-      offsets = reader.next();
-      if (offsets == -1) {
-        continue;
-      }
-      fileScanner.seek(offsets);
-      result = fileScanner.next();
-      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(0).asInt8()));
-      assertTrue("[seek check " + (i + 2) + " ]" , (i + 2) == (result.get(1).asFloat8()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java b/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
deleted file mode 100644
index 0a01dc4..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestReadWrite.java
+++ /dev/null
@@ -1,114 +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.parquet;
-
-import com.google.common.base.Charsets;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotNull;
-
-public class TestReadWrite {
-  private static final String HELLO = "hello";
-
-  private Path createTmpFile() throws IOException {
-    File tmp = File.createTempFile(getClass().getSimpleName(), ".tmp");
-    tmp.deleteOnExit();
-    tmp.delete();
-
-    // it prevents accessing HDFS namenode of TajoTestingCluster.
-    LocalFileSystem localFS = LocalFileSystem.getLocal(new Configuration());
-    return localFS.makeQualified(new Path(tmp.getPath()));
-  }
-
-  private Schema createAllTypesSchema() {
-    List<Column> columns = new ArrayList<Column>();
-    columns.add(new Column("myboolean", Type.BOOLEAN));
-    columns.add(new Column("mybit", Type.BIT));
-    columns.add(new Column("mychar", Type.CHAR));
-    columns.add(new Column("myint2", Type.INT2));
-    columns.add(new Column("myint4", Type.INT4));
-    columns.add(new Column("myint8", Type.INT8));
-    columns.add(new Column("myfloat4", Type.FLOAT4));
-    columns.add(new Column("myfloat8", Type.FLOAT8));
-    columns.add(new Column("mytext", Type.TEXT));
-    columns.add(new Column("myblob", Type.BLOB));
-    columns.add(new Column("mynull", Type.NULL_TYPE));
-    Column[] columnsArray = new Column[columns.size()];
-    columnsArray = columns.toArray(columnsArray);
-    return new Schema(columnsArray);
-  }
-
-  @Test
-  public void testAll() throws Exception {
-    Path file = createTmpFile();
-    Schema schema = createAllTypesSchema();
-    Tuple tuple = new VTuple(schema.size());
-    tuple.put(0, DatumFactory.createBool(true));
-    tuple.put(1, DatumFactory.createBit((byte)128));
-    tuple.put(2, DatumFactory.createChar('t'));
-    tuple.put(3, DatumFactory.createInt2((short)2048));
-    tuple.put(4, DatumFactory.createInt4(4096));
-    tuple.put(5, DatumFactory.createInt8(8192L));
-    tuple.put(6, DatumFactory.createFloat4(0.2f));
-    tuple.put(7, DatumFactory.createFloat8(4.1));
-    tuple.put(8, DatumFactory.createText(HELLO));
-    tuple.put(9, DatumFactory.createBlob(HELLO.getBytes(Charsets.UTF_8)));
-    tuple.put(10, NullDatum.get());
-
-    TajoParquetWriter writer = new TajoParquetWriter(file, schema);
-    writer.write(tuple);
-    writer.close();
-
-    TajoParquetReader reader = new TajoParquetReader(file, schema);
-    tuple = reader.read();
-
-    assertNotNull(tuple);
-    assertEquals(true, tuple.getBool(0));
-    assertEquals((byte)128, tuple.getByte(1));
-    assertTrue(String.valueOf('t').equals(String.valueOf(tuple.getChar(2))));
-    assertEquals((short)2048, tuple.getInt2(3));
-    assertEquals(4096, tuple.getInt4(4));
-    assertEquals(8192L, tuple.getInt8(5));
-    assertEquals(new Float(0.2f), new Float(tuple.getFloat4(6)));
-    assertEquals(new Double(4.1), new Double(tuple.getFloat8(7)));
-    assertTrue(HELLO.equals(tuple.getText(8)));
-    assertArrayEquals(HELLO.getBytes(Charsets.UTF_8), tuple.getBytes(9));
-    assertEquals(NullDatum.get(), tuple.get(10));
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java b/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
deleted file mode 100644
index 49a162b..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java
+++ /dev/null
@@ -1,132 +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.parquet;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes.Type;
-
-import org.junit.Test;
-
-import parquet.schema.MessageType;
-import parquet.schema.MessageTypeParser;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests for {@link TajoSchemaConverter}.
- */
-public class TestSchemaConverter {
-  private static final String ALL_PARQUET_SCHEMA =
-      "message table_schema {\n" +
-      "  optional boolean myboolean;\n" +
-      "  optional int32 myint;\n" +
-      "  optional int64 mylong;\n" +
-      "  optional float myfloat;\n" +
-      "  optional double mydouble;\n" +
-      "  optional binary mybytes;\n" +
-      "  optional binary mystring (UTF8);\n" +
-      "  optional fixed_len_byte_array(1) myfixed;\n" +
-      "}\n";
-
-  private static final String CONVERTED_ALL_PARQUET_SCHEMA =
-      "message table_schema {\n" +
-      "  optional boolean myboolean;\n" +
-      "  optional int32 mybit;\n" +
-      "  optional binary mychar (UTF8);\n" +
-      "  optional int32 myint2;\n" +
-      "  optional int32 myint4;\n" +
-      "  optional int64 myint8;\n" +
-      "  optional float myfloat4;\n" +
-      "  optional double myfloat8;\n" +
-      "  optional binary mytext (UTF8);\n" +
-      "  optional binary myblob;\n" +
-      // NULL_TYPE fields are not encoded.
-      "  optional binary myinet4;\n" +
-      "  optional binary myprotobuf;\n" +
-      "}\n";
-
-  private Schema createAllTypesSchema() {
-    List<Column> columns = new ArrayList<Column>();
-    columns.add(new Column("myboolean", Type.BOOLEAN));
-    columns.add(new Column("mybit", Type.BIT));
-    columns.add(new Column("mychar", Type.CHAR));
-    columns.add(new Column("myint2", Type.INT2));
-    columns.add(new Column("myint4", Type.INT4));
-    columns.add(new Column("myint8", Type.INT8));
-    columns.add(new Column("myfloat4", Type.FLOAT4));
-    columns.add(new Column("myfloat8", Type.FLOAT8));
-    columns.add(new Column("mytext", Type.TEXT));
-    columns.add(new Column("myblob", Type.BLOB));
-    columns.add(new Column("mynull", Type.NULL_TYPE));
-    columns.add(new Column("myinet4", Type.INET4));
-    columns.add(new Column("myprotobuf", Type.PROTOBUF));
-    Column[] columnsArray = new Column[columns.size()];
-    columnsArray = columns.toArray(columnsArray);
-    return new Schema(columnsArray);
-  }
-
-  private Schema createAllTypesConvertedSchema() {
-    List<Column> columns = new ArrayList<Column>();
-    columns.add(new Column("myboolean", Type.BOOLEAN));
-    columns.add(new Column("myint", Type.INT4));
-    columns.add(new Column("mylong", Type.INT8));
-    columns.add(new Column("myfloat", Type.FLOAT4));
-    columns.add(new Column("mydouble", Type.FLOAT8));
-    columns.add(new Column("mybytes", Type.BLOB));
-    columns.add(new Column("mystring", Type.TEXT));
-    columns.add(new Column("myfixed", Type.BLOB));
-    Column[] columnsArray = new Column[columns.size()];
-    columnsArray = columns.toArray(columnsArray);
-    return new Schema(columnsArray);
-  }
-
-  private void testTajoToParquetConversion(
-      Schema tajoSchema, String schemaString) throws Exception {
-    TajoSchemaConverter converter = new TajoSchemaConverter();
-    MessageType schema = converter.convert(tajoSchema);
-    MessageType expected = MessageTypeParser.parseMessageType(schemaString);
-    assertEquals("converting " + schema + " to " + schemaString,
-                 expected.toString(), schema.toString());
-  }
-
-  private void testParquetToTajoConversion(
-      Schema tajoSchema, String schemaString) throws Exception {
-    TajoSchemaConverter converter = new TajoSchemaConverter();
-    Schema schema = converter.convert(
-        MessageTypeParser.parseMessageType(schemaString));
-    assertEquals("converting " + schemaString + " to " + tajoSchema,
-                 tajoSchema.toString(), schema.toString());
-  }
-
-  @Test
-  public void testAllTypesToParquet() throws Exception {
-    Schema schema = createAllTypesSchema();
-    testTajoToParquetConversion(schema, CONVERTED_ALL_PARQUET_SCHEMA);
-  }
-
-  @Test
-  public void testAllTypesToTajo() throws Exception {
-    Schema schema = createAllTypesConvertedSchema();
-    testParquetToTajoConversion(schema, ALL_PARQUET_SCHEMA);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java
deleted file mode 100644
index 7b09937..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/INode.java
+++ /dev/null
@@ -1,124 +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.s3;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.io.IOUtils;
-
-import java.io.*;
-
-/**
- * Holds file metadata including type (regular file, or directory),
- * and the list of blocks that are pointers to the data.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class INode {
-
-  enum FileType {
-    DIRECTORY, FILE
-  }
-
-  public static final FileType[] FILE_TYPES = {
-      FileType.DIRECTORY,
-      FileType.FILE
-  };
-
-  public static final INode DIRECTORY_INODE = new INode(FileType.DIRECTORY, null);
-
-  private FileType fileType;
-  private Block[] blocks;
-
-  public INode(FileType fileType, Block[] blocks) {
-    this.fileType = fileType;
-    if (isDirectory() && blocks != null) {
-      throw new IllegalArgumentException("A directory cannot contain blocks.");
-    }
-    this.blocks = blocks;
-  }
-
-  public Block[] getBlocks() {
-    return blocks;
-  }
-
-  public FileType getFileType() {
-    return fileType;
-  }
-
-  public boolean isDirectory() {
-    return fileType == FileType.DIRECTORY;
-  }
-
-  public boolean isFile() {
-    return fileType == FileType.FILE;
-  }
-
-  public long getSerializedLength() {
-    return 1L + (blocks == null ? 0 : 4 + blocks.length * 16);
-  }
-
-
-  public InputStream serialize() throws IOException {
-    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(bytes);
-    try {
-      out.writeByte(fileType.ordinal());
-      if (isFile()) {
-        out.writeInt(blocks.length);
-        for (int i = 0; i < blocks.length; i++) {
-          out.writeLong(blocks[i].getId());
-          out.writeLong(blocks[i].getLength());
-        }
-      }
-      out.close();
-      out = null;
-    } finally {
-      IOUtils.closeStream(out);
-    }
-    return new ByteArrayInputStream(bytes.toByteArray());
-  }
-
-  public static INode deserialize(InputStream in) throws IOException {
-    if (in == null) {
-      return null;
-    }
-    DataInputStream dataIn = new DataInputStream(in);
-    FileType fileType = INode.FILE_TYPES[dataIn.readByte()];
-    switch (fileType) {
-      case DIRECTORY:
-        in.close();
-        return INode.DIRECTORY_INODE;
-      case FILE:
-        int numBlocks = dataIn.readInt();
-        Block[] blocks = new Block[numBlocks];
-        for (int i = 0; i < numBlocks; i++) {
-          long id = dataIn.readLong();
-          long length = dataIn.readLong();
-          blocks[i] = new Block(id, length);
-        }
-        in.close();
-        return new INode(fileType, blocks);
-      default:
-        throw new IllegalArgumentException("Cannot deserialize inode.");
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
deleted file mode 100644
index 40decc2..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/InMemoryFileSystemStore.java
+++ /dev/null
@@ -1,175 +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.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.fs.s3.FileSystemStore;
-import org.apache.hadoop.fs.s3.INode;
-import org.apache.tajo.common.exception.NotImplementedException;
-
-import java.io.*;
-import java.net.URI;
-import java.util.*;
-
-/**
- * A stub implementation of {@link FileSystemStore} for testing
- * {@link S3FileSystem} without actually connecting to S3.
- */
-public class InMemoryFileSystemStore implements FileSystemStore {
-
-  private Configuration conf;
-  private SortedMap<Path, INode> inodes = new TreeMap<Path, INode>();
-  private Map<Long, byte[]> blocks = new HashMap<Long, byte[]>();
-
-  @Override
-  public void initialize(URI uri, Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
-  public String getVersion() throws IOException {
-    return "0";
-  }
-
-  @Override
-  public void deleteINode(Path path) throws IOException {
-    inodes.remove(normalize(path));
-  }
-
-  @Override
-  public void deleteBlock(Block block) throws IOException {
-    blocks.remove(block.getId());
-  }
-
-  @Override
-  public boolean inodeExists(Path path) throws IOException {
-    return inodes.containsKey(normalize(path));
-  }
-
-  @Override
-  public boolean blockExists(long blockId) throws IOException {
-    return blocks.containsKey(blockId);
-  }
-
-  @Override
-  public INode retrieveINode(Path path) throws IOException {
-    return inodes.get(normalize(path));
-  }
-
-  @Override
-  public File retrieveBlock(Block block, long byteRangeStart) throws IOException {
-    byte[] data = blocks.get(block.getId());
-    File file = createTempFile();
-    BufferedOutputStream out = null;
-    try {
-      out = new BufferedOutputStream(new FileOutputStream(file));
-      out.write(data, (int) byteRangeStart, data.length - (int) byteRangeStart);
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-    }
-    return file;
-  }
-
-  private File createTempFile() throws IOException {
-    File dir = new File(conf.get("fs.s3.buffer.dir"));
-    if (!dir.exists() && !dir.mkdirs()) {
-      throw new IOException("Cannot create S3 buffer directory: " + dir);
-    }
-    File result = File.createTempFile("test-", ".tmp", dir);
-    result.deleteOnExit();
-    return result;
-  }
-
-  @Override
-  public Set<Path> listSubPaths(Path path) throws IOException {
-    Path normalizedPath = normalize(path);
-    // This is inefficient but more than adequate for testing purposes.
-    Set<Path> subPaths = new LinkedHashSet<Path>();
-    for (Path p : inodes.tailMap(normalizedPath).keySet()) {
-      if (normalizedPath.equals(p.getParent())) {
-        subPaths.add(p);
-      }
-    }
-    return subPaths;
-  }
-
-  @Override
-  public Set<Path> listDeepSubPaths(Path path) throws IOException {
-    Path normalizedPath = normalize(path);
-    String pathString = normalizedPath.toUri().getPath();
-    if (!pathString.endsWith("/")) {
-      pathString += "/";
-    }
-    // This is inefficient but more than adequate for testing purposes.
-    Set<Path> subPaths = new LinkedHashSet<Path>();
-    for (Path p : inodes.tailMap(normalizedPath).keySet()) {
-      if (p.toUri().getPath().startsWith(pathString)) {
-        subPaths.add(p);
-      }
-    }
-    return subPaths;
-  }
-
-  @Override
-  public void storeINode(Path path, INode inode) throws IOException {
-    inodes.put(normalize(path), inode);
-  }
-
-  @Override
-  public void storeBlock(Block block, File file) throws IOException {
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    byte[] buf = new byte[8192];
-    int numRead;
-    BufferedInputStream in = null;
-    try {
-      in = new BufferedInputStream(new FileInputStream(file));
-      while ((numRead = in.read(buf)) >= 0) {
-        out.write(buf, 0, numRead);
-      }
-    } finally {
-      if (in != null) {
-        in.close();
-      }
-    }
-    blocks.put(block.getId(), out.toByteArray());
-  }
-
-  private Path normalize(Path path) {
-    if (!path.isAbsolute()) {
-      throw new IllegalArgumentException("Path must be absolute: " + path);
-    }
-    return new Path(path.toUri().getPath());
-  }
-
-  @Override
-  public void purge() throws IOException {
-    inodes.clear();
-    blocks.clear();
-  }
-
-  @Override
-  public void dump() throws IOException {
-    throw new NotImplementedException();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
deleted file mode 100644
index d4034b9..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/S3OutputStream.java
+++ /dev/null
@@ -1,234 +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.s3;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.fs.s3.FileSystemStore;
-import org.apache.hadoop.fs.s3.INode;
-import org.apache.hadoop.util.Progressable;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class S3OutputStream extends OutputStream {
-
-  private Configuration conf;
-
-  private int bufferSize;
-
-  private FileSystemStore store;
-
-  private Path path;
-
-  private long blockSize;
-
-  private File backupFile;
-
-  private OutputStream backupStream;
-
-  private Random r = new Random();
-
-  private boolean closed;
-
-  private int pos = 0;
-
-  private long filePos = 0;
-
-  private int bytesWrittenToBlock = 0;
-
-  private byte[] outBuf;
-
-  private List<Block> blocks = new ArrayList<Block>();
-
-  private Block nextBlock;
-
-  private static final Log LOG =
-      LogFactory.getLog(S3OutputStream.class.getName());
-
-
-  public S3OutputStream(Configuration conf, FileSystemStore store,
-                        Path path, long blockSize, Progressable progress,
-                        int buffersize) throws IOException {
-
-    this.conf = conf;
-    this.store = store;
-    this.path = path;
-    this.blockSize = blockSize;
-    this.backupFile = newBackupFile();
-    this.backupStream = new FileOutputStream(backupFile);
-    this.bufferSize = buffersize;
-    this.outBuf = new byte[bufferSize];
-
-  }
-
-  private File newBackupFile() throws IOException {
-    File dir = new File(conf.get("fs.s3.buffer.dir"));
-    if (!dir.exists() && !dir.mkdirs()) {
-      throw new IOException("Cannot create S3 buffer directory: " + dir);
-    }
-    File result = File.createTempFile("output-", ".tmp", dir);
-    result.deleteOnExit();
-    return result;
-  }
-
-  public long getPos() throws IOException {
-    return filePos;
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    if (closed) {
-      throw new IOException("Stream closed");
-    }
-
-    if ((bytesWrittenToBlock + pos == blockSize) || (pos >= bufferSize)) {
-      flush();
-    }
-    outBuf[pos++] = (byte) b;
-    filePos++;
-  }
-
-  @Override
-  public synchronized void write(byte b[], int off, int len) throws IOException {
-    if (closed) {
-      throw new IOException("Stream closed");
-    }
-    while (len > 0) {
-      int remaining = bufferSize - pos;
-      int toWrite = Math.min(remaining, len);
-      System.arraycopy(b, off, outBuf, pos, toWrite);
-      pos += toWrite;
-      off += toWrite;
-      len -= toWrite;
-      filePos += toWrite;
-
-      if ((bytesWrittenToBlock + pos >= blockSize) || (pos == bufferSize)) {
-        flush();
-      }
-    }
-  }
-
-  @Override
-  public synchronized void flush() throws IOException {
-    if (closed) {
-      throw new IOException("Stream closed");
-    }
-
-    if (bytesWrittenToBlock + pos >= blockSize) {
-      flushData((int) blockSize - bytesWrittenToBlock);
-    }
-    if (bytesWrittenToBlock == blockSize) {
-      endBlock();
-    }
-    flushData(pos);
-  }
-
-  private synchronized void flushData(int maxPos) throws IOException {
-    int workingPos = Math.min(pos, maxPos);
-
-    if (workingPos > 0) {
-      //
-      // To the local block backup, write just the bytes
-      //
-      backupStream.write(outBuf, 0, workingPos);
-
-      //
-      // Track position
-      //
-      bytesWrittenToBlock += workingPos;
-      System.arraycopy(outBuf, workingPos, outBuf, 0, pos - workingPos);
-      pos -= workingPos;
-    }
-  }
-
-  private synchronized void endBlock() throws IOException {
-    //
-    // Done with local copy
-    //
-    backupStream.close();
-
-    //
-    // Send it to S3
-    //
-    // TODO: Use passed in Progressable to report progress.
-    nextBlockOutputStream();
-    store.storeBlock(nextBlock, backupFile);
-    Block[] arr = new Block[blocks.size()];
-    arr = blocks.toArray(arr);
-    store.storeINode(path, new INode(INode.FILE_TYPES[1], arr));
-
-    //
-    // Delete local backup, start new one
-    //
-    boolean b = backupFile.delete();
-    if (!b) {
-      LOG.warn("Ignoring failed delete");
-    }
-    backupFile = newBackupFile();
-    backupStream = new FileOutputStream(backupFile);
-    bytesWrittenToBlock = 0;
-  }
-
-  private synchronized void nextBlockOutputStream() throws IOException {
-    long blockId = r.nextLong();
-    while (store.blockExists(blockId)) {
-      blockId = r.nextLong();
-    }
-    nextBlock = new Block(blockId, bytesWrittenToBlock);
-    blocks.add(nextBlock);
-    bytesWrittenToBlock = 0;
-  }
-
-
-  @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
-      return;
-    }
-
-    flush();
-    if (filePos == 0 || bytesWrittenToBlock != 0) {
-      endBlock();
-    }
-
-    backupStream.close();
-    boolean b = backupFile.delete();
-    if (!b) {
-      LOG.warn("Ignoring failed delete");
-    }
-
-    super.close();
-
-    closed = true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java b/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
deleted file mode 100644
index fc1c908..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/storage/s3/SmallBlockS3FileSystem.java
+++ /dev/null
@@ -1,314 +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.s3;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.s3.Block;
-import org.apache.hadoop.fs.s3.FileSystemStore;
-import org.apache.hadoop.fs.s3.INode;
-import org.apache.hadoop.fs.s3.S3FileSystem;
-import org.apache.hadoop.util.Progressable;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-
-public class SmallBlockS3FileSystem extends S3FileSystem {
-
-  private URI uri;
-
-  private FileSystemStore store;
-
-  private Path workingDir;
-
-  static class Holder {
-    private static InMemoryFileSystemStore s;
-
-    public synchronized static FileSystemStore get() {
-      if(s != null) {
-        return s;
-      }
-      s = new InMemoryFileSystemStore();
-      return s;
-    }
-
-    public synchronized static void set(InMemoryFileSystemStore inMemoryFileSystemStore) {
-      s = inMemoryFileSystemStore;
-    }
-  }
-
-  public SmallBlockS3FileSystem() {
-  }
-
-
-  public SmallBlockS3FileSystem(
-      InMemoryFileSystemStore inMemoryFileSystemStore) {
-    Holder.set(inMemoryFileSystemStore);
-    this.store = inMemoryFileSystemStore;
-  }
-
-  @Override
-  public URI getUri() {
-    return uri;
-  }
-  @Override
-  public long getDefaultBlockSize() {
-    return 10;
-  }
-
-  @Override
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    if (store == null) {
-      store = Holder.get();
-    }
-    store.initialize(uri, conf);
-    setConf(conf);
-    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
-    this.workingDir =
-        new Path("/user", System.getProperty("user.name")).makeQualified(this);
-  }
-  @Override
-  public boolean isFile(Path path) throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(path));
-    if (inode == null) {
-      return false;
-    }
-    return inode.isFile();
-  }
-
-  private INode checkFile(Path path) throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(path));
-    if (inode == null) {
-      throw new IOException("No such file.");
-    }
-    if (inode.isDirectory()) {
-      throw new IOException("Path " + path + " is a directory.");
-    }
-    return inode;
-  }
-
-  @Override
-  public FileStatus[] listStatus(Path f) throws IOException {
-    Path absolutePath = makeAbsolute(f);
-    INode inode = store.retrieveINode(absolutePath);
-    if (inode == null) {
-      throw new FileNotFoundException("File " + f + " does not exist.");
-    }
-    if (inode.isFile()) {
-      return new FileStatus[] {
-          new S3FileStatus(f.makeQualified(this), inode)
-      };
-    }
-    ArrayList<FileStatus> ret = new ArrayList<FileStatus>();
-    for (Path p : store.listSubPaths(absolutePath)) {
-      ret.add(getFileStatus(p.makeQualified(this)));
-    }
-    return ret.toArray(new FileStatus[0]);
-  }
-  @Override
-  public FSDataOutputStream create(Path file, FsPermission permission,
-                                   boolean overwrite, int bufferSize,
-                                   short replication, long blockSize, Progressable progress)
-      throws IOException {
-
-    INode inode = store.retrieveINode(makeAbsolute(file));
-    if (inode != null) {
-      if (overwrite) {
-        delete(file, true);
-      } else {
-        throw new IOException("File already exists: " + file);
-      }
-    } else {
-      Path parent = file.getParent();
-      if (parent != null) {
-        if (!mkdirs(parent)) {
-          throw new IOException("Mkdirs failed to create " + parent.toString());
-        }
-      }
-    }
-    return new FSDataOutputStream
-        (new S3OutputStream(getConf(), store, makeAbsolute(file),
-            blockSize, progress, bufferSize),
-            statistics);
-  }
-  @Override
-  public boolean mkdirs(Path path, FsPermission permission) throws IOException {
-    Path absolutePath = makeAbsolute(path);
-    List<Path> paths = new ArrayList<Path>();
-    do {
-      paths.add(0, absolutePath);
-      absolutePath = absolutePath.getParent();
-    } while (absolutePath != null);
-
-    boolean result = true;
-    for (Path p : paths) {
-      result &= mkdir(p);
-    }
-    return result;
-  }
-
-  @Override
-  public Path getWorkingDirectory() {
-    return workingDir;
-  }
-
-  @Override
-  public boolean rename(Path src, Path dst) throws IOException {
-    Path absoluteSrc = makeAbsolute(src);
-    INode srcINode = store.retrieveINode(absoluteSrc);
-    if (srcINode == null) {
-      // src path doesn't exist
-      return false;
-    }
-    Path absoluteDst = makeAbsolute(dst);
-    INode dstINode = store.retrieveINode(absoluteDst);
-    if (dstINode != null && dstINode.isDirectory()) {
-      absoluteDst = new Path(absoluteDst, absoluteSrc.getName());
-      dstINode = store.retrieveINode(absoluteDst);
-    }
-    if (dstINode != null) {
-      // dst path already exists - can't overwrite
-      return false;
-    }
-    Path dstParent = absoluteDst.getParent();
-    if (dstParent != null) {
-      INode dstParentINode = store.retrieveINode(dstParent);
-      if (dstParentINode == null || dstParentINode.isFile()) {
-        // dst parent doesn't exist or is a file
-        return false;
-      }
-    }
-    return renameRecursive(absoluteSrc, absoluteDst);
-  }
-
-  private boolean renameRecursive(Path src, Path dst) throws IOException {
-    INode srcINode = store.retrieveINode(src);
-    store.storeINode(dst, srcINode);
-    store.deleteINode(src);
-    if (srcINode.isDirectory()) {
-      for (Path oldSrc : store.listDeepSubPaths(src)) {
-        INode inode = store.retrieveINode(oldSrc);
-        if (inode == null) {
-          return false;
-        }
-        String oldSrcPath = oldSrc.toUri().getPath();
-        String srcPath = src.toUri().getPath();
-        String dstPath = dst.toUri().getPath();
-        Path newDst = new Path(oldSrcPath.replaceFirst(srcPath, dstPath));
-        store.storeINode(newDst, inode);
-        store.deleteINode(oldSrc);
-      }
-    }
-    return true;
-  }
-
-  @Override
-  public boolean delete(Path path, boolean recursive) throws IOException {
-    Path absolutePath = makeAbsolute(path);
-    INode inode = store.retrieveINode(absolutePath);
-    if (inode == null) {
-      return false;
-    }
-    if (inode.isFile()) {
-      store.deleteINode(absolutePath);
-      for (Block block: inode.getBlocks()) {
-        store.deleteBlock(block);
-      }
-    } else {
-      FileStatus[] contents = null;
-      try {
-        contents = listStatus(absolutePath);
-      } catch(FileNotFoundException fnfe) {
-        return false;
-      }
-
-      if ((contents.length !=0) && (!recursive)) {
-        throw new IOException("Directory " + path.toString()
-            + " is not empty.");
-      }
-      for (FileStatus p:contents) {
-        if (!delete(p.getPath(), recursive)) {
-          return false;
-        }
-      }
-      store.deleteINode(absolutePath);
-    }
-    return true;
-  }
-
-  /**
-   * FileStatus for S3 file systems.
-   */
-  @Override
-  public FileStatus getFileStatus(Path f)  throws IOException {
-    INode inode = store.retrieveINode(makeAbsolute(f));
-    if (inode == null) {
-      throw new FileNotFoundException(f + ": No such file or directory.");
-    }
-    return new S3FileStatus(f.makeQualified(this), inode);
-  }
-  private boolean mkdir(Path path) throws IOException {
-    Path absolutePath = makeAbsolute(path);
-    INode inode = store.retrieveINode(absolutePath);
-    if (inode == null) {
-      store.storeINode(absolutePath, INode.DIRECTORY_INODE);
-    } else if (inode.isFile()) {
-      throw new IOException(String.format(
-          "Can't make directory for path %s since it is a file.",
-          absolutePath));
-    }
-    return true;
-  }
-  private Path makeAbsolute(Path path) {
-    if (path.isAbsolute()) {
-      return path;
-    }
-    return new Path(workingDir, path);
-  }
-
-  private static class S3FileStatus extends FileStatus {
-
-    S3FileStatus(Path f, INode inode) throws IOException {
-      super(findLength(inode), inode.isDirectory(), 1,
-          findBlocksize(inode), 0, f);
-    }
-
-    private static long findLength(INode inode) {
-      if (!inode.isDirectory()) {
-        long length = 0L;
-        for (Block block : inode.getBlocks()) {
-          length += block.getLength();
-        }
-        return length;
-      }
-      return 0;
-    }
-
-    private static long findBlocksize(INode inode) {
-      final Block[] ret = inode.getBlocks();
-      return ret == null ? 0L : ret[0].getLength();
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java b/tajo-storage/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
deleted file mode 100644
index b332364..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/tuple/TestBaseTupleBuilder.java
+++ /dev/null
@@ -1,76 +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.tuple;
-
-import org.apache.tajo.storage.RowStoreUtil;
-import org.apache.tajo.tuple.offheap.*;
-import org.junit.Test;
-
-public class TestBaseTupleBuilder {
-
-  @Test
-  public void testBuild() {
-    BaseTupleBuilder builder = new BaseTupleBuilder(TestOffHeapRowBlock.schema);
-
-    OffHeapRowBlock rowBlock = TestOffHeapRowBlock.createRowBlock(10248);
-    OffHeapRowBlockReader reader = rowBlock.getReader();
-
-    ZeroCopyTuple inputTuple = new ZeroCopyTuple();
-
-    HeapTuple heapTuple = null;
-    ZeroCopyTuple zcTuple = null;
-    int i = 0;
-    while(reader.next(inputTuple)) {
-      RowStoreUtil.convert(inputTuple, builder);
-
-      heapTuple = builder.buildToHeapTuple();
-      TestOffHeapRowBlock.validateTupleResult(i, heapTuple);
-
-      zcTuple = builder.buildToZeroCopyTuple();
-      TestOffHeapRowBlock.validateTupleResult(i, zcTuple);
-
-      i++;
-    }
-  }
-
-  @Test
-  public void testBuildWithNull() {
-    BaseTupleBuilder builder = new BaseTupleBuilder(TestOffHeapRowBlock.schema);
-
-    OffHeapRowBlock rowBlock = TestOffHeapRowBlock.createRowBlockWithNull(10248);
-    OffHeapRowBlockReader reader = rowBlock.getReader();
-
-    ZeroCopyTuple inputTuple = new ZeroCopyTuple();
-
-    HeapTuple heapTuple = null;
-    ZeroCopyTuple zcTuple = null;
-    int i = 0;
-    while(reader.next(inputTuple)) {
-      RowStoreUtil.convert(inputTuple, builder);
-
-      heapTuple = builder.buildToHeapTuple();
-      TestOffHeapRowBlock.validateNullity(i, heapTuple);
-
-      zcTuple = builder.buildToZeroCopyTuple();
-      TestOffHeapRowBlock.validateNullity(i, zcTuple);
-
-      i++;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java b/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
deleted file mode 100644
index 96f465a..0000000
--- a/tajo-storage/src/test/java/org/apache/tajo/tuple/offheap/TestHeapTuple.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/***
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.tuple.offheap;
-
-import org.apache.tajo.catalog.SchemaUtil;
-import org.junit.Test;
-
-public class TestHeapTuple {
-
-  @Test
-  public void testHeapTuple() {
-    OffHeapRowBlock rowBlock = TestOffHeapRowBlock.createRowBlock(1024);
-
-    OffHeapRowBlockReader reader = new OffHeapRowBlockReader(rowBlock);
-
-    ZeroCopyTuple zcTuple = new ZeroCopyTuple();
-    int i = 0;
-    while (reader.next(zcTuple)) {
-      byte [] bytes = new byte[zcTuple.nioBuffer().limit()];
-      zcTuple.nioBuffer().get(bytes);
-
-      HeapTuple heapTuple = new HeapTuple(bytes, SchemaUtil.toDataTypes(TestOffHeapRowBlock.schema));
-      TestOffHeapRowBlock.validateTupleResult(i, heapTuple);
-      i++;
-    }
-
-    rowBlock.release();
-  }
-}
\ No newline at end of file


[14/30] tajo git commit: TAJO-1122: Refactor the tajo-storage project structure.

Posted by hj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java
new file mode 100644
index 0000000..bfbe478
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java
@@ -0,0 +1,270 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.IntervalDatum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.exception.UnsupportedException;
+
+import java.util.Arrays;
+
+public class LazyTuple implements Tuple, Cloneable {
+  private long offset;
+  private Datum[] values;
+  private byte[][] textBytes;
+  private Schema schema;
+  private byte[] nullBytes;
+  private SerializerDeserializer serializeDeserialize;
+
+  public LazyTuple(Schema schema, byte[][] textBytes, long offset) {
+    this(schema, textBytes, offset, NullDatum.get().asTextBytes(), new TextSerializerDeserializer());
+  }
+
+  public LazyTuple(Schema schema, byte[][] textBytes, long offset, byte[] nullBytes, SerializerDeserializer serde) {
+    this.schema = schema;
+    this.textBytes = textBytes;
+    this.values = new Datum[schema.size()];
+    this.offset = offset;
+    this.nullBytes = nullBytes;
+    this.serializeDeserialize = serde;
+  }
+
+  public LazyTuple(LazyTuple tuple) {
+    this.values = tuple.getValues();
+    this.offset = tuple.offset;
+    this.schema = tuple.schema;
+    this.textBytes = new byte[size()][];
+    this.nullBytes = tuple.nullBytes;
+    this.serializeDeserialize = tuple.serializeDeserialize;
+  }
+
+  @Override
+  public int size() {
+    return values.length;
+  }
+
+  @Override
+  public boolean contains(int fieldid) {
+    return textBytes[fieldid] != null || values[fieldid] != null;
+  }
+
+  @Override
+  public boolean isNull(int fieldid) {
+    return get(fieldid).isNull();
+  }
+
+  @Override
+  public boolean isNotNull(int fieldid) {
+    return !isNull(fieldid);
+  }
+
+  @Override
+  public void clear() {
+    for (int i = 0; i < values.length; i++) {
+      values[i] = null;
+      textBytes[i] = null;
+    }
+  }
+
+  //////////////////////////////////////////////////////
+  // Setter
+  //////////////////////////////////////////////////////
+  @Override
+  public void put(int fieldId, Datum value) {
+    values[fieldId] = value;
+    textBytes[fieldId] = null;
+  }
+
+  @Override
+  public void put(int fieldId, Datum[] values) {
+    for (int i = fieldId, j = 0; j < values.length; i++, j++) {
+      this.values[i] = values[j];
+    }
+    this.textBytes = new byte[values.length][];
+  }
+
+  @Override
+  public void put(int fieldId, Tuple tuple) {
+    for (int i = fieldId, j = 0; j < tuple.size(); i++, j++) {
+      values[i] = tuple.get(j);
+      textBytes[i] = null;
+    }
+  }
+
+  @Override
+  public void put(Datum[] values) {
+    System.arraycopy(values, 0, this.values, 0, size());
+    this.textBytes = new byte[values.length][];
+  }
+
+  //////////////////////////////////////////////////////
+  // Getter
+  //////////////////////////////////////////////////////
+  @Override
+  public Datum get(int fieldId) {
+    if (values[fieldId] != null)
+      return values[fieldId];
+    else if (textBytes.length <= fieldId) {
+      values[fieldId] = NullDatum.get();  // split error. (col : 3, separator: ',', row text: "a,")
+    } else if (textBytes[fieldId] != null) {
+      try {
+        values[fieldId] = serializeDeserialize.deserialize(schema.getColumn(fieldId),
+            textBytes[fieldId], 0, textBytes[fieldId].length, nullBytes);
+      } catch (Exception e) {
+        values[fieldId] = NullDatum.get();
+      }
+      textBytes[fieldId] = null;
+    } else {
+      //non-projection
+    }
+    return values[fieldId];
+  }
+
+  @Override
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  @Override
+  public long getOffset() {
+    return this.offset;
+  }
+
+  @Override
+  public boolean getBool(int fieldId) {
+    return get(fieldId).asBool();
+  }
+
+  @Override
+  public byte getByte(int fieldId) {
+    return get(fieldId).asByte();
+  }
+
+  @Override
+  public char getChar(int fieldId) {
+    return get(fieldId).asChar();
+  }
+
+  @Override
+  public byte [] getBytes(int fieldId) {
+    return get(fieldId).asByteArray();
+  }
+
+  @Override
+  public short getInt2(int fieldId) {
+    return get(fieldId).asInt2();
+  }
+
+  @Override
+  public int getInt4(int fieldId) {
+    return get(fieldId).asInt4();
+  }
+
+  @Override
+  public long getInt8(int fieldId) {
+    return get(fieldId).asInt8();
+  }
+
+  @Override
+  public float getFloat4(int fieldId) {
+    return get(fieldId).asFloat4();
+  }
+
+  @Override
+  public double getFloat8(int fieldId) {
+    return get(fieldId).asFloat8();
+  }
+
+  @Override
+  public String getText(int fieldId) {
+    return get(fieldId).asChars();
+  }
+
+  @Override
+  public ProtobufDatum getProtobufDatum(int fieldId) {
+    throw new UnsupportedException();
+  }
+
+  @Override
+  public IntervalDatum getInterval(int fieldId) {
+    return (IntervalDatum) get(fieldId);
+  }
+
+  @Override
+  public char[] getUnicodeChars(int fieldId) {
+    return get(fieldId).asUnicodeChars();
+  }
+
+  public String toString() {
+    boolean first = true;
+    StringBuilder str = new StringBuilder();
+    str.append("(");
+    Datum d;
+    for (int i = 0; i < values.length; i++) {
+      d = get(i);
+      if (d != null) {
+        if (first) {
+          first = false;
+        } else {
+          str.append(", ");
+        }
+        str.append(i)
+            .append("=>")
+            .append(d);
+      }
+    }
+    str.append(")");
+    return str.toString();
+  }
+
+  @Override
+  public int hashCode() {
+    return Arrays.hashCode(values);
+  }
+
+  @Override
+  public Datum[] getValues() {
+    Datum[] datums = new Datum[values.length];
+    for (int i = 0; i < values.length; i++) {
+      datums[i] = get(i);
+    }
+    return datums;
+  }
+
+  @Override
+  public Tuple clone() throws CloneNotSupportedException {
+    LazyTuple lazyTuple = (LazyTuple) super.clone();
+
+    lazyTuple.values = getValues(); //shallow copy
+    lazyTuple.textBytes = new byte[size()][];
+    return lazyTuple;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof Tuple) {
+      Tuple other = (Tuple) obj;
+      return Arrays.equals(getValues(), other.getValues());
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java
new file mode 100644
index 0000000..f19b61f
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.datum.*;
+import org.apache.tajo.util.ClassSize;
+
+public class MemoryUtil {
+
+  /** Overhead for an NullDatum */
+  public static final long NULL_DATUM;
+
+  /** Overhead for an BoolDatum */
+  public static final long BOOL_DATUM;
+
+  /** Overhead for an CharDatum */
+  public static final long CHAR_DATUM;
+
+  /** Overhead for an BitDatum */
+  public static final long BIT_DATUM;
+
+  /** Overhead for an Int2Datum */
+  public static final long INT2_DATUM;
+
+  /** Overhead for an Int4Datum */
+  public static final long INT4_DATUM;
+
+  /** Overhead for an Int8Datum */
+  public static final long INT8_DATUM;
+
+  /** Overhead for an Float4Datum */
+  public static final long FLOAT4_DATUM;
+
+  /** Overhead for an Float8Datum */
+  public static final long FLOAT8_DATUM;
+
+  /** Overhead for an TextDatum */
+  public static final long TEXT_DATUM;
+
+  /** Overhead for an BlobDatum */
+  public static final long BLOB_DATUM;
+
+  /** Overhead for an DateDatum */
+  public static final long DATE_DATUM;
+
+  /** Overhead for an TimeDatum */
+  public static final long TIME_DATUM;
+
+  /** Overhead for an TimestampDatum */
+  public static final long TIMESTAMP_DATUM;
+
+  static {
+    NULL_DATUM = ClassSize.estimateBase(NullDatum.class, false);
+
+    CHAR_DATUM = ClassSize.estimateBase(CharDatum.class, false);
+
+    BOOL_DATUM = ClassSize.estimateBase(BooleanDatum.class, false);
+
+    BIT_DATUM = ClassSize.estimateBase(BitDatum.class, false);
+
+    INT2_DATUM = ClassSize.estimateBase(Int2Datum.class, false);
+
+    INT4_DATUM = ClassSize.estimateBase(Int4Datum.class, false);
+
+    INT8_DATUM = ClassSize.estimateBase(Int8Datum.class, false);
+
+    FLOAT4_DATUM = ClassSize.estimateBase(Float4Datum.class, false);
+
+    FLOAT8_DATUM = ClassSize.estimateBase(Float8Datum.class, false);
+
+    TEXT_DATUM = ClassSize.estimateBase(TextDatum.class, false);
+
+    BLOB_DATUM = ClassSize.estimateBase(BlobDatum.class, false);
+
+    DATE_DATUM = ClassSize.estimateBase(DateDatum.class, false);
+
+    TIME_DATUM = ClassSize.estimateBase(TimeDatum.class, false);
+
+    TIMESTAMP_DATUM = ClassSize.estimateBase(TimestampDatum.class, false);
+  }
+
+  public static long calculateMemorySize(Tuple tuple) {
+    long total = ClassSize.OBJECT;
+    for (Datum datum : tuple.getValues()) {
+      switch (datum.type()) {
+
+      case NULL_TYPE:
+        total += NULL_DATUM;
+        break;
+
+      case BOOLEAN:
+        total += BOOL_DATUM;
+        break;
+
+      case BIT:
+        total += BIT_DATUM;
+        break;
+
+      case CHAR:
+        total += CHAR_DATUM + datum.size();
+        break;
+
+      case INT1:
+      case INT2:
+        total += INT2_DATUM;
+        break;
+
+      case INT4:
+        total += INT4_DATUM;
+        break;
+
+      case INT8:
+        total += INT8_DATUM;
+        break;
+
+      case FLOAT4:
+        total += FLOAT4_DATUM;
+        break;
+
+      case FLOAT8:
+        total += FLOAT4_DATUM;
+        break;
+
+      case TEXT:
+        total += TEXT_DATUM + datum.size();
+        break;
+
+      case DATE:
+        total += DATE_DATUM;
+        break;
+
+      case TIME:
+        total += TIME_DATUM;
+        break;
+
+      case TIMESTAMP:
+        total += TIMESTAMP_DATUM;
+        break;
+
+      default:
+        break;
+      }
+    }
+
+    return total;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..66b3667
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MergeScanner.java
@@ -0,0 +1,201 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class MergeScanner implements Scanner {
+  private Configuration conf;
+  private TableMeta meta;
+  private Schema schema;
+  private List<Fragment> fragments;
+  private Iterator<Fragment> iterator;
+  private Fragment currentFragment;
+  private Scanner currentScanner;
+  private Tuple tuple;
+  private boolean projectable = false;
+  private boolean selectable = false;
+  private Schema target;
+  private float progress;
+  protected TableStats tableStats;
+
+  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<Fragment> rawFragmentList)
+      throws IOException {
+    this(conf, schema, meta, rawFragmentList, schema);
+  }
+
+  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, List<Fragment> rawFragmentList,
+                      Schema target)
+      throws IOException {
+    this.conf = conf;
+    this.schema = schema;
+    this.meta = meta;
+    this.target = target;
+
+    this.fragments = new ArrayList<Fragment>();
+
+    long numBytes = 0;
+    for (Fragment eachFileFragment: rawFragmentList) {
+      long fragmentLength = StorageManager.getFragmentLength((TajoConf)conf, eachFileFragment);
+      if (fragmentLength > 0) {
+        numBytes += fragmentLength;
+        fragments.add(eachFileFragment);
+      }
+    }
+
+    // it should keep the input order. Otherwise, it causes wrong result of sort queries.
+    this.reset();
+
+    if (currentScanner != null) {
+      this.projectable = currentScanner.isProjectable();
+      this.selectable = currentScanner.isSelectable();
+    }
+
+    tableStats = new TableStats();
+
+    tableStats.setNumBytes(numBytes);
+    tableStats.setNumBlocks(fragments.size());
+
+    for(Column eachColumn: schema.getColumns()) {
+      ColumnStats columnStats = new ColumnStats(eachColumn);
+      tableStats.addColumnStat(columnStats);
+    }
+  }
+
+  @Override
+  public void init() throws IOException {
+    progress = 0.0f;
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (currentScanner != null)
+      tuple = currentScanner.next();
+
+    if (tuple != null) {
+      return tuple;
+    } else {
+      if (currentScanner != null) {
+        currentScanner.close();
+        TableStats scannerTableStsts = currentScanner.getInputStats();
+        if (scannerTableStsts != null) {
+          tableStats.setReadBytes(tableStats.getReadBytes() + scannerTableStsts.getReadBytes());
+          tableStats.setNumRows(tableStats.getNumRows() + scannerTableStsts.getNumRows());
+        }
+      }
+      currentScanner = getNextScanner();
+      if (currentScanner != null) {
+        tuple = currentScanner.next();
+      }
+    }
+    return tuple;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    this.iterator = fragments.iterator();
+    if (currentScanner != null) {
+      currentScanner.close();
+    }
+    this.currentScanner = getNextScanner();
+  }
+
+  private Scanner getNextScanner() throws IOException {
+    if (iterator.hasNext()) {
+      currentFragment = iterator.next();
+      currentScanner = StorageManager.getStorageManager((TajoConf)conf, meta.getStoreType()).getScanner(meta, schema,
+          currentFragment, target);
+      currentScanner.init();
+      return currentScanner;
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(currentScanner != null) {
+      currentScanner.close();
+      currentScanner = null;
+    }
+    iterator = null;
+    progress = 1.0f;
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return projectable;
+  }
+
+  @Override
+  public void setTarget(Column[] targets) {
+    this.target = new Schema(targets);
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return selectable;
+  }
+
+  @Override
+  public void setSearchCondition(Object expr) {
+  }
+
+  @Override
+  public Schema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public boolean isSplittable(){
+    return false;
+  }
+
+  @Override
+  public float getProgress() {
+    if (currentScanner != null && iterator != null && tableStats.getNumBytes() > 0) {
+      TableStats scannerTableStsts = currentScanner.getInputStats();
+      long currentScannerReadBytes = 0;
+      if (scannerTableStsts != null) {
+        currentScannerReadBytes = scannerTableStsts.getReadBytes();
+      }
+
+      return (float)(tableStats.getReadBytes() + currentScannerReadBytes) / (float)tableStats.getNumBytes();
+    } else {
+      return progress;
+    }
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    return tableStats;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java
new file mode 100644
index 0000000..4272228
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NullScanner.java
@@ -0,0 +1,109 @@
+package org.apache.tajo.storage; /**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+
+public class NullScanner implements Scanner {
+  protected final Configuration conf;
+  protected final TableMeta meta;
+  protected final Schema schema;
+  protected final Fragment fragment;
+  protected final int columnNum;
+  protected Column [] targets;
+  protected float progress;
+  protected TableStats tableStats;
+
+  public NullScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) {
+    this.conf = conf;
+    this.meta = meta;
+    this.schema = schema;
+    this.fragment = fragment;
+    this.tableStats = new TableStats();
+    this.columnNum = this.schema.size();
+  }
+
+  @Override
+  public void init() throws IOException {
+    progress = 0.0f;
+    tableStats.setNumBytes(0);
+    tableStats.setNumBlocks(0);
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    progress = 1.0f;
+    return null;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    progress = 0.0f;
+  }
+
+  @Override
+  public void close() throws IOException {
+    progress = 1.0f;
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return false;
+  }
+
+  @Override
+  public void setTarget(Column[] targets) {
+    this.targets = targets;
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return true;
+  }
+
+  @Override
+  public void setSearchCondition(Object expr) {
+
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return true;
+  }
+
+  @Override
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public TableStats getInputStats() {
+    return tableStats;
+  }
+
+  @Override
+  public Schema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NumericPathComparator.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
new file mode 100644
index 0000000..94d13ee
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.fs.Path;
+
+import java.util.Comparator;
+
+public class NumericPathComparator implements Comparator<Path> {
+
+  @Override
+  public int compare(Path p1, Path p2) {
+    int num1 = Integer.parseInt(p1.getName());
+    int num2 = Integer.parseInt(p2.getName());
+
+    return num1 - num2;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
new file mode 100644
index 0000000..24b6280
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
@@ -0,0 +1,377 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.IntervalDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.storage.exception.UnknownDataTypeException;
+import org.apache.tajo.tuple.offheap.RowWriter;
+import org.apache.tajo.util.BitArray;
+
+import java.nio.ByteBuffer;
+
+public class RowStoreUtil {
+  public static int[] getTargetIds(Schema inSchema, Schema outSchema) {
+    int[] targetIds = new int[outSchema.size()];
+    int i = 0;
+    for (Column target : outSchema.getColumns()) {
+      targetIds[i] = inSchema.getColumnId(target.getQualifiedName());
+      i++;
+    }
+
+    return targetIds;
+  }
+
+  public static Tuple project(Tuple in, Tuple out, int[] targetIds) {
+    out.clear();
+    for (int idx = 0; idx < targetIds.length; idx++) {
+      out.put(idx, in.get(targetIds[idx]));
+    }
+    return out;
+  }
+
+  public static RowStoreEncoder createEncoder(Schema schema) {
+    return new RowStoreEncoder(schema);
+  }
+
+  public static RowStoreDecoder createDecoder(Schema schema) {
+    return new RowStoreDecoder(schema);
+  }
+
+  public static class RowStoreDecoder {
+
+    private Schema schema;
+    private BitArray nullFlags;
+    private int headerSize;
+
+    private RowStoreDecoder(Schema schema) {
+      this.schema = schema;
+      nullFlags = new BitArray(schema.size());
+      headerSize = nullFlags.bytesLength();
+    }
+
+
+    public Tuple toTuple(byte [] bytes) {
+      nullFlags.clear();
+      ByteBuffer bb = ByteBuffer.wrap(bytes);
+      Tuple tuple = new VTuple(schema.size());
+      Column col;
+      TajoDataTypes.DataType type;
+
+      bb.limit(headerSize);
+      nullFlags.fromByteBuffer(bb);
+      bb.limit(bytes.length);
+
+      for (int i =0; i < schema.size(); i++) {
+        if (nullFlags.get(i)) {
+          tuple.put(i, DatumFactory.createNullDatum());
+          continue;
+        }
+
+        col = schema.getColumn(i);
+        type = col.getDataType();
+        switch (type.getType()) {
+          case BOOLEAN: tuple.put(i, DatumFactory.createBool(bb.get())); break;
+          case BIT:
+            byte b = bb.get();
+            tuple.put(i, DatumFactory.createBit(b));
+            break;
+
+          case CHAR:
+            byte c = bb.get();
+            tuple.put(i, DatumFactory.createChar(c));
+            break;
+
+          case INT2:
+            short s = bb.getShort();
+            tuple.put(i, DatumFactory.createInt2(s));
+            break;
+
+          case INT4:
+          case DATE:
+            int i_ = bb.getInt();
+            tuple.put(i, DatumFactory.createFromInt4(type, i_));
+            break;
+
+          case INT8:
+          case TIME:
+          case TIMESTAMP:
+            long l = bb.getLong();
+            tuple.put(i, DatumFactory.createFromInt8(type, l));
+            break;
+
+        case INTERVAL:
+            int month  = bb.getInt();
+            long milliseconds  = bb.getLong();
+            tuple.put(i, new IntervalDatum(month, milliseconds));
+            break;
+
+          case FLOAT4:
+            float f = bb.getFloat();
+            tuple.put(i, DatumFactory.createFloat4(f));
+            break;
+
+          case FLOAT8:
+            double d = bb.getDouble();
+            tuple.put(i, DatumFactory.createFloat8(d));
+            break;
+
+          case TEXT:
+            byte [] _string = new byte[bb.getInt()];
+            bb.get(_string);
+            tuple.put(i, DatumFactory.createText(_string));
+            break;
+
+          case BLOB:
+            byte [] _bytes = new byte[bb.getInt()];
+            bb.get(_bytes);
+            tuple.put(i, DatumFactory.createBlob(_bytes));
+            break;
+
+          case INET4:
+            byte [] _ipv4 = new byte[4];
+            bb.get(_ipv4);
+            tuple.put(i, DatumFactory.createInet4(_ipv4));
+            break;
+          case INET6:
+            // TODO - to be implemented
+            throw new UnsupportedException(type.getType().name());
+          default:
+            throw new RuntimeException(new UnknownDataTypeException(type.getType().name()));
+        }
+      }
+      return tuple;
+    }
+
+    public Schema getSchema() {
+      return schema;
+    }
+  }
+
+  public static class RowStoreEncoder {
+    private Schema schema;
+    private BitArray nullFlags;
+    private int headerSize;
+
+    private RowStoreEncoder(Schema schema) {
+      this.schema = schema;
+      nullFlags = new BitArray(schema.size());
+      headerSize = nullFlags.bytesLength();
+    }
+
+    public byte[] toBytes(Tuple tuple) {
+      nullFlags.clear();
+      int size = estimateTupleDataSize(tuple);
+      ByteBuffer bb = ByteBuffer.allocate(size + headerSize);
+      bb.position(headerSize);
+      Column col;
+      for (int i = 0; i < schema.size(); i++) {
+        if (tuple.isNull(i)) {
+          nullFlags.set(i);
+          continue;
+        }
+
+        col = schema.getColumn(i);
+        switch (col.getDataType().getType()) {
+        case NULL_TYPE:
+          nullFlags.set(i);
+          break;
+        case BOOLEAN:
+          bb.put(tuple.get(i).asByte());
+          break;
+        case BIT:
+          bb.put(tuple.get(i).asByte());
+          break;
+        case CHAR:
+          bb.put(tuple.get(i).asByte());
+          break;
+        case INT2:
+          bb.putShort(tuple.get(i).asInt2());
+          break;
+        case INT4:
+          bb.putInt(tuple.get(i).asInt4());
+          break;
+        case INT8:
+          bb.putLong(tuple.get(i).asInt8());
+          break;
+        case FLOAT4:
+          bb.putFloat(tuple.get(i).asFloat4());
+          break;
+        case FLOAT8:
+          bb.putDouble(tuple.get(i).asFloat8());
+          break;
+        case TEXT:
+          byte[] _string = tuple.get(i).asByteArray();
+          bb.putInt(_string.length);
+          bb.put(_string);
+          break;
+        case DATE:
+          bb.putInt(tuple.get(i).asInt4());
+          break;
+        case TIME:
+        case TIMESTAMP:
+          bb.putLong(tuple.get(i).asInt8());
+          break;
+        case INTERVAL:
+          IntervalDatum interval = (IntervalDatum) tuple.get(i);
+          bb.putInt(interval.getMonths());
+          bb.putLong(interval.getMilliSeconds());
+          break;
+        case BLOB:
+          byte[] bytes = tuple.get(i).asByteArray();
+          bb.putInt(bytes.length);
+          bb.put(bytes);
+          break;
+        case INET4:
+          byte[] ipBytes = tuple.get(i).asByteArray();
+          bb.put(ipBytes);
+          break;
+        case INET6:
+          bb.put(tuple.get(i).asByteArray());
+          break;
+        default:
+          throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
+        }
+      }
+
+      byte[] flags = nullFlags.toArray();
+      int finalPosition = bb.position();
+      bb.position(0);
+      bb.put(flags);
+
+      bb.position(finalPosition);
+      bb.flip();
+      byte[] buf = new byte[bb.limit()];
+      bb.get(buf);
+      return buf;
+    }
+
+    // Note that, NULL values are treated separately
+    private int estimateTupleDataSize(Tuple tuple) {
+      int size = 0;
+      Column col;
+
+      for (int i = 0; i < schema.size(); i++) {
+        if (tuple.isNull(i)) {
+          continue;
+        }
+
+        col = schema.getColumn(i);
+        switch (col.getDataType().getType()) {
+        case BOOLEAN:
+        case BIT:
+        case CHAR:
+          size += 1;
+          break;
+        case INT2:
+          size += 2;
+          break;
+        case DATE:
+        case INT4:
+        case FLOAT4:
+          size += 4;
+          break;
+        case TIME:
+        case TIMESTAMP:
+        case INT8:
+        case FLOAT8:
+          size += 8;
+          break;
+        case INTERVAL:
+          size += 12;
+          break;
+        case TEXT:
+        case BLOB:
+          size += (4 + tuple.get(i).asByteArray().length);
+          break;
+        case INET4:
+        case INET6:
+          size += tuple.get(i).asByteArray().length;
+          break;
+        default:
+          throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name()));
+        }
+      }
+
+      size += 100; // optimistic reservation
+
+      return size;
+    }
+
+    public Schema getSchema() {
+      return schema;
+    }
+  }
+
+  public static void convert(Tuple tuple, RowWriter writer) {
+    writer.startRow();
+
+    for (int i = 0; i < writer.dataTypes().length; i++) {
+      if (tuple.isNull(i)) {
+        writer.skipField();
+        continue;
+      }
+      switch (writer.dataTypes()[i].getType()) {
+      case BOOLEAN:
+        writer.putBool(tuple.getBool(i));
+        break;
+      case INT1:
+      case INT2:
+        writer.putInt2(tuple.getInt2(i));
+        break;
+      case INT4:
+      case DATE:
+      case INET4:
+        writer.putInt4(tuple.getInt4(i));
+        break;
+      case INT8:
+      case TIMESTAMP:
+      case TIME:
+        writer.putInt8(tuple.getInt8(i));
+        break;
+      case FLOAT4:
+        writer.putFloat4(tuple.getFloat4(i));
+        break;
+      case FLOAT8:
+        writer.putFloat8(tuple.getFloat8(i));
+        break;
+      case TEXT:
+        writer.putText(tuple.getBytes(i));
+        break;
+      case INTERVAL:
+        writer.putInterval((IntervalDatum) tuple.getInterval(i));
+        break;
+      case PROTOBUF:
+        writer.putProtoDatum((ProtobufDatum) tuple.getProtobufDatum(i));
+        break;
+      case NULL_TYPE:
+        writer.skipField();
+        break;
+      default:
+        throw new UnsupportedException("Unknown data type: " + writer.dataTypes()[i]);
+      }
+    }
+    writer.endRow();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Scanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Scanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Scanner.java
new file mode 100644
index 0000000..0356b19
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Scanner.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.SchemaObject;
+import org.apache.tajo.catalog.statistics.TableStats;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Scanner Interface
+ */
+public interface Scanner extends SchemaObject, Closeable {
+
+  void init() throws IOException;
+
+  /**
+   * It returns one tuple at each call. 
+   * 
+   * @return retrieve null if the scanner has no more tuples. 
+   * Otherwise it returns one tuple.
+   * 
+   * @throws java.io.IOException if internal I/O error occurs during next method
+   */
+  Tuple next() throws IOException;
+
+  /**
+   * Reset the cursor. After executed, the scanner
+   * will retrieve the first tuple.
+   *
+   * @throws java.io.IOException if internal I/O error occurs during reset method
+   */
+  void reset() throws IOException;
+
+  /**
+   * Close scanner
+   *
+   * @throws java.io.IOException if internal I/O error occurs during close method
+   */
+  void close() throws IOException;
+
+
+  /**
+   * It returns if the projection is executed in the underlying scanner layer.
+   *
+   * @return true if this scanner can project the given columns.
+   */
+  boolean isProjectable();
+
+  /**
+   * Set target columns
+   * @param targets columns to be projected
+   */
+  void setTarget(Column[] targets);
+
+  /**
+   * It returns if the selection is executed in the underlying scanner layer.
+   *
+   * @return true if this scanner can filter tuples against a given condition.
+   */
+  boolean isSelectable();
+
+  /**
+   * Set a search condition
+   * @param expr to be searched
+   *
+   * TODO - to be changed Object type
+   */
+  void setSearchCondition(Object expr);
+
+  /**
+   * It returns if the file is splittable.
+   *
+   * @return true if this scanner can split the a file.
+   */
+  boolean isSplittable();
+
+  /**
+   * How much of the input has the Scanner consumed
+   * @return progress from <code>0.0</code> to <code>1.0</code>.
+   */
+  float getProgress();
+
+  TableStats getInputStats();
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SeekableScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SeekableScanner.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SeekableScanner.java
new file mode 100644
index 0000000..894e7ee
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SeekableScanner.java
@@ -0,0 +1,28 @@
+/**
+ * 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 java.io.IOException;
+
+public interface SeekableScanner extends Scanner {
+
+  public abstract long getNextOffset() throws IOException;
+
+  public abstract void seek(long offset) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
new file mode 100644
index 0000000..564a9f5
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+@Deprecated
+public interface SerializerDeserializer {
+
+  public int serialize(Column col, Datum datum, OutputStream out, byte[] nullCharacters) throws IOException;
+
+  public Datum deserialize(Column col, byte[] bytes, int offset, int length, byte[] nullCharacters) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
new file mode 100644
index 0000000..d2a692d
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageManager.java
@@ -0,0 +1,933 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.NodeType;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.plan.rewrite.RewriteRule;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.URI;
+import java.text.NumberFormat;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * StorageManager manages the functions of storing and reading data.
+ * StorageManager is a abstract class.
+ * For supporting such as HDFS, HBASE, a specific StorageManager should be implemented by inheriting this class.
+ *
+ */
+public abstract class StorageManager {
+  private final Log LOG = LogFactory.getLog(StorageManager.class);
+
+  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
+      Configuration.class,
+      Schema.class,
+      TableMeta.class,
+      Fragment.class
+  };
+
+  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
+      Configuration.class,
+      QueryUnitAttemptId.class,
+      Schema.class,
+      TableMeta.class,
+      Path.class
+  };
+
+  protected TajoConf conf;
+  protected StoreType storeType;
+
+  /**
+   * Cache of StorageManager.
+   * Key is manager key(warehouse path) + store type
+   */
+  private static final Map<String, StorageManager> storageManagers = Maps.newHashMap();
+
+  /**
+   * Cache of scanner handlers for each storage type.
+   */
+  protected static final Map<String, Class<? extends Scanner>> SCANNER_HANDLER_CACHE
+      = new ConcurrentHashMap<String, Class<? extends Scanner>>();
+
+  /**
+   * Cache of appender handlers for each storage type.
+   */
+  protected static final Map<String, Class<? extends Appender>> APPENDER_HANDLER_CACHE
+      = new ConcurrentHashMap<String, Class<? extends Appender>>();
+
+  /**
+   * Cache of constructors for each class. Pins the classes so they
+   * can't be garbage collected until ReflectionUtils can be collected.
+   */
+  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
+      new ConcurrentHashMap<Class<?>, Constructor<?>>();
+
+  public StorageManager(StoreType storeType) {
+    this.storeType = storeType;
+  }
+
+  /**
+   * Initialize storage manager.
+   * @throws java.io.IOException
+   */
+  protected abstract void storageInit() throws IOException;
+
+  /**
+   * This method is called after executing "CREATE TABLE" statement.
+   * If a storage is a file based storage, a storage manager may create directory.
+   *
+   * @param tableDesc Table description which is created.
+   * @param ifNotExists Creates the table only when the table does not exist.
+   * @throws java.io.IOException
+   */
+  public abstract void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException;
+
+  /**
+   * This method is called after executing "DROP TABLE" statement with the 'PURGE' option
+   * which is the option to delete all the data.
+   *
+   * @param tableDesc
+   * @throws java.io.IOException
+   */
+  public abstract void purgeTable(TableDesc tableDesc) throws IOException;
+
+  /**
+   * Returns the splits that will serve as input for the scan tasks. The
+   * number of splits matches the number of regions in a table.
+   * @param fragmentId The table name or previous ExecutionBlockId
+   * @param tableDesc The table description for the target data.
+   * @param scanNode The logical node for scanning.
+   * @return The list of input fragments.
+   * @throws java.io.IOException
+   */
+  public abstract List<Fragment> getSplits(String fragmentId, TableDesc tableDesc,
+                                           ScanNode scanNode) throws IOException;
+
+  /**
+   * It returns the splits that will serve as input for the non-forward query scanner such as 'select * from table1'.
+   * The result list should be small. If there is many fragments for scanning, TajoMaster uses the paging navigation.
+   * @param tableDesc The table description for the target data.
+   * @param currentPage The current page number within the entire list.
+   * @param numFragments The number of fragments in the result.
+   * @return The list of input fragments.
+   * @throws java.io.IOException
+   */
+  public abstract List<Fragment> getNonForwardSplit(TableDesc tableDesc, int currentPage, int numFragments)
+      throws IOException;
+
+  /**
+   * It returns the storage property.
+   * @return The storage property
+   */
+  public abstract StorageProperty getStorageProperty();
+
+  /**
+   * Release storage manager resource
+   */
+  public abstract void closeStorageManager();
+
+  /**
+   * It is called by a Repartitioner for range shuffling when the SortRangeType of SortNode is USING_STORAGE_MANAGER.
+   * In general Repartitioner determines the partition range using previous output statistics data.
+   * In the special cases, such as HBase Repartitioner uses the result of this method.
+   *
+   * @param queryContext The current query context which contains query properties.
+   * @param tableDesc The table description for the target data.
+   * @param inputSchema The input schema
+   * @param sortSpecs The sort specification that contains the sort column and sort order.
+   * @return The list of sort ranges.
+   * @throws java.io.IOException
+   */
+  public abstract TupleRange[] getInsertSortRanges(OverridableConf queryContext, TableDesc tableDesc,
+                                                   Schema inputSchema, SortSpec[] sortSpecs,
+                                                   TupleRange dataRange) throws IOException;
+
+  /**
+   * This method is called before executing 'INSERT' or 'CREATE TABLE as SELECT'.
+   * In general Tajo creates the target table after finishing the final sub-query of CATS.
+   * But In the special cases, such as HBase INSERT or CAST query uses the target table information.
+   * That kind of the storage should implements the logic related to creating table in this method.
+   *
+   * @param node The child node of the root node.
+   * @throws java.io.IOException
+   */
+  public abstract void beforeInsertOrCATS(LogicalNode node) throws IOException;
+
+  /**
+   * It is called when the query failed.
+   * Each storage manager should implement to be processed when the query fails in this method.
+   *
+   * @param node The child node of the root node.
+   * @throws java.io.IOException
+   */
+  public abstract void rollbackOutputCommit(LogicalNode node) throws IOException;
+
+  /**
+   * Returns the current storage type.
+   * @return
+   */
+  public StoreType getStoreType() {
+    return storeType;
+  }
+
+  /**
+   * Initialize StorageManager instance. It should be called before using.
+   *
+   * @param tajoConf
+   * @throws java.io.IOException
+   */
+  public void init(TajoConf tajoConf) throws IOException {
+    this.conf = tajoConf;
+    storageInit();
+  }
+
+  /**
+   * Close StorageManager
+   * @throws java.io.IOException
+   */
+  public void close() throws IOException {
+    synchronized(storageManagers) {
+      for (StorageManager eachStorageManager: storageManagers.values()) {
+        eachStorageManager.closeStorageManager();
+      }
+    }
+  }
+
+  /**
+   * Returns the splits that will serve as input for the scan tasks. The
+   * number of splits matches the number of regions in a table.
+   *
+   * @param fragmentId The table name or previous ExecutionBlockId
+   * @param tableDesc The table description for the target data.
+   * @return The list of input fragments.
+   * @throws java.io.IOException
+   */
+  public List<Fragment> getSplits(String fragmentId, TableDesc tableDesc) throws IOException {
+    return getSplits(fragmentId, tableDesc, null);
+  }
+
+  /**
+   * Returns FileStorageManager instance.
+   *
+   * @param tajoConf Tajo system property.
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getFileStorageManager(TajoConf tajoConf) throws IOException {
+    return getFileStorageManager(tajoConf, null);
+  }
+
+  /**
+   * Returns FileStorageManager instance and sets WAREHOUSE_DIR property in tajoConf with warehousePath parameter.
+   *
+   * @param tajoConf Tajo system property.
+   * @param warehousePath The warehouse directory to be set in the tajoConf.
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getFileStorageManager(TajoConf tajoConf, Path warehousePath) throws IOException {
+    URI uri;
+    TajoConf copiedConf = new TajoConf(tajoConf);
+    if (warehousePath != null) {
+      copiedConf.setVar(ConfVars.WAREHOUSE_DIR, warehousePath.toUri().toString());
+    }
+    uri = TajoConf.getWarehouseDir(copiedConf).toUri();
+    String key = "file".equals(uri.getScheme()) ? "file" : uri.toString();
+    return getStorageManager(copiedConf, StoreType.CSV, key);
+  }
+
+  /**
+   * Returns the proper StorageManager instance according to the storeType.
+   *
+   * @param tajoConf Tajo system property.
+   * @param storeType Storage type
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getStorageManager(TajoConf tajoConf, String storeType) throws IOException {
+    if ("HBASE".equals(storeType)) {
+      return getStorageManager(tajoConf, StoreType.HBASE);
+    } else {
+      return getStorageManager(tajoConf, StoreType.CSV);
+    }
+  }
+
+  /**
+   * Returns the proper StorageManager instance according to the storeType.
+   *
+   * @param tajoConf Tajo system property.
+   * @param storeType Storage type
+   * @return
+   * @throws java.io.IOException
+   */
+  public static StorageManager getStorageManager(TajoConf tajoConf, StoreType storeType) throws IOException {
+    return getStorageManager(tajoConf, storeType, null);
+  }
+
+  /**
+   * Returns the proper StorageManager instance according to the storeType
+   *
+   * @param tajoConf Tajo system property.
+   * @param storeType Storage type
+   * @param managerKey Key that can identify each storage manager(may be a path)
+   * @return
+   * @throws java.io.IOException
+   */
+  public static synchronized StorageManager getStorageManager (
+      TajoConf tajoConf, StoreType storeType, String managerKey) throws IOException {
+    synchronized (storageManagers) {
+      String storeKey = storeType + managerKey;
+      StorageManager manager = storageManagers.get(storeKey);
+      if (manager == null) {
+        String typeName = "hdfs";
+
+        switch (storeType) {
+          case HBASE:
+            typeName = "hbase";
+            break;
+          default:
+            typeName = "hdfs";
+        }
+
+        Class<? extends StorageManager> storageManagerClass =
+            tajoConf.getClass(String.format("tajo.storage.manager.%s.class", typeName), null, StorageManager.class);
+
+        if (storageManagerClass == null) {
+          throw new IOException("Unknown Storage Type: " + typeName);
+        }
+
+        try {
+          Constructor<? extends StorageManager> constructor =
+              (Constructor<? extends StorageManager>) CONSTRUCTOR_CACHE.get(storageManagerClass);
+          if (constructor == null) {
+            constructor = storageManagerClass.getDeclaredConstructor(new Class<?>[]{StoreType.class});
+            constructor.setAccessible(true);
+            CONSTRUCTOR_CACHE.put(storageManagerClass, constructor);
+          }
+          manager = constructor.newInstance(new Object[]{storeType});
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+        manager.init(tajoConf);
+        storageManagers.put(storeKey, manager);
+      }
+
+      return manager;
+    }
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @param target Columns which are selected.
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment, Schema target) throws IOException {
+    return getScanner(meta, schema, FragmentConvertor.convert(conf, fragment), target);
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment) throws IOException {
+    return getScanner(meta, schema, fragment, schema);
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @param target The output schema
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
+    if (fragment.isEmpty()) {
+      Scanner scanner = new NullScanner(conf, schema, meta, fragment);
+      scanner.setTarget(target.toArray());
+
+      return scanner;
+    }
+
+    Scanner scanner;
+
+    Class<? extends Scanner> scannerClass = getScannerClass(meta.getStoreType());
+    scanner = newScannerInstance(scannerClass, conf, schema, meta, fragment);
+    if (scanner.isProjectable()) {
+      scanner.setTarget(target.toArray());
+    }
+
+    return scanner;
+  }
+
+  /**
+   * Returns Scanner instance.
+   *
+   * @param conf The system property
+   * @param meta The table meta
+   * @param schema The input schema
+   * @param fragment The fragment for scanning
+   * @param target The output schema
+   * @return Scanner instance
+   * @throws java.io.IOException
+   */
+  public static synchronized SeekableScanner getSeekableScanner(
+      TajoConf conf, TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
+    return (SeekableScanner)getStorageManager(conf, meta.getStoreType()).getScanner(meta, schema, fragment, target);
+  }
+
+  /**
+   * Returns Appender instance.
+   * @param queryContext Query property.
+   * @param taskAttemptId Task id.
+   * @param meta Table meta data.
+   * @param schema Output schema.
+   * @param workDir Working directory
+   * @return Appender instance
+   * @throws java.io.IOException
+   */
+  public Appender getAppender(OverridableConf queryContext,
+                              QueryUnitAttemptId taskAttemptId, TableMeta meta, Schema schema, Path workDir)
+      throws IOException {
+    Appender appender;
+
+    Class<? extends Appender> appenderClass;
+
+    String handlerName = meta.getStoreType().name().toLowerCase();
+    appenderClass = APPENDER_HANDLER_CACHE.get(handlerName);
+    if (appenderClass == null) {
+      appenderClass = conf.getClass(
+          String.format("tajo.storage.appender-handler.%s.class",
+              meta.getStoreType().name().toLowerCase()), null, Appender.class);
+      APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
+    }
+
+    if (appenderClass == null) {
+      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
+    }
+
+    appender = newAppenderInstance(appenderClass, conf, taskAttemptId, meta, schema, workDir);
+
+    return appender;
+  }
+
+  /**
+   * Creates a scanner instance.
+   *
+   * @param theClass Concrete class of scanner
+   * @param conf System property
+   * @param schema Input schema
+   * @param meta Table meta data
+   * @param fragment The fragment for scanning
+   * @param <T>
+   * @return The scanner instance
+   */
+  public static <T> T newScannerInstance(Class<T> theClass, Configuration conf, Schema schema, TableMeta meta,
+                                         Fragment fragment) {
+    T result;
+    try {
+      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
+      if (meth == null) {
+        meth = theClass.getDeclaredConstructor(DEFAULT_SCANNER_PARAMS);
+        meth.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(theClass, meth);
+      }
+      result = meth.newInstance(new Object[]{conf, schema, meta, fragment});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result;
+  }
+
+  /**
+   * Creates a scanner instance.
+   *
+   * @param theClass Concrete class of scanner
+   * @param conf System property
+   * @param taskAttemptId Task id
+   * @param meta Table meta data
+   * @param schema Input schema
+   * @param workDir Working directory
+   * @param <T>
+   * @return The scanner instance
+   */
+  public static <T> T newAppenderInstance(Class<T> theClass, Configuration conf, QueryUnitAttemptId taskAttemptId,
+                                          TableMeta meta, Schema schema, Path workDir) {
+    T result;
+    try {
+      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
+      if (meth == null) {
+        meth = theClass.getDeclaredConstructor(DEFAULT_APPENDER_PARAMS);
+        meth.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(theClass, meth);
+      }
+      result = meth.newInstance(new Object[]{conf, taskAttemptId, schema, meta, workDir});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result;
+  }
+
+  /**
+   * Return the Scanner class for the StoreType that is defined in storage-default.xml.
+   *
+   * @param storeType store type
+   * @return The Scanner class
+   * @throws java.io.IOException
+   */
+  public Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException {
+    String handlerName = storeType.name().toLowerCase();
+    Class<? extends Scanner> scannerClass = SCANNER_HANDLER_CACHE.get(handlerName);
+    if (scannerClass == null) {
+      scannerClass = conf.getClass(
+          String.format("tajo.storage.scanner-handler.%s.class",storeType.name().toLowerCase()), null, Scanner.class);
+      SCANNER_HANDLER_CACHE.put(handlerName, scannerClass);
+    }
+
+    if (scannerClass == null) {
+      throw new IOException("Unknown Storage Type: " + storeType.name());
+    }
+
+    return scannerClass;
+  }
+
+  /**
+   * Return length of the fragment.
+   * In the UNKNOWN_LENGTH case get FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH from the configuration.
+   *
+   * @param conf Tajo system property
+   * @param fragment Fragment
+   * @return
+   */
+  public static long getFragmentLength(TajoConf conf, Fragment fragment) {
+    if (fragment.getLength() == TajoConstants.UNKNOWN_LENGTH) {
+      return conf.getLongVar(ConfVars.FRAGMENT_ALTERNATIVE_UNKNOWN_LENGTH);
+    } else {
+      return fragment.getLength();
+    }
+  }
+
+  /**
+   * It is called after making logical plan. Storage manager should verify the schema for inserting.
+   *
+   * @param tableDesc The table description of insert target.
+   * @param outSchema  The output schema of select query for inserting.
+   * @throws java.io.IOException
+   */
+  public void verifyInsertTableSchema(TableDesc tableDesc, Schema outSchema) throws IOException {
+    // nothing to do
+  }
+
+  /**
+   * Returns the list of storage specified rewrite rules.
+   * This values are used by LogicalOptimizer.
+   *
+   * @param queryContext The query property
+   * @param tableDesc The description of the target table.
+   * @return The list of storage specified rewrite rules
+   * @throws java.io.IOException
+   */
+  public List<RewriteRule> getRewriteRules(OverridableConf queryContext, TableDesc tableDesc) throws IOException {
+    return null;
+  }
+
+  /**
+   * Finalizes result data. Tajo stores result data in the staging directory.
+   * If the query fails, clean up the staging directory.
+   * Otherwise the query is successful, move to the final directory from the staging directory.
+   *
+   * @param queryContext The query property
+   * @param finalEbId The final execution block id
+   * @param plan The query plan
+   * @param schema The final output schema
+   * @param tableDesc The description of the target table
+   * @return Saved path
+   * @throws java.io.IOException
+   */
+  public Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
+                               LogicalPlan plan, Schema schema,
+                               TableDesc tableDesc) throws IOException {
+    return commitOutputData(queryContext, finalEbId, plan, schema, tableDesc, true);
+  }
+
+  /**
+   * Finalizes result data. Tajo stores result data in the staging directory.
+   * If the query fails, clean up the staging directory.
+   * Otherwise the query is successful, move to the final directory from the staging directory.
+   *
+   * @param queryContext The query property
+   * @param finalEbId The final execution block id
+   * @param plan The query plan
+   * @param schema The final output schema
+   * @param tableDesc The description of the target table
+   * @param changeFileSeq If true change result file name with max sequence.
+   * @return Saved path
+   * @throws java.io.IOException
+   */
+  protected Path commitOutputData(OverridableConf queryContext, ExecutionBlockId finalEbId,
+                               LogicalPlan plan, Schema schema,
+                               TableDesc tableDesc, boolean changeFileSeq) throws IOException {
+    Path stagingDir = new Path(queryContext.get(QueryVars.STAGING_DIR));
+    Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    Path finalOutputDir;
+    if (!queryContext.get(QueryVars.OUTPUT_TABLE_PATH, "").isEmpty()) {
+      finalOutputDir = new Path(queryContext.get(QueryVars.OUTPUT_TABLE_PATH));
+      FileSystem fs = stagingResultDir.getFileSystem(conf);
+
+      if (queryContext.getBool(QueryVars.OUTPUT_OVERWRITE, false)) { // INSERT OVERWRITE INTO
+
+        // It moves the original table into the temporary location.
+        // Then it moves the new result table into the original table location.
+        // Upon failed, it recovers the original table if possible.
+        boolean movedToOldTable = false;
+        boolean committed = false;
+        Path oldTableDir = new Path(stagingDir, TajoConstants.INSERT_OVERWIRTE_OLD_TABLE_NAME);
+
+        if (!queryContext.get(QueryVars.OUTPUT_PARTITIONS, "").isEmpty()) {
+          // This is a map for existing non-leaf directory to rename. A key is current directory and a value is
+          // renaming directory.
+          Map<Path, Path> renameDirs = TUtil.newHashMap();
+          // This is a map for recovering existing partition directory. A key is current directory and a value is
+          // temporary directory to back up.
+          Map<Path, Path> recoveryDirs = TUtil.newHashMap();
+
+          try {
+            if (!fs.exists(finalOutputDir)) {
+              fs.mkdirs(finalOutputDir);
+            }
+
+            visitPartitionedDirectory(fs, stagingResultDir, finalOutputDir, stagingResultDir.toString(),
+                renameDirs, oldTableDir);
+
+            // Rename target partition directories
+            for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
+              // Backup existing data files for recovering
+              if (fs.exists(entry.getValue())) {
+                String recoveryPathString = entry.getValue().toString().replaceAll(finalOutputDir.toString(),
+                    oldTableDir.toString());
+                Path recoveryPath = new Path(recoveryPathString);
+                fs.rename(entry.getValue(), recoveryPath);
+                fs.exists(recoveryPath);
+                recoveryDirs.put(entry.getValue(), recoveryPath);
+              }
+              // Delete existing directory
+              fs.delete(entry.getValue(), true);
+              // Rename staging directory to final output directory
+              fs.rename(entry.getKey(), entry.getValue());
+            }
+
+          } catch (IOException ioe) {
+            // Remove created dirs
+            for(Map.Entry<Path, Path> entry : renameDirs.entrySet()) {
+              fs.delete(entry.getValue(), true);
+            }
+
+            // Recovery renamed dirs
+            for(Map.Entry<Path, Path> entry : recoveryDirs.entrySet()) {
+              fs.delete(entry.getValue(), true);
+              fs.rename(entry.getValue(), entry.getKey());
+            }
+            throw new IOException(ioe.getMessage());
+          }
+        } else {
+          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 {
+        String queryType = queryContext.get(QueryVars.COMMAND_TYPE);
+
+        if (queryType != null && queryType.equals(NodeType.INSERT.name())) { // INSERT INTO an existing table
+
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(3);
+
+          if (!queryContext.get(QueryVars.OUTPUT_PARTITIONS, "").isEmpty()) {
+            for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
+              if (eachFile.isFile()) {
+                LOG.warn("Partition table can't have file in a staging dir: " + eachFile.getPath());
+                continue;
+              }
+              moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, -1, changeFileSeq);
+            }
+          } else {
+            int maxSeq = StorageUtil.getMaxFileSequence(fs, finalOutputDir, false) + 1;
+            for(FileStatus eachFile: fs.listStatus(stagingResultDir)) {
+              if (eachFile.getPath().getName().startsWith("_")) {
+                continue;
+              }
+              moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputDir, fmt, maxSeq++, changeFileSeq);
+            }
+          }
+          // checking all file moved and remove empty dir
+          verifyAllFileMoved(fs, stagingResultDir);
+          FileStatus[] files = fs.listStatus(stagingResultDir);
+          if (files != null && files.length != 0) {
+            for (FileStatus eachFile: files) {
+              LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
+            }
+          }
+        } else { // CREATE TABLE AS SELECT (CTAS)
+          fs.rename(stagingResultDir, finalOutputDir);
+          LOG.info("Moved from the staging dir to the output directory '" + finalOutputDir);
+        }
+      }
+    } else {
+      finalOutputDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+    }
+
+    return finalOutputDir;
+  }
+
+  /**
+   * Attach the sequence number to the output file name and than move the file into the final result path.
+   *
+   * @param fs FileSystem
+   * @param stagingResultDir The staging result dir
+   * @param fileStatus The file status
+   * @param finalOutputPath Final output path
+   * @param nf Number format
+   * @param fileSeq The sequence number
+   * @throws java.io.IOException
+   */
+  private void moveResultFromStageToFinal(FileSystem fs, Path stagingResultDir,
+                                          FileStatus fileStatus, Path finalOutputPath,
+                                          NumberFormat nf,
+                                          int fileSeq, boolean changeFileSeq) throws IOException {
+    if (fileStatus.isDirectory()) {
+      String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
+      if (subPath != null) {
+        Path finalSubPath = new Path(finalOutputPath, subPath);
+        if (!fs.exists(finalSubPath)) {
+          fs.mkdirs(finalSubPath);
+        }
+        int maxSeq = StorageUtil.getMaxFileSequence(fs, finalSubPath, false);
+        for (FileStatus eachFile : fs.listStatus(fileStatus.getPath())) {
+          if (eachFile.getPath().getName().startsWith("_")) {
+            continue;
+          }
+          moveResultFromStageToFinal(fs, stagingResultDir, eachFile, finalOutputPath, nf, ++maxSeq, changeFileSeq);
+        }
+      } else {
+        throw new IOException("Wrong staging dir:" + stagingResultDir + "," + fileStatus.getPath());
+      }
+    } else {
+      String subPath = extractSubPath(stagingResultDir, fileStatus.getPath());
+      if (subPath != null) {
+        Path finalSubPath = new Path(finalOutputPath, subPath);
+        if (changeFileSeq) {
+          finalSubPath = new Path(finalSubPath.getParent(), replaceFileNameSeq(finalSubPath, fileSeq, nf));
+        }
+        if (!fs.exists(finalSubPath.getParent())) {
+          fs.mkdirs(finalSubPath.getParent());
+        }
+        if (fs.exists(finalSubPath)) {
+          throw new IOException("Already exists data file:" + finalSubPath);
+        }
+        boolean success = fs.rename(fileStatus.getPath(), finalSubPath);
+        if (success) {
+          LOG.info("Moving staging file[" + fileStatus.getPath() + "] + " +
+              "to final output[" + finalSubPath + "]");
+        } else {
+          LOG.error("Can't move staging file[" + fileStatus.getPath() + "] + " +
+              "to final output[" + finalSubPath + "]");
+        }
+      }
+    }
+  }
+
+  /**
+   * Removes the path of the parent.
+   * @param parentPath
+   * @param childPath
+   * @return
+   */
+  private String extractSubPath(Path parentPath, Path childPath) {
+    String parentPathStr = parentPath.toUri().getPath();
+    String childPathStr = childPath.toUri().getPath();
+
+    if (parentPathStr.length() > childPathStr.length()) {
+      return null;
+    }
+
+    int index = childPathStr.indexOf(parentPathStr);
+    if (index != 0) {
+      return null;
+    }
+
+    return childPathStr.substring(parentPathStr.length() + 1);
+  }
+
+  /**
+   * Attach the sequence number to a path.
+   *
+   * @param path Path
+   * @param seq sequence number
+   * @param nf Number format
+   * @return New path attached with sequence number
+   * @throws java.io.IOException
+   */
+  private String replaceFileNameSeq(Path path, int seq, NumberFormat nf) throws IOException {
+    String[] tokens = path.getName().split("-");
+    if (tokens.length != 4) {
+      throw new IOException("Wrong result file name:" + path);
+    }
+    return tokens[0] + "-" + tokens[1] + "-" + tokens[2] + "-" + nf.format(seq);
+  }
+
+  /**
+   * Make sure all files are moved.
+   * @param fs FileSystem
+   * @param stagingPath The stagind directory
+   * @return
+   * @throws java.io.IOException
+   */
+  private boolean verifyAllFileMoved(FileSystem fs, Path stagingPath) throws IOException {
+    FileStatus[] files = fs.listStatus(stagingPath);
+    if (files != null && files.length != 0) {
+      for (FileStatus eachFile: files) {
+        if (eachFile.isFile()) {
+          LOG.error("There are some unmoved files in staging dir:" + eachFile.getPath());
+          return false;
+        } else {
+          if (verifyAllFileMoved(fs, eachFile.getPath())) {
+            fs.delete(eachFile.getPath(), false);
+          } else {
+            return false;
+          }
+        }
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * This method sets a rename map which includes renamed staging directory to final output directory recursively.
+   * If there exists some data files, this delete it for duplicate data.
+   *
+   *
+   * @param fs
+   * @param stagingPath
+   * @param outputPath
+   * @param stagingParentPathString
+   * @throws java.io.IOException
+   */
+  private void visitPartitionedDirectory(FileSystem fs, Path stagingPath, Path outputPath,
+                                         String stagingParentPathString,
+                                         Map<Path, Path> renameDirs, Path oldTableDir) throws IOException {
+    FileStatus[] files = fs.listStatus(stagingPath);
+
+    for(FileStatus eachFile : files) {
+      if (eachFile.isDirectory()) {
+        Path oldPath = eachFile.getPath();
+
+        // Make recover directory.
+        String recoverPathString = oldPath.toString().replaceAll(stagingParentPathString,
+            oldTableDir.toString());
+        Path recoveryPath = new Path(recoverPathString);
+        if (!fs.exists(recoveryPath)) {
+          fs.mkdirs(recoveryPath);
+        }
+
+        visitPartitionedDirectory(fs, eachFile.getPath(), outputPath, stagingParentPathString,
+            renameDirs, oldTableDir);
+        // Find last order partition for renaming
+        String newPathString = oldPath.toString().replaceAll(stagingParentPathString,
+            outputPath.toString());
+        Path newPath = new Path(newPathString);
+        if (!isLeafDirectory(fs, eachFile.getPath())) {
+          renameDirs.put(eachFile.getPath(), newPath);
+        } else {
+          if (!fs.exists(newPath)) {
+            fs.mkdirs(newPath);
+          }
+        }
+      }
+    }
+  }
+
+  private boolean isLeafDirectory(FileSystem fs, Path path) throws IOException {
+    boolean retValue = false;
+
+    FileStatus[] files = fs.listStatus(path);
+    for (FileStatus file : files) {
+      if (fs.isDirectory(file.getPath())) {
+        retValue = true;
+        break;
+      }
+    }
+
+    return retValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..6816d08
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageProperty.java
@@ -0,0 +1,40 @@
+/**
+ * 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 StorageProperty {
+  private boolean supportsInsertInto;
+  private boolean sortedInsert;
+
+  public boolean isSupportsInsertInto() {
+    return supportsInsertInto;
+  }
+
+  public void setSupportsInsertInto(boolean supportsInsertInto) {
+    this.supportsInsertInto = supportsInsertInto;
+  }
+
+  public boolean isSortedInsert() {
+    return sortedInsert;
+  }
+
+  public void setSortedInsert(boolean sortedInsert) {
+    this.sortedInsert = sortedInsert;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/dfd7f996/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
new file mode 100644
index 0000000..54fdb69
--- /dev/null
+++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/StorageUtil.java
@@ -0,0 +1,221 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.util.FileUtil;
+import sun.nio.ch.DirectBuffer;
+
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class StorageUtil extends StorageConstants {
+  public static int getRowByteSize(Schema schema) {
+    int sum = 0;
+    for(Column col : schema.getColumns()) {
+      sum += StorageUtil.getColByteSize(col);
+    }
+
+    return sum;
+  }
+
+  public static int getColByteSize(Column col) {
+    switch (col.getDataType().getType()) {
+      case BOOLEAN:
+        return 1;
+      case CHAR:
+        return 1;
+      case BIT:
+        return 1;
+      case INT2:
+        return 2;
+      case INT4:
+        return 4;
+      case INT8:
+        return 8;
+      case FLOAT4:
+        return 4;
+      case FLOAT8:
+        return 8;
+      case INET4:
+        return 4;
+      case INET6:
+        return 32;
+      case TEXT:
+        return 256;
+      case BLOB:
+        return 256;
+      case DATE:
+        return 4;
+      case TIME:
+        return 8;
+      case TIMESTAMP:
+        return 8;
+      default:
+        return 0;
+    }
+  }
+
+  public static void writeTableMeta(Configuration conf, Path tableroot, TableMeta meta) throws IOException {
+    FileSystem fs = tableroot.getFileSystem(conf);
+    FSDataOutputStream out = fs.create(new Path(tableroot, ".meta"));
+    FileUtil.writeProto(out, meta.getProto());
+    out.flush();
+    out.close();
+  }
+  
+  public static Path concatPath(String parent, String...childs) {
+    return concatPath(new Path(parent), childs);
+  }
+  
+  public static Path concatPath(Path parent, String...childs) {
+    StringBuilder sb = new StringBuilder();
+    
+    for(int i=0; i < childs.length; i++) {      
+      sb.append(childs[i]);
+      if(i < childs.length - 1)
+        sb.append("/");
+    }
+    
+    return new Path(parent, sb.toString());
+  }
+
+  static final String fileNamePatternV08 = "part-[0-9]*-[0-9]*";
+  static final String fileNamePatternV09 = "part-[0-9]*-[0-9]*-[0-9]*";
+
+  /**
+   * Written files can be one of two forms: "part-[0-9]*-[0-9]*" or "part-[0-9]*-[0-9]*-[0-9]*".
+   *
+   * This method finds the maximum sequence number from existing data files through the above patterns.
+   * If it cannot find any matched file or the maximum number, it will return -1.
+   *
+   * @param fs
+   * @param path
+   * @param recursive
+   * @return The maximum sequence number
+   * @throws java.io.IOException
+   */
+  public static int getMaxFileSequence(FileSystem fs, Path path, boolean recursive) throws IOException {
+    if (!fs.isDirectory(path)) {
+      return -1;
+    }
+
+    FileStatus[] files = fs.listStatus(path);
+
+    if (files == null || files.length == 0) {
+      return -1;
+    }
+
+    int maxValue = -1;
+    List<Path> fileNamePatternMatchedList = new ArrayList<Path>();
+
+    for (FileStatus eachFile: files) {
+      // In the case of partition table, return largest value within all partition dirs.
+      if (eachFile.isDirectory() && recursive) {
+        int value = getMaxFileSequence(fs, eachFile.getPath(), recursive);
+        if (value > maxValue) {
+          maxValue = value;
+        }
+      } else {
+        if (eachFile.getPath().getName().matches(fileNamePatternV08) ||
+            eachFile.getPath().getName().matches(fileNamePatternV09)) {
+          fileNamePatternMatchedList.add(eachFile.getPath());
+        }
+      }
+    }
+
+    if (fileNamePatternMatchedList.isEmpty()) {
+      return maxValue;
+    }
+    Path lastFile = fileNamePatternMatchedList.get(fileNamePatternMatchedList.size() - 1);
+    String pathName = lastFile.getName();
+
+    // 0.8: pathName = part-<ExecutionBlockId.seq>-<QueryUnitId.seq>
+    // 0.9: pathName = part-<ExecutionBlockId.seq>-<QueryUnitId.seq>-<Sequence>
+    String[] pathTokens = pathName.split("-");
+    if (pathTokens.length == 3) {
+      return -1;
+    } else if(pathTokens.length == 4) {
+      return Integer.parseInt(pathTokens[3]);
+    } else {
+      return -1;
+    }
+  }
+
+  public static void closeBuffer(ByteBuffer buffer) {
+    if (buffer != null) {
+      if (buffer.isDirect()) {
+        ((DirectBuffer) buffer).cleaner().clean();
+      } else {
+        buffer.clear();
+      }
+    }
+  }
+
+  public static int readFully(InputStream is, byte[] buffer, int offset, int length)
+      throws IOException {
+    int nread = 0;
+    while (nread < length) {
+      int nbytes = is.read(buffer, offset + nread, length - nread);
+      if (nbytes < 0) {
+        return nread > 0 ? nread : nbytes;
+      }
+      nread += nbytes;
+    }
+    return nread;
+  }
+
+  /**
+   * Similar to readFully(). Skips bytes in a loop.
+   * @param in The DataInput to skip bytes from
+   * @param len number of bytes to skip.
+   * @throws java.io.IOException if it could not skip requested number of bytes
+   * for any reason (including EOF)
+   */
+  public static void skipFully(DataInput in, int len) throws IOException {
+    int amt = len;
+    while (amt > 0) {
+      long ret = in.skipBytes(amt);
+      if (ret == 0) {
+        // skip may return 0 even if we're not at EOF.  Luckily, we can
+        // use the read() method to figure out if we're at the end.
+        int b = in.readByte();
+        if (b == -1) {
+          throw new EOFException( "Premature EOF from inputStream after " +
+              "skipping " + (len - amt) + " byte(s).");
+        }
+        ret = 1;
+      }
+      amt -= ret;
+    }
+  }
+}