You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by jh...@apache.org on 2014/01/28 13:35:38 UTC

[01/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Updated Branches:
  refs/heads/master 6aa96fa9b -> bbf9b7bf8


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..34c362c
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
@@ -0,0 +1,948 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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.getColumnNum());
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + storeType + ".idx"), keySchema, comp);
+    reader.open();
+    scanner = StorageManagerFactory.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) StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema,
+        tablePath);
+    appender.init();
+
+    SortSpec[] sortKeys = new SortSpec[2];
+    sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    TupleComparator comp = new TupleComparator(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.getColumnNum());
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    SeekableScanner scanner = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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.getColumnNum());
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    scanner = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), false, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), false, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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.getColumnNum());
+
+    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
+        keySchema, comp);
+    reader.open();
+    scanner = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), false, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), false, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..6d5d90f
--- /dev/null
+++ b/tajo-storage/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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("long", Type.INT8));
+    keySchema.addColumn(new Column("double", Type.FLOAT8));
+
+    TupleComparator comp = new TupleComparator(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.getColumnNum());
+    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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
+    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+
+    Schema keySchema = new Schema();
+    keySchema.addColumn(new Column("int", Type.INT4));
+    keySchema.addColumn(new Column("long", Type.INT8));
+
+    TupleComparator comp = new TupleComparator(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/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java b/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java
new file mode 100644
index 0000000..7e95b8b
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java
@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.*;
+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;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes;
+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.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.assertEquals;
+
+@RunWith(Parameterized.class)
+public class TestCSVCompression {
+  private TajoConf conf;
+  private static String TEST_PATH = "target/test-data/v2/TestCSVCompression";
+
+  private CatalogProtos.StoreType storeType;
+  private Path testDir;
+  private FileSystem fs;
+
+  public TestCSVCompression(CatalogProtos.StoreType type) throws IOException {
+    this.storeType = type;
+    conf = new TajoConf();
+    conf.setBoolVar(TajoConf.ConfVars.STORAGE_MANAGER_VERSION_2, true);
+
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return Arrays.asList(new Object[][]{
+        {CatalogProtos.StoreType.CSV}
+    });
+  }
+
+  @Test
+  public void testDeflateCodecCompressionData() throws IOException {
+    storageCompressionTest(storeType, DeflateCodec.class);
+  }
+
+  @Test
+  public void testGzipCodecCompressionData() throws IOException {
+    storageCompressionTest(storeType, GzipCodec.class);
+  }
+
+  @Test
+  public void testSnappyCodecCompressionData() throws IOException {
+    if (SnappyCodec.isNativeCodeLoaded()) {
+      storageCompressionTest(storeType, SnappyCodec.class);
+    }
+  }
+
+  @Test
+  public void testBzip2CodecCompressionData() throws IOException {
+    storageCompressionTest(storeType, BZip2Codec.class);
+  }
+
+  @Test
+  public void testLz4CodecCompressionData() throws IOException {
+    if(NativeCodeLoader.isNativeCodeLoaded() && Lz4Codec.isNativeCodeLoaded())
+      storageCompressionTest(storeType, Lz4Codec.class);
+  }
+
+  // TODO - See https://issues.apache.org/jira/browse/HADOOP-9622
+  //@Test
+  public void testSplitCompressionData() throws IOException {
+
+    Schema schema = new Schema();
+    schema.addColumn("id", TajoDataTypes.Type.INT4);
+    schema.addColumn("age", TajoDataTypes.Type.INT8);
+
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV);
+    meta.putOption("compression.codec", BZip2Codec.class.getCanonicalName());
+
+    Path tablePath = new Path(testDir, "SplitCompression");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    appender.enableStats();
+
+    appender.init();
+
+    String extention = "";
+    if (appender instanceof CSVFile.CSVAppender) {
+      extention = ((CSVFile.CSVAppender) appender).getExtension();
+    }
+
+    int tupleNum = 100000;
+    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());
+    tablePath = tablePath.suffix(extention);
+
+    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("SplitCompression", tablePath, 0, randomNum);
+    tablets[1] = new FileFragment("SplitCompression", tablePath, randomNum, (fileLen - randomNum));
+
+    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+    scanner.init();
+    int tupleCnt = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      tupleCnt++;
+    }
+    scanner.close();
+
+    scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
+    scanner.init();
+    while ((tuple = scanner.next()) != null) {
+      tupleCnt++;
+    }
+
+    scanner.close();
+    assertEquals(tupleNum, tupleCnt);
+  }
+
+  private void storageCompressionTest(CatalogProtos.StoreType storeType, Class<? extends CompressionCodec> codec)
+      throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("id", TajoDataTypes.Type.INT4);
+    schema.addColumn("age", TajoDataTypes.Type.INT8);
+
+    TableMeta meta = CatalogUtil.newTableMeta(storeType);
+    meta.putOption("compression.codec", codec.getCanonicalName());
+
+    String fileName = "Compression_" + codec.getSimpleName();
+    Path tablePath = new Path(testDir, fileName);
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    appender.enableStats();
+
+    appender.init();
+
+    String extension = "";
+    if (appender instanceof CSVFile.CSVAppender) {
+      extension = ((CSVFile.CSVAppender) appender).getExtension();
+    }
+
+    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());
+    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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+    scanner.init();
+    int tupleCnt = 0;
+    while (scanner.next() != null) {
+      tupleCnt++;
+    }
+    scanner.close();
+    assertEquals(tupleCnt, tupleNum);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java b/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java
new file mode 100644
index 0000000..c356548
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java
@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+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;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.CommonTestingUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+import static org.junit.Assert.assertEquals;
+
+public class TestCSVScanner {
+  private TajoConf conf;
+  private static String TEST_PATH = "target/test-data/v2/TestCSVScanner";
+  AbstractStorageManager sm = null;
+  private Path testDir;
+  private FileSystem fs;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new TajoConf();
+    conf.setBoolVar(ConfVars.STORAGE_MANAGER_VERSION_2, true);
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+    sm = StorageManagerFactory.getStorageManager(conf, testDir);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public final void testGetScannerAndAppender() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("id", TajoDataTypes.Type.INT4);
+    schema.addColumn("age", TajoDataTypes.Type.INT4);
+    schema.addColumn("name", TajoDataTypes.Type.TEXT);
+
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.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 = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, path);
+    appender.init();
+    for(Tuple t : tuples) {
+      appender.addTuple(t);
+    }
+    appender.close();
+
+    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getFileScanner(meta, schema, path);
+    scanner.init();
+    int i=0;
+    Tuple tuple = null;
+    while( (tuple = scanner.next()) != null) {
+      i++;
+    }
+    assertEquals(4,i);
+  }
+
+  @Test
+  public final void testPartitionFile() throws IOException {
+    Schema schema = new Schema();
+    schema.addColumn("key", TajoDataTypes.Type.TEXT);
+    schema.addColumn("age", TajoDataTypes.Type.INT4);
+    schema.addColumn("name", TajoDataTypes.Type.TEXT);
+
+    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV);
+
+
+    Path path = StorageUtil.concatPath(testDir, "testPartitionFile", "table.csv");
+    fs.mkdirs(path.getParent());
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, path);
+    appender.init();
+
+    String keyValue = "";
+    for(int i = 0; i < 100; i++) {
+      keyValue += "0123456789";
+    }
+    keyValue = "key_" + keyValue + "_";
+
+    String nameValue = "";
+    for(int i = 0; i < 100; i++) {
+      nameValue += "0123456789";
+    }
+    nameValue = "name_" + nameValue + "_";
+
+    int numTuples = 100000;
+    for(int i = 0; i < numTuples; i++) {
+      Tuple tuple = new VTuple(3);
+      tuple.put(new Datum[] {
+          DatumFactory.createText(keyValue + i),
+          DatumFactory.createInt4(i + 32),
+          DatumFactory.createText(nameValue + i)});
+      appender.addTuple(tuple);
+    }
+    appender.close();
+
+    long fileLength = fs.getLength(path);
+    long totalTupleCount = 0;
+
+    int scanCount = 0;
+    Tuple startTuple = null;
+    Tuple lastTuple = null;
+    while(true) {
+      long startOffset = (64 * 1024 * 1024) * scanCount;
+      long length = Math.min(64 * 1024 * 1024, fileLength - startOffset);
+
+      FileFragment fragment = new FileFragment("Test", path, startOffset, length, null);
+      Scanner scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, fragment, schema);
+      scanner.init();
+      Tuple tuple = null;
+      while( (tuple = scanner.next()) != null) {
+        if(startTuple == null) {
+          startTuple = tuple;
+        }
+        lastTuple = tuple;
+        totalTupleCount++;
+      }
+      scanCount++;
+      if(length < 64 * 1024 * 1024) {
+        break;
+      }
+    }
+    assertEquals(numTuples, totalTupleCount);
+    assertEquals(keyValue + 0, startTuple.get(0).toString());
+    assertEquals(keyValue + (numTuples - 1), lastTuple.get(0).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java b/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java
new file mode 100644
index 0000000..a3d3133
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java
@@ -0,0 +1,242 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+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.Options;
+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.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.rcfile.RCFile;
+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.apache.tajo.conf.TajoConf.ConfVars;
+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/v2/TestStorages";
+
+  private StoreType storeType;
+  private boolean splitable;
+  private boolean statsable;
+  private Path testDir;
+  private FileSystem fs;
+
+  public TestStorages(StoreType type, boolean splitable, boolean statsable) throws IOException {
+    this.storeType = type;
+    this.splitable = splitable;
+    this.statsable = statsable;
+
+    conf = new TajoConf();
+    conf.setBoolVar(ConfVars.STORAGE_MANAGER_VERSION_2, true);
+
+    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[][]{
+        {StoreType.CSV, true, true},
+        {StoreType.RCFILE, true, true},
+        {StoreType.TREVNI, false, true},
+        {StoreType.RAW, false, 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 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+      scanner.init();
+      int tupleCnt = 0;
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
+      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);
+
+    Path tablePath = new Path(testDir, "testProjection.data");
+    Appender appender = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, fragment, target);
+    scanner.init();
+    int tupleCnt = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      if (storeType == StoreType.RCFILE || storeType == StoreType.TREVNI || storeType == StoreType.CSV) {
+        assertTrue(tuple.get(0) == null || tuple.get(0) instanceof NullDatum);
+      }
+      assertEquals(DatumFactory.createInt8(tupleCnt + 2), tuple.getLong(1));
+      assertEquals(DatumFactory.createFloat4(tupleCnt + 3), tuple.getFloat(2));
+      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);
+
+    Options options = new Options();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    appender.init();
+
+    Tuple tuple = new VTuple(12);
+    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()
+    });
+    appender.addTuple(tuple);
+    appender.flush();
+    appender.close();
+
+    FileStatus status = fs.getFileStatus(tablePath);
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  StorageManagerFactory.getStorageManager(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));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..5bf4453
--- /dev/null
+++ b/tajo-storage/src/test/resources/storage-default.xml
@@ -0,0 +1,149 @@
+<?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.v2</name>
+    <value>false</value>
+  </property>
+
+  <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>csv,raw,rcfile,row,trevni</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>
+
+  <!--- 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.v2.csv.class</name>
+    <value>org.apache.tajo.storage.v2.CSVFileScanner</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.v2.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.v2.rcfile.class</name>
+    <value>org.apache.tajo.storage.v2.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.v2.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.v2.trevni.class</name>
+    <value>org.apache.tajo.storage.trevni.TrevniScanner</value>
+  </property>
+
+  <!--- Appender Handler -->
+  <property>
+    <name>tajo.storage.appender-handler</name>
+    <value>csv,raw,rcfile,row,trevni</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>
+</configuration>


[06/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..5e200a0
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..c83b505
--- /dev/null
+++ b/tajo-storage/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 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..352776f
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..707d55a
--- /dev/null
+++ b/tajo-storage/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 BytesRefWritable
+ */
+public interface LazyDecompressionCallback {
+
+  byte[] decompress() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..bb6af22
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..53a3dca
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..46745ab
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..3944f38
--- /dev/null
+++ b/tajo-storage/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;
+    }
+  }
+}


[13/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskDeviceInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskDeviceInfo.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskDeviceInfo.java
deleted file mode 100644
index 7802c91..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/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.v2;
-
-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java
deleted file mode 100644
index 1babf99..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java
+++ /dev/null
@@ -1,205 +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.v2;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-
-public final class DiskFileScanScheduler extends Thread {
-  private static final Log LOG = LogFactory.getLog(DiskFileScanScheduler.class);
-
-	private Queue<FileScannerV2> requestQueue = new LinkedList<FileScannerV2>();
-
-  List<FileScannerV2> fetchingScanners = new ArrayList<FileScannerV2>();
-
-  private int scanConcurrency;
-
-	private AtomicInteger numOfRunningScanners = new AtomicInteger(0);
-
-	private Object requestQueueMonitor = new Object(); // c++ code style
-
-	private StorageManagerV2.StorgaeManagerContext smContext;
-
-	private DiskDeviceInfo diskDeviceInfo;
-
-	private AtomicBoolean stopped = new AtomicBoolean(false);
-
-  private long totalScanCount = 0;
-
-  private FetchWaitingThread fetchWaitingThread;
-
-  private AtomicLong totalReadBytesForFetch = new AtomicLong(0);
-
-  private AtomicLong totalReadBytesFromDisk = new AtomicLong(0);
-
-  private long[] lastReportReadBytes;
-
-  private long lastReportTime = 0;
-
-	public DiskFileScanScheduler(
-			StorageManagerV2.StorgaeManagerContext smContext,
-			DiskDeviceInfo diskDeviceInfo) {
-		super("DiskFileScanner:" + diskDeviceInfo);
-		this.smContext = smContext;
-		this.diskDeviceInfo = diskDeviceInfo;
-		initScannerPool();
-		this.fetchWaitingThread = new FetchWaitingThread();
-		this.fetchWaitingThread.start();
-	}
-
-  public void incrementReadBytes(long[] readBytes) {
-    totalReadBytesForFetch.addAndGet(readBytes[0]);
-    totalReadBytesFromDisk.addAndGet(readBytes[1]);
-  }
-
-  public int getDiskId() {
-    return diskDeviceInfo.getId();
-  }
-
-  public void run() {
-    synchronized (requestQueueMonitor) {
-      while(!stopped.get()) {
-        if(isAllScannerRunning()) {
-          try {
-            requestQueueMonitor.wait(2000);
-            continue;
-          } catch (InterruptedException e) {
-            break;
-          }
-        } else {
-          FileScannerV2 fileScanner = requestQueue.poll();
-          if(fileScanner == null) {
-            try {
-              requestQueueMonitor.wait(2000);
-              continue;
-            } catch (InterruptedException e) {
-              break;
-            }
-          }
-          if(fileScanner.isStopScanScheduling()) {
-            LOG.info("Exit from Disk Queue:" + fileScanner.getId());
-            continue;
-          }
-          if(fileScanner.isFetchProcessing()) {
-            synchronized(fetchingScanners) {
-              fetchingScanners.add(fileScanner);
-              //fetchingScanners.notifyAll();
-            }
-          } else {
-            numOfRunningScanners.incrementAndGet();
-            FileScanRunner fileScanRunner = new FileScanRunner(
-                DiskFileScanScheduler.this, smContext,
-                fileScanner, requestQueueMonitor,
-                numOfRunningScanners);
-            totalScanCount++;
-            fileScanRunner.start();
-          }
-        }
-      }
-    }
-  }
-
-	protected void requestScanFile(FileScannerV2 fileScannerV2) {
-		synchronized (requestQueueMonitor) {
-			requestQueue.offer(fileScannerV2);
-			requestQueueMonitor.notifyAll();
-		}
-	}
-
-  public class FetchWaitingThread extends Thread {
-    List<FileScannerV2> workList = new ArrayList<FileScannerV2>(20);
-    public void run() {
-      while(!stopped.get()) {
-        try {
-          Thread.sleep(100);
-        } catch (InterruptedException e) {
-          break;
-        }
-        workList.clear();
-        synchronized(fetchingScanners) {
-          workList.addAll(fetchingScanners);
-          fetchingScanners.clear();
-        }
-        synchronized(requestQueueMonitor) {
-          for(FileScannerV2 eachScanner: workList) {
-            requestQueue.offer(eachScanner);
-          }
-          requestQueueMonitor.notifyAll();
-        }
-      }
-    }
-  }
-
-	private void initScannerPool() {
-		// TODO finally implements heuristic, currently set with property
-		scanConcurrency = smContext.getConf().getIntVar(ConfVars.STORAGE_MANAGER_CONCURRENCY_PER_DISK);
-	}
-
-  public int getTotalQueueSize() {
-      return requestQueue.size();
-  }
-
-  boolean isAllScannerRunning() {
-    return numOfRunningScanners.get() >= scanConcurrency;
-  }
-
-  public long getTotalScanCount() {
-    return totalScanCount;
-  }
-
-	public void stopScan() {
-		stopped.set(true);
-		if (fetchWaitingThread != null) {
-      fetchWaitingThread.interrupt();
-		}
-
-		this.interrupt();
-	}
-
-  public void printDiskSchedulerInfo() {
-    long currentReadBytes[] = new long[]{totalReadBytesForFetch.get(), totalReadBytesFromDisk.get()};
-    int[] throughput = new int[2];
-    if(lastReportTime != 0 && lastReportReadBytes != null) {
-      int sec = (int)((System.currentTimeMillis() - lastReportTime)/1000);
-      throughput[0] = (int)((currentReadBytes[0] - lastReportReadBytes[0])/sec);
-      throughput[1] = (int)((currentReadBytes[1] - lastReportReadBytes[1])/sec);
-    }
-    lastReportTime = System.currentTimeMillis();
-
-    LOG.info("===>" + DiskFileScanScheduler.this.diskDeviceInfo
-        + ", request=" + requestQueue.size()
-        + ", fetching=" + fetchingScanners.size()
-        + ", running=" + numOfRunningScanners.get()
-        + ", totalScan=" + totalScanCount
-        + ", FetchThroughput=" + throughput[0]/1024 + "KB"
-        + ", DiskScanThroughput=" + throughput[1]/1024 + "KB");
-
-    lastReportReadBytes = currentReadBytes;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskInfo.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskInfo.java
deleted file mode 100644
index d71154c..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/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.v2;
-
-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskMountInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskMountInfo.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskMountInfo.java
deleted file mode 100644
index 56100f2..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/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.v2;
-
-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java
deleted file mode 100644
index bb90c39..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java
+++ /dev/null
@@ -1,199 +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.v2;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
-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);
-	}
-	
-	private 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 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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java
deleted file mode 100644
index 07fbe6c..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java
+++ /dev/null
@@ -1,70 +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.v2;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class FileScanRunner extends Thread {
-  private static final Log LOG = LogFactory.getLog(FileScanRunner.class);
-
-  StorageManagerV2.StorgaeManagerContext smContext;
-	FileScannerV2 fileScanner;
-	Object requestQueueMonitor;
-	AtomicInteger numOfRunningScanners;
-	DiskFileScanScheduler diskFileScanScheduler;
-	
-	int maxReadBytes;
-	
-	public FileScanRunner(DiskFileScanScheduler diskFileScanScheduler, 
-			StorageManagerV2.StorgaeManagerContext smContext,
-      FileScannerV2 fileScanner, Object requestQueueMonitor,
-			AtomicInteger numOfRunningScanners) {
-		super("FileScanRunner:" + fileScanner.getId());
-		this.diskFileScanScheduler = diskFileScanScheduler;
-		this.fileScanner = fileScanner;
-		this.smContext = smContext;
-		this.requestQueueMonitor = requestQueueMonitor;
-		this.numOfRunningScanners = numOfRunningScanners;
-		
-		this.maxReadBytes = smContext.getMaxReadBytesPerScheduleSlot();
-	}
-
-	public void run() {
-    try {
-//      long startTime = System.currentTimeMillis();
-//      boolean fetching = fileScanner.isFetchProcessing();
-      fileScanner.scan(maxReadBytes);
-//      if(diskFileScanScheduler.getDiskId() == 1) {
-//        LOG.info("========>" + diskFileScanScheduler.getDiskId() + "," + fileScanner.getId() +
-//            ",fetching=" + fetching +
-//            ", scanTime:" + (System.currentTimeMillis() - startTime) + " ms");
-//      }
-    } catch (Exception e) {
-      LOG.error(e.getMessage(), e);
-    } finally {
-      synchronized(requestQueueMonitor) {
-        numOfRunningScanners.decrementAndGet();
-        requestQueueMonitor.notifyAll();
-      }
-    }
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.java
deleted file mode 100644
index 0d5b33d..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.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.v2;
-
-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.storage.fragment.FileFragment;
-import org.apache.tajo.storage.Scanner;
-import org.apache.tajo.storage.Tuple;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public abstract class FileScannerV2 implements Scanner {
-  private static final Log LOG = LogFactory.getLog(FileScannerV2.class);
-
-	protected AtomicBoolean closed = new AtomicBoolean(false);
-
-	protected FileSystem fs;
-
-  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 long totalScanTime = 0;
-  protected int allocatedDiskId;
-
-  protected StorageManagerV2.StorgaeManagerContext smContext;
-
-  protected AtomicBoolean firstSchdeuled = new AtomicBoolean(true);
-
-  protected abstract boolean scanNext(int length) throws IOException;
-
-  protected abstract boolean initFirstScan(int maxBytesPerSchedule) throws IOException;
-
-  protected abstract long getFilePosition() throws IOException;
-
-  protected abstract Tuple nextTuple() throws IOException;
-
-  public abstract boolean isFetchProcessing();
-
-  public abstract boolean isStopScanScheduling();
-
-  public abstract void scannerReset();
-
-  protected abstract long[] reportReadBytes();
-
-	public FileScannerV2(final Configuration conf,
-                       final TableMeta meta,
-                       final Schema schema,
-                       final FileFragment fragment) throws IOException {
-    this.conf = conf;
-    this.meta = meta;
-    this.schema = schema;
-    this.fragment = fragment;
-    this.columnNum = this.schema.getColumnNum();
-
-    this.fs = fragment.getPath().getFileSystem(conf);
-	}
-
-  public void init() throws IOException {
-    closed.set(false);
-    firstSchdeuled.set(true);
-
-    if(!inited) {
-      smContext.requestFileScan(this);
-    }
-    inited = true;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    scannerReset();
-    close();
-    inited = false;
-    init();
-  }
-
-  public void setAllocatedDiskId(int allocatedDiskId) {
-    this.allocatedDiskId = allocatedDiskId;
-  }
-
-  public String getId() {
-    return fragment.getPath().getName() + ":" + fragment.getStartKey() + ":" +
-        fragment.getEndKey() + "_" + System.currentTimeMillis();
-  }
-
-  @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 Path getPath() {
-    return fragment.getPath();
-  }
-
-  public int getDiskId() {
-    if(fragment.getDiskIds().length <= 0) {
-      //LOG.warn("===> No DiskId:" + fragment.getPath() + ":" + fragment.getStartKey());
-      return -1;
-    } else {
-      return fragment.getDiskIds()[0];
-    }
-  }
-
-  public void setSearchCondition(Object expr) {
-    if (inited) {
-      throw new IllegalStateException("Should be called before init()");
-    }
-  }
-
-  public void setStorageManagerContext(StorageManagerV2.StorgaeManagerContext context) {
-    this.smContext = context;
-  }
-
-  public String toString() {
-    return fragment.getPath() + ":" + fragment.getStartKey();
-  }
-
-  public void scan(int maxBytesPerSchedule) throws IOException {
-    long startTime = System.currentTimeMillis();
-    try {
-    synchronized(firstSchdeuled) {
-      if(firstSchdeuled.get()) {
-        boolean moreData = initFirstScan(maxBytesPerSchedule);
-        firstSchdeuled.set(false);
-        firstSchdeuled.notifyAll();
-        if(moreData) {
-          smContext.requestFileScan(this);
-        }
-        return;
-      }
-    }
-    boolean moreData = scanNext(maxBytesPerSchedule);
-
-    if(moreData) {
-      smContext.requestFileScan(this);
-    }
-    } finally {
-      totalScanTime += System.currentTimeMillis() - startTime;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if(closed.get()) {
-      return;
-    }
-    long[] readBytes = reportReadBytes();
-    smContext.incrementReadBytes(allocatedDiskId, readBytes);
-    closed.set(true);
-    LOG.info(toString() + " closed, totalScanTime=" + totalScanTime);
-  }
-
-  public boolean isClosed() {
-    return closed.get();
-  }
-
-  public Tuple next() throws IOException {
-    synchronized(firstSchdeuled) {
-      if(firstSchdeuled.get()) {
-        try {
-          firstSchdeuled.wait();
-        } catch (InterruptedException e) {
-        }
-      }
-    }
-    return nextTuple();
-  }
-}


[17/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileAppender.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileAppender.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileAppender.java
deleted file mode 100644
index 064841f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileAppender.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;
-
-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 java.io.IOException;
-
-public abstract class FileAppender implements Appender {
-  protected boolean inited = false;
-
-  protected final Configuration conf;
-  protected final TableMeta meta;
-  protected final Schema schema;
-  protected final Path path;
-
-  protected boolean enabledStats;
-  
-  public FileAppender(Configuration conf, Schema schema, TableMeta meta, Path path) {
-    this.conf = conf;
-    this.meta = meta;
-    this.schema = schema;
-    this.path = path;
-  }
-
-  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 abstract long getOffset() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
deleted file mode 100644
index c831822..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FileScanner.java
+++ /dev/null
@@ -1,93 +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.conf.TajoConf;
-import org.apache.tajo.storage.fragment.FileFragment;
-
-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;
-  
-  public FileScanner(Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment) {
-    this.conf = conf;
-    this.meta = meta;
-    this.schema = schema;
-    this.fragment = fragment;
-    this.columnNum = this.schema.getColumnNum();
-  }
-
-  public void init() throws IOException {
-    inited = true;
-  }
-
-  @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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
deleted file mode 100644
index f05a316..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
+++ /dev/null
@@ -1,231 +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.*;
-import org.apache.tajo.exception.UnimplementedException;
-import org.apache.tajo.exception.UnsupportedException;
-
-import java.net.InetAddress;
-
-/**
- * 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) instanceof NullDatum;
-  }
-
-  @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 BooleanDatum getBoolean(int fieldId) {
-    return (BooleanDatum) get(fieldId);
-  }
-
-  @Override
-  public BitDatum getByte(int fieldId) {
-    return (BitDatum) get(fieldId);
-  }
-
-  @Override
-  public CharDatum getChar(int fieldId) {
-    return (CharDatum) get(fieldId);
-  }
-
-  @Override
-  public BlobDatum getBytes(int fieldId) {
-    return (BlobDatum) get(fieldId);
-  }
-
-  @Override
-  public Int2Datum getShort(int fieldId) {
-    return (Int2Datum) get(fieldId);
-  }
-
-  @Override
-  public Int4Datum getInt(int fieldId) {
-    return (Int4Datum) get(fieldId);
-  }
-
-  @Override
-  public Int8Datum getLong(int fieldId) {
-    return (Int8Datum) get(fieldId);
-  }
-
-  @Override
-  public Float4Datum getFloat(int fieldId) {
-    return (Float4Datum) get(fieldId);
-  }
-
-  @Override
-  public Float8Datum getDouble(int fieldId) {
-    return (Float8Datum) get(fieldId);
-  }
-
-  @Override
-  public Inet4Datum getIPv4(int fieldId) {
-    return (Inet4Datum) get(fieldId);
-  }
-
-  @Override
-  public byte[] getIPv4Bytes(int fieldId) { 
-    return get(fieldId).asByteArray();
-  }
-
-  @Override
-  public InetAddress getIPv6(int fieldId) {
-    throw new UnimplementedException();
-  }
-  
-  @Override
-  public byte[] getIPv6Bytes(int fieldId) {
-    throw new UnimplementedException();
-  }
-
-  @Override
-  public TextDatum getString(int fieldId) {
-    return (TextDatum) get(fieldId);
-  }
-
-  @Override
-  public TextDatum getText(int fieldId) {
-    return (TextDatum) get(fieldId);
-  }
-
-  @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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
deleted file mode 100644
index 4d484df..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
+++ /dev/null
@@ -1,291 +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.*;
-import org.apache.tajo.datum.exception.InvalidCastException;
-
-import java.net.InetAddress;
-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.getColumnNum()];
-    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) instanceof NullDatum;
-  }
-
-  @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 BooleanDatum getBoolean(int fieldId) {
-    return (BooleanDatum) get(fieldId);
-  }
-
-  @Override
-  public BitDatum getByte(int fieldId) {
-    return (BitDatum) get(fieldId);
-  }
-
-  @Override
-  public CharDatum getChar(int fieldId) {
-    return (CharDatum) get(fieldId);
-  }
-
-  @Override
-  public BlobDatum getBytes(int fieldId) {
-    return (BlobDatum) get(fieldId);
-  }
-
-  @Override
-  public Int2Datum getShort(int fieldId) {
-    return (Int2Datum) get(fieldId);
-  }
-
-  @Override
-  public Int4Datum getInt(int fieldId) {
-    return (Int4Datum) get(fieldId);
-  }
-
-  @Override
-  public Int8Datum getLong(int fieldId) {
-    return (Int8Datum) get(fieldId);
-  }
-
-  @Override
-  public Float4Datum getFloat(int fieldId) {
-    return (Float4Datum) get(fieldId);
-  }
-
-  @Override
-  public Float8Datum getDouble(int fieldId) {
-    return (Float8Datum) get(fieldId);
-  }
-
-  @Override
-  public Inet4Datum getIPv4(int fieldId) {
-    return (Inet4Datum) get(fieldId);
-  }
-
-  @Override
-  public byte[] getIPv4Bytes(int fieldId) {
-    return get(fieldId).asByteArray();
-  }
-
-  @Override
-  public InetAddress getIPv6(int fieldId) {
-    throw new InvalidCastException("IPv6 is unsupported yet");
-  }
-
-  @Override
-  public byte[] getIPv6Bytes(int fieldId) {
-    throw new InvalidCastException("IPv6 is unsupported yet");
-  }
-
-  @Override
-  public TextDatum getString(int fieldId) {
-    return (TextDatum) get(fieldId);
-  }
-
-  @Override
-  public TextDatum getText(int fieldId) {
-    return (TextDatum) get(fieldId);
-  }
-
-  public byte[] getTextBytes(int fieldId) {
-    if(textBytes[fieldId] != null)
-      return textBytes[fieldId];
-    else {
-      return get(fieldId).asTextBytes();
-    }
-  }
-
-  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() {
-    int hashCode = 37;
-    for (int i = 0; i < values.length; i++) {
-      Datum d = get(i);
-      if (d != null) {
-        hashCode ^= (d.hashCode() * 41);
-      } else {
-        hashCode = hashCode ^ (i + 17);
-      }
-    }
-
-    return hashCode;
-  }
-
-  @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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/LineReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/LineReader.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/LineReader.java
deleted file mode 100644
index 66c610a..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
deleted file mode 100644
index e4439f3..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.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;
-
-import com.google.common.collect.Lists;
-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.conf.TajoConf;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-
-public class MergeScanner implements Scanner {
-  private Configuration conf;
-  private TableMeta meta;
-  private Schema schema;
-  private List<FileFragment> fragments;
-  private Iterator<FileFragment> iterator;
-  private FileFragment currentFragment;
-  private Scanner currentScanner;
-  private Tuple tuple;
-  private boolean projectable = false;
-  private boolean selectable = false;
-  private Schema target;
-
-  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, Collection<FileFragment> rawFragmentList)
-      throws IOException {
-    this(conf, schema, meta, rawFragmentList, schema);
-  }
-
-  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, Collection<FileFragment> rawFragmentList,
-                      Schema target)
-      throws IOException {
-    this.conf = conf;
-    this.schema = schema;
-    this.meta = meta;
-    this.fragments = Lists.newArrayList();
-    for (Fragment f : rawFragmentList) {
-      fragments.add((FileFragment) f);
-    }
-    Collections.sort(fragments);
-
-    this.target = target;
-    this.reset();
-    if (currentScanner != null) {
-      this.projectable = currentScanner.isProjectable();
-      this.selectable = currentScanner.isSelectable();
-    }
-  }
-
-  @Override
-  public void init() throws IOException {
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    if (currentScanner != null)
-      tuple = currentScanner.next();
-
-    if (tuple != null) {
-      return tuple;
-    } else {
-      if (currentScanner != null) {
-        currentScanner.close();
-      }
-      currentScanner = getNextScanner();
-      if (currentScanner != null) {
-        tuple = currentScanner.next();
-      }
-    }
-    return tuple;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    this.iterator = fragments.iterator();
-    this.currentScanner = getNextScanner();
-  }
-
-  private Scanner getNextScanner() throws IOException {
-    if (iterator.hasNext()) {
-      currentFragment = iterator.next();
-      currentScanner = StorageManagerFactory.getStorageManager((TajoConf)conf).getScanner(meta, schema,
-          currentFragment, target);
-      currentScanner.init();
-      return currentScanner;
-    } else {
-      return null;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if(currentScanner != null) {
-      currentScanner.close();
-    }
-    iterator = null;
-    if(fragments != null) {
-      fragments.clear();
-    }
-  }
-
-  @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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/NumericPathComparator.java
deleted file mode 100644
index 94d13ee..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java
deleted file mode 100644
index db511dc..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RawFile.java
+++ /dev/null
@@ -1,532 +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.logging.Log;
-import org.apache.commons.logging.LogFactory;
-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.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.datum.TimestampDatum;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.util.BitArray;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.Arrays;
-
-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 Path path;
-
-    private ByteBuffer buffer;
-    private Tuple tuple;
-
-    private int headerSize = 0;
-    private BitArray nullFlags;
-    private static final int RECORD_SIZE = 4;
-    private boolean eof = false;
-    private long fileSize;
-    private FileInputStream fis;
-
-    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
-      super(conf, schema, meta, null);
-      this.path = path;
-      init();
-    }
-
-    @SuppressWarnings("unused")
-    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, FileFragment fragment) throws IOException {
-      this(conf, schema, meta, fragment.getPath());
-    }
-
-    public void init() throws IOException {
-      //Preconditions.checkArgument(FileUtil.isLocalPath(path));
-      // TODO - to make it unified one.
-      URI uri = path.toUri();
-      fis = new FileInputStream(new File(uri));
-      channel = fis.getChannel();
-      fileSize = channel.size();
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RawFileScanner open:" + path + "," + channel.position() + ", size :" + channel.size());
-      }
-
-      buffer = ByteBuffer.allocateDirect(128 * 1024);
-
-      columnTypes = new DataType[schema.getColumnNum()];
-      for (int i = 0; i < schema.getColumnNum(); i++) {
-        columnTypes[i] = schema.getColumn(i).getDataType();
-      }
-
-      tuple = new VTuple(columnTypes.length);
-
-      // initial read
-      channel.read(buffer);
-      buffer.flip();
-
-      nullFlags = new BitArray(schema.getColumnNum());
-      headerSize = RECORD_SIZE + 2 + nullFlags.bytesLength();
-
-      super.init();
-    }
-
-    @Override
-    public long getNextOffset() throws IOException {
-      return channel.position() - buffer.remaining();
-    }
-
-    @Override
-    public void seek(long offset) throws IOException {
-      long currentPos = channel.position();
-      if(currentPos < offset &&  offset < currentPos + buffer.limit()){
-        buffer.position((int)(offset - currentPos));
-      } else {
-        buffer.clear();
-        channel.position(offset);
-        channel.read(buffer);
-        buffer.flip();
-        eof = false;
-      }
-    }
-
-    private boolean fillBuffer() throws IOException {
-      buffer.compact();
-      if (channel.read(buffer) == -1) {
-        eof = true;
-        return false;
-      } else {
-        buffer.flip();
-        return true;
-      }
-    }
-
-    @Override
-    public Tuple next() throws IOException {
-      if(eof) return null;
-
-      if (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);
-      //buffer.position(recordOffset + headerSize);
-      if (buffer.remaining() < (recordSize - headerSize)) {
-        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 = buffer.getInt();
-            byte[] buf = new byte[columnTypes[i].getLength()];
-            buffer.get(buf);
-            byte[] charBuf = Arrays.copyOf(buf, realLen);
-            tuple.put(i, DatumFactory.createChar(charBuf));
-            break;
-
-          case INT2 :
-            tuple.put(i, DatumFactory.createInt2(buffer.getShort()));
-            break;
-
-          case INT4 :
-            tuple.put(i, DatumFactory.createInt4(buffer.getInt()));
-            break;
-
-          case INT8 :
-            tuple.put(i, DatumFactory.createInt8(buffer.getLong()));
-            break;
-
-          case FLOAT4 :
-            tuple.put(i, DatumFactory.createFloat4(buffer.getFloat()));
-            break;
-
-          case FLOAT8 :
-            tuple.put(i, DatumFactory.createFloat8(buffer.getDouble()));
-            break;
-
-          case TEXT :
-            // TODO - shoud use CharsetEncoder / CharsetDecoder
-            //byte [] rawBytes = getColumnBytes();
-            int strSize2 = buffer.getInt();
-            byte [] strBytes2 = new byte[strSize2];
-            buffer.get(strBytes2);
-            tuple.put(i, DatumFactory.createText(new String(strBytes2)));
-            break;
-
-          case TIMESTAMP:
-            tuple.put(i, DatumFactory.createTimeStampFromMillis(buffer.getLong()));
-            break;
-
-          case BLOB : {
-            //byte [] rawBytes = getColumnBytes();
-            int byteSize = buffer.getInt();
-            byte [] rawBytes = new byte[byteSize];
-            buffer.get(rawBytes);
-            tuple.put(i, DatumFactory.createBlob(rawBytes));
-            break;
-          }
-
-          case PROTOBUF: {
-            //byte [] rawBytes = getColumnBytes();
-            int byteSize = buffer.getInt();
-            byte [] rawBytes = new byte[byteSize];
-            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 NULL_TYPE:
-            tuple.put(i, NullDatum.get());
-            break;
-
-          default:
-        }
-      }
-
-      if(!buffer.hasRemaining() && channel.position() == fileSize){
-        eof = true;
-      }
-      return tuple;
-    }
-
-    @Override
-    public void reset() throws IOException {
-      // clear the buffer
-      buffer.clear();
-      // reload initial buffer
-      channel.position(0);
-      channel.read(buffer);
-      buffer.flip();
-      eof = false;
-    }
-
-    @Override
-    public void close() throws IOException {
-      buffer.clear();
-      channel.close();
-      fis.close();
-    }
-
-    @Override
-    public boolean isProjectable() {
-      return false;
-    }
-
-    @Override
-    public boolean isSelectable() {
-      return false;
-    }
-
-    @Override
-    public boolean isSplittable(){
-      return false;
-    }
-  }
-
-  public static class RawFileAppender extends FileAppender {
-    private FileChannel channel;
-    private RandomAccessFile randomAccessFile;
-    private DataType[] columnTypes;
-
-    private ByteBuffer buffer;
-    private BitArray nullFlags;
-    private int headerSize = 0;
-    private static final int RECORD_SIZE = 4;
-    private long pos;
-
-    private TableStatistics stats;
-
-    public RawFileAppender(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
-      super(conf, schema, meta, path);
-    }
-
-    public void init() throws IOException {
-      // TODO - RawFile only works on Local File System.
-      //Preconditions.checkArgument(FileUtil.isLocalPath(path));
-      File file = new File(path.toUri());
-      randomAccessFile = new RandomAccessFile(file, "rw");
-      channel = randomAccessFile.getChannel();
-      pos = 0;
-
-      columnTypes = new DataType[schema.getColumnNum()];
-      for (int i = 0; i < schema.getColumnNum(); i++) {
-        columnTypes[i] = schema.getColumn(i).getDataType();
-      }
-
-      buffer = ByteBuffer.allocateDirect(64 * 1024);
-
-      // comput the number of bytes, representing the null flags
-
-      nullFlags = new BitArray(schema.getColumnNum());
-      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.limit(buffer.position());
-      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;
-      }
-    }
-
-    @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.getColumnNum(); 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.get(i).asByte());
-            break;
-
-          case CHAR :
-            byte[] src = t.getChar(i).asByteArray();
-            byte[] dst = Arrays.copyOf(src, columnTypes[i].getLength());
-            buffer.putInt(src.length);
-            buffer.put(dst);
-            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 TEXT:
-            byte [] strBytes2 = t.get(i).asByteArray();
-            if (flushBufferAndReplace(recordOffset, strBytes2.length + 4)) {
-              recordOffset = 0;
-            }
-            buffer.putInt(strBytes2.length);
-            buffer.put(strBytes2);
-            break;
-
-          case TIMESTAMP:
-            buffer.putLong(((TimestampDatum)t.get(i)).getMillis());
-            break;
-
-          case BLOB : {
-            byte [] rawBytes = t.get(i).asByteArray();
-            if (flushBufferAndReplace(recordOffset, rawBytes.length + 4)) {
-              recordOffset = 0;
-            }
-            buffer.putInt(rawBytes.length);
-            buffer.put(rawBytes);
-            break;
-          }
-
-          case PROTOBUF: {
-            // TODO - to be fixed
-//            byte [] lengthByte = new byte[4];
-//            byte [] byteArray = t.get(i).asByteArray();
-//            CodedOutputStream outputStream = CodedOutputStream.newInstance(lengthByte);
-//            outputStream.writeUInt32NoTag(byteArray.length);
-//            outputStream.flush();
-//            int legnthByteLength = CodedOutputStream.computeInt32SizeNoTag(byteArray.length);
-//            if (flushBufferAndReplace(recordOffset, byteArray.length + legnthByteLength)) {
-//              recordOffset = 0;
-//            }
-//            buffer.put(lengthByte, 0, legnthByteLength);
-            byte [] rawBytes = t.get(i).asByteArray();
-            if (flushBufferAndReplace(recordOffset, rawBytes.length + 4)) {
-              recordOffset = 0;
-            }
-            buffer.putInt(rawBytes.length);
-            buffer.put(rawBytes);
-            break;
-          }
-
-          case INET4 :
-            buffer.put(t.get(i).asByteArray());
-            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 {
-      flushBuffer();
-    }
-
-    @Override
-    public void close() throws IOException {
-      flush();
-      if (enabledStats) {
-        stats.setNumBytes(getOffset());
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RawFileAppender written: " + getOffset() + " bytes, path: " + path);
-      }
-      channel.close();
-      randomAccessFile.close();
-    }
-
-    @Override
-    public TableStats getStats() {
-      if (enabledStats) {
-        return stats.getTableStat();
-      } else {
-        return null;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowFile.java
deleted file mode 100644
index 1e89f31..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowFile.java
+++ /dev/null
@@ -1,506 +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.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.FileFragment;
-import org.apache.tajo.util.BitArray;
-import org.apache.tajo.util.Bytes;
-
-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 FileFragment fragment)
-        throws IOException {
-      super(conf, schema, meta, fragment);
-
-      SYNC_INTERVAL =
-          conf.getInt(ConfVars.RAWFILE_SYNC_INTERVAL.varname,
-              SYNC_SIZE * 100);
-
-      nullFlags = new BitArray(schema.getColumnNum());
-      tupleHeaderSize = nullFlags.bytesLength() + (2 * Short.SIZE / 8);
-      this.start = fragment.getStartKey();
-      this.end = this.start + fragment.getEndKey();
-    }
-
-    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.getColumnNum());
-      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();
-      Bytes.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.getColumnNum());
-
-      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.getColumnNum(); 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 len = buffer.getShort();
-//              byte[] buf = new byte[len];
-//              buffer.get(buf, 0, len);
-//              datum = DatumFactory.createText(buf);
-//              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 Schema schema, final TableMeta meta, final Path path)
-        throws IOException {
-      super(conf, schema, meta, path);
-    }
-
-    public void init() throws IOException {
-      SYNC_INTERVAL = conf.getInt(ConfVars.RAWFILE_SYNC_INTERVAL.varname, 100);
-
-      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.getColumnNum());
-
-      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.getColumnNum(); 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.getBoolean(i).asByte());
-              break;
-            case BIT:
-              buffer.put(t.getByte(i).asByte());
-              break;
-            case CHAR:
-              byte[] src = t.getChar(i).asByteArray();
-              byte[] dst = Arrays.copyOf(src, col.getDataType().getLength());
-              buffer.putInt(src.length);
-              buffer.put(dst);
-              break;
-            case TEXT:
-              byte [] strbytes = t.getText(i).asByteArray();
-              buffer.putShort((short)strbytes.length);
-              buffer.put(strbytes, 0, strbytes.length);
-              break;
-            case INT2:
-              buffer.putShort(t.getShort(i).asInt2());
-              break;
-            case INT4:
-              buffer.putInt(t.getInt(i).asInt4());
-              break;
-            case INT8:
-              buffer.putLong(t.getLong(i).asInt8());
-              break;
-            case FLOAT4:
-              buffer.putFloat(t.getFloat(i).asFloat4());
-              break;
-            case FLOAT8:
-              buffer.putDouble(t.getDouble(i).asFloat8());
-              break;
-            case BLOB:
-              byte [] bytes = t.getBytes(i).asByteArray();
-              buffer.putShort((short)bytes.length);
-              buffer.put(bytes);
-              break;
-            case INET4:
-              buffer.put(t.getIPv4Bytes(i));
-              break;
-            case INET6:
-              buffer.put(t.getIPv6Bytes(i));
-              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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.java
deleted file mode 100644
index 9f32028..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.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 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.TimestampDatum;
-import org.apache.tajo.util.Bytes;
-
-import java.nio.ByteBuffer;
-
-public class RowStoreUtil {
-  public static int[] getTargetIds(Schema inSchema, Schema outSchema) {
-    int[] targetIds = new int[outSchema.getColumnNum()];
-    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 class RowStoreDecoder {
-
-    public static Tuple toTuple(Schema schema, byte [] bytes) {
-      ByteBuffer bb = ByteBuffer.wrap(bytes);
-      Tuple tuple = new VTuple(schema.getColumnNum());
-      Column col;
-      TajoDataTypes.DataType type;
-      for (int i =0; i < schema.getColumnNum(); i++) {
-        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();
-            if(b == 0) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            } else {
-              tuple.put(i, DatumFactory.createBit(b));
-            }
-            break;
-
-          case CHAR:
-            byte c = bb.get();
-            if(c == 0) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            } else {
-              tuple.put(i, DatumFactory.createChar(c));
-            }
-            break;
-
-          case INT2:
-            short s = bb.getShort();
-            if(s < Short.MIN_VALUE + 1) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            }else {
-              tuple.put(i, DatumFactory.createInt2(s));
-            }
-            break;
-
-          case INT4:
-          case DATE:
-            int i_ = bb.getInt();
-            if ( i_ < Integer.MIN_VALUE + 1) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            } else {
-              tuple.put(i, DatumFactory.createFromInt4(type, i_));
-            }
-            break;
-
-          case INT8:
-          case TIME:
-          case TIMESTAMP:
-            long l = bb.getLong();
-            if ( l < Long.MIN_VALUE + 1) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            }else {
-              tuple.put(i, DatumFactory.createFromInt8(type, l));
-            }
-            break;
-
-          case FLOAT4:
-            float f = bb.getFloat();
-            if (Float.isNaN(f)) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            }else {
-              tuple.put(i, DatumFactory.createFloat4(f));
-            }
-            break;
-
-          case FLOAT8:
-            double d = bb.getDouble();
-            if(Double.isNaN(d)) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            }else {
-              tuple.put(i, DatumFactory.createFloat8(d));
-            }
-            break;
-
-          case TEXT:
-            byte [] _string = new byte[bb.getInt()];
-            bb.get(_string);
-            String str = new String(_string);
-            if(str.compareTo("NULL") == 0) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            }else {
-            tuple.put(i, DatumFactory.createText(str));
-            }
-            break;
-
-          case BLOB:
-            byte [] _bytes = new byte[bb.getInt()];
-            bb.get(_bytes);
-            if(Bytes.compareTo(bytes, Bytes.toBytes("NULL")) == 0) {
-              tuple.put(i, DatumFactory.createNullDatum());
-            } else {
-              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
-        }
-      }
-      return tuple;
-    }
-  }
-
-  public static class RowStoreEncoder {
-
-    public static byte [] toBytes(Schema schema, Tuple tuple) {
-      int size = StorageUtil.getRowByteSize(schema);
-      ByteBuffer bb = ByteBuffer.allocate(size);
-      Column col;
-      for (int i = 0; i < schema.getColumnNum(); i++) {
-        col = schema.getColumn(i);
-        switch (col.getDataType().getType()) {
-          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 TIMESTAMP: bb.putLong(((TimestampDatum)tuple.get(i)).getMillis()); break;
-          case BLOB:
-            byte [] bytes = tuple.get(i).asByteArray();
-            bb.putInt(bytes.length);
-            bb.put(bytes);
-            break;
-          case INET4:
-            byte [] ipBytes = tuple.getIPv4Bytes(i);
-            bb.put(ipBytes);
-            break;
-          case INET6: bb.put(tuple.getIPv6Bytes(i)); break;
-          default:
-        }
-      }
-
-      bb.flip();
-      byte [] buf = new byte [bb.limit()];
-      bb.get(buf);
-      return buf;
-    }
-  }
-}


[04/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskDeviceInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskDeviceInfo.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskDeviceInfo.java
new file mode 100644
index 0000000..7802c91
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/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.v2;
+
+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/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java
new file mode 100644
index 0000000..1babf99
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskFileScanScheduler.java
@@ -0,0 +1,205 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+
+public final class DiskFileScanScheduler extends Thread {
+  private static final Log LOG = LogFactory.getLog(DiskFileScanScheduler.class);
+
+	private Queue<FileScannerV2> requestQueue = new LinkedList<FileScannerV2>();
+
+  List<FileScannerV2> fetchingScanners = new ArrayList<FileScannerV2>();
+
+  private int scanConcurrency;
+
+	private AtomicInteger numOfRunningScanners = new AtomicInteger(0);
+
+	private Object requestQueueMonitor = new Object(); // c++ code style
+
+	private StorageManagerV2.StorgaeManagerContext smContext;
+
+	private DiskDeviceInfo diskDeviceInfo;
+
+	private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private long totalScanCount = 0;
+
+  private FetchWaitingThread fetchWaitingThread;
+
+  private AtomicLong totalReadBytesForFetch = new AtomicLong(0);
+
+  private AtomicLong totalReadBytesFromDisk = new AtomicLong(0);
+
+  private long[] lastReportReadBytes;
+
+  private long lastReportTime = 0;
+
+	public DiskFileScanScheduler(
+			StorageManagerV2.StorgaeManagerContext smContext,
+			DiskDeviceInfo diskDeviceInfo) {
+		super("DiskFileScanner:" + diskDeviceInfo);
+		this.smContext = smContext;
+		this.diskDeviceInfo = diskDeviceInfo;
+		initScannerPool();
+		this.fetchWaitingThread = new FetchWaitingThread();
+		this.fetchWaitingThread.start();
+	}
+
+  public void incrementReadBytes(long[] readBytes) {
+    totalReadBytesForFetch.addAndGet(readBytes[0]);
+    totalReadBytesFromDisk.addAndGet(readBytes[1]);
+  }
+
+  public int getDiskId() {
+    return diskDeviceInfo.getId();
+  }
+
+  public void run() {
+    synchronized (requestQueueMonitor) {
+      while(!stopped.get()) {
+        if(isAllScannerRunning()) {
+          try {
+            requestQueueMonitor.wait(2000);
+            continue;
+          } catch (InterruptedException e) {
+            break;
+          }
+        } else {
+          FileScannerV2 fileScanner = requestQueue.poll();
+          if(fileScanner == null) {
+            try {
+              requestQueueMonitor.wait(2000);
+              continue;
+            } catch (InterruptedException e) {
+              break;
+            }
+          }
+          if(fileScanner.isStopScanScheduling()) {
+            LOG.info("Exit from Disk Queue:" + fileScanner.getId());
+            continue;
+          }
+          if(fileScanner.isFetchProcessing()) {
+            synchronized(fetchingScanners) {
+              fetchingScanners.add(fileScanner);
+              //fetchingScanners.notifyAll();
+            }
+          } else {
+            numOfRunningScanners.incrementAndGet();
+            FileScanRunner fileScanRunner = new FileScanRunner(
+                DiskFileScanScheduler.this, smContext,
+                fileScanner, requestQueueMonitor,
+                numOfRunningScanners);
+            totalScanCount++;
+            fileScanRunner.start();
+          }
+        }
+      }
+    }
+  }
+
+	protected void requestScanFile(FileScannerV2 fileScannerV2) {
+		synchronized (requestQueueMonitor) {
+			requestQueue.offer(fileScannerV2);
+			requestQueueMonitor.notifyAll();
+		}
+	}
+
+  public class FetchWaitingThread extends Thread {
+    List<FileScannerV2> workList = new ArrayList<FileScannerV2>(20);
+    public void run() {
+      while(!stopped.get()) {
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          break;
+        }
+        workList.clear();
+        synchronized(fetchingScanners) {
+          workList.addAll(fetchingScanners);
+          fetchingScanners.clear();
+        }
+        synchronized(requestQueueMonitor) {
+          for(FileScannerV2 eachScanner: workList) {
+            requestQueue.offer(eachScanner);
+          }
+          requestQueueMonitor.notifyAll();
+        }
+      }
+    }
+  }
+
+	private void initScannerPool() {
+		// TODO finally implements heuristic, currently set with property
+		scanConcurrency = smContext.getConf().getIntVar(ConfVars.STORAGE_MANAGER_CONCURRENCY_PER_DISK);
+	}
+
+  public int getTotalQueueSize() {
+      return requestQueue.size();
+  }
+
+  boolean isAllScannerRunning() {
+    return numOfRunningScanners.get() >= scanConcurrency;
+  }
+
+  public long getTotalScanCount() {
+    return totalScanCount;
+  }
+
+	public void stopScan() {
+		stopped.set(true);
+		if (fetchWaitingThread != null) {
+      fetchWaitingThread.interrupt();
+		}
+
+		this.interrupt();
+	}
+
+  public void printDiskSchedulerInfo() {
+    long currentReadBytes[] = new long[]{totalReadBytesForFetch.get(), totalReadBytesFromDisk.get()};
+    int[] throughput = new int[2];
+    if(lastReportTime != 0 && lastReportReadBytes != null) {
+      int sec = (int)((System.currentTimeMillis() - lastReportTime)/1000);
+      throughput[0] = (int)((currentReadBytes[0] - lastReportReadBytes[0])/sec);
+      throughput[1] = (int)((currentReadBytes[1] - lastReportReadBytes[1])/sec);
+    }
+    lastReportTime = System.currentTimeMillis();
+
+    LOG.info("===>" + DiskFileScanScheduler.this.diskDeviceInfo
+        + ", request=" + requestQueue.size()
+        + ", fetching=" + fetchingScanners.size()
+        + ", running=" + numOfRunningScanners.get()
+        + ", totalScan=" + totalScanCount
+        + ", FetchThroughput=" + throughput[0]/1024 + "KB"
+        + ", DiskScanThroughput=" + throughput[1]/1024 + "KB");
+
+    lastReportReadBytes = currentReadBytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskInfo.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskInfo.java
new file mode 100644
index 0000000..d71154c
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/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.v2;
+
+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/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskMountInfo.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskMountInfo.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskMountInfo.java
new file mode 100644
index 0000000..56100f2
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/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.v2;
+
+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/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java
new file mode 100644
index 0000000..bb90c39
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/DiskUtil.java
@@ -0,0 +1,199 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+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);
+	}
+	
+	private 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 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/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java
new file mode 100644
index 0000000..07fbe6c
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScanRunner.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class FileScanRunner extends Thread {
+  private static final Log LOG = LogFactory.getLog(FileScanRunner.class);
+
+  StorageManagerV2.StorgaeManagerContext smContext;
+	FileScannerV2 fileScanner;
+	Object requestQueueMonitor;
+	AtomicInteger numOfRunningScanners;
+	DiskFileScanScheduler diskFileScanScheduler;
+	
+	int maxReadBytes;
+	
+	public FileScanRunner(DiskFileScanScheduler diskFileScanScheduler, 
+			StorageManagerV2.StorgaeManagerContext smContext,
+      FileScannerV2 fileScanner, Object requestQueueMonitor,
+			AtomicInteger numOfRunningScanners) {
+		super("FileScanRunner:" + fileScanner.getId());
+		this.diskFileScanScheduler = diskFileScanScheduler;
+		this.fileScanner = fileScanner;
+		this.smContext = smContext;
+		this.requestQueueMonitor = requestQueueMonitor;
+		this.numOfRunningScanners = numOfRunningScanners;
+		
+		this.maxReadBytes = smContext.getMaxReadBytesPerScheduleSlot();
+	}
+
+	public void run() {
+    try {
+//      long startTime = System.currentTimeMillis();
+//      boolean fetching = fileScanner.isFetchProcessing();
+      fileScanner.scan(maxReadBytes);
+//      if(diskFileScanScheduler.getDiskId() == 1) {
+//        LOG.info("========>" + diskFileScanScheduler.getDiskId() + "," + fileScanner.getId() +
+//            ",fetching=" + fetching +
+//            ", scanTime:" + (System.currentTimeMillis() - startTime) + " ms");
+//      }
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    } finally {
+      synchronized(requestQueueMonitor) {
+        numOfRunningScanners.decrementAndGet();
+        requestQueueMonitor.notifyAll();
+      }
+    }
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.java
new file mode 100644
index 0000000..0d5b33d
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/FileScannerV2.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.v2;
+
+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.storage.fragment.FileFragment;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.Tuple;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public abstract class FileScannerV2 implements Scanner {
+  private static final Log LOG = LogFactory.getLog(FileScannerV2.class);
+
+	protected AtomicBoolean closed = new AtomicBoolean(false);
+
+	protected FileSystem fs;
+
+  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 long totalScanTime = 0;
+  protected int allocatedDiskId;
+
+  protected StorageManagerV2.StorgaeManagerContext smContext;
+
+  protected AtomicBoolean firstSchdeuled = new AtomicBoolean(true);
+
+  protected abstract boolean scanNext(int length) throws IOException;
+
+  protected abstract boolean initFirstScan(int maxBytesPerSchedule) throws IOException;
+
+  protected abstract long getFilePosition() throws IOException;
+
+  protected abstract Tuple nextTuple() throws IOException;
+
+  public abstract boolean isFetchProcessing();
+
+  public abstract boolean isStopScanScheduling();
+
+  public abstract void scannerReset();
+
+  protected abstract long[] reportReadBytes();
+
+	public FileScannerV2(final Configuration conf,
+                       final TableMeta meta,
+                       final Schema schema,
+                       final FileFragment fragment) throws IOException {
+    this.conf = conf;
+    this.meta = meta;
+    this.schema = schema;
+    this.fragment = fragment;
+    this.columnNum = this.schema.getColumnNum();
+
+    this.fs = fragment.getPath().getFileSystem(conf);
+	}
+
+  public void init() throws IOException {
+    closed.set(false);
+    firstSchdeuled.set(true);
+
+    if(!inited) {
+      smContext.requestFileScan(this);
+    }
+    inited = true;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    scannerReset();
+    close();
+    inited = false;
+    init();
+  }
+
+  public void setAllocatedDiskId(int allocatedDiskId) {
+    this.allocatedDiskId = allocatedDiskId;
+  }
+
+  public String getId() {
+    return fragment.getPath().getName() + ":" + fragment.getStartKey() + ":" +
+        fragment.getEndKey() + "_" + System.currentTimeMillis();
+  }
+
+  @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 Path getPath() {
+    return fragment.getPath();
+  }
+
+  public int getDiskId() {
+    if(fragment.getDiskIds().length <= 0) {
+      //LOG.warn("===> No DiskId:" + fragment.getPath() + ":" + fragment.getStartKey());
+      return -1;
+    } else {
+      return fragment.getDiskIds()[0];
+    }
+  }
+
+  public void setSearchCondition(Object expr) {
+    if (inited) {
+      throw new IllegalStateException("Should be called before init()");
+    }
+  }
+
+  public void setStorageManagerContext(StorageManagerV2.StorgaeManagerContext context) {
+    this.smContext = context;
+  }
+
+  public String toString() {
+    return fragment.getPath() + ":" + fragment.getStartKey();
+  }
+
+  public void scan(int maxBytesPerSchedule) throws IOException {
+    long startTime = System.currentTimeMillis();
+    try {
+    synchronized(firstSchdeuled) {
+      if(firstSchdeuled.get()) {
+        boolean moreData = initFirstScan(maxBytesPerSchedule);
+        firstSchdeuled.set(false);
+        firstSchdeuled.notifyAll();
+        if(moreData) {
+          smContext.requestFileScan(this);
+        }
+        return;
+      }
+    }
+    boolean moreData = scanNext(maxBytesPerSchedule);
+
+    if(moreData) {
+      smContext.requestFileScan(this);
+    }
+    } finally {
+      totalScanTime += System.currentTimeMillis() - startTime;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(closed.get()) {
+      return;
+    }
+    long[] readBytes = reportReadBytes();
+    smContext.incrementReadBytes(allocatedDiskId, readBytes);
+    closed.set(true);
+    LOG.info(toString() + " closed, totalScanTime=" + totalScanTime);
+  }
+
+  public boolean isClosed() {
+    return closed.get();
+  }
+
+  public Tuple next() throws IOException {
+    synchronized(firstSchdeuled) {
+      if(firstSchdeuled.get()) {
+        try {
+          firstSchdeuled.wait();
+        } catch (InterruptedException e) {
+        }
+      }
+    }
+    return nextTuple();
+  }
+}


[09/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java b/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
new file mode 100644
index 0000000..91a535e
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
@@ -0,0 +1,690 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.net.util.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.util.Bytes;
+import org.apache.tajo.util.FileUtil;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public abstract class AbstractStorageManager {
+  private final Log LOG = LogFactory.getLog(AbstractStorageManager.class);
+
+  protected final TajoConf conf;
+  protected final FileSystem fs;
+  protected final Path tableBaseDir;
+  protected final boolean blocksMetadataEnabled;
+
+  /**
+   * Cache of scanner handlers for each storage type.
+   */
+  protected static final Map<String, Class<? extends Scanner>> SCANNER_HANDLER_CACHE
+      = new ConcurrentHashMap<String, Class<? extends Scanner>>();
+
+  /**
+   * Cache of appender handlers for each storage type.
+   */
+  protected static final Map<String, Class<? extends FileAppender>> APPENDER_HANDLER_CACHE
+      = new ConcurrentHashMap<String, Class<? extends FileAppender>>();
+
+  /**
+   * Cache of constructors for each class. Pins the classes so they
+   * can't be garbage collected until ReflectionUtils can be collected.
+   */
+  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
+      new ConcurrentHashMap<Class<?>, Constructor<?>>();
+
+  public abstract Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException;
+
+  public abstract Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException;
+
+  protected AbstractStorageManager(TajoConf conf) throws IOException {
+    this.conf = conf;
+    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);
+    FileFragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
+    return getScanner(meta, schema, fragment);
+  }
+
+  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment) throws IOException {
+    return getScanner(meta, schema, FragmentConvertor.convert(conf, meta.getStoreType(), fragment), schema);
+  }
+
+  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment, Schema target) throws IOException {
+    return getScanner(meta, schema, FragmentConvertor.convert(conf, meta.getStoreType(), fragment), target);
+  }
+
+  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment) throws IOException {
+    return getScanner(meta, schema, fragment, schema);
+  }
+
+  public FileSystem getFileSystem() {
+    return this.fs;
+  }
+
+  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);
+  }
+
+  public Appender getAppender(TableMeta meta, Schema schema, Path path)
+      throws IOException {
+    Appender appender;
+
+    Class<? extends FileAppender> appenderClass;
+
+    String handlerName = meta.getStoreType().name().toLowerCase();
+    appenderClass = APPENDER_HANDLER_CACHE.get(handlerName);
+    if (appenderClass == null) {
+      appenderClass = conf.getClass(
+          String.format("tajo.storage.appender-handler.%s.class",
+              meta.getStoreType().name().toLowerCase()), null,
+          FileAppender.class);
+      APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
+    }
+
+    if (appenderClass == null) {
+      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
+    }
+
+    appender = newAppenderInstance(appenderClass, conf, meta, schema, path);
+
+    return appender;
+  }
+
+
+  public TableMeta getTableMeta(Path tablePath) throws IOException {
+    TableMeta meta;
+
+    FileSystem fs = tablePath.getFileSystem(conf);
+    Path tableMetaPath = new Path(tablePath, ".meta");
+    if (!fs.exists(tableMetaPath)) {
+      throw new FileNotFoundException(".meta file not found in " + tablePath.toString());
+    }
+
+    FSDataInputStream tableMetaIn = fs.open(tableMetaPath);
+
+    CatalogProtos.TableProto tableProto = (CatalogProtos.TableProto) FileUtil.loadProto(tableMetaIn,
+        CatalogProtos.TableProto.getDefaultInstance());
+    meta = new TableMeta(tableProto);
+
+    return meta;
+  }
+
+  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
+  /////////////////////////////////////////////////////////////////////////////
+
+  private static final PathFilter hiddenFileFilter = new PathFilter() {
+    public boolean accept(Path p) {
+      String name = p.getName();
+      return !name.startsWith("_") && !name.startsWith(".");
+    }
+  };
+
+  /**
+   * 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 path) throws IOException {
+    List<FileStatus> result = new ArrayList<FileStatus>();
+    Path[] dirs = new Path[]{path};
+    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;
+  }
+
+  /**
+   * Get the lower bound on split size imposed by the format.
+   *
+   * @return the number of bytes of the minimal split for this format
+   */
+  protected long getFormatMinSplitSize() {
+    return 1;
+  }
+
+  /**
+   * 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 filename the file name to check
+   * @return is this file isSplittable?
+   */
+  protected boolean isSplittable(TableMeta meta, Schema schema, Path filename) throws IOException {
+    Scanner scanner = getFileScanner(meta, schema, filename);
+    return scanner.isSplittable();
+  }
+
+  @Deprecated
+  protected long computeSplitSize(long blockSize, long minSize,
+                                  long maxSize) {
+    return Math.max(minSize, Math.min(maxSize, blockSize));
+  }
+
+  @Deprecated
+  private static final double SPLIT_SLOP = 1.1;   // 10% slop
+
+  @Deprecated
+  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, TableMeta meta, Path file, long start, long length) {
+    return new FileFragment(fragmentId, file, start, length);
+  }
+
+  protected FileFragment makeSplit(String fragmentId, TableMeta meta, Path file, BlockLocation blockLocation,
+                               int[] diskIds) throws IOException {
+    return new FileFragment(fragmentId, file, blockLocation, diskIds);
+  }
+
+  // for Non Splittable. eg, compressed gzip TextFile
+  protected FileFragment makeNonSplit(String fragmentId, TableMeta meta, 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 maximum split size.
+   *
+   * @return the maximum number of bytes a split can include
+   */
+  @Deprecated
+  public static long getMaxSplitSize() {
+    // TODO - to be configurable
+    return 536870912L;
+  }
+
+  /**
+   * Get the minimum split size
+   *
+   * @return the minimum number of bytes that can be in a split
+   */
+  @Deprecated
+  public static long getMinSplitSize() {
+    // TODO - to be configurable
+    return 67108864L;
+  }
+
+  /**
+   * 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].isValid()) {
+        String volumeIdString = volumeIds[i].toString();
+        byte[] volumeIdBytes = Base64.decodeBase64(volumeIdString);
+
+        if (volumeIdBytes.length == 4) {
+          diskId = Bytes.toInt(volumeIdBytes);
+        } else if (volumeIdBytes.length == 1) {
+          diskId = (int) volumeIdBytes[0];  // support hadoop-2.0.2
+        }
+      }
+      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<FileFragment> getSplits(String tableName, TableMeta meta, Schema schema, Path inputPath) throws IOException {
+    // generate splits'
+
+    List<FileFragment> splits = new ArrayList<FileFragment>();
+    FileSystem fs = inputPath.getFileSystem(conf);
+    List<FileStatus> files;
+    if (fs.isFile(inputPath)) {
+      files = Lists.newArrayList(fs.getFileStatus(inputPath));
+    } else {
+      files = listStatus(inputPath);
+    }
+    for (FileStatus file : files) {
+      Path path = file.getPath();
+      long length = file.getLen();
+      if (length > 0) {
+        BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
+        boolean splittable = isSplittable(meta, schema, path);
+        if (blocksMetadataEnabled && fs instanceof DistributedFileSystem) {
+          // supported disk volume
+          BlockStorageLocation[] blockStorageLocations = ((DistributedFileSystem) fs)
+              .getFileBlockStorageLocations(Arrays.asList(blkLocations));
+          if (splittable) {
+            for (BlockStorageLocation blockStorageLocation : blockStorageLocations) {
+              splits.add(makeSplit(tableName, meta, path, blockStorageLocation, getDiskIds(blockStorageLocation
+                  .getVolumeIds())));
+            }
+          } else { // Non splittable
+            long blockSize = blockStorageLocations[0].getLength();
+            if (blockSize >= length) {
+              for (BlockStorageLocation blockStorageLocation : blockStorageLocations) {
+                splits.add(makeSplit(tableName, meta, path, blockStorageLocation, getDiskIds(blockStorageLocation
+                    .getVolumeIds())));
+              }
+            } else {
+              splits.add(makeNonSplit(tableName, meta, path, 0, length, blockStorageLocations));
+            }
+          }
+
+        } else {
+          if (splittable) {
+            for (BlockLocation blockLocation : blkLocations) {
+              splits.add(makeSplit(tableName, meta, path, blockLocation, null));
+            }
+          } else { // Non splittable
+            splits.add(makeNonSplit(tableName, meta, path, 0, length, blkLocations));
+          }
+        }
+      } else {
+        //for zero length files
+        splits.add(makeSplit(tableName, meta, path, 0, length));
+      }
+    }
+
+    LOG.info("Total # of splits: " + splits.size());
+    return splits;
+  }
+
+  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();
+    }
+  }
+
+  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
+      Configuration.class,
+      Schema.class,
+      TableMeta.class,
+      FileFragment.class
+  };
+
+  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
+      Configuration.class,
+      Schema.class,
+      TableMeta.class,
+      Path.class
+  };
+
+  /**
+   * create a scanner instance.
+   */
+  public static <T> T newScannerInstance(Class<T> theClass, Configuration conf, Schema schema, TableMeta meta,
+                                         Fragment fragment) {
+    T result;
+    try {
+      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
+      if (meth == null) {
+        meth = theClass.getDeclaredConstructor(DEFAULT_SCANNER_PARAMS);
+        meth.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(theClass, meth);
+      }
+      result = meth.newInstance(new Object[]{conf, schema, meta, fragment});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result;
+  }
+
+  /**
+   * create a scanner instance.
+   */
+  public static <T> T newAppenderInstance(Class<T> theClass, Configuration conf, TableMeta meta, Schema schema,
+                                          Path path) {
+    T result;
+    try {
+      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
+      if (meth == null) {
+        meth = theClass.getDeclaredConstructor(DEFAULT_APPENDER_PARAMS);
+        meth.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(theClass, meth);
+      }
+      result = meth.newInstance(new Object[]{conf, schema, meta, path});
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..ed6ea34
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/Appender.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.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;
+  
+  void close() throws IOException;
+
+  void enableStats();
+  
+  TableStats getStats();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..ed034be
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
@@ -0,0 +1,257 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+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 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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..5d05d6f
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -0,0 +1,531 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+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.FileFragment;
+import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
+import org.apache.tajo.util.Bytes;
+
+import java.io.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+public class CSVFile {
+
+  public static final String DELIMITER = "csvfile.delimiter";
+  public static final String NULL = "csvfile.null";     //read only
+  public static final String SERDE = "csvfile.serde";
+  public static final String DELIMITER_DEFAULT = "|";
+  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 NonSyncByteArrayOutputStream os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
+    private SerializerDeserializer serde;
+
+    public CSVAppender(Configuration conf, final Schema schema, final TableMeta meta, final Path path) throws IOException {
+      super(conf, schema, meta, path);
+      this.fs = path.getFileSystem(conf);
+      this.meta = meta;
+      this.schema = schema;
+      this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(DELIMITER, DELIMITER_DEFAULT)).charAt(0);
+      this.columnNum = schema.getColumnNum();
+      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(NULL));
+      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());
+      }
+
+      String codecName = this.meta.getOption(TableMeta.COMPRESSION_CODEC);
+      if(!StringUtils.isEmpty(codecName)){
+        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 {
+        String serdeClass = this.meta.getOption(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 (enabledStats) {
+          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 FileFragment fragment)
+        throws IOException {
+      super(conf, schema, meta, fragment);
+      factory = new CompressionCodecFactory(conf);
+      codec = factory.getCodec(fragment.getPath());
+      if (codec == null || codec instanceof SplittableCompressionCodec) {
+        splittable = true;
+      }
+
+      //Delimiter
+      String delim  = meta.getOption(DELIMITER, DELIMITER_DEFAULT);
+      this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0);
+
+      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(NULL));
+      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 = new ArrayList<Long>();
+    private ArrayList<Integer> rowLengthList = new ArrayList<Integer>();
+    private ArrayList<Integer> startOffsets = new ArrayList<Integer>();
+    private NonSyncByteArrayOutputStream buffer = new NonSyncByteArrayOutputStream(DEFAULT_PAGE_SIZE);
+    private SerializerDeserializer serde;
+
+    @Override
+    public void init() throws IOException {
+
+      // 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.getEndKey();
+
+      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.getColumnIdByName(targets[i].getColumnName());
+      }
+
+      try {
+        String serdeClass = this.meta.getOption(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) {
+        startOffset += reader.readLine(new Text(), 0, maxBytesToConsume(startOffset));
+        pos = startOffset;
+      }
+      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;
+        }
+      }
+    }
+
+    @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 = Bytes.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 {
+        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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..4f58e68
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..8841a31
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..064841f
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/FileAppender.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;
+
+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 java.io.IOException;
+
+public abstract class FileAppender implements Appender {
+  protected boolean inited = false;
+
+  protected final Configuration conf;
+  protected final TableMeta meta;
+  protected final Schema schema;
+  protected final Path path;
+
+  protected boolean enabledStats;
+  
+  public FileAppender(Configuration conf, Schema schema, TableMeta meta, Path path) {
+    this.conf = conf;
+    this.meta = meta;
+    this.schema = schema;
+    this.path = path;
+  }
+
+  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 abstract long getOffset() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..c831822
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/FileScanner.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;
+
+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.conf.TajoConf;
+import org.apache.tajo.storage.fragment.FileFragment;
+
+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;
+  
+  public FileScanner(Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment) {
+    this.conf = conf;
+    this.meta = meta;
+    this.schema = schema;
+    this.fragment = fragment;
+    this.columnNum = this.schema.getColumnNum();
+  }
+
+  public void init() throws IOException {
+    inited = true;
+  }
+
+  @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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..f05a316
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/FrameTuple.java
@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.*;
+import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.exception.UnsupportedException;
+
+import java.net.InetAddress;
+
+/**
+ * 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) instanceof NullDatum;
+  }
+
+  @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 BooleanDatum getBoolean(int fieldId) {
+    return (BooleanDatum) get(fieldId);
+  }
+
+  @Override
+  public BitDatum getByte(int fieldId) {
+    return (BitDatum) get(fieldId);
+  }
+
+  @Override
+  public CharDatum getChar(int fieldId) {
+    return (CharDatum) get(fieldId);
+  }
+
+  @Override
+  public BlobDatum getBytes(int fieldId) {
+    return (BlobDatum) get(fieldId);
+  }
+
+  @Override
+  public Int2Datum getShort(int fieldId) {
+    return (Int2Datum) get(fieldId);
+  }
+
+  @Override
+  public Int4Datum getInt(int fieldId) {
+    return (Int4Datum) get(fieldId);
+  }
+
+  @Override
+  public Int8Datum getLong(int fieldId) {
+    return (Int8Datum) get(fieldId);
+  }
+
+  @Override
+  public Float4Datum getFloat(int fieldId) {
+    return (Float4Datum) get(fieldId);
+  }
+
+  @Override
+  public Float8Datum getDouble(int fieldId) {
+    return (Float8Datum) get(fieldId);
+  }
+
+  @Override
+  public Inet4Datum getIPv4(int fieldId) {
+    return (Inet4Datum) get(fieldId);
+  }
+
+  @Override
+  public byte[] getIPv4Bytes(int fieldId) { 
+    return get(fieldId).asByteArray();
+  }
+
+  @Override
+  public InetAddress getIPv6(int fieldId) {
+    throw new UnimplementedException();
+  }
+  
+  @Override
+  public byte[] getIPv6Bytes(int fieldId) {
+    throw new UnimplementedException();
+  }
+
+  @Override
+  public TextDatum getString(int fieldId) {
+    return (TextDatum) get(fieldId);
+  }
+
+  @Override
+  public TextDatum getText(int fieldId) {
+    return (TextDatum) get(fieldId);
+  }
+
+  @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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..4d484df
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/LazyTuple.java
@@ -0,0 +1,291 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.*;
+import org.apache.tajo.datum.exception.InvalidCastException;
+
+import java.net.InetAddress;
+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.getColumnNum()];
+    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) instanceof NullDatum;
+  }
+
+  @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 BooleanDatum getBoolean(int fieldId) {
+    return (BooleanDatum) get(fieldId);
+  }
+
+  @Override
+  public BitDatum getByte(int fieldId) {
+    return (BitDatum) get(fieldId);
+  }
+
+  @Override
+  public CharDatum getChar(int fieldId) {
+    return (CharDatum) get(fieldId);
+  }
+
+  @Override
+  public BlobDatum getBytes(int fieldId) {
+    return (BlobDatum) get(fieldId);
+  }
+
+  @Override
+  public Int2Datum getShort(int fieldId) {
+    return (Int2Datum) get(fieldId);
+  }
+
+  @Override
+  public Int4Datum getInt(int fieldId) {
+    return (Int4Datum) get(fieldId);
+  }
+
+  @Override
+  public Int8Datum getLong(int fieldId) {
+    return (Int8Datum) get(fieldId);
+  }
+
+  @Override
+  public Float4Datum getFloat(int fieldId) {
+    return (Float4Datum) get(fieldId);
+  }
+
+  @Override
+  public Float8Datum getDouble(int fieldId) {
+    return (Float8Datum) get(fieldId);
+  }
+
+  @Override
+  public Inet4Datum getIPv4(int fieldId) {
+    return (Inet4Datum) get(fieldId);
+  }
+
+  @Override
+  public byte[] getIPv4Bytes(int fieldId) {
+    return get(fieldId).asByteArray();
+  }
+
+  @Override
+  public InetAddress getIPv6(int fieldId) {
+    throw new InvalidCastException("IPv6 is unsupported yet");
+  }
+
+  @Override
+  public byte[] getIPv6Bytes(int fieldId) {
+    throw new InvalidCastException("IPv6 is unsupported yet");
+  }
+
+  @Override
+  public TextDatum getString(int fieldId) {
+    return (TextDatum) get(fieldId);
+  }
+
+  @Override
+  public TextDatum getText(int fieldId) {
+    return (TextDatum) get(fieldId);
+  }
+
+  public byte[] getTextBytes(int fieldId) {
+    if(textBytes[fieldId] != null)
+      return textBytes[fieldId];
+    else {
+      return get(fieldId).asTextBytes();
+    }
+  }
+
+  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() {
+    int hashCode = 37;
+    for (int i = 0; i < values.length; i++) {
+      Datum d = get(i);
+      if (d != null) {
+        hashCode ^= (d.hashCode() * 41);
+      } else {
+        hashCode = hashCode ^ (i + 17);
+      }
+    }
+
+    return hashCode;
+  }
+
+  @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;
+  }
+}


[12/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java
deleted file mode 100644
index 4b79c51..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java
+++ /dev/null
@@ -1,1823 +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.v2;
-
-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.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.rcfile.*;
-import org.apache.tajo.util.Bytes;
-
-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>.
- *
- * 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 org.apache.tajo.storage.v2.RCFile.Writer}, {@link org.apache.tajo.storage.v2.RCFile.Reader} and classes for
- * writing, reading respectively.
- * </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>
- * 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>
- * The {@link org.apache.tajo.storage.v2.RCFile.Reader} is used to read and explain the bytes of RCFile.
- * </p>
- *
- * <h4 id="Formats">RCFile Formats</h4>
- *
- *
- * <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>
- *
- * <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>
- *
- */
-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";
-
-  public static final String COLUMN_NUMBER_CONF_STR = "hive.io.rcfile.column.number.conf";
-
-  public static final String TOLERATE_CORRUPTIONS_CONF_STR =
-    "hive.io.rcfile.tolerate.corruptions";
-
-  // HACK: We actually need BlockMissingException, but that is not available
-  // in all hadoop versions.
-  public static final String BLOCK_MISSING_MESSAGE =
-    "Could not obtain block";
-
-  // 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;
-
-  /**
-   * KeyBuffer is the key of each record in RCFile. Its on-disk layout is as
-   * below:
-   *
-   * <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 implements WritableComparable {
-    // 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 NonSyncDataOutputBuffer[] allCellValLenBuffer = null;
-    // how many rows in this split
-    private int numberRows = 0;
-    // how many columns
-    private int columnNumber = 0;
-
-    // return the number of columns recorded in this file's header
-    public int getColumnNumber() {
-      return columnNumber;
-    }
-
-    @SuppressWarnings("unused")
-    @Deprecated
-    public KeyBuffer(){
-    }
-
-    KeyBuffer(int columnNum) {
-      columnNumber = columnNum;
-      eachColumnValueLen = new int[columnNumber];
-      eachColumnUncompressedValueLen = new int[columnNumber];
-      allCellValLenBuffer = new NonSyncDataOutputBuffer[columnNumber];
-    }
-
-    @SuppressWarnings("unused")
-    @Deprecated
-    KeyBuffer(int numberRows, int columnNum) {
-      this(columnNum);
-      this.numberRows = numberRows;
-    }
-
-    /**
-     * add in a new column's meta data.
-     *
-     * @param columnValueLen
-     *          this total bytes number of this column's values in this split
-     * @param colValLenBuffer
-     *          each cell's length of this column's in this split
-     */
-    void setColumnLenInfo(int columnValueLen,
-        NonSyncDataOutputBuffer colValLenBuffer,
-        int columnUncompressedValueLen, int columnIndex) {
-      eachColumnValueLen[columnIndex] = columnValueLen;
-      eachColumnUncompressedValueLen[columnIndex] = columnUncompressedValueLen;
-      allCellValLenBuffer[columnIndex] = colValLenBuffer;
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      eachColumnValueLen = new int[columnNumber];
-      eachColumnUncompressedValueLen = new int[columnNumber];
-      allCellValLenBuffer = new NonSyncDataOutputBuffer[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 NonSyncDataOutputBuffer();
-        } else {
-          allCellValLenBuffer[i].reset();
-        }
-        allCellValLenBuffer[i].write(in, bufLen);
-      }
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      // out.writeInt(numberRows);
-      WritableUtils.writeVLong(out, numberRows);
-      for (int i = 0; i < eachColumnValueLen.length; i++) {
-        WritableUtils.writeVLong(out, eachColumnValueLen[i]);
-        WritableUtils.writeVLong(out, eachColumnUncompressedValueLen[i]);
-        NonSyncDataOutputBuffer colRowsLenBuf = allCellValLenBuffer[i];
-        int bufLen = colRowsLenBuf.getLength();
-        WritableUtils.writeVLong(out, bufLen);
-        out.write(colRowsLenBuf.getData(), 0, bufLen);
-      }
-    }
-
-    /**
-     * 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 getSize() throws IOException {
-      int ret = 0;
-      ret += WritableUtils.getVIntSize(numberRows);
-      for (int i = 0; i < eachColumnValueLen.length; i++) {
-        ret += WritableUtils.getVIntSize(eachColumnValueLen[i]);
-        ret += WritableUtils.getVIntSize(eachColumnUncompressedValueLen[i]);
-        ret += WritableUtils.getVIntSize(allCellValLenBuffer[i].getLength());
-        ret += allCellValLenBuffer[i].getLength();
-      }
-
-      return ret;
-    }
-
-    @Override
-    public int compareTo(Object arg0) {
-      throw new RuntimeException("compareTo not supported in class "
-          + this.getClass().getName());
-    }
-
-    @Override
-    public boolean equals(Object obj){
-      return super.equals(obj);
-    }
-
-    @Override
-    public int hashCode(){
-      return super.hashCode();
-    }
-  }
-
-  /**
-   * 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 WritableComparable {
-
-    class LazyDecompressionCallbackImpl implements LazyDecompressionCallback {
-
-      int index = -1;
-      int colIndex = -1;
-
-      public LazyDecompressionCallbackImpl(int index, int colIndex) {
-        super();
-        this.index = index;
-        this.colIndex = colIndex;
-      }
-
-      @Override
-      public byte[] decompress() throws IOException {
-
-        if (decompressedFlag[index] || codec == null) {
-          return loadedColumnsValueBuffer[index].getData();
-        }
-
-        NonSyncDataOutputBuffer compressedData = compressedColumnsValueBuffer[index];
-        decompressBuffer.reset();
-        DataInputStream valueIn = new DataInputStream(deflatFilter);
-        deflatFilter.resetState();
-        if (deflatFilter instanceof SchemaAwareCompressionInputStream) {
-          ((SchemaAwareCompressionInputStream)deflatFilter).setColumnIndex(colIndex);
-        }
-        decompressBuffer.reset(compressedData.getData(),
-            keyBuffer.eachColumnValueLen[colIndex]);
-
-        NonSyncDataOutputBuffer decompressedColBuf = loadedColumnsValueBuffer[index];
-        decompressedColBuf.reset();
-        decompressedColBuf.write(valueIn,
-            keyBuffer.eachColumnUncompressedValueLen[colIndex]);
-        decompressedFlag[index] = true;
-        numCompressed--;
-        return decompressedColBuf.getData();
-      }
-    }
-
-    // used to load columns' value into memory
-    private NonSyncDataOutputBuffer[] loadedColumnsValueBuffer = null;
-    private NonSyncDataOutputBuffer[] compressedColumnsValueBuffer = null;
-    private boolean[] decompressedFlag = null;
-    private int numCompressed;
-    private LazyDecompressionCallbackImpl[] lazyDecompressCallbackObjs = null;
-    private boolean lazyDecompress = true;
-
-    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 valDecompressor = null;
-    NonSyncDataInputBuffer decompressBuffer = new NonSyncDataInputBuffer();
-    CompressionInputStream deflatFilter = null;
-
-    @SuppressWarnings("unused")
-    @Deprecated
-    public ValueBuffer() throws IOException {
-    }
-
-    @SuppressWarnings("unused")
-    @Deprecated
-    public ValueBuffer(KeyBuffer keyBuffer) throws IOException {
-      this(keyBuffer, keyBuffer.columnNumber, null, null, true);
-    }
-
-    @SuppressWarnings("unused")
-    @Deprecated
-    public ValueBuffer(KeyBuffer keyBuffer, boolean[] skippedColIDs)
-        throws IOException {
-      this(keyBuffer, keyBuffer.columnNumber, skippedColIDs, null, true);
-    }
-
-    @SuppressWarnings("unused")
-    @Deprecated
-    public ValueBuffer(KeyBuffer currentKey, int columnNumber,
-        boolean[] skippedCols, CompressionCodec codec) throws IOException {
-      this(currentKey, columnNumber, skippedCols, codec, true);
-    }
-
-    public ValueBuffer(KeyBuffer currentKey, int columnNumber,
-      boolean[] skippedCols, CompressionCodec codec, boolean lazyDecompress)
-        throws IOException {
-      this.lazyDecompress = lazyDecompress;
-      keyBuffer = currentKey;
-      this.columnNumber = columnNumber;
-
-      if (skippedCols != null && skippedCols.length > 0) {
-        skippedColIDs = skippedCols;
-      } else {
-        skippedColIDs = new boolean[columnNumber];
-        for (int i = 0; i < skippedColIDs.length; i++) {
-          skippedColIDs[i] = false;
-        }
-      }
-
-      int skipped = 0;
-      for (boolean currentSkip : skippedColIDs) {
-        if (currentSkip) {
-          skipped++;
-        }
-      }
-      loadedColumnsValueBuffer = new NonSyncDataOutputBuffer[columnNumber
-          - skipped];
-      decompressedFlag = new boolean[columnNumber - skipped];
-      lazyDecompressCallbackObjs = new LazyDecompressionCallbackImpl[columnNumber
-          - skipped];
-      compressedColumnsValueBuffer = new NonSyncDataOutputBuffer[columnNumber
-                                                                 - skipped];
-      this.codec = codec;
-      if (codec != null) {
-        valDecompressor = CodecPool.getDecompressor(codec);
-        deflatFilter = codec.createInputStream(decompressBuffer,
-            valDecompressor);
-      }
-      if (codec != null) {
-        numCompressed = decompressedFlag.length;
-      } else {
-        numCompressed = 0;
-      }
-      for (int k = 0, readIndex = 0; k < columnNumber; k++) {
-        if (skippedColIDs[k]) {
-          continue;
-        }
-        loadedColumnsValueBuffer[readIndex] = new NonSyncDataOutputBuffer();
-        if (codec != null) {
-          decompressedFlag[readIndex] = false;
-          lazyDecompressCallbackObjs[readIndex] = new LazyDecompressionCallbackImpl(
-              readIndex, k);
-          compressedColumnsValueBuffer[readIndex] = new NonSyncDataOutputBuffer();
-        } else {
-          decompressedFlag[readIndex] = true;
-        }
-        readIndex++;
-      }
-    }
-
-    @SuppressWarnings("unused")
-    @Deprecated
-    public void setColumnValueBuffer(NonSyncDataOutputBuffer valBuffer,
-        int addIndex) {
-      loadedColumnsValueBuffer[addIndex] = valBuffer;
-    }
-
-    @Override
-    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) {
-          Bytes.skipFully(in, skipTotal);
-          skipTotal = 0;
-        }
-
-        NonSyncDataOutputBuffer valBuf;
-        if (codec != null){
-           // load into compressed buf first
-          valBuf = compressedColumnsValueBuffer[addIndex];
-        } else {
-          valBuf = loadedColumnsValueBuffer[addIndex];
-        }
-        valBuf.reset();
-        valBuf.write(in, vaRowsLen);
-        if (codec != null) {
-          decompressedFlag[addIndex] = false;
-          if (!lazyDecompress) {
-            lazyDecompressCallbackObjs[addIndex].decompress();
-            decompressedFlag[addIndex] = true;
-          }
-        }
-        addIndex++;
-      }
-      if (codec != null) {
-        numCompressed = decompressedFlag.length;
-      }
-
-      if (skipTotal != 0) {
-        Bytes.skipFully(in, skipTotal);
-      }
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      if (codec != null) {
-        for (NonSyncDataOutputBuffer currentBuf : compressedColumnsValueBuffer) {
-          out.write(currentBuf.getData(), 0, currentBuf.getLength());
-        }
-      } else {
-        for (NonSyncDataOutputBuffer currentBuf : loadedColumnsValueBuffer) {
-          out.write(currentBuf.getData(), 0, currentBuf.getLength());
-        }
-      }
-    }
-
-    public void clearColumnBuffer() throws IOException {
-      decompressBuffer.reset();
-    }
-
-    public void close() {
-      for (NonSyncDataOutputBuffer element : loadedColumnsValueBuffer) {
-        IOUtils.closeStream(element);
-      }
-      if (codec != null) {
-        IOUtils.closeStream(decompressBuffer);
-        CodecPool.returnDecompressor(valDecompressor);
-      }
-    }
-
-    @Override
-    public int compareTo(Object arg0) {
-      throw new RuntimeException("compareTo not supported in class "
-          + this.getClass().getName());
-    }
-
-    @Override
-    public boolean equals(Object obj){
-      return super.equals(obj);
-    }
-
-    @Override
-    public int hashCode(){
-      return super.hashCode();
-    }
-  }
-
-  /**
-   * 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 Writer {
-
-    Configuration conf;
-    FSDataOutputStream out;
-
-    CompressionCodec codec = null;
-    Metadata metadata = null;
-
-    // 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);
-      }
-    }
-
-    // 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 columnsBufferSize = 4 * 1024 * 1024; // 4M
-    // 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 final ColumnBuffer[] columnBuffers;
-
-    private int columnNumber = 0;
-
-    private final int[] columnValuePlainLength;
-
-    KeyBuffer key = null;
-    private final int[] plainTotalColumnLength;
-    private final int[] comprTotalColumnLength;
-
-    boolean useNewMagic = true;
-
-    /*
-     * used for buffering appends before flush them out
-     */
-    static class ColumnBuffer {
-      // used for buffer a column's values
-      NonSyncDataOutputBuffer columnValBuffer;
-      // used to store each value's length
-      NonSyncDataOutputBuffer 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 runLength = 0;
-      int prevValueLength = -1;
-
-      ColumnBuffer() throws IOException {
-        columnValBuffer = new NonSyncDataOutputBuffer();
-        valLenBuffer = new NonSyncDataOutputBuffer();
-      }
-
-      public void append(BytesRefWritable data) throws IOException {
-        data.writeDataTo(columnValBuffer);
-        int currentLen = data.getLength();
-
-        if (prevValueLength < 0) {
-          startNewGroup(currentLen);
-          return;
-        }
-
-        if (currentLen != prevValueLength) {
-          flushGroup();
-          startNewGroup(currentLen);
-        } else {
-          runLength++;
-        }
-      }
-
-      private void startNewGroup(int currentLen) {
-        prevValueLength = currentLen;
-        runLength = 0;
-      }
-
-      public void clear() throws IOException {
-        valLenBuffer.reset();
-        columnValBuffer.reset();
-        prevValueLength = -1;
-        runLength = 0;
-      }
-
-      public void flushGroup() throws IOException {
-        if (prevValueLength >= 0) {
-          WritableUtils.writeVLong(valLenBuffer, prevValueLength);
-          if (runLength > 0) {
-            WritableUtils.writeVLong(valLenBuffer, ~runLength);
-          }
-          runLength = -1;
-          prevValueLength = -1;
-        }
-      }
-    }
-
-    public long getLength() throws IOException {
-      return out.getPos();
-    }
-
-    /** Constructs a RCFile Writer. */
-    public Writer(FileSystem fs, Configuration conf, Path name) throws IOException {
-      this(fs, conf, name, null, new Metadata(), null);
-    }
-
-    /**
-     * Constructs a RCFile Writer.
-     *
-     * @param fs
-     *          the file system used
-     * @param conf
-     *          the configuration file
-     * @param name
-     *          the file name
-     * @throws java.io.IOException
-     */
-    public Writer(FileSystem fs, Configuration conf, Path name,
-        Progressable progress, CompressionCodec codec) throws IOException {
-      this(fs, conf, name, progress, new Metadata(), codec);
-    }
-
-    /**
-     * Constructs a RCFile Writer.
-     *
-     * @param fs
-     *          the file system used
-     * @param conf
-     *          the configuration file
-     * @param name
-     *          the file name
-     * @param progress a progress meter to update as the file is written
-     * @param metadata a string to string map in the file header
-     * @throws java.io.IOException
-     */
-    public Writer(FileSystem fs, Configuration conf, Path name,
-        Progressable progress, Metadata metadata, CompressionCodec codec) throws IOException {
-      this(fs, conf, name, fs.getConf().getInt("io.file.buffer.size", 4096),
-          fs.getDefaultReplication(), fs.getDefaultBlockSize(), progress,
-          metadata, codec);
-    }
-
-    /**
-     *
-     * Constructs a RCFile Writer.
-     *
-     * @param fs
-     *          the file system used
-     * @param conf
-     *          the configuration file
-     * @param name
-     *          the file name
-     * @param bufferSize the size of the file buffer
-     * @param replication the number of replicas for the file
-     * @param blockSize the block size of the file
-     * @param progress the progress meter for writing the file
-     * @param metadata a string to string map in the file header
-     * @throws java.io.IOException
-     */
-    public Writer(FileSystem fs, Configuration conf, Path name, int bufferSize,
-        short replication, long blockSize, Progressable progress,
-        Metadata metadata, CompressionCodec codec) throws IOException {
-      RECORD_INTERVAL = conf.getInt(RECORD_INTERVAL_CONF_STR, RECORD_INTERVAL);
-      columnNumber = conf.getInt(COLUMN_NUMBER_CONF_STR, 0);
-
-      if (metadata == null) {
-        metadata = new Metadata();
-      }
-      metadata.set(new Text(COLUMN_NUMBER_METADATA_STR), new Text(""
-          + columnNumber));
-
-      columnsBufferSize = conf.getInt(COLUMNS_BUFFER_SIZE_CONF_STR,
-          4 * 1024 * 1024);
-
-      columnValuePlainLength = new int[columnNumber];
-
-      columnBuffers = new ColumnBuffer[columnNumber];
-      for (int i = 0; i < columnNumber; i++) {
-        columnBuffers[i] = new ColumnBuffer();
-      }
-
-      init(conf, fs.create(name, true, bufferSize, replication,
-        blockSize, progress), codec, metadata);
-      initializeFileHeader();
-      writeFileHeader();
-      finalizeFileHeader();
-      key = new KeyBuffer(columnNumber);
-
-      plainTotalColumnLength = new int[columnNumber];
-      comprTotalColumnLength = new int[columnNumber];
-    }
-
-    /** 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, KeyBuffer.class.getName());
-        Text.writeString(out, ValueBuffer.class.getName());
-        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.conf = conf;
-      this.out = out;
-      this.codec = codec;
-      this.metadata = metadata;
-      this.useNewMagic =
-          conf.getBoolean(TajoConf.ConfVars.HIVEUSEEXPLICITRCFILEHEADER.varname, true);
-    }
-
-    /** Returns the compression codec of data in this file. */
-    @SuppressWarnings("unused")
-    @Deprecated
-    public CompressionCodec getCompressionCodec() {
-      return codec;
-    }
-
-    /** 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
-      }
-    }
-
-    /** Returns the configuration of this file. */
-    @SuppressWarnings("unused")
-    @Deprecated
-    Configuration getConf() {
-      return conf;
-    }
-
-    private void checkAndWriteSync() throws IOException {
-      if (sync != null && out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
-        sync();
-      }
-    }
-
-    private int columnBufferSize = 0;
-
-    /**
-     * Append a row of values. Currently it only can accept <
-     * {@link BytesRefArrayWritable}. 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 val a BytesRefArrayWritable with the list of serialized columns
-     * @throws java.io.IOException
-     */
-    public void append(Writable val) throws IOException {
-
-      if (!(val instanceof BytesRefArrayWritable)) {
-        throw new UnsupportedOperationException(
-            "Currently the writer can only accept BytesRefArrayWritable");
-      }
-
-      BytesRefArrayWritable columns = (BytesRefArrayWritable) val;
-      int size = columns.size();
-      for (int i = 0; i < size; i++) {
-        BytesRefWritable cu = columns.get(i);
-        int plainLen = cu.getLength();
-        columnBufferSize += plainLen;
-        columnValuePlainLength[i] += plainLen;
-        columnBuffers[i].append(cu);
-      }
-
-      if (size < columnNumber) {
-        for (int i = columns.size(); i < columnNumber; i++) {
-          columnBuffers[i].append(BytesRefWritable.ZeroBytesRefWritable);
-        }
-      }
-
-      bufferedRecords++;
-      if ((columnBufferSize > columnsBufferSize)
-          || (bufferedRecords >= RECORD_INTERVAL)) {
-        flushRecords();
-      }
-    }
-
-    private void flushRecords() throws IOException {
-
-      key.numberRows = bufferedRecords;
-
-      Compressor compressor = null;
-      NonSyncDataOutputBuffer valueBuffer = null;
-      CompressionOutputStream deflateFilter = null;
-      DataOutputStream deflateOut = null;
-      boolean isCompressed = isCompressed();
-      int valueLength = 0;
-      if (isCompressed) {
-        ReflectionUtils.setConf(codec, this.conf);
-        compressor = CodecPool.getCompressor(codec);
-        valueBuffer = new NonSyncDataOutputBuffer();
-        deflateFilter = codec.createOutputStream(valueBuffer, compressor);
-        deflateOut = new DataOutputStream(deflateFilter);
-      }
-
-      try {
-        for (int columnIndex = 0; columnIndex < columnNumber; columnIndex++) {
-          ColumnBuffer currentBuf = columnBuffers[columnIndex];
-          currentBuf.flushGroup();
-
-          NonSyncDataOutputBuffer columnValue = currentBuf.columnValBuffer;
-          int colLen;
-          int plainLen = columnValuePlainLength[columnIndex];
-
-          if (isCompressed) {
-            if (deflateFilter instanceof SchemaAwareCompressionOutputStream) {
-              ((SchemaAwareCompressionOutputStream) deflateFilter).
-                  setColumnIndex(columnIndex);
-            }
-            deflateFilter.resetState();
-            deflateOut.write(columnValue.getData(), 0, columnValue.getLength());
-            deflateOut.flush();
-            deflateFilter.finish();
-            // find how much compressed data was added for this column
-            colLen = valueBuffer.getLength() - valueLength;
-          } else {
-            colLen = columnValuePlainLength[columnIndex];
-          }
-          valueLength += colLen;
-          key.setColumnLenInfo(colLen, currentBuf.valLenBuffer, plainLen,
-              columnIndex);
-          plainTotalColumnLength[columnIndex] += plainLen;
-          comprTotalColumnLength[columnIndex] += colLen;
-          columnValuePlainLength[columnIndex] = 0;
-        }
-      } catch (IOException e) {
-        IOUtils.cleanup(LOG, deflateOut);
-        throw e;
-      }
-
-      int keyLength = key.getSize();
-      if (keyLength < 0) {
-        throw new IOException("negative length keys not allowed: " + key);
-      }
-      if (compressor != null) {
-        CodecPool.returnCompressor(compressor);
-      }
-
-      // Write the key out
-      writeKey(key, keyLength + valueLength, keyLength);
-      // write the value out
-      if (isCompressed) {
-        out.write(valueBuffer.getData(), 0, valueBuffer.getLength());
-      } else {
-        for(int columnIndex=0; columnIndex < columnNumber; ++columnIndex) {
-          NonSyncDataOutputBuffer buf =
-            columnBuffers[columnIndex].columnValBuffer;
-          out.write(buf.getData(), 0, buf.getLength());
-        }
-      }
-
-      // clear the columnBuffers
-      clearColumnBuffers();
-
-      bufferedRecords = 0;
-      columnBufferSize = 0;
-    }
-
-    /**
-     * flush a block out without doing anything except compressing the key part.
-     */
-    public void flushBlock(KeyBuffer keyBuffer, ValueBuffer valueBuffer,
-        int recordLen, int keyLength,
-        @SuppressWarnings("unused") int compressedKeyLen) throws IOException {
-      writeKey(keyBuffer, recordLen, keyLength);
-      valueBuffer.write(out);
-    }
-
-    private void writeKey(KeyBuffer keyBuffer, 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 = CodecPool.getCompressor(codec);
-        NonSyncDataOutputBuffer compressionBuffer =
-          new NonSyncDataOutputBuffer();
-        CompressionOutputStream deflateFilter =
-          codec.createOutputStream(compressionBuffer, compressor);
-        DataOutputStream deflateOut = new DataOutputStream(deflateFilter);
-        //compress key and write key out
-        compressionBuffer.reset();
-        deflateFilter.resetState();
-        keyBuffer.write(deflateOut);
-        deflateOut.flush();
-        deflateFilter.finish();
-        int compressedKeyLen = compressionBuffer.getLength();
-        out.writeInt(compressedKeyLen);
-        out.write(compressionBuffer.getData(), 0, compressedKeyLen);
-        CodecPool.returnCompressor(compressor);
-      } else {
-        out.writeInt(keyLength);
-        keyBuffer.write(out);
-      }
-    }
-
-    private void clearColumnBuffers() throws IOException {
-      for (int i = 0; i < columnNumber; i++) {
-        columnBuffers[i].clear();
-      }
-    }
-
-    public void close() throws IOException {
-      if (bufferedRecords > 0) {
-        flushRecords();
-      }
-      clearColumnBuffers();
-
-      if (out != null) {
-
-        // Close the underlying stream if we own it...
-        out.flush();
-        out.close();
-        out = null;
-      }
-      for (int i = 0; i < columnNumber; i++) {
-        LOG.info("Column#" + i + " : Plain Total Column Value Length: "
-          + plainTotalColumnLength[i]
-          + ",  Compr Total Column Value Length: " + comprTotalColumnLength[i]);
-      }
-    }
-  }
-
-  /**
-   * Read KeyBuffer/ValueBuffer pairs from a RCFile.
-   *
-   */
-  public static class Reader {
-    private static class SelectedColumn {
-      public int colIndex;
-      public int rowReadIndex;
-      public int runLength;
-      public int prvLength;
-    }
-    private final Path file;
-//    private final FSDataInputStream in;
-
-    private byte version;
-
-    private CompressionCodec codec = null;
-    private Metadata metadata = null;
-
-    private final byte[] sync = new byte[SYNC_HASH_SIZE];
-    private final byte[] syncCheck = new byte[SYNC_HASH_SIZE];
-    private boolean syncSeen;
-    private long lastSeenSyncPos = 0;
-
-    private long headerEnd;
-    private final long end;
-    private int currentKeyLength;
-    private int currentRecordLength;
-
-    private final Configuration conf;
-
-    private final ValueBuffer currentValue;
-
-    private int readRowsIndexInBuffer = 0;
-
-    private int recordsNumInValBuffer = 0;
-
-    private int columnNumber = 0;
-
-    private int loadColumnNum;
-
-    private int passedRowsNum = 0;
-
-    // Should we try to tolerate corruption? Default is No.
-    private boolean tolerateCorruptions = false;
-
-    private boolean decompress = false;
-
-    private Decompressor keyDecompressor;
-    NonSyncDataOutputBuffer keyDecompressedData = new NonSyncDataOutputBuffer();
-
-    //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 final SelectedColumn[] selectedColumns;
-
-    // map of original column id -> index among selected columns
-    private final int[] revPrjColIDs;
-
-    // column value lengths for each of the selected columns
-    private final NonSyncDataInputBuffer[] colValLenBufferReadIn;
-
-    private ScheduledInputStream sin;
-
-    public Reader(Path file, ScheduledInputStream sin, FileSystem fs, Configuration conf) throws IOException {
-        this(file, sin, conf.getInt("io.file.buffer.size", 4096), conf, 0, fs.getFileStatus(file).getLen());
-    }
-    /** Create a new RCFile reader. */
-    public Reader(Path file, ScheduledInputStream sin, int bufferSize, Configuration conf,
-        long start, long length) throws IOException {
-      tolerateCorruptions = conf.getBoolean(
-        TOLERATE_CORRUPTIONS_CONF_STR, false);
-      conf.setInt("io.file.buffer.size", bufferSize);
-//      in = openFile(fs, file, bufferSize, length);
-      this.file = file;
-      this.sin = sin;
-      this.conf = conf;
-      end = start + length;
-      boolean succeed = false;
-      try {
-        if (start > 0) {
-          seek(0);
-          init();
-          seek(start);
-        } else {
-          seek(0);
-          init();
-        }
-        succeed = true;
-      } finally {
-        if (!succeed) {
-          if (sin != null) {
-            try {
-              sin.close();
-            } catch(IOException e) {
-              if (LOG != null && LOG.isDebugEnabled()) {
-                LOG.debug("Exception in closing " + sin, e);
-              }
-            }
-          }
-        }
-      }
-
-      columnNumber = Integer.parseInt(metadata.get(
-          new Text(COLUMN_NUMBER_METADATA_STR)).toString());
-
-      java.util.ArrayList<Integer> notSkipIDs = ColumnProjectionUtils
-          .getReadColumnIDs(conf);
-      boolean[] skippedColIDs = new boolean[columnNumber];
-      if (notSkipIDs.size() > 0) {
-        for (int i = 0; i < skippedColIDs.length; i++) {
-          skippedColIDs[i] = true;
-        }
-        for (int read : notSkipIDs) {
-          if (read < columnNumber) {
-            skippedColIDs[read] = false;
-          }
-        }
-      } else {
-        // TODO: if no column name is specified e.g, in select count(1) from tt;
-        // skip all columns, this should be distinguished from the case:
-        // select * from tt;
-        for (int i = 0; i < skippedColIDs.length; i++) {
-          skippedColIDs[i] = false;
-        }
-      }
-
-      loadColumnNum = columnNumber;
-      if (skippedColIDs.length > 0) {
-        for (boolean skippedColID : skippedColIDs) {
-          if (skippedColID) {
-            loadColumnNum -= 1;
-          }
-        }
-      }
-
-
-      revPrjColIDs = new int[columnNumber];
-      // get list of selected column IDs
-      selectedColumns = new SelectedColumn[loadColumnNum];
-      colValLenBufferReadIn = new NonSyncDataInputBuffer[loadColumnNum];
-      for (int i = 0, j = 0; i < columnNumber; ++i) {
-        if (!skippedColIDs[i]) {
-          SelectedColumn col = new SelectedColumn();
-          col.colIndex = i;
-          col.runLength = 0;
-          col.prvLength = -1;
-          col.rowReadIndex = 0;
-          selectedColumns[j] = col;
-          colValLenBufferReadIn[j] = new NonSyncDataInputBuffer();
-          revPrjColIDs[i] = j;
-          j++;
-        } else {
-          revPrjColIDs[i] = -1;
-        }
-      }
-
-      currentKey = createKeyBuffer();
-      boolean lazyDecompress = !tolerateCorruptions;
-      currentValue = new ValueBuffer(
-        null, columnNumber, skippedColIDs, codec, lazyDecompress);
-    }
-
-    /**
-     * 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, long length) throws IOException {
-      return fs.open(file, bufferSize);
-    }
-
-    private void init() throws IOException {
-      byte[] magic = new byte[MAGIC.length];
-      sin.readFully(magic);
-
-      if (Arrays.equals(magic, ORIGINAL_MAGIC)) {
-        byte vers = sin.readByte();
-        if (vers != ORIGINAL_MAGIC_VERSION_WITH_METADATA) {
-          throw new IOException(file + " is a version " + vers +
-                                " SequenceFile instead of an RCFile.");
-        }
-        version = ORIGINAL_VERSION;
-      } else {
-        if (!Arrays.equals(magic, MAGIC)) {
-          throw new IOException(file + " not a RCFile and has magic of " +
-                                new String(magic));
-        }
-
-        // Set 'version'
-        version = sin.readByte();
-        if (version > CURRENT_VERSION) {
-          throw new VersionMismatchException((byte) CURRENT_VERSION, version);
-        }
-      }
-
-      if (version == ORIGINAL_VERSION) {
-        try {
-          Class<?> keyCls = conf.getClassByName(Text.readString(sin));
-          Class<?> valCls = conf.getClassByName(Text.readString(sin));
-          if (!keyCls.equals(KeyBuffer.class)
-              || !valCls.equals(ValueBuffer.class)) {
-            throw new IOException(file + " not a RCFile");
-          }
-        } catch (ClassNotFoundException e) {
-          throw new IOException(file + " not a RCFile", e);
-        }
-      }
-
-      decompress = sin.readBoolean(); // is compressed?
-
-      if (version == ORIGINAL_VERSION) {
-        // is block-compressed? it should be always false.
-        boolean blkCompressed = sin.readBoolean();
-        if (blkCompressed) {
-          throw new IOException(file + " not a RCFile.");
-        }
-      }
-
-      // setup the compression codec
-      if (decompress) {
-        String codecClassname = Text.readString(sin);
-        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 = CodecPool.getDecompressor(codec);
-      }
-
-      metadata = new Metadata();
-      metadata.readFields(sin);
-
-      sin.readFully(sync); // read sync bytes
-      headerEnd = sin.getPos();
-    }
-
-    /** Return the current byte position in the input file. */
-    public long getPosition() throws IOException {
-      return sin.getPos();
-    }
-
-    /**
-     * Set the current byte position in the input file.
-     *
-     * <p>
-     * The position passed must be a position returned by
-     * {@link org.apache.tajo.storage.v2.RCFile.Writer#getLength()} when writing this file. To seek to an
-     * arbitrary position, use {@link org.apache.tajo.storage.v2.RCFile.Reader#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.v2.RCFile.Reader#sync(long)}.
-     */
-    public void seek(long position) throws IOException {
-      sin.seek(position);
-      sin.readNext(128 * 1024);
-    }
-
-    /**
-     * Resets the values which determine if there are more rows in the buffer
-     *
-     * 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
-        sin.seek(headerEnd);
-        // note the sync marker "seen" in the header
-        syncSeen = true;
-        return;
-      }
-
-      try {
-//        if(sin.getAvaliableSize() < 64 * 1024) {
-//      	  sin.readNext(256 * 1024);
-//        }
-        seek(position + 4); // skip escape
-        sin.readFully(syncCheck);
-        int syncLen = sync.length;
-        for (int i = 0; sin.getPos() < end; i++) {
-          int j = 0;
-          for (; j < syncLen; j++) {
-            if (sync[j] != syncCheck[(i + j) % syncLen]) {
-              break;
-            }
-          }
-          if (j == syncLen) {
-        	sin.seek(sin.getPos() - SYNC_SIZE); // position before
-            // sync
-            return;
-          }
-//          if(sin.getAvaliableSize() < 64 * 1024) {
-//        	  sin.readNext(256 * 1024);
-//          }
-          syncCheck[i % syncLen] = sin.readByte();
-        }
-      } 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 (sin.getPos() >= end) {
-        return -1;
-      }
-      int length = sin.readInt();
-      if (sync != null && length == SYNC_ESCAPE) { // process
-        // a
-        // sync entry
-        lastSeenSyncPos = sin.getPos() - 4; // minus SYNC_ESCAPE's length
-        sin.readFully(syncCheck); // read syncCheck
-        if (!Arrays.equals(sync, syncCheck)) {
-          throw new IOException("File is corrupt!");
-        }
-        syncSeen = true;
-        if (sin.getPos() >= end) {
-          return -1;
-        }
-        length = sin.readInt(); // re-read length
-      } else {
-        syncSeen = false;
-      }
-      return length;
-    }
-
-    private void seekToNextKeyBuffer() throws IOException {
-      if (!keyInit) {
-        return;
-      }
-      if (!currentValue.inited) {
-        IOUtils.skipFully(sin, currentRecordLength - currentKeyLength);
-      }
-    }
-
-    private int compressedKeyLen = 0;
-    NonSyncDataInputBuffer keyDataIn = new NonSyncDataInputBuffer();
-    NonSyncDataInputBuffer keyDecompressBuffer = new NonSyncDataInputBuffer();
-    NonSyncDataOutputBuffer keyTempBuffer = new NonSyncDataOutputBuffer();
-
-    KeyBuffer currentKey = null;
-    boolean keyInit = false;
-
-    protected int nextKeyBuffer() throws IOException {
-      seekToNextKeyBuffer();
-      currentRecordLength = readRecordLength();
-      if (currentRecordLength == -1) {
-        keyInit = false;
-        return -1;
-      }
-      
-      currentKeyLength = sin.readInt();
-      compressedKeyLen = sin.readInt();
-      
-//      System.out.println(">>>currentRecordLength=" + currentRecordLength + 
-//    		  ",currentKeyLength=" + currentKeyLength + 
-//    		  ",compressedKeyLen=" + compressedKeyLen + 
-//    		  ",decompress=" + decompress);
-      
-      if (decompress) {
-        keyTempBuffer.reset();
-        keyTempBuffer.write(sin, compressedKeyLen);
-        keyDecompressBuffer.reset(keyTempBuffer.getData(), compressedKeyLen);
-        CompressionInputStream deflatFilter = codec.createInputStream(
-            keyDecompressBuffer, keyDecompressor);
-        DataInputStream compressedIn = new DataInputStream(deflatFilter);
-        deflatFilter.resetState();
-        keyDecompressedData.reset();
-        keyDecompressedData.write(compressedIn, currentKeyLength);
-        keyDataIn.reset(keyDecompressedData.getData(), currentKeyLength);
-        currentKey.readFields(keyDataIn);
-      } else {
-        currentKey.readFields(sin);
-      }
-
-      keyInit = true;
-      currentValue.inited = false;
-
-      readRowsIndexInBuffer = 0;
-      recordsNumInValBuffer = currentKey.numberRows;
-
-      for (int selIx = 0; selIx < selectedColumns.length; selIx++) {
-        SelectedColumn col = selectedColumns[selIx];
-        int colIx = col.colIndex;
-        NonSyncDataOutputBuffer buf = currentKey.allCellValLenBuffer[colIx];
-        colValLenBufferReadIn[selIx].reset(buf.getData(), buf.getLength());
-        col.rowReadIndex = 0;
-        col.runLength = 0;
-        col.prvLength = -1;
-      }
-
-      return currentKeyLength;
-    }
-
-    protected void currentValueBuffer() throws IOException {
-      if (!keyInit) {
-        nextKeyBuffer();
-      }
-      currentValue.keyBuffer = currentKey;
-      currentValue.clearColumnBuffer();
-      currentValue.readFields(sin);
-      currentValue.inited = true;
-    }
-
-    public boolean nextBlock() throws IOException {
-      int keyLength = nextKeyBuffer();
-      if(keyLength > 0) {
-        currentValueBuffer();
-        return true;
-      }
-      return false;
-    }
-
-    private boolean rowFetched = false;
-
-    // use this buffer to hold column's cells value length for usages in
-    // getColumn(), instead of using colValLenBufferReadIn directly.
-    private final NonSyncDataInputBuffer fetchColumnTempBuf = new NonSyncDataInputBuffer();
-
-    /**
-     * Fetch all data in the buffer for a given column. This is useful for
-     * columnar operators, which perform operations on an array data of one
-     * column. It should be used together with {@link #nextColumnsBatch()}.
-     * Calling getColumn() with not change the result of
-     * {@link #next(org.apache.hadoop.io.LongWritable)} and
-     * {@link #getCurrentRow(BytesRefArrayWritable)}.
-     *
-     * @param columnID the number of the column to get 0 to N-1
-     * @throws java.io.IOException
-     */
-    public BytesRefArrayWritable getColumn(int columnID,
-        BytesRefArrayWritable rest) throws IOException {
-      int selColIdx = revPrjColIDs[columnID];
-      if (selColIdx == -1) {
-        return null;
-      }
-
-      if (rest == null) {
-        rest = new BytesRefArrayWritable();
-      }
-
-      rest.resetValid(recordsNumInValBuffer);
-
-      if (!currentValue.inited) {
-        currentValueBuffer();
-      }
-
-      int columnNextRowStart = 0;
-      fetchColumnTempBuf.reset(currentKey.allCellValLenBuffer[columnID]
-          .getData(), currentKey.allCellValLenBuffer[columnID].getLength());
-      SelectedColumn selCol = selectedColumns[selColIdx];
-      byte[] uncompData = null;
-      ValueBuffer.LazyDecompressionCallbackImpl decompCallBack = null;
-      boolean decompressed = currentValue.decompressedFlag[selColIdx];
-      if (decompressed) {
-        uncompData =
-              currentValue.loadedColumnsValueBuffer[selColIdx].getData();
-      } else {
-        decompCallBack = currentValue.lazyDecompressCallbackObjs[selColIdx];
-      }
-      for (int i = 0; i < recordsNumInValBuffer; i++) {
-        colAdvanceRow(selColIdx, selCol);
-        int length = selCol.prvLength;
-
-        BytesRefWritable currentCell = rest.get(i);
-
-        if (decompressed) {
-          currentCell.set(uncompData, columnNextRowStart, length);
-        } else {
-          currentCell.set(decompCallBack, columnNextRowStart, length);
-        }
-        columnNextRowStart = columnNextRowStart + length;
-      }
-      return rest;
-    }
-
-    /**
-     * Read in next key buffer and throw any data in current key buffer and
-     * current value buffer. It will influence the result of
-     * {@link #next(org.apache.hadoop.io.LongWritable)} and
-     * {@link #getCurrentRow(BytesRefArrayWritable)}
-     *
-     * @return whether there still has records or not
-     * @throws java.io.IOException
-     */
-    @SuppressWarnings("unused")
-    @Deprecated
-    public boolean nextColumnsBatch() throws IOException {
-      passedRowsNum += (recordsNumInValBuffer - readRowsIndexInBuffer);
-      return nextKeyBuffer() > 0;
-    }
-
-    /**
-     * Returns how many rows we fetched with next(). It only means how many rows
-     * are read by next(). The returned result may be smaller than actual number
-     * of rows passed by, because {@link #seek(long)},
-     * {@link #nextColumnsBatch()} can change the underlying key buffer and
-     * value buffer.
-     *
-     * @return next row number
-     * @throws java.io.IOException
-     */
-    public boolean next(LongWritable readRows) throws IOException {
-      if (hasRecordsInBuffer()) {
-        readRows.set(passedRowsNum);
-        readRowsIndexInBuffer++;
-        passedRowsNum++;
-        rowFetched = false;
-        return true;
-      } else {
-        keyInit = false;
-      }
-
-      int ret = -1;
-      if (tolerateCorruptions) {
-        ret = nextKeyValueTolerateCorruptions();
-      } else {
-        try {
-          ret = nextKeyBuffer();
-        } catch (EOFException eof) {
-          eof.printStackTrace();
-        }
-      }
-      return (ret > 0) && next(readRows);
-    }
-
-    private int nextKeyValueTolerateCorruptions() throws IOException {
-      long currentOffset = sin.getPos();
-      int ret;
-      try {
-        ret = nextKeyBuffer();
-        this.currentValueBuffer();
-      } catch (IOException ioe) {
-        // A BlockMissingException indicates a temporary error,
-        // not a corruption. Re-throw this exception.
-        String msg = ioe.getMessage();
-        if (msg != null && msg.startsWith(BLOCK_MISSING_MESSAGE)) {
-          LOG.warn("Re-throwing block-missing exception" + ioe);
-          throw ioe;
-        }
-        // We have an IOException other than a BlockMissingException.
-        LOG.warn("Ignoring IOException in file " + file +
-                 " after offset " + currentOffset, ioe);
-        ret = -1;
-      } catch (Throwable t) {
-        // We got an exception that is not IOException
-        // (typically OOM, IndexOutOfBounds, InternalError).
-        // This is most likely a corruption.
-        LOG.warn("Ignoring unknown error in " + file +
-                 " after offset " + currentOffset, t);
-        ret = -1;
-      }
-      return ret;
-    }
-
-    public boolean hasRecordsInBuffer() {
-      return readRowsIndexInBuffer < recordsNumInValBuffer;
-    }
-
-    /**
-     * get the current row used,make sure called {@link #next(org.apache.hadoop.io.LongWritable)}
-     * first.
-     *
-     * @throws java.io.IOException
-     */
-    public void getCurrentRow(BytesRefArrayWritable ret) throws IOException {
-
-      if (!keyInit || rowFetched) {
-        return;
-      }
-
-      if (tolerateCorruptions) {
-        if (!currentValue.inited) {
-          currentValueBuffer();
-        }
-        ret.resetValid(columnNumber);
-      } else {
-        if (!currentValue.inited) {
-          currentValueBuffer();
-          // do this only when not initialized, but we may need to find a way to
-          // tell the caller how to initialize the valid size
-          ret.resetValid(columnNumber);
-        }
-      }
-
-      // we do not use BytesWritable here to avoid the byte-copy from
-      // DataOutputStream to BytesWritable
-      if (currentValue.numCompressed > 0) {
-        for (int j = 0; j < selectedColumns.length; ++j) {
-          SelectedColumn col = selectedColumns[j];
-          int i = col.colIndex;
-
-          BytesRefWritable ref = ret.unCheckedGet(i);
-
-          colAdvanceRow(j, col);
-
-          if (currentValue.decompressedFlag[j]) {
-            ref.set(currentValue.loadedColumnsValueBuffer[j].getData(),
-                col.rowReadIndex, col.prvLength);
-          } else {
-            ref.set(currentValue.lazyDecompressCallbackObjs[j],
-                col.rowReadIndex, col.prvLength);
-          }
-          col.rowReadIndex += col.prvLength;
-        }
-      } else {
-        // This version of the loop eliminates a condition check and branch
-        // and is measurably faster (20% or so)
-        for (int j = 0; j < selectedColumns.length; ++j) {
-          SelectedColumn col = selectedColumns[j];
-          int i = col.colIndex;
-
-          BytesRefWritable ref = ret.unCheckedGet(i);
-
-          colAdvanceRow(j, col);
-          ref.set(currentValue.loadedColumnsValueBuffer[j].getData(),
-                col.rowReadIndex, col.prvLength);
-          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 iff the previous call to next passed a sync mark. */
-    @SuppressWarnings("unused")
-    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 file.toString();
-    }
-
-    @SuppressWarnings("unused")
-    public boolean isCompressedRCFile() {
-      return this.decompress;
-    }
-
-    /** Close the reader. */
-    public void close() {
-      IOUtils.closeStream(sin);
-      currentValue.close();
-      if (decompress) {
-        IOUtils.closeStream(keyDecompressedData);
-        CodecPool.returnDecompressor(keyDecompressor);
-      }
-    }
-
-    /**
-     * return the KeyBuffer object used in the reader. Internally in each
-     * reader, there is only one KeyBuffer object, which gets reused for every
-     * block.
-     */
-    public KeyBuffer getCurrentKeyBufferObj() {
-      return this.currentKey;
-    }
-
-    /**
-     * return the ValueBuffer object used in the reader. Internally in each
-     * reader, there is only one ValueBuffer object, which gets reused for every
-     * block.
-     */
-    public ValueBuffer getCurrentValueBufferObj() {
-      return this.currentValue;
-    }
-
-    //return the current block's length
-    public int getCurrentBlockLength() {
-      return this.currentRecordLength;
-    }
-
-    //return the current block's key length
-    public int getCurrentKeyLength() {
-      return this.currentKeyLength;
-    }
-
-    //return the current block's compressed key length
-    public int getCurrentCompressedKeyLen() {
-      return this.compressedKeyLen;
-    }
-
-    //return the CompressionCodec used for this file
-    public CompressionCodec getCompressionCodec() {
-      return this.codec;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
deleted file mode 100644
index a549a51..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
+++ /dev/null
@@ -1,297 +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.v2;
-
-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.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.rcfile.BytesRefArrayWritable;
-import org.apache.tajo.storage.rcfile.ColumnProjectionUtils;
-import org.apache.tajo.util.TUtil;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-public class RCFileScanner extends FileScannerV2 {
-  private static final Log LOG = LogFactory.getLog(RCFileScanner.class);
-  public static final String SERDE = "rcfile.serde";
-  public static final String NULL = "rcfile.null";
-
-  private RCFile.Reader in;
-  private long start;
-  private long end;
-  private boolean more = true;
-  private LongWritable key;
-  private BytesRefArrayWritable column;
-  private Integer [] projectionMap;
-  private ScheduledInputStream sin;
-  private boolean first = true;
-  private int maxBytesPerSchedule;
-  private SerializerDeserializer serde;
-  private byte[] nullChars;
-  private Object lock = new Object();
-
-  public RCFileScanner(final Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment)
-      throws IOException {
-    super(conf, meta, schema, fragment);
-
-    this.start = fragment.getStartKey();
-    this.end = start + fragment.getEndKey();
-    key = new LongWritable();
-    column = new BytesRefArrayWritable();
-
-    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(NULL));
-    if (StringUtils.isEmpty(nullCharacters)) {
-      nullChars = NullDatum.get().asTextBytes();
-    } else {
-      nullChars = nullCharacters.getBytes();
-    }
-	}
-
-  @Override
-  protected Tuple nextTuple() throws IOException {
-    if(first) {
-      first = false;
-      if (start > in.getPosition()) {
-        in.sync(start); // sync to start
-      }
-      this.start = in.getPosition();
-      more = start < end;
-      if(!more) {
-        return null;
-      }
-    }
-
-    more = next(key);
-
-    if (more) {
-      column.clear();
-      in.getCurrentRow(column);
-    }
-
-    if(more) {
-      Tuple tuple = makeTuple();
-      return tuple;
-    } else {
-      close();
-      return null;
-    }
-  }
-
-  private Tuple makeTuple() throws IOException {
-    Tuple tuple = new VTuple(schema.getColumnNum());
-    synchronized (lock) {
-      column.resetValid(schema.getColumnNum());
-      int tid; // target column id
-      for (int i = 0; i < projectionMap.length; i++) {
-        tid = projectionMap[i];
-
-        byte[] bytes = column.get(tid).getBytesCopy();
-        Datum datum = serde.deserialize(targets[i], bytes, 0, bytes.length, nullChars);
-        tuple.put(tid, datum);
-      }
-    }
-    return tuple;
-  }
-
-  @Override
-  public void init() throws IOException {
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-
-    prepareProjection(targets);
-
-    super.init();
-  }
-
-  private void prepareProjection(Column[] targets) {
-    projectionMap = new Integer[targets.length];
-    int tid;
-    for (int i = 0; i < targets.length; i++) {
-      tid = schema.getColumnIdByName(targets[i].getColumnName());
-      projectionMap[i] = tid;
-    }
-    ArrayList<Integer> projectionIdList = new ArrayList<Integer>(TUtil.newList(projectionMap));
-    ColumnProjectionUtils.setReadColumnIDs(conf, projectionIdList);
-  }
-
-  @Override
-  public void close() throws IOException {
-    if(closed.get()) {
-      return;
-    }
-    try {
-      if(in != null) {
-        in.close();
-        in = null;
-        sin = null;
-      }
-    } catch (Exception e) {
-      LOG.warn(e.getMessage(), e);
-    }
-
-    if(column != null) {
-      column.clear();
-      column = null;
-    }
-    super.close();
-  }
-
-  private boolean next(LongWritable key) throws IOException {
-    if (!more) {
-      return false;
-    }
-
-    more = in.next(key);
-    if (!more) {
-      return false;
-    }
-
-    long lastSeenSyncPos = in.lastSeenSyncPos();
-    if (lastSeenSyncPos >= end) {
-      more = false;
-      return more;
-    }
-    return more;
-  }
-
-  @Override
-  protected boolean initFirstScan(int maxBytesPerSchedule) throws IOException {
-    synchronized(lock) {
-      first = true;
-      this.maxBytesPerSchedule = maxBytesPerSchedule;
-      if(sin == null) {
-        sin = new ScheduledInputStream(
-            fragment.getPath(),
-            fs.open(fragment.getPath()),
-            fragment.getStartKey(),
-            fragment.getEndKey(),
-            fs.getLength(fragment.getPath()));
-
-        this.in = new RCFile.Reader(fragment.getPath(), sin, fs, fs.getConf());
-
-        Text text = this.in.getMetadata().get(new Text(SERDE));
-
-        try {
-          String serdeClass;
-          if(text != null && !text.toString().isEmpty()){
-            serdeClass = text.toString();
-          } else{
-            serdeClass = this.meta.getOption(SERDE, BinarySerializerDeserializer.class.getName());
-          }
-          serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
-        } catch (Exception e) {
-          LOG.error(e.getMessage(), e);
-          throw new IOException(e);
-        }
-      }
-    }
-    return true;
-  }
-
-  @Override
-  public boolean isStopScanScheduling() {
-    if(sin != null && sin.isEndOfStream()) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  protected boolean scanNext(int length) throws IOException {
-    synchronized(lock) {
-      if(isClosed()) {
-        return false;
-      }
-      return sin.readNext(length);
-    }
-  }
-
-
-  @Override
-  public boolean isFetchProcessing() {
-    //TODO row group size
-    if(sin != null && sin.getAvaliableSize() > maxBytesPerSchedule * 3) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  protected long getFilePosition() throws IOException {
-    return in.getPosition();
-  }
-
-  @Override
-  public void scannerReset() {
-    if(in != null) {
-      try {
-        in.seek(0);
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-      }
-    }
-    if(sin != null) {
-      try {
-        sin.seek(0);
-        sin.reset();
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-      }
-    }
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return true;
-  }
-
-  @Override
-  public boolean isSelectable() {
-    return false;
-  }
-
-  @Override
-  public boolean isSplittable(){
-    return true;
-  }
-
-  @Override
-  protected long[] reportReadBytes() {
-    if(sin == null) {
-      return new long[]{0, 0};
-    } else {
-      return new long[]{sin.getTotalReadBytesForFetch(), sin.getTotalReadBytesFromDisk()};
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java
deleted file mode 100644
index 71c4d93..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java
+++ /dev/null
@@ -1,189 +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.v2;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.storage.v2.StorageManagerV2.StorgaeManagerContext;
-
-import java.util.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-
-public final class ScanScheduler extends Thread {
-  private static final Log LOG = LogFactory.getLog(ScanScheduler.class);
-
-  private final Object scanQueueLock;
-  private StorgaeManagerContext context;
-
-  private Map<String, FileScannerV2> requestMap = new HashMap<String, FileScannerV2>();
-
-  private final Map<Integer, DiskFileScanScheduler> diskFileScannerMap = new HashMap<Integer, DiskFileScanScheduler>();
-
-  private Map<Integer, DiskDeviceInfo> diskDeviceInfoMap = new HashMap<Integer, DiskDeviceInfo>();
-
-  private SortedSet<DiskMountInfo> diskMountInfos = new TreeSet<DiskMountInfo>();
-
-  private AtomicBoolean stopped = new AtomicBoolean(false);
-
-  private Random rand = new Random(System.currentTimeMillis());
-
-  private Thread schedulerStatusReportThread;
-
-  public ScanScheduler(StorgaeManagerContext context) {
-    this.context = context;
-    this.scanQueueLock = context.getScanQueueLock();
-
-    try {
-      List<DiskDeviceInfo> deviceInfos = DiskUtil.getDiskDeviceInfos();
-      for(DiskDeviceInfo eachInfo: deviceInfos) {
-        LOG.info("Create DiskScanQueue:" + eachInfo.getName());
-        diskDeviceInfoMap.put(eachInfo.getId(), eachInfo);
-
-        diskMountInfos.addAll(eachInfo.getMountInfos());
-      }
-
-      initFileScanners();
-    } catch (Exception e) {
-      LOG.error(e.getMessage(), e);
-    }
-
-    final int reportInterval = context.getConf().getIntVar(ConfVars.STORAGE_MANAGER_DISK_SCHEDULER_REPORT_INTERVAL);
-    if(reportInterval  > 0) {
-      schedulerStatusReportThread = new Thread() {
-        public void run() {
-          while (true) {
-            try {
-              Thread.sleep(reportInterval);
-            } catch (InterruptedException e) {
-              break;
-            }
-            synchronized (diskFileScannerMap) {
-              for (DiskFileScanScheduler eachScheduler : diskFileScannerMap
-                  .values()) {
-                eachScheduler.printDiskSchedulerInfo();
-              }
-            }
-          }
-        }
-      };
-
-      schedulerStatusReportThread.start();
-    }
-  }
-
-  public void run() {
-    synchronized(scanQueueLock) {
-      while(!stopped.get()) {
-        FileScannerV2 fileScannerV2 = context.getScanQueue().poll();
-        if(fileScannerV2 == null) {
-          try {
-            scanQueueLock.wait();
-          } catch (InterruptedException e) {
-            break;
-          }
-        } else {
-          int diskId = fileScannerV2.getDiskId();
-
-          int emptyDiskId = findEmptyDisk();
-          if(emptyDiskId < 0) {
-            if(diskId < 0 || diskId >= diskDeviceInfoMap.size()) {
-              diskId = findDiskPartitionPath(fileScannerV2.getPath().toString());
-              if(diskId < 0) {
-
-                diskId = findMinQueueDisk();
-                if(diskId < 0) {
-                  diskId = rand.nextInt(diskDeviceInfoMap.size());
-                }
-              }
-            }
-          } else {
-            diskId = emptyDiskId;
-          }
-          synchronized(diskFileScannerMap) {
-            requestMap.put(fileScannerV2.getId(), fileScannerV2);
-            DiskFileScanScheduler diskScheduler = diskFileScannerMap.get(diskId);
-            fileScannerV2.setAllocatedDiskId(diskId);
-            diskScheduler.requestScanFile(fileScannerV2);
-          }
-        }
-      }
-    }
-  }
-
-  private int findEmptyDisk() {
-    synchronized(diskFileScannerMap) {
-      for(DiskFileScanScheduler eachDiskScanner: diskFileScannerMap.values()) {
-        int queueSize = eachDiskScanner.getTotalQueueSize();
-        if(queueSize == 0) {
-          return eachDiskScanner.getDiskId();
-        }
-      }
-      return -1;
-    }
-  }
-  
-  private int findMinQueueDisk() {
-    int minValue = Integer.MAX_VALUE;
-    int minId = -1;
-    synchronized(diskFileScannerMap) {
-      for(DiskFileScanScheduler eachDiskScanner: diskFileScannerMap.values()) {
-        int queueSize = eachDiskScanner.getTotalQueueSize();
-        if(queueSize <= minValue) {
-          minValue = queueSize;
-          minId = eachDiskScanner.getDiskId();
-        }
-      }
-    }
-
-    return minId;
-  }
-
-  private int findDiskPartitionPath(String fullPath) {
-    for (DiskMountInfo eachMountInfo : diskMountInfos) {
-      if (fullPath.indexOf(eachMountInfo.getMountPath()) == 0) {
-        return eachMountInfo.getDeviceId();
-      }
-    }
-
-    return -1;
-  }
-
-  public void incrementReadBytes(int diskId, long[] readBytes) {
-    diskFileScannerMap.get(diskId).incrementReadBytes(readBytes);
-  }
-
-  private void initFileScanners() {
-    for(Integer eachId: diskDeviceInfoMap.keySet()) {
-      DiskFileScanScheduler scanner = new DiskFileScanScheduler(context, diskDeviceInfoMap.get(eachId));
-      scanner.start();
-
-      diskFileScannerMap.put(eachId, scanner);
-    }
-  }
-
-  public void stopScheduler() {
-    stopped.set(true);
-    for(DiskFileScanScheduler eachDiskScanner: diskFileScannerMap.values()) {
-      eachDiskScanner.stopScan();
-    }
-    this.interrupt();
-  }
-}


[16/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Scanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Scanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Scanner.java
deleted file mode 100644
index 6dca3f2..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Scanner.java
+++ /dev/null
@@ -1,94 +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 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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SeekableScanner.java
deleted file mode 100644
index 894e7ee..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SerializerDeserializer.java
deleted file mode 100644
index 333f205..0000000
--- a/tajo-core/tajo-core-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;
-
-
-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/SplitLineReader.java
deleted file mode 100644
index 3579674..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Storage.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Storage.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Storage.java
deleted file mode 100644
index cc85c1d..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManager.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
deleted file mode 100644
index 1b852d4..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
+++ /dev/null
@@ -1,67 +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.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-
-/**
- * StorageManager
- */
-public class StorageManager extends AbstractStorageManager {
-
-  protected StorageManager(TajoConf conf) throws IOException {
-    super(conf);
-  }
-
-  @Override
-  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;
-  }
-
-  @Override
-  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
-    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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java
deleted file mode 100644
index 85bb861..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java
+++ /dev/null
@@ -1,98 +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.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.v2.StorageManagerV2;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Map;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-
-public class StorageManagerFactory {
-  private static final Map<String, AbstractStorageManager> storageManagers = Maps.newHashMap();
-
-  public static AbstractStorageManager getStorageManager(TajoConf conf) throws IOException {
-    return getStorageManager(conf, null);
-  }
-
-  public static synchronized AbstractStorageManager getStorageManager (
-      TajoConf conf, Path warehouseDir) throws IOException {
-    return getStorageManager(conf, warehouseDir, conf.getBoolVar(ConfVars.STORAGE_MANAGER_VERSION_2));
-  }
-
-  private static synchronized AbstractStorageManager getStorageManager (
-      TajoConf conf, Path warehouseDir, boolean v2) throws IOException {
-
-    URI uri;
-    TajoConf localConf = new TajoConf(conf);
-    if (warehouseDir != null) {
-      localConf.setVar(ConfVars.WAREHOUSE_DIR, warehouseDir.toUri().toString());
-    }
-
-    uri = TajoConf.getWarehouseDir(localConf).toUri();
-
-    String key = "file".equals(uri.getScheme()) ? "file" : uri.toString();
-
-    if(v2) {
-      key += "_v2";
-    }
-
-    if(storageManagers.containsKey(key)) {
-      AbstractStorageManager sm = storageManagers.get(key);
-      return sm;
-    } else {
-      AbstractStorageManager storageManager;
-
-      if(v2) {
-        storageManager = new StorageManagerV2(localConf);
-      } else {
-        storageManager = new StorageManager(localConf);
-      }
-
-      storageManagers.put(key, storageManager);
-
-      return storageManager;
-    }
-  }
-
-  public static synchronized SeekableScanner getSeekableScanner(
-      TajoConf conf, TableMeta meta, Schema schema, FileFragment fragment, Schema target) throws IOException {
-    return (SeekableScanner)getStorageManager(conf, null, false).getScanner(meta, schema, fragment, target);
-  }
-
-  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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
deleted file mode 100644
index 9627a5d..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
+++ /dev/null
@@ -1,83 +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.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.util.FileUtil;
-
-import java.io.IOException;
-
-public class StorageUtil {
-  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;
-    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());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TableStatistics.java
deleted file mode 100644
index 41c9d61..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TableStatistics.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;
-
-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 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.getColumnNum());
-    maxValues = new VTuple(schema.getColumnNum());
-
-    numNulls = new long[schema.getColumnNum()];
-    comparable = new boolean[schema.getColumnNum()];
-
-    DataType type;
-    for (int i = 0; i < schema.getColumnNum(); 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.getColumnNum(); i++) {
-      columnStats = new ColumnStats(schema.getColumn(i));
-      columnStats.setNumNulls(numNulls[i]);
-      columnStats.setMinValue(minValues.get(i));
-      columnStats.setMaxValue(maxValues.get(i));
-      stat.addColumnStat(columnStats);
-    }
-
-    stat.setNumRows(this.numRows);
-    stat.setNumBytes(this.numBytes);
-
-    return stat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java
deleted file mode 100644
index 07ea79b..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.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 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.datum.*;
-import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
-import org.apache.tajo.util.Bytes;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-//Compatibility with Apache Hive
-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 TIME:
-      case TIMESTAMP:
-        bytes = datum.asTextBytes();
-        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 INT2:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createInt2(new String(bytes, offset, length));
-        break;
-      case INT4:
-        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
-            : DatumFactory.createInt4(new String(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(new String(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 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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Tuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Tuple.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Tuple.java
deleted file mode 100644
index ba35988..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Tuple.java
+++ /dev/null
@@ -1,82 +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 java.net.InetAddress;
-
-public interface Tuple extends Cloneable {
-  
-	public int size();
-	
-	public boolean contains(int fieldid);
-
-  public boolean isNull(int fieldid);
-	
-	public void clear();
-	
-	public void put(int fieldId, Datum value);
-
-  public void put(int fieldId, Datum [] values);
-
-  public void put(int fieldId, Tuple tuple);
-	
-	public void put(Datum [] values);
-	
-	public Datum get(int fieldId);
-	
-	public void setOffset(long offset);
-	
-	public long getOffset();
-
-	public BooleanDatum getBoolean(int fieldId);
-	
-	public BitDatum getByte(int fieldId);
-
-  public CharDatum getChar(int fieldId);
-	
-	public BlobDatum getBytes(int fieldId);
-	
-	public Int2Datum getShort(int fieldId);
-	
-	public Int4Datum getInt(int fieldId);
-	
-	public Int8Datum getLong(int fieldId);
-	
-	public Float4Datum getFloat(int fieldId);
-	
-	public Float8Datum getDouble(int fieldId);
-	
-	public Inet4Datum getIPv4(int fieldId);
-	
-	public byte [] getIPv4Bytes(int fieldId);
-	
-	public InetAddress getIPv6(int fieldId);
-	
-	public byte [] getIPv6Bytes(int fieldId);
-	
-	public TextDatum getString(int fieldId);
-
-  public TextDatum getText(int fieldId);
-
-  public Tuple clone() throws CloneNotSupportedException;
-
-  public Datum[] getValues();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
deleted file mode 100644
index 69c1c04..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
+++ /dev/null
@@ -1,159 +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 org.apache.tajo.datum.NullDatum;
-
-import java.util.Comparator;
-
-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 TupleComparator implements Comparator<Tuple>, ProtoObject<TupleComparatorProto> {
-  private final int[] sortKeyIds;
-  private final boolean[] asc;
-  @SuppressWarnings("unused")
-  private final boolean[] nullFirsts;  
-
-  private Datum left;
-  private Datum right;
-  private int compVal;
-
-  public TupleComparator(Schema schema, SortSpec[] sortKeys) {
-    Preconditions.checkArgument(sortKeys.length > 0, 
-        "At least one sort key must be specified.");
-
-    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++) {
-      this.sortKeyIds[i] = schema.getColumnId(sortKeys[i].getSortKey().getQualifiedName());
-          
-      this.asc[i] = sortKeys[i].isAscending();
-      this.nullFirsts[i]= sortKeys[i].isNullFirst();
-    }
-  }
-
-  public TupleComparator(TupleComparatorProto proto) {
-    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 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 instanceof NullDatum || right instanceof NullDatum) {
-        if (!left.equals(right)) {
-          if (left instanceof NullDatum) {
-            compVal = 1;
-          } else if (right instanceof NullDatum) {
-            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 TupleComparator) {
-      TupleComparator other = (TupleComparator) 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();
-    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();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleRange.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleRange.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleRange.java
deleted file mode 100644
index 7d0f674..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/TupleRange.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 com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.SortSpec;
-
-import java.util.Comparator;
-
-public class TupleRange implements Comparable<TupleRange> {
-  private final Schema schema;
-  private final Tuple start;
-  private final Tuple end;
-  private final TupleComparator comp;
-
-  public TupleRange(final Schema schema, final Tuple start, final Tuple end) {
-    this.comp = new TupleComparator(schema, schemaToSortSpecs(schema));
-    // if there is only one value, start == end
-    Preconditions.checkArgument(comp.compare(start, end) <= 0, ("start=" + start) + ", end=" + end);
-    this.schema = schema;
-    this.start = start;
-    this.end = end;
-  }
-
-  public static SortSpec[] schemaToSortSpecs(Schema schema) {
-    SortSpec[] specs = new SortSpec[schema.getColumnNum()];
-
-    for (int i = 0; i < schema.getColumnNum(); i++) {
-      specs[i] = new SortSpec(schema.getColumn(i), true, false);
-    }
-
-    return specs;
-  }
-
-  public final Schema getSchema() {
-    return this.schema;
-  }
-
-  public final Tuple getStart() {
-    return this.start;
-  }
-
-  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 -(left.compareTo(right));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/VTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/VTuple.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/VTuple.java
deleted file mode 100644
index 878c05e..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/VTuple.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.gson.annotations.Expose;
-import org.apache.tajo.datum.*;
-import org.apache.tajo.datum.exception.InvalidCastException;
-
-import java.net.InetAddress;
-import java.util.Arrays;
-
-public class VTuple implements Tuple, Cloneable {
-	@Expose public Datum [] values;
-	@Expose private long offset;
-	
-	public VTuple(int size) {
-		values = new Datum[size];
-	}
-
-  public VTuple(Tuple tuple) {
-    this.values = new Datum[tuple.size()];
-    System.arraycopy(((VTuple)tuple).values, 0, values, 0, tuple.size());
-    this.offset = ((VTuple)tuple).offset;
-  }
-
-  public VTuple(Datum [] datum) {
-    this(datum.length);
-    put(datum);
-  }
-
-	@Override
-	public int size() {	
-		return values.length;
-	}
-	
-	public boolean contains(int fieldId) {
-		return values[fieldId] != null;
-	}
-
-  @Override
-  public boolean isNull(int fieldid) {
-    return values[fieldid] instanceof NullDatum;
-  }
-
-  @Override
-  public void clear() {   
-    for (int i=0; i < values.length; i++) {
-      values[i] = null;
-    }
-  }
-	
-	//////////////////////////////////////////////////////
-	// Setter
-	//////////////////////////////////////////////////////	
-	public void put(int fieldId, Datum value) {
-		values[fieldId] = value;
-	}
-
-  @Override
-  public void put(int fieldId, Datum[] values) {
-    for (int i = fieldId, j = 0; j < values.length; i++, j++) {
-      values[i] = values[j];
-    }
-  }
-
-  @Override
-  public void put(int fieldId, Tuple tuple) {
-    for (int i = fieldId, j = 0; j < tuple.size(); i++, j++) {
-      values[i] = tuple.get(j);
-    }
-  }
-
-  public void put(Datum [] values) {
-    System.arraycopy(values, 0, this.values, 0, size());
-	}
-	
-	//////////////////////////////////////////////////////
-	// Getter
-	//////////////////////////////////////////////////////
-	public Datum get(int fieldId) {
-		return this.values[fieldId];
-	}
-	
-	public void setOffset(long offset) {
-	  this.offset = offset;
-	}
-	
-	public long getOffset() {
-	  return this.offset;
-	}
-	
-	@Override
-	public BooleanDatum getBoolean(int fieldId) {
-		return (BooleanDatum) values[fieldId];
-	}
-
-	public BitDatum getByte(int fieldId) {
-		return (BitDatum) values[fieldId];
-	}
-
-  public CharDatum getChar(int fieldId) {
-    return (CharDatum) values[fieldId];
-  }
-
-	public BlobDatum getBytes(int fieldId) {
-		return (BlobDatum) values[fieldId];
-	}
-
-	public Int2Datum getShort(int fieldId) {
-		return (Int2Datum) values[fieldId];
-	}
-
-	public Int4Datum getInt(int fieldId) {
-		return (Int4Datum) values[fieldId];
-	}
-
-	public Int8Datum getLong(int fieldId) {
-		return (Int8Datum) values[fieldId];
-	}
-
-	public Float4Datum getFloat(int fieldId) {
-		return (Float4Datum) values[fieldId];
-	}
-
-	public Float8Datum getDouble(int fieldId) {
-		return (Float8Datum) values[fieldId];
-	}
-
-	public Inet4Datum getIPv4(int fieldId) {
-		return (Inet4Datum) values[fieldId];
-	}
-
-	public byte[] getIPv4Bytes(int fieldId) {
-		return values[fieldId].asByteArray();
-	}
-
-	public InetAddress getIPv6(int fieldId) {
-		throw new InvalidCastException("IPv6 is unsupported yet");
-	}
-
-	public byte[] getIPv6Bytes(int fieldId) {
-	  throw new InvalidCastException("IPv6 is unsupported yet");
-	}
-
-	public TextDatum getString(int fieldId) {
-		return (TextDatum) values[fieldId];
-	}
-
-  @Override
-  public TextDatum getText(int fieldId) {
-    return (TextDatum) values[fieldId];
-  }
-
-  @Override
-  public Tuple clone() throws CloneNotSupportedException {
-    VTuple tuple = (VTuple) super.clone();
-
-    tuple.values = new Datum[size()];
-    System.arraycopy(values, 0, tuple.values, 0, size()); //shallow copy
-    return tuple;
-  }
-
-  public String toString() {
-		boolean first = true;
-		StringBuilder str = new StringBuilder();
-		str.append("(");
-		for(int i=0; i < values.length; i++) {			
-			if(values[i] != null) {
-				if(first) {
-					first = false;
-				} else {
-					str.append(", ");
-				}
-				str.append(i)
-				.append("=>")
-				.append(values[i]);
-			}
-		}
-		str.append(")");
-		return str.toString();
-	}
-	
-	@Override
-	public int hashCode() {
-	  int hashCode = 37;
-	  for (int i=0; i < values.length; i++) {
-	    if(values[i] != null) {
-        hashCode ^= (values[i].hashCode() * 41);
-	    } else {
-	      hashCode = hashCode ^ (i + 17);
-	    }
-	  }
-	  
-	  return hashCode;
-	}
-
-  @Override
-  public Datum[] getValues() {
-    return values;
-  }
-
-  @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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/annotation/ForSplitableStore.java
deleted file mode 100644
index ad19101..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/compress/CodecPool.java
deleted file mode 100644
index baeda8c..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/AlreadyExistsStorageException.java
deleted file mode 100644
index bb035a8..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnknownCodecException.java
deleted file mode 100644
index a67d1f7..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnknownDataTypeException.java
deleted file mode 100644
index d18b5a0..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/exception/UnsupportedFileTypeException.java
deleted file mode 100644
index 8b197d6..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
deleted file mode 100644
index ea8bf9f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java
+++ /dev/null
@@ -1,219 +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.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 private Long startOffset; // required
-  @Expose private 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, int[] diskIds)
-      throws IOException {
-    this.set(tableName, uri, blockLocation.getOffset(), blockLocation.getLength(),
-        blockLocation.getHosts(), diskIds);
-  }
-
-  // Non splittable
-  public FileFragment(String tableName, Path uri, long start, long length, String[] hosts) {
-    this.set(tableName, uri, start, length, null, null);
-    this.hosts = hosts;
-  }
-
-  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 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;
-  }
-
-  public Long getEndKey() {
-    return this.length;
-  }
-
-  /**
-   * 
-   * 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.getEndKey(), this.getEndKey())) {
-        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\": "
-        + getEndKey() + "}" ;
-  }
-
-  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.setContents(builder.buildPartial().toByteString());
-    return fragmentBuilder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
deleted file mode 100644
index 3f9c160..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
+++ /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.
- */
-
-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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
deleted file mode 100644
index 3bfe96f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
+++ /dev/null
@@ -1,123 +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;
-import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-
-@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, StoreType storeType)
-      throws IOException {
-    String handlerName = storeType.name().toLowerCase();
-    Class<? extends Fragment> fragmentClass = CACHED_FRAGMENT_CLASSES.get(handlerName);
-    if (fragmentClass == null) {
-      fragmentClass = conf.getClass(
-          String.format("tajo.storage.fragment.%s.class", storeType.name().toLowerCase()), null, Fragment.class);
-      CACHED_FRAGMENT_CLASSES.put(handlerName, fragmentClass);
-    }
-
-    if (fragmentClass == null) {
-      throw new IOException("No such a fragment for " + storeType.name());
-    }
-
-    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, StoreType storeType, FragmentProto fragment)
-      throws IOException {
-    Class<T> fragmentClass = (Class<T>) getFragmentClass(conf, storeType);
-    if (fragmentClass == null) {
-      throw new IOException("No such a fragment class for " + storeType.name());
-    }
-    return convert(fragmentClass, fragment);
-  }
-
-  public static <T extends Fragment> List<T> convert(Class<T> clazz, FragmentProto...fragments)
-      throws IOException {
-    List<T> list = Lists.newArrayList();
-    for (FragmentProto proto : fragments) {
-      list.add(convert(clazz, proto));
-    }
-    return list;
-  }
-
-  public static <T extends Fragment> List<T> convert(Configuration conf, StoreType storeType,
-                                                           FragmentProto...fragments) throws IOException {
-    List<T> list = Lists.newArrayList();
-    for (FragmentProto proto : fragments) {
-      list.add((T) convert(conf, storeType, proto));
-    }
-    return list;
-  }
-
-  public static List<FragmentProto> toFragmentProtoList(Fragment... fragments) {
-    List<FragmentProto> list = Lists.newArrayList();
-    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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.java
deleted file mode 100644
index 74be7ff..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexMethod.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.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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexReader.java
deleted file mode 100644
index 7baf7aa..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/IndexWriter.java
deleted file mode 100644
index 04738f8..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/OrderIndexReader.java
deleted file mode 100644
index 688bbc7..0000000
--- a/tajo-core/tajo-core-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;
-}


[15/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/index/bst/BSTIndex.java
deleted file mode 100644
index bc8fe96..0000000
--- a/tajo-core/tajo-core-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.RowStoreUtil;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.TupleComparator;
-import org.apache.tajo.storage.index.IndexMethod;
-import org.apache.tajo.storage.index.IndexWriter;
-import org.apache.tajo.storage.index.OrderIndexReader;
-import org.apache.tajo.util.Bytes;
-
-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 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);
-    }
-
-   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 = RowStoreUtil.RowStoreEncoder.toBytes(keySchema,
-            firstKey);
-        out.writeInt(minBytes.length);
-        out.write(minBytes);
-        byte [] maxBytes = RowStoreUtil.RowStoreEncoder.toBytes(keySchema,
-            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 = RowStoreUtil.RowStoreEncoder.toBytes(this.keySchema, 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 = RowStoreUtil.RowStoreEncoder.toBytes(keySchema, 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();
-
-    /**
-     *
-     * @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;
-    }
-
-    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];
-      Bytes.readFully(indexIn, schemaBytes, 0, schemaByteSize);
-
-      SchemaProto.Builder builder = SchemaProto.newBuilder();
-      builder.mergeFrom(schemaBytes);
-      SchemaProto proto = builder.build();
-      this.keySchema = new Schema(proto);
-
-      // comparator
-      int compByteSize = indexIn.readInt();
-      byte [] compBytes = new byte[compByteSize];
-      Bytes.readFully(indexIn, compBytes, 0, compByteSize);
-
-      TupleComparatorProto.Builder compProto = TupleComparatorProto.newBuilder();
-      compProto.mergeFrom(compBytes);
-      this.comparator = new TupleComparator(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()];
-        Bytes.readFully(indexIn, minBytes, 0, minBytes.length);
-        this.firstKey = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, minBytes);
-
-        byte [] maxBytes = new byte[indexIn.readInt()];
-        Bytes.readFully(indexIn, maxBytes, 0, maxBytes.length);
-        this.lastKey = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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;
-        switch (this.level) {
-          case ONE_LEVEL_INDEX:
-            pos = oneLevBS(key);
-            break;
-          case TWO_LEVEL_INDEX:
-            pos = twoLevBS(key, this.loadNum + 1);
-            break;
-        }
-
-        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()];
-          Bytes.readFully(in, buf, 0, buf.length);
-          dataSubIndex[i] = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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()];
-          Bytes.readFully(in, buf, 0, buf.length);
-          dataSubIndex[i] = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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()];
-        Bytes.readFully(in, buf, 0, buf.length);
-        keyTuple = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefArrayWritable.java
deleted file mode 100644
index 5e200a0..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/BytesRefWritable.java
deleted file mode 100644
index c83b505..0000000
--- a/tajo-core/tajo-core-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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/ColumnProjectionUtils.java
deleted file mode 100644
index 352776f..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/LazyDecompressionCallback.java
deleted file mode 100644
index 707d55a..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayInputStream.java
deleted file mode 100644
index bb6af22..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncByteArrayOutputStream.java
deleted file mode 100644
index 53a3dca..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataInputBuffer.java
deleted file mode 100644
index 46745ab..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/NonSyncDataOutputBuffer.java
deleted file mode 100644
index 3944f38..0000000
--- a/tajo-core/tajo-core-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;
-    }
-  }
-}


[14/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
deleted file mode 100644
index a6b8781..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
+++ /dev/null
@@ -1,1739 +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.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;
-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.util.Bytes;
-
-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 {
-
-    // 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();
-
-
-    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) {
-          Bytes.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);
-        }
-        addIndex++;
-      }
-
-      if (skipTotal != 0) {
-        Bytes.skipFully(in, skipTotal);
-      }
-    }
-
-    public void clearColumnBuffer() throws IOException {
-      decompressBuffer.reset();
-    }
-
-    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; // 4M
-    // 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;
-    SerializerDeserializer serde;
-
-    // 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 Schema schema, final TableMeta meta, final Path path) throws IOException {
-      super(conf, schema, meta, path);
-
-      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.getColumnNum();
-    }
-
-    public void init() throws IOException {
-      fs = path.getFileSystem(conf);
-
-      if (!fs.exists(path.getParent())) {
-        throw new FileNotFoundException(path.toString());
-      }
-
-      String codecClassname = this.meta.getOption(TableMeta.COMPRESSION_CODEC);
-      if (!StringUtils.isEmpty(codecClassname)) {
-        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(NULL));
-      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(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(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.getColumnNum();
-
-      for (int i = 0; i < size; i++) {
-        Datum datum = tuple.get(i);
-        int length = columnBuffers[i].append(schema.getColumn(i), datum);
-        columnBufferSize += length;
-        if (enabledStats) {
-          stats.analyzeField(i, datum);
-        }
-      }
-
-      if (size < columnNumber) {
-        for (int i = size; i < columnNumber; i++) {
-          columnBuffers[i].append(schema.getColumn(i), NullDatum.get());
-          if (enabledStats) {
-            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(new BufferedOutputStream(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);
-        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, deflateOut, deflateFilter);
-        }
-      } 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(new BufferedOutputStream(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) {
-        // Close the underlying stream if we own it...
-        out.flush();
-        out.close();
-        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 final byte[] sync = new byte[SYNC_HASH_SIZE];
-    private final byte[] syncCheck = new byte[SYNC_HASH_SIZE];
-    private boolean syncSeen;
-    private long lastSeenSyncPos = 0;
-
-    private long headerEnd;
-    private long start, end;
-    private 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;
-
-
-    //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 FileFragment fragment) throws IOException {
-      super(conf, schema, meta, fragment);
-
-      rowId = new LongWritable();
-      conf.setInt("io.file.buffer.size", 4096); //TODO remove
-
-
-      startOffset = fragment.getStartKey();
-      endOffset = startOffset + fragment.getEndKey();
-      more = startOffset < endOffset;
-      start = 0;
-    }
-
-    @Override
-    public void init() throws IOException {
-      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(NULL));
-      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.getColumnIdByName(targets[i].getColumnName());
-      }
-      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(SERDE));
-
-      try {
-        String serdeClass;
-        if(text != null && !text.toString().isEmpty()){
-          serdeClass = text.toString();
-        } else{
-          serdeClass = this.meta.getOption(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();
-    }
-
-    /**
-     * 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);
-          /* 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();
-      if (sync != null && length == SYNC_ESCAPE) { // process
-        // a
-        // sync entry
-        lastSeenSyncPos = in.getPos() - 4; // minus SYNC_ESCAPE's length
-        in.readFully(syncCheck); // read syncCheck
-        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
-      } 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();
-      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);
-      }
-
-      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;
-    }
-
-    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.getColumnNum());
-      getCurrentRow(tuple);
-      return tuple;
-    }
-
-    /**
-     * 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 {
-      IOUtils.closeStream(in);
-      currentValue.close();
-      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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionInputStream.java
deleted file mode 100644
index 60f1b06..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/rcfile/SchemaAwareCompressionOutputStream.java
deleted file mode 100644
index c0ce8b3..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.java
deleted file mode 100644
index 3209469..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.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.trevni;
-
-import org.apache.commons.io.IOUtils;
-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.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.common.TajoDataTypes.Type;
-import org.apache.tajo.storage.FileAppender;
-import org.apache.tajo.storage.TableStatistics;
-import org.apache.tajo.storage.Tuple;
-import org.apache.trevni.ColumnFileMetaData;
-import org.apache.trevni.ColumnFileWriter;
-import org.apache.trevni.ColumnMetaData;
-import org.apache.trevni.ValueType;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-public class TrevniAppender extends FileAppender {
-  private FileSystem fs;
-  private ColumnFileWriter writer;
-  private FSDataOutputStream fos;
-
-  private TableStatistics stats = null;
-  private boolean flushed = false;
-
-  public TrevniAppender(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
-    super(conf, schema, meta, path);
-  }
-
-  public void init() throws IOException {
-    fs = path.getFileSystem(conf);
-
-    if (!fs.exists(path.getParent())) {
-      throw new FileNotFoundException(path.toString());
-    }
-
-    fos = fs.create(path);
-
-    ColumnMetaData [] trevniMetas =
-        new ColumnMetaData[schema.getColumnNum()];
-    int i = 0;
-    for (Column column : schema.getColumns()) {
-      trevniMetas[i++] = new ColumnMetaData(column.getColumnName(),
-          getType(column.getDataType().getType()));
-    }
-
-    writer = new ColumnFileWriter(createFileMeta(), trevniMetas);
-
-    if (enabledStats) {
-      this.stats = new TableStatistics(this.schema);
-    }
-
-    super.init();
-  }
-
-  private ColumnFileMetaData createFileMeta() {
-    return new ColumnFileMetaData()
-        .setCodec("null")
-        .setChecksum("null");
-  }
-
-  private static ValueType getType(Type type) {
-    switch (type) {
-      case BOOLEAN:
-        return ValueType.INT;
-      case BIT:
-        return ValueType.INT;
-      case CHAR:
-        return ValueType.STRING;
-      case INT2:
-        return ValueType.INT;
-      case INT4:
-        return ValueType.INT;
-      case INT8:
-        return ValueType.LONG;
-      case FLOAT4:
-        return ValueType.FLOAT;
-      case FLOAT8:
-        return ValueType.DOUBLE;
-      case TEXT:
-        return ValueType.STRING;
-      case BLOB:
-        return ValueType.BYTES;
-      case INET4:
-        return ValueType.BYTES;
-      case INET6:
-        return ValueType.BYTES;
-      case PROTOBUF:
-        return ValueType.BYTES;
-      case NULL_TYPE:
-        return ValueType.NULL;
-      default:
-        return null;
-    }
-  }
-
-  @Override
-  public long getOffset() throws IOException {
-    return 0;
-  }
-
-  @Override
-  public void addTuple(Tuple t) throws IOException {
-    Column col;
-    writer.startRow();
-    for (int i = 0; i < schema.getColumnNum(); i++) {
-      if (enabledStats) {
-        stats.analyzeField(i, t.get(i));
-      }
-
-      if (!t.isNull(i)) {
-        col = schema.getColumn(i);
-        switch (col.getDataType().getType()) {
-          case NULL_TYPE:
-            break;
-          case BOOLEAN:
-          case BIT:
-          case INT2:
-          case INT4:
-            writer.writeValue(t.get(i).asInt4(), i);
-            break;
-          case INT8:
-            writer.writeValue(t.get(i).asInt8(), i);
-            break;
-          case FLOAT4:
-            writer.writeValue(t.get(i).asFloat4(), i);
-            break;
-          case FLOAT8:
-            writer.writeValue(t.get(i).asFloat8(), i);
-            break;
-          case CHAR:
-          case TEXT:
-            writer.writeValue(t.get(i).asChars(), i);
-            break;
-          case PROTOBUF:
-          case BLOB:
-          case INET4:
-          case INET6:
-            writer.writeValue(t.get(i).asByteArray(), i);
-
-          default:
-            break;
-        }
-      }
-    }
-    writer.endRow();
-
-    // Statistical section
-    if (enabledStats) {
-      stats.incrementRow();
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    if (!flushed) {
-      writer.writeTo(fos);
-      fos.flush();
-      flushed = true;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    flush();
-    IOUtils.closeQuietly(fos);
-  }
-
-  @Override
-  public TableStats getStats() {
-    if (enabledStats) {
-      return stats.getTableStat();
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java
deleted file mode 100644
index 2c2037f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java
+++ /dev/null
@@ -1,193 +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.trevni;
-
-import com.google.protobuf.Message;
-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.datum.BlobDatum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.NullDatum;
-import org.apache.tajo.datum.ProtobufDatumFactory;
-import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.trevni.ColumnFileReader;
-import org.apache.trevni.ColumnValues;
-import org.apache.trevni.avro.HadoopInput;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import static org.apache.tajo.common.TajoDataTypes.DataType;
-
-public class TrevniScanner extends FileScanner {
-  private ColumnFileReader reader;
-  private int [] projectionMap;
-  private ColumnValues [] columns;
-
-  public TrevniScanner(Configuration conf, Schema schema, TableMeta meta, FileFragment fragment) throws IOException {
-    super(conf, schema, meta, fragment);
-    reader = new ColumnFileReader(new HadoopInput(fragment.getPath(), conf));
-  }
-
-  @Override
-  public void init() throws IOException {
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-
-    prepareProjection(targets);
-
-    columns = new ColumnValues[projectionMap.length];
-
-    for (int i = 0; i < projectionMap.length; i++) {
-      columns[i] = reader.getValues(projectionMap[i]);
-    }
-
-    super.init();
-  }
-
-  private void prepareProjection(Column [] targets) {
-    projectionMap = new int[targets.length];
-    int tid;
-    for (int i = 0; i < targets.length; i++) {
-      tid = schema.getColumnIdByName(targets[i].getColumnName());
-      projectionMap[i] = tid;
-    }
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    Tuple tuple = new VTuple(schema.getColumnNum());
-
-    if (!columns[0].hasNext()) {
-      return null;
-    }
-
-    int tid; // column id of the original input schema
-    for (int i = 0; i < projectionMap.length; i++) {
-      tid = projectionMap[i];
-      columns[i].startRow();
-      DataType dataType = schema.getColumn(tid).getDataType();
-      switch (dataType.getType()) {
-        case BOOLEAN:
-          tuple.put(tid,
-              DatumFactory.createBool(((Integer)columns[i].nextValue()).byteValue()));
-          break;
-        case BIT:
-          tuple.put(tid,
-              DatumFactory.createBit(((Integer) columns[i].nextValue()).byteValue()));
-          break;
-        case CHAR:
-          String str = (String) columns[i].nextValue();
-          tuple.put(tid,
-              DatumFactory.createChar(str));
-          break;
-
-        case INT2:
-          tuple.put(tid,
-              DatumFactory.createInt2(((Integer) columns[i].nextValue()).shortValue()));
-          break;
-        case INT4:
-          tuple.put(tid,
-              DatumFactory.createInt4((Integer) columns[i].nextValue()));
-          break;
-
-        case INT8:
-          tuple.put(tid,
-              DatumFactory.createInt8((Long) columns[i].nextValue()));
-          break;
-
-        case FLOAT4:
-          tuple.put(tid,
-              DatumFactory.createFloat4((Float) columns[i].nextValue()));
-          break;
-
-        case FLOAT8:
-          tuple.put(tid,
-              DatumFactory.createFloat8((Double) columns[i].nextValue()));
-          break;
-
-        case INET4:
-          tuple.put(tid,
-              DatumFactory.createInet4(((ByteBuffer) columns[i].nextValue()).array()));
-          break;
-
-        case TEXT:
-          tuple.put(tid,
-              DatumFactory.createText((String) columns[i].nextValue()));
-          break;
-
-        case PROTOBUF: {
-          ProtobufDatumFactory factory = ProtobufDatumFactory.get(dataType.getCode());
-          Message.Builder builder = factory.newBuilder();
-          builder.mergeFrom(((ByteBuffer)columns[i].nextValue()).array());
-          tuple.put(tid, factory.createDatum(builder));
-          break;
-        }
-
-        case BLOB:
-          tuple.put(tid,
-              new BlobDatum(((ByteBuffer) columns[i].nextValue())));
-          break;
-
-        case NULL_TYPE:
-          tuple.put(tid, NullDatum.get());
-          break;
-
-        default:
-          throw new IOException("Unsupport data type");
-      }
-    }
-
-    return tuple;
-  }
-
-  @Override
-  public void reset() throws IOException {
-    for (int i = 0; i < projectionMap.length; i++) {
-      columns[i] = reader.getValues(projectionMap[i]);
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    reader.close();
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return true;
-  }
-
-
-  @Override
-  public boolean isSelectable() {
-    return false;
-  }
-
-  @Override
-  public boolean isSplittable(){
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.java
deleted file mode 100644
index b93672b..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.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.v2;
-
-import org.apache.commons.lang.ArrayUtils;
-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.Seekable;
-import org.apache.hadoop.io.compress.*;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.storage.LazyTuple;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.compress.CodecPool;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.util.Bytes;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-public class CSVFileScanner extends FileScannerV2 {
-  public static final String DELIMITER = "csvfile.delimiter";
-  public static final String DELIMITER_DEFAULT = "|";
-  public static final byte LF = '\n';
-  private static final Log LOG = LogFactory.getLog(CSVFileScanner.class);
-
-  private final static int DEFAULT_BUFFER_SIZE = 256 * 1024;
-  private int bufSize;
-  private char delimiter;
-  private ScheduledInputStream sin;
-  private InputStream is; // decompressd stream
-  private CompressionCodecFactory factory;
-  private CompressionCodec codec;
-  private Decompressor decompressor;
-  private Seekable filePosition;
-  private boolean splittable = true;
-  private long startOffset, length;
-  private byte[] buf = null;
-  private byte[][] tuples = null;
-  private long[] tupleOffsets = null;
-  private int currentIdx = 0, validIdx = 0;
-  private byte[] tail = null;
-  private long pageStart = -1;
-  private long prevTailLen = -1;
-  private int[] targetColumnIndexes;
-  private boolean eof = false;
-  private boolean first = true;
-
-  private long totalReadBytesForFetch;
-  private long totalReadBytesFromDisk;
-
-  public CSVFileScanner(Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment)
-      throws IOException {
-    super(conf, meta, schema, fragment);
-    factory = new CompressionCodecFactory(conf);
-    codec = factory.getCodec(this.fragment.getPath());
-    if (isCompress() && !(codec instanceof SplittableCompressionCodec)) {
-      splittable = false;
-    }
-  }
-
-  @Override
-  public void init() throws IOException {
-    // Buffer size, Delimiter
-    this.bufSize = DEFAULT_BUFFER_SIZE;
-    String delim  = meta.getOption(DELIMITER, DELIMITER_DEFAULT);
-    this.delimiter = delim.charAt(0);
-
-    super.init();
-  }
-
-  @Override
-  protected boolean initFirstScan(int maxBytesPerSchedule) throws IOException {
-    synchronized(this) {
-      eof = false;
-      first = true;
-      if(sin == null) {
-        FSDataInputStream fin = fs.open(fragment.getPath(), 128 * 1024);
-        sin = new ScheduledInputStream(fragment.getPath(), fin,
-            fragment.getStartKey(), fragment.getEndKey(), fs.getLength(fragment.getPath()));
-        startOffset = fragment.getStartKey();
-        length = fragment.getEndKey();
-
-        if (startOffset > 0) {
-          startOffset--; // prev line feed
-        }
-      }
-    }
-    return true;
-  }
-
-  private boolean scanFirst() throws IOException {
-    if (codec != null) {
-      decompressor = CodecPool.getDecompressor(codec);
-      if (codec instanceof SplittableCompressionCodec) {
-        SplitCompressionInputStream cIn = ((SplittableCompressionCodec) codec).createInputStream(
-            sin, decompressor, startOffset, startOffset + length,
-            SplittableCompressionCodec.READ_MODE.BYBLOCK);
-
-        startOffset = cIn.getAdjustedStart();
-        length = cIn.getAdjustedEnd() - startOffset;
-        filePosition = cIn;
-        is = cIn;
-      } else {
-        is = new DataInputStream(codec.createInputStream(sin, decompressor));
-      }
-    } else {
-      sin.seek(startOffset);
-      filePosition = sin;
-      is = sin;
-    }
-
-    tuples = new byte[0][];
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-
-    targetColumnIndexes = new int[targets.length];
-    for (int i = 0; i < targets.length; i++) {
-      targetColumnIndexes[i] = schema.getColumnIdByName(targets[i].getColumnName());
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("CSVScanner open:" + fragment.getPath() + "," + startOffset + "," + length +
-          "," + fs.getFileStatus(fragment.getPath()).getLen());
-    }
-
-    if (startOffset != 0) {
-      int rbyte;
-      while ((rbyte = is.read()) != LF) {
-        if(rbyte == -1) break;
-      }
-    }
-
-    if (fragmentable() < 1) {
-      close();
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public boolean isStopScanScheduling() {
-    if(sin != null && sin.isEndOfStream()) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  private long fragmentable() throws IOException {
-    return startOffset + length - getFilePosition();
-  }
-
-  @Override
-  protected long getFilePosition() throws IOException {
-    long retVal;
-    if (filePosition != null) {
-      retVal = filePosition.getPos();
-    } else {
-      retVal = sin.getPos();
-    }
-    return retVal;
-  }
-
-  @Override
-  public boolean isFetchProcessing() {
-    if(sin != null &&
-        (sin.getAvaliableSize() >= 64 * 1024 * 1024)) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  private void page() throws IOException {
-    // Index initialization
-    currentIdx = 0;
-
-    // Buffer size set
-    if (isSplittable() && fragmentable() < DEFAULT_BUFFER_SIZE) {
-      bufSize = (int) fragmentable();
-    }
-
-    if (this.tail == null || this.tail.length == 0) {
-      this.pageStart = getFilePosition();
-      this.prevTailLen = 0;
-    } else {
-      this.pageStart = getFilePosition() - this.tail.length;
-      this.prevTailLen = this.tail.length;
-    }
-
-    // Read
-    int rbyte;
-    buf = new byte[bufSize];
-    rbyte = is.read(buf);
-
-    if (rbyte < 0) {
-      eof = true; // EOF
-      return;
-    }
-
-    if (prevTailLen == 0) {
-      tail = new byte[0];
-      tuples = Bytes.splitPreserveAllTokens(buf, rbyte, (char) LF);
-    } else {
-      byte[] lastRow = ArrayUtils.addAll(tail, buf);
-      tuples = Bytes.splitPreserveAllTokens(lastRow, rbyte + tail.length, (char) LF);
-      tail = null;
-    }
-
-    // Check tail
-    if ((char) buf[rbyte - 1] != LF) {
-      if ((fragmentable() < 1 || rbyte != bufSize)) {
-        int lineFeedPos = 0;
-        byte[] temp = new byte[DEFAULT_BUFFER_SIZE];
-
-        // find line feed
-        while ((temp[lineFeedPos] = (byte)is.read()) != (byte)LF) {
-          if(temp[lineFeedPos] < 0) {
-            break;
-          }
-          lineFeedPos++;
-        }
-
-        tuples[tuples.length - 1] = ArrayUtils.addAll(tuples[tuples.length - 1],
-            ArrayUtils.subarray(temp, 0, lineFeedPos));
-        validIdx = tuples.length;
-      } else {
-        tail = tuples[tuples.length - 1];
-        validIdx = tuples.length - 1;
-      }
-    } else {
-      tail = new byte[0];
-      validIdx = tuples.length - 1;
-    }
-
-    if(!isCompress()) makeTupleOffset();
-  }
-
-  private void makeTupleOffset() {
-    long curTupleOffset = 0;
-    this.tupleOffsets = new long[this.validIdx];
-    for (int i = 0; i < this.validIdx; i++) {
-      this.tupleOffsets[i] = curTupleOffset + this.pageStart;
-      curTupleOffset += this.tuples[i].length + 1;//tuple byte +  1byte line feed
-    }
-  }
-
-  protected Tuple nextTuple() throws IOException {
-    if(first) {
-      boolean more = scanFirst();
-      first = false;
-      if(!more) {
-        return null;
-      }
-    }
-    try {
-      if (currentIdx == validIdx) {
-        if (isSplittable() && fragmentable() < 1) {
-          close();
-          return null;
-        } else {
-          page();
-        }
-
-        if(eof){
-          close();
-          return null;
-        }
-      }
-
-      long offset = -1;
-      if(!isCompress()){
-        offset = this.tupleOffsets[currentIdx];
-      }
-
-      byte[][] cells = Bytes.splitPreserveAllTokens(tuples[currentIdx++], delimiter, targetColumnIndexes);
-      return new LazyTuple(schema, cells, offset);
-    } catch (Throwable t) {
-      LOG.error(t.getMessage(), t);
-    }
-    return null;
-  }
-
-  private boolean isCompress() {
-    return codec != null;
-  }
-
-  @Override
-  public void scannerReset() {
-    if(sin != null) {
-      try {
-        filePosition.seek(0);
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-      }
-    }
-    if(sin != null) {
-      try {
-        sin.seek(0);
-        sin.reset();
-      } catch (IOException e) {
-        LOG.error(e.getMessage(), e);
-      }
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if(closed.get()) {
-      return;
-    }
-    if(sin != null) {
-      totalReadBytesForFetch = sin.getTotalReadBytesForFetch();
-      totalReadBytesFromDisk = sin.getTotalReadBytesFromDisk();
-    }
-    try {
-      if(is != null) {
-        is.close();
-      }
-      is = null;
-      sin = null;
-    } finally {
-      if (decompressor != null) {
-        CodecPool.returnDecompressor(decompressor);
-        decompressor = null;
-      }
-      tuples = null;
-      super.close();
-    }
-  }
-
-  @Override
-  protected boolean scanNext(int length) throws IOException {
-    synchronized(this) {
-      if(isClosed()) {
-        return false;
-      }
-      return sin.readNext(length);
-    }
-  }
-
-  @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
-  protected long[] reportReadBytes() {
-    return new long[]{totalReadBytesForFetch, totalReadBytesFromDisk};
-  }
-}


[03/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java
new file mode 100644
index 0000000..4b79c51
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFile.java
@@ -0,0 +1,1823 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+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.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.rcfile.*;
+import org.apache.tajo.util.Bytes;
+
+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>.
+ *
+ * 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 org.apache.tajo.storage.v2.RCFile.Writer}, {@link org.apache.tajo.storage.v2.RCFile.Reader} and classes for
+ * writing, reading respectively.
+ * </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>
+ * 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>
+ * The {@link org.apache.tajo.storage.v2.RCFile.Reader} is used to read and explain the bytes of RCFile.
+ * </p>
+ *
+ * <h4 id="Formats">RCFile Formats</h4>
+ *
+ *
+ * <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>
+ *
+ * <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>
+ *
+ */
+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";
+
+  public static final String COLUMN_NUMBER_CONF_STR = "hive.io.rcfile.column.number.conf";
+
+  public static final String TOLERATE_CORRUPTIONS_CONF_STR =
+    "hive.io.rcfile.tolerate.corruptions";
+
+  // HACK: We actually need BlockMissingException, but that is not available
+  // in all hadoop versions.
+  public static final String BLOCK_MISSING_MESSAGE =
+    "Could not obtain block";
+
+  // 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;
+
+  /**
+   * KeyBuffer is the key of each record in RCFile. Its on-disk layout is as
+   * below:
+   *
+   * <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 implements WritableComparable {
+    // 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 NonSyncDataOutputBuffer[] allCellValLenBuffer = null;
+    // how many rows in this split
+    private int numberRows = 0;
+    // how many columns
+    private int columnNumber = 0;
+
+    // return the number of columns recorded in this file's header
+    public int getColumnNumber() {
+      return columnNumber;
+    }
+
+    @SuppressWarnings("unused")
+    @Deprecated
+    public KeyBuffer(){
+    }
+
+    KeyBuffer(int columnNum) {
+      columnNumber = columnNum;
+      eachColumnValueLen = new int[columnNumber];
+      eachColumnUncompressedValueLen = new int[columnNumber];
+      allCellValLenBuffer = new NonSyncDataOutputBuffer[columnNumber];
+    }
+
+    @SuppressWarnings("unused")
+    @Deprecated
+    KeyBuffer(int numberRows, int columnNum) {
+      this(columnNum);
+      this.numberRows = numberRows;
+    }
+
+    /**
+     * add in a new column's meta data.
+     *
+     * @param columnValueLen
+     *          this total bytes number of this column's values in this split
+     * @param colValLenBuffer
+     *          each cell's length of this column's in this split
+     */
+    void setColumnLenInfo(int columnValueLen,
+        NonSyncDataOutputBuffer colValLenBuffer,
+        int columnUncompressedValueLen, int columnIndex) {
+      eachColumnValueLen[columnIndex] = columnValueLen;
+      eachColumnUncompressedValueLen[columnIndex] = columnUncompressedValueLen;
+      allCellValLenBuffer[columnIndex] = colValLenBuffer;
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      eachColumnValueLen = new int[columnNumber];
+      eachColumnUncompressedValueLen = new int[columnNumber];
+      allCellValLenBuffer = new NonSyncDataOutputBuffer[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 NonSyncDataOutputBuffer();
+        } else {
+          allCellValLenBuffer[i].reset();
+        }
+        allCellValLenBuffer[i].write(in, bufLen);
+      }
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      // out.writeInt(numberRows);
+      WritableUtils.writeVLong(out, numberRows);
+      for (int i = 0; i < eachColumnValueLen.length; i++) {
+        WritableUtils.writeVLong(out, eachColumnValueLen[i]);
+        WritableUtils.writeVLong(out, eachColumnUncompressedValueLen[i]);
+        NonSyncDataOutputBuffer colRowsLenBuf = allCellValLenBuffer[i];
+        int bufLen = colRowsLenBuf.getLength();
+        WritableUtils.writeVLong(out, bufLen);
+        out.write(colRowsLenBuf.getData(), 0, bufLen);
+      }
+    }
+
+    /**
+     * 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 getSize() throws IOException {
+      int ret = 0;
+      ret += WritableUtils.getVIntSize(numberRows);
+      for (int i = 0; i < eachColumnValueLen.length; i++) {
+        ret += WritableUtils.getVIntSize(eachColumnValueLen[i]);
+        ret += WritableUtils.getVIntSize(eachColumnUncompressedValueLen[i]);
+        ret += WritableUtils.getVIntSize(allCellValLenBuffer[i].getLength());
+        ret += allCellValLenBuffer[i].getLength();
+      }
+
+      return ret;
+    }
+
+    @Override
+    public int compareTo(Object arg0) {
+      throw new RuntimeException("compareTo not supported in class "
+          + this.getClass().getName());
+    }
+
+    @Override
+    public boolean equals(Object obj){
+      return super.equals(obj);
+    }
+
+    @Override
+    public int hashCode(){
+      return super.hashCode();
+    }
+  }
+
+  /**
+   * 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 WritableComparable {
+
+    class LazyDecompressionCallbackImpl implements LazyDecompressionCallback {
+
+      int index = -1;
+      int colIndex = -1;
+
+      public LazyDecompressionCallbackImpl(int index, int colIndex) {
+        super();
+        this.index = index;
+        this.colIndex = colIndex;
+      }
+
+      @Override
+      public byte[] decompress() throws IOException {
+
+        if (decompressedFlag[index] || codec == null) {
+          return loadedColumnsValueBuffer[index].getData();
+        }
+
+        NonSyncDataOutputBuffer compressedData = compressedColumnsValueBuffer[index];
+        decompressBuffer.reset();
+        DataInputStream valueIn = new DataInputStream(deflatFilter);
+        deflatFilter.resetState();
+        if (deflatFilter instanceof SchemaAwareCompressionInputStream) {
+          ((SchemaAwareCompressionInputStream)deflatFilter).setColumnIndex(colIndex);
+        }
+        decompressBuffer.reset(compressedData.getData(),
+            keyBuffer.eachColumnValueLen[colIndex]);
+
+        NonSyncDataOutputBuffer decompressedColBuf = loadedColumnsValueBuffer[index];
+        decompressedColBuf.reset();
+        decompressedColBuf.write(valueIn,
+            keyBuffer.eachColumnUncompressedValueLen[colIndex]);
+        decompressedFlag[index] = true;
+        numCompressed--;
+        return decompressedColBuf.getData();
+      }
+    }
+
+    // used to load columns' value into memory
+    private NonSyncDataOutputBuffer[] loadedColumnsValueBuffer = null;
+    private NonSyncDataOutputBuffer[] compressedColumnsValueBuffer = null;
+    private boolean[] decompressedFlag = null;
+    private int numCompressed;
+    private LazyDecompressionCallbackImpl[] lazyDecompressCallbackObjs = null;
+    private boolean lazyDecompress = true;
+
+    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 valDecompressor = null;
+    NonSyncDataInputBuffer decompressBuffer = new NonSyncDataInputBuffer();
+    CompressionInputStream deflatFilter = null;
+
+    @SuppressWarnings("unused")
+    @Deprecated
+    public ValueBuffer() throws IOException {
+    }
+
+    @SuppressWarnings("unused")
+    @Deprecated
+    public ValueBuffer(KeyBuffer keyBuffer) throws IOException {
+      this(keyBuffer, keyBuffer.columnNumber, null, null, true);
+    }
+
+    @SuppressWarnings("unused")
+    @Deprecated
+    public ValueBuffer(KeyBuffer keyBuffer, boolean[] skippedColIDs)
+        throws IOException {
+      this(keyBuffer, keyBuffer.columnNumber, skippedColIDs, null, true);
+    }
+
+    @SuppressWarnings("unused")
+    @Deprecated
+    public ValueBuffer(KeyBuffer currentKey, int columnNumber,
+        boolean[] skippedCols, CompressionCodec codec) throws IOException {
+      this(currentKey, columnNumber, skippedCols, codec, true);
+    }
+
+    public ValueBuffer(KeyBuffer currentKey, int columnNumber,
+      boolean[] skippedCols, CompressionCodec codec, boolean lazyDecompress)
+        throws IOException {
+      this.lazyDecompress = lazyDecompress;
+      keyBuffer = currentKey;
+      this.columnNumber = columnNumber;
+
+      if (skippedCols != null && skippedCols.length > 0) {
+        skippedColIDs = skippedCols;
+      } else {
+        skippedColIDs = new boolean[columnNumber];
+        for (int i = 0; i < skippedColIDs.length; i++) {
+          skippedColIDs[i] = false;
+        }
+      }
+
+      int skipped = 0;
+      for (boolean currentSkip : skippedColIDs) {
+        if (currentSkip) {
+          skipped++;
+        }
+      }
+      loadedColumnsValueBuffer = new NonSyncDataOutputBuffer[columnNumber
+          - skipped];
+      decompressedFlag = new boolean[columnNumber - skipped];
+      lazyDecompressCallbackObjs = new LazyDecompressionCallbackImpl[columnNumber
+          - skipped];
+      compressedColumnsValueBuffer = new NonSyncDataOutputBuffer[columnNumber
+                                                                 - skipped];
+      this.codec = codec;
+      if (codec != null) {
+        valDecompressor = CodecPool.getDecompressor(codec);
+        deflatFilter = codec.createInputStream(decompressBuffer,
+            valDecompressor);
+      }
+      if (codec != null) {
+        numCompressed = decompressedFlag.length;
+      } else {
+        numCompressed = 0;
+      }
+      for (int k = 0, readIndex = 0; k < columnNumber; k++) {
+        if (skippedColIDs[k]) {
+          continue;
+        }
+        loadedColumnsValueBuffer[readIndex] = new NonSyncDataOutputBuffer();
+        if (codec != null) {
+          decompressedFlag[readIndex] = false;
+          lazyDecompressCallbackObjs[readIndex] = new LazyDecompressionCallbackImpl(
+              readIndex, k);
+          compressedColumnsValueBuffer[readIndex] = new NonSyncDataOutputBuffer();
+        } else {
+          decompressedFlag[readIndex] = true;
+        }
+        readIndex++;
+      }
+    }
+
+    @SuppressWarnings("unused")
+    @Deprecated
+    public void setColumnValueBuffer(NonSyncDataOutputBuffer valBuffer,
+        int addIndex) {
+      loadedColumnsValueBuffer[addIndex] = valBuffer;
+    }
+
+    @Override
+    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) {
+          Bytes.skipFully(in, skipTotal);
+          skipTotal = 0;
+        }
+
+        NonSyncDataOutputBuffer valBuf;
+        if (codec != null){
+           // load into compressed buf first
+          valBuf = compressedColumnsValueBuffer[addIndex];
+        } else {
+          valBuf = loadedColumnsValueBuffer[addIndex];
+        }
+        valBuf.reset();
+        valBuf.write(in, vaRowsLen);
+        if (codec != null) {
+          decompressedFlag[addIndex] = false;
+          if (!lazyDecompress) {
+            lazyDecompressCallbackObjs[addIndex].decompress();
+            decompressedFlag[addIndex] = true;
+          }
+        }
+        addIndex++;
+      }
+      if (codec != null) {
+        numCompressed = decompressedFlag.length;
+      }
+
+      if (skipTotal != 0) {
+        Bytes.skipFully(in, skipTotal);
+      }
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      if (codec != null) {
+        for (NonSyncDataOutputBuffer currentBuf : compressedColumnsValueBuffer) {
+          out.write(currentBuf.getData(), 0, currentBuf.getLength());
+        }
+      } else {
+        for (NonSyncDataOutputBuffer currentBuf : loadedColumnsValueBuffer) {
+          out.write(currentBuf.getData(), 0, currentBuf.getLength());
+        }
+      }
+    }
+
+    public void clearColumnBuffer() throws IOException {
+      decompressBuffer.reset();
+    }
+
+    public void close() {
+      for (NonSyncDataOutputBuffer element : loadedColumnsValueBuffer) {
+        IOUtils.closeStream(element);
+      }
+      if (codec != null) {
+        IOUtils.closeStream(decompressBuffer);
+        CodecPool.returnDecompressor(valDecompressor);
+      }
+    }
+
+    @Override
+    public int compareTo(Object arg0) {
+      throw new RuntimeException("compareTo not supported in class "
+          + this.getClass().getName());
+    }
+
+    @Override
+    public boolean equals(Object obj){
+      return super.equals(obj);
+    }
+
+    @Override
+    public int hashCode(){
+      return super.hashCode();
+    }
+  }
+
+  /**
+   * 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 Writer {
+
+    Configuration conf;
+    FSDataOutputStream out;
+
+    CompressionCodec codec = null;
+    Metadata metadata = null;
+
+    // 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);
+      }
+    }
+
+    // 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 columnsBufferSize = 4 * 1024 * 1024; // 4M
+    // 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 final ColumnBuffer[] columnBuffers;
+
+    private int columnNumber = 0;
+
+    private final int[] columnValuePlainLength;
+
+    KeyBuffer key = null;
+    private final int[] plainTotalColumnLength;
+    private final int[] comprTotalColumnLength;
+
+    boolean useNewMagic = true;
+
+    /*
+     * used for buffering appends before flush them out
+     */
+    static class ColumnBuffer {
+      // used for buffer a column's values
+      NonSyncDataOutputBuffer columnValBuffer;
+      // used to store each value's length
+      NonSyncDataOutputBuffer 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 runLength = 0;
+      int prevValueLength = -1;
+
+      ColumnBuffer() throws IOException {
+        columnValBuffer = new NonSyncDataOutputBuffer();
+        valLenBuffer = new NonSyncDataOutputBuffer();
+      }
+
+      public void append(BytesRefWritable data) throws IOException {
+        data.writeDataTo(columnValBuffer);
+        int currentLen = data.getLength();
+
+        if (prevValueLength < 0) {
+          startNewGroup(currentLen);
+          return;
+        }
+
+        if (currentLen != prevValueLength) {
+          flushGroup();
+          startNewGroup(currentLen);
+        } else {
+          runLength++;
+        }
+      }
+
+      private void startNewGroup(int currentLen) {
+        prevValueLength = currentLen;
+        runLength = 0;
+      }
+
+      public void clear() throws IOException {
+        valLenBuffer.reset();
+        columnValBuffer.reset();
+        prevValueLength = -1;
+        runLength = 0;
+      }
+
+      public void flushGroup() throws IOException {
+        if (prevValueLength >= 0) {
+          WritableUtils.writeVLong(valLenBuffer, prevValueLength);
+          if (runLength > 0) {
+            WritableUtils.writeVLong(valLenBuffer, ~runLength);
+          }
+          runLength = -1;
+          prevValueLength = -1;
+        }
+      }
+    }
+
+    public long getLength() throws IOException {
+      return out.getPos();
+    }
+
+    /** Constructs a RCFile Writer. */
+    public Writer(FileSystem fs, Configuration conf, Path name) throws IOException {
+      this(fs, conf, name, null, new Metadata(), null);
+    }
+
+    /**
+     * Constructs a RCFile Writer.
+     *
+     * @param fs
+     *          the file system used
+     * @param conf
+     *          the configuration file
+     * @param name
+     *          the file name
+     * @throws java.io.IOException
+     */
+    public Writer(FileSystem fs, Configuration conf, Path name,
+        Progressable progress, CompressionCodec codec) throws IOException {
+      this(fs, conf, name, progress, new Metadata(), codec);
+    }
+
+    /**
+     * Constructs a RCFile Writer.
+     *
+     * @param fs
+     *          the file system used
+     * @param conf
+     *          the configuration file
+     * @param name
+     *          the file name
+     * @param progress a progress meter to update as the file is written
+     * @param metadata a string to string map in the file header
+     * @throws java.io.IOException
+     */
+    public Writer(FileSystem fs, Configuration conf, Path name,
+        Progressable progress, Metadata metadata, CompressionCodec codec) throws IOException {
+      this(fs, conf, name, fs.getConf().getInt("io.file.buffer.size", 4096),
+          fs.getDefaultReplication(), fs.getDefaultBlockSize(), progress,
+          metadata, codec);
+    }
+
+    /**
+     *
+     * Constructs a RCFile Writer.
+     *
+     * @param fs
+     *          the file system used
+     * @param conf
+     *          the configuration file
+     * @param name
+     *          the file name
+     * @param bufferSize the size of the file buffer
+     * @param replication the number of replicas for the file
+     * @param blockSize the block size of the file
+     * @param progress the progress meter for writing the file
+     * @param metadata a string to string map in the file header
+     * @throws java.io.IOException
+     */
+    public Writer(FileSystem fs, Configuration conf, Path name, int bufferSize,
+        short replication, long blockSize, Progressable progress,
+        Metadata metadata, CompressionCodec codec) throws IOException {
+      RECORD_INTERVAL = conf.getInt(RECORD_INTERVAL_CONF_STR, RECORD_INTERVAL);
+      columnNumber = conf.getInt(COLUMN_NUMBER_CONF_STR, 0);
+
+      if (metadata == null) {
+        metadata = new Metadata();
+      }
+      metadata.set(new Text(COLUMN_NUMBER_METADATA_STR), new Text(""
+          + columnNumber));
+
+      columnsBufferSize = conf.getInt(COLUMNS_BUFFER_SIZE_CONF_STR,
+          4 * 1024 * 1024);
+
+      columnValuePlainLength = new int[columnNumber];
+
+      columnBuffers = new ColumnBuffer[columnNumber];
+      for (int i = 0; i < columnNumber; i++) {
+        columnBuffers[i] = new ColumnBuffer();
+      }
+
+      init(conf, fs.create(name, true, bufferSize, replication,
+        blockSize, progress), codec, metadata);
+      initializeFileHeader();
+      writeFileHeader();
+      finalizeFileHeader();
+      key = new KeyBuffer(columnNumber);
+
+      plainTotalColumnLength = new int[columnNumber];
+      comprTotalColumnLength = new int[columnNumber];
+    }
+
+    /** 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, KeyBuffer.class.getName());
+        Text.writeString(out, ValueBuffer.class.getName());
+        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.conf = conf;
+      this.out = out;
+      this.codec = codec;
+      this.metadata = metadata;
+      this.useNewMagic =
+          conf.getBoolean(TajoConf.ConfVars.HIVEUSEEXPLICITRCFILEHEADER.varname, true);
+    }
+
+    /** Returns the compression codec of data in this file. */
+    @SuppressWarnings("unused")
+    @Deprecated
+    public CompressionCodec getCompressionCodec() {
+      return codec;
+    }
+
+    /** 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
+      }
+    }
+
+    /** Returns the configuration of this file. */
+    @SuppressWarnings("unused")
+    @Deprecated
+    Configuration getConf() {
+      return conf;
+    }
+
+    private void checkAndWriteSync() throws IOException {
+      if (sync != null && out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
+        sync();
+      }
+    }
+
+    private int columnBufferSize = 0;
+
+    /**
+     * Append a row of values. Currently it only can accept <
+     * {@link BytesRefArrayWritable}. 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 val a BytesRefArrayWritable with the list of serialized columns
+     * @throws java.io.IOException
+     */
+    public void append(Writable val) throws IOException {
+
+      if (!(val instanceof BytesRefArrayWritable)) {
+        throw new UnsupportedOperationException(
+            "Currently the writer can only accept BytesRefArrayWritable");
+      }
+
+      BytesRefArrayWritable columns = (BytesRefArrayWritable) val;
+      int size = columns.size();
+      for (int i = 0; i < size; i++) {
+        BytesRefWritable cu = columns.get(i);
+        int plainLen = cu.getLength();
+        columnBufferSize += plainLen;
+        columnValuePlainLength[i] += plainLen;
+        columnBuffers[i].append(cu);
+      }
+
+      if (size < columnNumber) {
+        for (int i = columns.size(); i < columnNumber; i++) {
+          columnBuffers[i].append(BytesRefWritable.ZeroBytesRefWritable);
+        }
+      }
+
+      bufferedRecords++;
+      if ((columnBufferSize > columnsBufferSize)
+          || (bufferedRecords >= RECORD_INTERVAL)) {
+        flushRecords();
+      }
+    }
+
+    private void flushRecords() throws IOException {
+
+      key.numberRows = bufferedRecords;
+
+      Compressor compressor = null;
+      NonSyncDataOutputBuffer valueBuffer = null;
+      CompressionOutputStream deflateFilter = null;
+      DataOutputStream deflateOut = null;
+      boolean isCompressed = isCompressed();
+      int valueLength = 0;
+      if (isCompressed) {
+        ReflectionUtils.setConf(codec, this.conf);
+        compressor = CodecPool.getCompressor(codec);
+        valueBuffer = new NonSyncDataOutputBuffer();
+        deflateFilter = codec.createOutputStream(valueBuffer, compressor);
+        deflateOut = new DataOutputStream(deflateFilter);
+      }
+
+      try {
+        for (int columnIndex = 0; columnIndex < columnNumber; columnIndex++) {
+          ColumnBuffer currentBuf = columnBuffers[columnIndex];
+          currentBuf.flushGroup();
+
+          NonSyncDataOutputBuffer columnValue = currentBuf.columnValBuffer;
+          int colLen;
+          int plainLen = columnValuePlainLength[columnIndex];
+
+          if (isCompressed) {
+            if (deflateFilter instanceof SchemaAwareCompressionOutputStream) {
+              ((SchemaAwareCompressionOutputStream) deflateFilter).
+                  setColumnIndex(columnIndex);
+            }
+            deflateFilter.resetState();
+            deflateOut.write(columnValue.getData(), 0, columnValue.getLength());
+            deflateOut.flush();
+            deflateFilter.finish();
+            // find how much compressed data was added for this column
+            colLen = valueBuffer.getLength() - valueLength;
+          } else {
+            colLen = columnValuePlainLength[columnIndex];
+          }
+          valueLength += colLen;
+          key.setColumnLenInfo(colLen, currentBuf.valLenBuffer, plainLen,
+              columnIndex);
+          plainTotalColumnLength[columnIndex] += plainLen;
+          comprTotalColumnLength[columnIndex] += colLen;
+          columnValuePlainLength[columnIndex] = 0;
+        }
+      } catch (IOException e) {
+        IOUtils.cleanup(LOG, deflateOut);
+        throw e;
+      }
+
+      int keyLength = key.getSize();
+      if (keyLength < 0) {
+        throw new IOException("negative length keys not allowed: " + key);
+      }
+      if (compressor != null) {
+        CodecPool.returnCompressor(compressor);
+      }
+
+      // Write the key out
+      writeKey(key, keyLength + valueLength, keyLength);
+      // write the value out
+      if (isCompressed) {
+        out.write(valueBuffer.getData(), 0, valueBuffer.getLength());
+      } else {
+        for(int columnIndex=0; columnIndex < columnNumber; ++columnIndex) {
+          NonSyncDataOutputBuffer buf =
+            columnBuffers[columnIndex].columnValBuffer;
+          out.write(buf.getData(), 0, buf.getLength());
+        }
+      }
+
+      // clear the columnBuffers
+      clearColumnBuffers();
+
+      bufferedRecords = 0;
+      columnBufferSize = 0;
+    }
+
+    /**
+     * flush a block out without doing anything except compressing the key part.
+     */
+    public void flushBlock(KeyBuffer keyBuffer, ValueBuffer valueBuffer,
+        int recordLen, int keyLength,
+        @SuppressWarnings("unused") int compressedKeyLen) throws IOException {
+      writeKey(keyBuffer, recordLen, keyLength);
+      valueBuffer.write(out);
+    }
+
+    private void writeKey(KeyBuffer keyBuffer, 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 = CodecPool.getCompressor(codec);
+        NonSyncDataOutputBuffer compressionBuffer =
+          new NonSyncDataOutputBuffer();
+        CompressionOutputStream deflateFilter =
+          codec.createOutputStream(compressionBuffer, compressor);
+        DataOutputStream deflateOut = new DataOutputStream(deflateFilter);
+        //compress key and write key out
+        compressionBuffer.reset();
+        deflateFilter.resetState();
+        keyBuffer.write(deflateOut);
+        deflateOut.flush();
+        deflateFilter.finish();
+        int compressedKeyLen = compressionBuffer.getLength();
+        out.writeInt(compressedKeyLen);
+        out.write(compressionBuffer.getData(), 0, compressedKeyLen);
+        CodecPool.returnCompressor(compressor);
+      } else {
+        out.writeInt(keyLength);
+        keyBuffer.write(out);
+      }
+    }
+
+    private void clearColumnBuffers() throws IOException {
+      for (int i = 0; i < columnNumber; i++) {
+        columnBuffers[i].clear();
+      }
+    }
+
+    public void close() throws IOException {
+      if (bufferedRecords > 0) {
+        flushRecords();
+      }
+      clearColumnBuffers();
+
+      if (out != null) {
+
+        // Close the underlying stream if we own it...
+        out.flush();
+        out.close();
+        out = null;
+      }
+      for (int i = 0; i < columnNumber; i++) {
+        LOG.info("Column#" + i + " : Plain Total Column Value Length: "
+          + plainTotalColumnLength[i]
+          + ",  Compr Total Column Value Length: " + comprTotalColumnLength[i]);
+      }
+    }
+  }
+
+  /**
+   * Read KeyBuffer/ValueBuffer pairs from a RCFile.
+   *
+   */
+  public static class Reader {
+    private static class SelectedColumn {
+      public int colIndex;
+      public int rowReadIndex;
+      public int runLength;
+      public int prvLength;
+    }
+    private final Path file;
+//    private final FSDataInputStream in;
+
+    private byte version;
+
+    private CompressionCodec codec = null;
+    private Metadata metadata = null;
+
+    private final byte[] sync = new byte[SYNC_HASH_SIZE];
+    private final byte[] syncCheck = new byte[SYNC_HASH_SIZE];
+    private boolean syncSeen;
+    private long lastSeenSyncPos = 0;
+
+    private long headerEnd;
+    private final long end;
+    private int currentKeyLength;
+    private int currentRecordLength;
+
+    private final Configuration conf;
+
+    private final ValueBuffer currentValue;
+
+    private int readRowsIndexInBuffer = 0;
+
+    private int recordsNumInValBuffer = 0;
+
+    private int columnNumber = 0;
+
+    private int loadColumnNum;
+
+    private int passedRowsNum = 0;
+
+    // Should we try to tolerate corruption? Default is No.
+    private boolean tolerateCorruptions = false;
+
+    private boolean decompress = false;
+
+    private Decompressor keyDecompressor;
+    NonSyncDataOutputBuffer keyDecompressedData = new NonSyncDataOutputBuffer();
+
+    //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 final SelectedColumn[] selectedColumns;
+
+    // map of original column id -> index among selected columns
+    private final int[] revPrjColIDs;
+
+    // column value lengths for each of the selected columns
+    private final NonSyncDataInputBuffer[] colValLenBufferReadIn;
+
+    private ScheduledInputStream sin;
+
+    public Reader(Path file, ScheduledInputStream sin, FileSystem fs, Configuration conf) throws IOException {
+        this(file, sin, conf.getInt("io.file.buffer.size", 4096), conf, 0, fs.getFileStatus(file).getLen());
+    }
+    /** Create a new RCFile reader. */
+    public Reader(Path file, ScheduledInputStream sin, int bufferSize, Configuration conf,
+        long start, long length) throws IOException {
+      tolerateCorruptions = conf.getBoolean(
+        TOLERATE_CORRUPTIONS_CONF_STR, false);
+      conf.setInt("io.file.buffer.size", bufferSize);
+//      in = openFile(fs, file, bufferSize, length);
+      this.file = file;
+      this.sin = sin;
+      this.conf = conf;
+      end = start + length;
+      boolean succeed = false;
+      try {
+        if (start > 0) {
+          seek(0);
+          init();
+          seek(start);
+        } else {
+          seek(0);
+          init();
+        }
+        succeed = true;
+      } finally {
+        if (!succeed) {
+          if (sin != null) {
+            try {
+              sin.close();
+            } catch(IOException e) {
+              if (LOG != null && LOG.isDebugEnabled()) {
+                LOG.debug("Exception in closing " + sin, e);
+              }
+            }
+          }
+        }
+      }
+
+      columnNumber = Integer.parseInt(metadata.get(
+          new Text(COLUMN_NUMBER_METADATA_STR)).toString());
+
+      java.util.ArrayList<Integer> notSkipIDs = ColumnProjectionUtils
+          .getReadColumnIDs(conf);
+      boolean[] skippedColIDs = new boolean[columnNumber];
+      if (notSkipIDs.size() > 0) {
+        for (int i = 0; i < skippedColIDs.length; i++) {
+          skippedColIDs[i] = true;
+        }
+        for (int read : notSkipIDs) {
+          if (read < columnNumber) {
+            skippedColIDs[read] = false;
+          }
+        }
+      } else {
+        // TODO: if no column name is specified e.g, in select count(1) from tt;
+        // skip all columns, this should be distinguished from the case:
+        // select * from tt;
+        for (int i = 0; i < skippedColIDs.length; i++) {
+          skippedColIDs[i] = false;
+        }
+      }
+
+      loadColumnNum = columnNumber;
+      if (skippedColIDs.length > 0) {
+        for (boolean skippedColID : skippedColIDs) {
+          if (skippedColID) {
+            loadColumnNum -= 1;
+          }
+        }
+      }
+
+
+      revPrjColIDs = new int[columnNumber];
+      // get list of selected column IDs
+      selectedColumns = new SelectedColumn[loadColumnNum];
+      colValLenBufferReadIn = new NonSyncDataInputBuffer[loadColumnNum];
+      for (int i = 0, j = 0; i < columnNumber; ++i) {
+        if (!skippedColIDs[i]) {
+          SelectedColumn col = new SelectedColumn();
+          col.colIndex = i;
+          col.runLength = 0;
+          col.prvLength = -1;
+          col.rowReadIndex = 0;
+          selectedColumns[j] = col;
+          colValLenBufferReadIn[j] = new NonSyncDataInputBuffer();
+          revPrjColIDs[i] = j;
+          j++;
+        } else {
+          revPrjColIDs[i] = -1;
+        }
+      }
+
+      currentKey = createKeyBuffer();
+      boolean lazyDecompress = !tolerateCorruptions;
+      currentValue = new ValueBuffer(
+        null, columnNumber, skippedColIDs, codec, lazyDecompress);
+    }
+
+    /**
+     * 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, long length) throws IOException {
+      return fs.open(file, bufferSize);
+    }
+
+    private void init() throws IOException {
+      byte[] magic = new byte[MAGIC.length];
+      sin.readFully(magic);
+
+      if (Arrays.equals(magic, ORIGINAL_MAGIC)) {
+        byte vers = sin.readByte();
+        if (vers != ORIGINAL_MAGIC_VERSION_WITH_METADATA) {
+          throw new IOException(file + " is a version " + vers +
+                                " SequenceFile instead of an RCFile.");
+        }
+        version = ORIGINAL_VERSION;
+      } else {
+        if (!Arrays.equals(magic, MAGIC)) {
+          throw new IOException(file + " not a RCFile and has magic of " +
+                                new String(magic));
+        }
+
+        // Set 'version'
+        version = sin.readByte();
+        if (version > CURRENT_VERSION) {
+          throw new VersionMismatchException((byte) CURRENT_VERSION, version);
+        }
+      }
+
+      if (version == ORIGINAL_VERSION) {
+        try {
+          Class<?> keyCls = conf.getClassByName(Text.readString(sin));
+          Class<?> valCls = conf.getClassByName(Text.readString(sin));
+          if (!keyCls.equals(KeyBuffer.class)
+              || !valCls.equals(ValueBuffer.class)) {
+            throw new IOException(file + " not a RCFile");
+          }
+        } catch (ClassNotFoundException e) {
+          throw new IOException(file + " not a RCFile", e);
+        }
+      }
+
+      decompress = sin.readBoolean(); // is compressed?
+
+      if (version == ORIGINAL_VERSION) {
+        // is block-compressed? it should be always false.
+        boolean blkCompressed = sin.readBoolean();
+        if (blkCompressed) {
+          throw new IOException(file + " not a RCFile.");
+        }
+      }
+
+      // setup the compression codec
+      if (decompress) {
+        String codecClassname = Text.readString(sin);
+        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 = CodecPool.getDecompressor(codec);
+      }
+
+      metadata = new Metadata();
+      metadata.readFields(sin);
+
+      sin.readFully(sync); // read sync bytes
+      headerEnd = sin.getPos();
+    }
+
+    /** Return the current byte position in the input file. */
+    public long getPosition() throws IOException {
+      return sin.getPos();
+    }
+
+    /**
+     * Set the current byte position in the input file.
+     *
+     * <p>
+     * The position passed must be a position returned by
+     * {@link org.apache.tajo.storage.v2.RCFile.Writer#getLength()} when writing this file. To seek to an
+     * arbitrary position, use {@link org.apache.tajo.storage.v2.RCFile.Reader#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.v2.RCFile.Reader#sync(long)}.
+     */
+    public void seek(long position) throws IOException {
+      sin.seek(position);
+      sin.readNext(128 * 1024);
+    }
+
+    /**
+     * Resets the values which determine if there are more rows in the buffer
+     *
+     * 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
+        sin.seek(headerEnd);
+        // note the sync marker "seen" in the header
+        syncSeen = true;
+        return;
+      }
+
+      try {
+//        if(sin.getAvaliableSize() < 64 * 1024) {
+//      	  sin.readNext(256 * 1024);
+//        }
+        seek(position + 4); // skip escape
+        sin.readFully(syncCheck);
+        int syncLen = sync.length;
+        for (int i = 0; sin.getPos() < end; i++) {
+          int j = 0;
+          for (; j < syncLen; j++) {
+            if (sync[j] != syncCheck[(i + j) % syncLen]) {
+              break;
+            }
+          }
+          if (j == syncLen) {
+        	sin.seek(sin.getPos() - SYNC_SIZE); // position before
+            // sync
+            return;
+          }
+//          if(sin.getAvaliableSize() < 64 * 1024) {
+//        	  sin.readNext(256 * 1024);
+//          }
+          syncCheck[i % syncLen] = sin.readByte();
+        }
+      } 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 (sin.getPos() >= end) {
+        return -1;
+      }
+      int length = sin.readInt();
+      if (sync != null && length == SYNC_ESCAPE) { // process
+        // a
+        // sync entry
+        lastSeenSyncPos = sin.getPos() - 4; // minus SYNC_ESCAPE's length
+        sin.readFully(syncCheck); // read syncCheck
+        if (!Arrays.equals(sync, syncCheck)) {
+          throw new IOException("File is corrupt!");
+        }
+        syncSeen = true;
+        if (sin.getPos() >= end) {
+          return -1;
+        }
+        length = sin.readInt(); // re-read length
+      } else {
+        syncSeen = false;
+      }
+      return length;
+    }
+
+    private void seekToNextKeyBuffer() throws IOException {
+      if (!keyInit) {
+        return;
+      }
+      if (!currentValue.inited) {
+        IOUtils.skipFully(sin, currentRecordLength - currentKeyLength);
+      }
+    }
+
+    private int compressedKeyLen = 0;
+    NonSyncDataInputBuffer keyDataIn = new NonSyncDataInputBuffer();
+    NonSyncDataInputBuffer keyDecompressBuffer = new NonSyncDataInputBuffer();
+    NonSyncDataOutputBuffer keyTempBuffer = new NonSyncDataOutputBuffer();
+
+    KeyBuffer currentKey = null;
+    boolean keyInit = false;
+
+    protected int nextKeyBuffer() throws IOException {
+      seekToNextKeyBuffer();
+      currentRecordLength = readRecordLength();
+      if (currentRecordLength == -1) {
+        keyInit = false;
+        return -1;
+      }
+      
+      currentKeyLength = sin.readInt();
+      compressedKeyLen = sin.readInt();
+      
+//      System.out.println(">>>currentRecordLength=" + currentRecordLength + 
+//    		  ",currentKeyLength=" + currentKeyLength + 
+//    		  ",compressedKeyLen=" + compressedKeyLen + 
+//    		  ",decompress=" + decompress);
+      
+      if (decompress) {
+        keyTempBuffer.reset();
+        keyTempBuffer.write(sin, compressedKeyLen);
+        keyDecompressBuffer.reset(keyTempBuffer.getData(), compressedKeyLen);
+        CompressionInputStream deflatFilter = codec.createInputStream(
+            keyDecompressBuffer, keyDecompressor);
+        DataInputStream compressedIn = new DataInputStream(deflatFilter);
+        deflatFilter.resetState();
+        keyDecompressedData.reset();
+        keyDecompressedData.write(compressedIn, currentKeyLength);
+        keyDataIn.reset(keyDecompressedData.getData(), currentKeyLength);
+        currentKey.readFields(keyDataIn);
+      } else {
+        currentKey.readFields(sin);
+      }
+
+      keyInit = true;
+      currentValue.inited = false;
+
+      readRowsIndexInBuffer = 0;
+      recordsNumInValBuffer = currentKey.numberRows;
+
+      for (int selIx = 0; selIx < selectedColumns.length; selIx++) {
+        SelectedColumn col = selectedColumns[selIx];
+        int colIx = col.colIndex;
+        NonSyncDataOutputBuffer buf = currentKey.allCellValLenBuffer[colIx];
+        colValLenBufferReadIn[selIx].reset(buf.getData(), buf.getLength());
+        col.rowReadIndex = 0;
+        col.runLength = 0;
+        col.prvLength = -1;
+      }
+
+      return currentKeyLength;
+    }
+
+    protected void currentValueBuffer() throws IOException {
+      if (!keyInit) {
+        nextKeyBuffer();
+      }
+      currentValue.keyBuffer = currentKey;
+      currentValue.clearColumnBuffer();
+      currentValue.readFields(sin);
+      currentValue.inited = true;
+    }
+
+    public boolean nextBlock() throws IOException {
+      int keyLength = nextKeyBuffer();
+      if(keyLength > 0) {
+        currentValueBuffer();
+        return true;
+      }
+      return false;
+    }
+
+    private boolean rowFetched = false;
+
+    // use this buffer to hold column's cells value length for usages in
+    // getColumn(), instead of using colValLenBufferReadIn directly.
+    private final NonSyncDataInputBuffer fetchColumnTempBuf = new NonSyncDataInputBuffer();
+
+    /**
+     * Fetch all data in the buffer for a given column. This is useful for
+     * columnar operators, which perform operations on an array data of one
+     * column. It should be used together with {@link #nextColumnsBatch()}.
+     * Calling getColumn() with not change the result of
+     * {@link #next(org.apache.hadoop.io.LongWritable)} and
+     * {@link #getCurrentRow(BytesRefArrayWritable)}.
+     *
+     * @param columnID the number of the column to get 0 to N-1
+     * @throws java.io.IOException
+     */
+    public BytesRefArrayWritable getColumn(int columnID,
+        BytesRefArrayWritable rest) throws IOException {
+      int selColIdx = revPrjColIDs[columnID];
+      if (selColIdx == -1) {
+        return null;
+      }
+
+      if (rest == null) {
+        rest = new BytesRefArrayWritable();
+      }
+
+      rest.resetValid(recordsNumInValBuffer);
+
+      if (!currentValue.inited) {
+        currentValueBuffer();
+      }
+
+      int columnNextRowStart = 0;
+      fetchColumnTempBuf.reset(currentKey.allCellValLenBuffer[columnID]
+          .getData(), currentKey.allCellValLenBuffer[columnID].getLength());
+      SelectedColumn selCol = selectedColumns[selColIdx];
+      byte[] uncompData = null;
+      ValueBuffer.LazyDecompressionCallbackImpl decompCallBack = null;
+      boolean decompressed = currentValue.decompressedFlag[selColIdx];
+      if (decompressed) {
+        uncompData =
+              currentValue.loadedColumnsValueBuffer[selColIdx].getData();
+      } else {
+        decompCallBack = currentValue.lazyDecompressCallbackObjs[selColIdx];
+      }
+      for (int i = 0; i < recordsNumInValBuffer; i++) {
+        colAdvanceRow(selColIdx, selCol);
+        int length = selCol.prvLength;
+
+        BytesRefWritable currentCell = rest.get(i);
+
+        if (decompressed) {
+          currentCell.set(uncompData, columnNextRowStart, length);
+        } else {
+          currentCell.set(decompCallBack, columnNextRowStart, length);
+        }
+        columnNextRowStart = columnNextRowStart + length;
+      }
+      return rest;
+    }
+
+    /**
+     * Read in next key buffer and throw any data in current key buffer and
+     * current value buffer. It will influence the result of
+     * {@link #next(org.apache.hadoop.io.LongWritable)} and
+     * {@link #getCurrentRow(BytesRefArrayWritable)}
+     *
+     * @return whether there still has records or not
+     * @throws java.io.IOException
+     */
+    @SuppressWarnings("unused")
+    @Deprecated
+    public boolean nextColumnsBatch() throws IOException {
+      passedRowsNum += (recordsNumInValBuffer - readRowsIndexInBuffer);
+      return nextKeyBuffer() > 0;
+    }
+
+    /**
+     * Returns how many rows we fetched with next(). It only means how many rows
+     * are read by next(). The returned result may be smaller than actual number
+     * of rows passed by, because {@link #seek(long)},
+     * {@link #nextColumnsBatch()} can change the underlying key buffer and
+     * value buffer.
+     *
+     * @return next row number
+     * @throws java.io.IOException
+     */
+    public boolean next(LongWritable readRows) throws IOException {
+      if (hasRecordsInBuffer()) {
+        readRows.set(passedRowsNum);
+        readRowsIndexInBuffer++;
+        passedRowsNum++;
+        rowFetched = false;
+        return true;
+      } else {
+        keyInit = false;
+      }
+
+      int ret = -1;
+      if (tolerateCorruptions) {
+        ret = nextKeyValueTolerateCorruptions();
+      } else {
+        try {
+          ret = nextKeyBuffer();
+        } catch (EOFException eof) {
+          eof.printStackTrace();
+        }
+      }
+      return (ret > 0) && next(readRows);
+    }
+
+    private int nextKeyValueTolerateCorruptions() throws IOException {
+      long currentOffset = sin.getPos();
+      int ret;
+      try {
+        ret = nextKeyBuffer();
+        this.currentValueBuffer();
+      } catch (IOException ioe) {
+        // A BlockMissingException indicates a temporary error,
+        // not a corruption. Re-throw this exception.
+        String msg = ioe.getMessage();
+        if (msg != null && msg.startsWith(BLOCK_MISSING_MESSAGE)) {
+          LOG.warn("Re-throwing block-missing exception" + ioe);
+          throw ioe;
+        }
+        // We have an IOException other than a BlockMissingException.
+        LOG.warn("Ignoring IOException in file " + file +
+                 " after offset " + currentOffset, ioe);
+        ret = -1;
+      } catch (Throwable t) {
+        // We got an exception that is not IOException
+        // (typically OOM, IndexOutOfBounds, InternalError).
+        // This is most likely a corruption.
+        LOG.warn("Ignoring unknown error in " + file +
+                 " after offset " + currentOffset, t);
+        ret = -1;
+      }
+      return ret;
+    }
+
+    public boolean hasRecordsInBuffer() {
+      return readRowsIndexInBuffer < recordsNumInValBuffer;
+    }
+
+    /**
+     * get the current row used,make sure called {@link #next(org.apache.hadoop.io.LongWritable)}
+     * first.
+     *
+     * @throws java.io.IOException
+     */
+    public void getCurrentRow(BytesRefArrayWritable ret) throws IOException {
+
+      if (!keyInit || rowFetched) {
+        return;
+      }
+
+      if (tolerateCorruptions) {
+        if (!currentValue.inited) {
+          currentValueBuffer();
+        }
+        ret.resetValid(columnNumber);
+      } else {
+        if (!currentValue.inited) {
+          currentValueBuffer();
+          // do this only when not initialized, but we may need to find a way to
+          // tell the caller how to initialize the valid size
+          ret.resetValid(columnNumber);
+        }
+      }
+
+      // we do not use BytesWritable here to avoid the byte-copy from
+      // DataOutputStream to BytesWritable
+      if (currentValue.numCompressed > 0) {
+        for (int j = 0; j < selectedColumns.length; ++j) {
+          SelectedColumn col = selectedColumns[j];
+          int i = col.colIndex;
+
+          BytesRefWritable ref = ret.unCheckedGet(i);
+
+          colAdvanceRow(j, col);
+
+          if (currentValue.decompressedFlag[j]) {
+            ref.set(currentValue.loadedColumnsValueBuffer[j].getData(),
+                col.rowReadIndex, col.prvLength);
+          } else {
+            ref.set(currentValue.lazyDecompressCallbackObjs[j],
+                col.rowReadIndex, col.prvLength);
+          }
+          col.rowReadIndex += col.prvLength;
+        }
+      } else {
+        // This version of the loop eliminates a condition check and branch
+        // and is measurably faster (20% or so)
+        for (int j = 0; j < selectedColumns.length; ++j) {
+          SelectedColumn col = selectedColumns[j];
+          int i = col.colIndex;
+
+          BytesRefWritable ref = ret.unCheckedGet(i);
+
+          colAdvanceRow(j, col);
+          ref.set(currentValue.loadedColumnsValueBuffer[j].getData(),
+                col.rowReadIndex, col.prvLength);
+          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 iff the previous call to next passed a sync mark. */
+    @SuppressWarnings("unused")
+    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 file.toString();
+    }
+
+    @SuppressWarnings("unused")
+    public boolean isCompressedRCFile() {
+      return this.decompress;
+    }
+
+    /** Close the reader. */
+    public void close() {
+      IOUtils.closeStream(sin);
+      currentValue.close();
+      if (decompress) {
+        IOUtils.closeStream(keyDecompressedData);
+        CodecPool.returnDecompressor(keyDecompressor);
+      }
+    }
+
+    /**
+     * return the KeyBuffer object used in the reader. Internally in each
+     * reader, there is only one KeyBuffer object, which gets reused for every
+     * block.
+     */
+    public KeyBuffer getCurrentKeyBufferObj() {
+      return this.currentKey;
+    }
+
+    /**
+     * return the ValueBuffer object used in the reader. Internally in each
+     * reader, there is only one ValueBuffer object, which gets reused for every
+     * block.
+     */
+    public ValueBuffer getCurrentValueBufferObj() {
+      return this.currentValue;
+    }
+
+    //return the current block's length
+    public int getCurrentBlockLength() {
+      return this.currentRecordLength;
+    }
+
+    //return the current block's key length
+    public int getCurrentKeyLength() {
+      return this.currentKeyLength;
+    }
+
+    //return the current block's compressed key length
+    public int getCurrentCompressedKeyLen() {
+      return this.compressedKeyLen;
+    }
+
+    //return the CompressionCodec used for this file
+    public CompressionCodec getCompressionCodec() {
+      return this.codec;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
new file mode 100644
index 0000000..a549a51
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/RCFileScanner.java
@@ -0,0 +1,297 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+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.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.*;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.rcfile.BytesRefArrayWritable;
+import org.apache.tajo.storage.rcfile.ColumnProjectionUtils;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+public class RCFileScanner extends FileScannerV2 {
+  private static final Log LOG = LogFactory.getLog(RCFileScanner.class);
+  public static final String SERDE = "rcfile.serde";
+  public static final String NULL = "rcfile.null";
+
+  private RCFile.Reader in;
+  private long start;
+  private long end;
+  private boolean more = true;
+  private LongWritable key;
+  private BytesRefArrayWritable column;
+  private Integer [] projectionMap;
+  private ScheduledInputStream sin;
+  private boolean first = true;
+  private int maxBytesPerSchedule;
+  private SerializerDeserializer serde;
+  private byte[] nullChars;
+  private Object lock = new Object();
+
+  public RCFileScanner(final Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment)
+      throws IOException {
+    super(conf, meta, schema, fragment);
+
+    this.start = fragment.getStartKey();
+    this.end = start + fragment.getEndKey();
+    key = new LongWritable();
+    column = new BytesRefArrayWritable();
+
+    String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(NULL));
+    if (StringUtils.isEmpty(nullCharacters)) {
+      nullChars = NullDatum.get().asTextBytes();
+    } else {
+      nullChars = nullCharacters.getBytes();
+    }
+	}
+
+  @Override
+  protected Tuple nextTuple() throws IOException {
+    if(first) {
+      first = false;
+      if (start > in.getPosition()) {
+        in.sync(start); // sync to start
+      }
+      this.start = in.getPosition();
+      more = start < end;
+      if(!more) {
+        return null;
+      }
+    }
+
+    more = next(key);
+
+    if (more) {
+      column.clear();
+      in.getCurrentRow(column);
+    }
+
+    if(more) {
+      Tuple tuple = makeTuple();
+      return tuple;
+    } else {
+      close();
+      return null;
+    }
+  }
+
+  private Tuple makeTuple() throws IOException {
+    Tuple tuple = new VTuple(schema.getColumnNum());
+    synchronized (lock) {
+      column.resetValid(schema.getColumnNum());
+      int tid; // target column id
+      for (int i = 0; i < projectionMap.length; i++) {
+        tid = projectionMap[i];
+
+        byte[] bytes = column.get(tid).getBytesCopy();
+        Datum datum = serde.deserialize(targets[i], bytes, 0, bytes.length, nullChars);
+        tuple.put(tid, datum);
+      }
+    }
+    return tuple;
+  }
+
+  @Override
+  public void init() throws IOException {
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+
+    prepareProjection(targets);
+
+    super.init();
+  }
+
+  private void prepareProjection(Column[] targets) {
+    projectionMap = new Integer[targets.length];
+    int tid;
+    for (int i = 0; i < targets.length; i++) {
+      tid = schema.getColumnIdByName(targets[i].getColumnName());
+      projectionMap[i] = tid;
+    }
+    ArrayList<Integer> projectionIdList = new ArrayList<Integer>(TUtil.newList(projectionMap));
+    ColumnProjectionUtils.setReadColumnIDs(conf, projectionIdList);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(closed.get()) {
+      return;
+    }
+    try {
+      if(in != null) {
+        in.close();
+        in = null;
+        sin = null;
+      }
+    } catch (Exception e) {
+      LOG.warn(e.getMessage(), e);
+    }
+
+    if(column != null) {
+      column.clear();
+      column = null;
+    }
+    super.close();
+  }
+
+  private boolean next(LongWritable key) throws IOException {
+    if (!more) {
+      return false;
+    }
+
+    more = in.next(key);
+    if (!more) {
+      return false;
+    }
+
+    long lastSeenSyncPos = in.lastSeenSyncPos();
+    if (lastSeenSyncPos >= end) {
+      more = false;
+      return more;
+    }
+    return more;
+  }
+
+  @Override
+  protected boolean initFirstScan(int maxBytesPerSchedule) throws IOException {
+    synchronized(lock) {
+      first = true;
+      this.maxBytesPerSchedule = maxBytesPerSchedule;
+      if(sin == null) {
+        sin = new ScheduledInputStream(
+            fragment.getPath(),
+            fs.open(fragment.getPath()),
+            fragment.getStartKey(),
+            fragment.getEndKey(),
+            fs.getLength(fragment.getPath()));
+
+        this.in = new RCFile.Reader(fragment.getPath(), sin, fs, fs.getConf());
+
+        Text text = this.in.getMetadata().get(new Text(SERDE));
+
+        try {
+          String serdeClass;
+          if(text != null && !text.toString().isEmpty()){
+            serdeClass = text.toString();
+          } else{
+            serdeClass = this.meta.getOption(SERDE, BinarySerializerDeserializer.class.getName());
+          }
+          serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance();
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+          throw new IOException(e);
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public boolean isStopScanScheduling() {
+    if(sin != null && sin.isEndOfStream()) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  protected boolean scanNext(int length) throws IOException {
+    synchronized(lock) {
+      if(isClosed()) {
+        return false;
+      }
+      return sin.readNext(length);
+    }
+  }
+
+
+  @Override
+  public boolean isFetchProcessing() {
+    //TODO row group size
+    if(sin != null && sin.getAvaliableSize() > maxBytesPerSchedule * 3) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  protected long getFilePosition() throws IOException {
+    return in.getPosition();
+  }
+
+  @Override
+  public void scannerReset() {
+    if(in != null) {
+      try {
+        in.seek(0);
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+    if(sin != null) {
+      try {
+        sin.seek(0);
+        sin.reset();
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return false;
+  }
+
+  @Override
+  public boolean isSplittable(){
+    return true;
+  }
+
+  @Override
+  protected long[] reportReadBytes() {
+    if(sin == null) {
+      return new long[]{0, 0};
+    } else {
+      return new long[]{sin.getTotalReadBytesForFetch(), sin.getTotalReadBytesFromDisk()};
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java
new file mode 100644
index 0000000..71c4d93
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScanScheduler.java
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.storage.v2.StorageManagerV2.StorgaeManagerContext;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+
+public final class ScanScheduler extends Thread {
+  private static final Log LOG = LogFactory.getLog(ScanScheduler.class);
+
+  private final Object scanQueueLock;
+  private StorgaeManagerContext context;
+
+  private Map<String, FileScannerV2> requestMap = new HashMap<String, FileScannerV2>();
+
+  private final Map<Integer, DiskFileScanScheduler> diskFileScannerMap = new HashMap<Integer, DiskFileScanScheduler>();
+
+  private Map<Integer, DiskDeviceInfo> diskDeviceInfoMap = new HashMap<Integer, DiskDeviceInfo>();
+
+  private SortedSet<DiskMountInfo> diskMountInfos = new TreeSet<DiskMountInfo>();
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private Random rand = new Random(System.currentTimeMillis());
+
+  private Thread schedulerStatusReportThread;
+
+  public ScanScheduler(StorgaeManagerContext context) {
+    this.context = context;
+    this.scanQueueLock = context.getScanQueueLock();
+
+    try {
+      List<DiskDeviceInfo> deviceInfos = DiskUtil.getDiskDeviceInfos();
+      for(DiskDeviceInfo eachInfo: deviceInfos) {
+        LOG.info("Create DiskScanQueue:" + eachInfo.getName());
+        diskDeviceInfoMap.put(eachInfo.getId(), eachInfo);
+
+        diskMountInfos.addAll(eachInfo.getMountInfos());
+      }
+
+      initFileScanners();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    final int reportInterval = context.getConf().getIntVar(ConfVars.STORAGE_MANAGER_DISK_SCHEDULER_REPORT_INTERVAL);
+    if(reportInterval  > 0) {
+      schedulerStatusReportThread = new Thread() {
+        public void run() {
+          while (true) {
+            try {
+              Thread.sleep(reportInterval);
+            } catch (InterruptedException e) {
+              break;
+            }
+            synchronized (diskFileScannerMap) {
+              for (DiskFileScanScheduler eachScheduler : diskFileScannerMap
+                  .values()) {
+                eachScheduler.printDiskSchedulerInfo();
+              }
+            }
+          }
+        }
+      };
+
+      schedulerStatusReportThread.start();
+    }
+  }
+
+  public void run() {
+    synchronized(scanQueueLock) {
+      while(!stopped.get()) {
+        FileScannerV2 fileScannerV2 = context.getScanQueue().poll();
+        if(fileScannerV2 == null) {
+          try {
+            scanQueueLock.wait();
+          } catch (InterruptedException e) {
+            break;
+          }
+        } else {
+          int diskId = fileScannerV2.getDiskId();
+
+          int emptyDiskId = findEmptyDisk();
+          if(emptyDiskId < 0) {
+            if(diskId < 0 || diskId >= diskDeviceInfoMap.size()) {
+              diskId = findDiskPartitionPath(fileScannerV2.getPath().toString());
+              if(diskId < 0) {
+
+                diskId = findMinQueueDisk();
+                if(diskId < 0) {
+                  diskId = rand.nextInt(diskDeviceInfoMap.size());
+                }
+              }
+            }
+          } else {
+            diskId = emptyDiskId;
+          }
+          synchronized(diskFileScannerMap) {
+            requestMap.put(fileScannerV2.getId(), fileScannerV2);
+            DiskFileScanScheduler diskScheduler = diskFileScannerMap.get(diskId);
+            fileScannerV2.setAllocatedDiskId(diskId);
+            diskScheduler.requestScanFile(fileScannerV2);
+          }
+        }
+      }
+    }
+  }
+
+  private int findEmptyDisk() {
+    synchronized(diskFileScannerMap) {
+      for(DiskFileScanScheduler eachDiskScanner: diskFileScannerMap.values()) {
+        int queueSize = eachDiskScanner.getTotalQueueSize();
+        if(queueSize == 0) {
+          return eachDiskScanner.getDiskId();
+        }
+      }
+      return -1;
+    }
+  }
+  
+  private int findMinQueueDisk() {
+    int minValue = Integer.MAX_VALUE;
+    int minId = -1;
+    synchronized(diskFileScannerMap) {
+      for(DiskFileScanScheduler eachDiskScanner: diskFileScannerMap.values()) {
+        int queueSize = eachDiskScanner.getTotalQueueSize();
+        if(queueSize <= minValue) {
+          minValue = queueSize;
+          minId = eachDiskScanner.getDiskId();
+        }
+      }
+    }
+
+    return minId;
+  }
+
+  private int findDiskPartitionPath(String fullPath) {
+    for (DiskMountInfo eachMountInfo : diskMountInfos) {
+      if (fullPath.indexOf(eachMountInfo.getMountPath()) == 0) {
+        return eachMountInfo.getDeviceId();
+      }
+    }
+
+    return -1;
+  }
+
+  public void incrementReadBytes(int diskId, long[] readBytes) {
+    diskFileScannerMap.get(diskId).incrementReadBytes(readBytes);
+  }
+
+  private void initFileScanners() {
+    for(Integer eachId: diskDeviceInfoMap.keySet()) {
+      DiskFileScanScheduler scanner = new DiskFileScanScheduler(context, diskDeviceInfoMap.get(eachId));
+      scanner.start();
+
+      diskFileScannerMap.put(eachId, scanner);
+    }
+  }
+
+  public void stopScheduler() {
+    stopped.set(true);
+    for(DiskFileScanScheduler eachDiskScanner: diskFileScannerMap.values()) {
+      eachDiskScanner.stopScan();
+    }
+    this.interrupt();
+  }
+}


[18/18] git commit: TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)


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

Branch: refs/heads/master
Commit: bbf9b7bf8b60a930b4e44754c732f9629762d1bf
Parents: 6aa96fa
Author: jinossy <ji...@gmail.com>
Authored: Tue Jan 28 21:34:41 2014 +0900
Committer: jinossy <ji...@gmail.com>
Committed: Tue Jan 28 21:34:41 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |    2 +
 pom.xml                                         |    1 +
 tajo-client/pom.xml                             |    2 +-
 tajo-core/pom.xml                               |    2 -
 tajo-core/tajo-core-backend/pom.xml             |    2 +-
 tajo-core/tajo-core-pullserver/pom.xml          |    2 +-
 tajo-core/tajo-core-storage/pom.xml             |  301 ---
 .../tajo/storage/AbstractStorageManager.java    |  690 -------
 .../java/org/apache/tajo/storage/Appender.java  |   39 -
 .../storage/BinarySerializerDeserializer.java   |  257 ---
 .../java/org/apache/tajo/storage/CSVFile.java   |  531 -----
 .../tajo/storage/CompressedSplitLineReader.java |  182 --
 .../org/apache/tajo/storage/DataLocation.java   |   45 -
 .../org/apache/tajo/storage/FileAppender.java   |   61 -
 .../org/apache/tajo/storage/FileScanner.java    |   93 -
 .../org/apache/tajo/storage/FrameTuple.java     |  231 ---
 .../java/org/apache/tajo/storage/LazyTuple.java |  291 ---
 .../org/apache/tajo/storage/LineReader.java     |  559 ------
 .../org/apache/tajo/storage/MergeScanner.java   |  154 --
 .../tajo/storage/NumericPathComparator.java     |   34 -
 .../java/org/apache/tajo/storage/RawFile.java   |  532 -----
 .../java/org/apache/tajo/storage/RowFile.java   |  506 -----
 .../org/apache/tajo/storage/RowStoreUtil.java   |  206 --
 .../java/org/apache/tajo/storage/Scanner.java   |   94 -
 .../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 |   67 -
 .../tajo/storage/StorageManagerFactory.java     |   98 -
 .../org/apache/tajo/storage/StorageUtil.java    |   83 -
 .../apache/tajo/storage/TableStatistics.java    |  117 --
 .../storage/TextSerializerDeserializer.java     |  209 --
 .../java/org/apache/tajo/storage/Tuple.java     |   82 -
 .../apache/tajo/storage/TupleComparator.java    |  159 --
 .../org/apache/tajo/storage/TupleRange.java     |  103 -
 .../java/org/apache/tajo/storage/VTuple.java    |  226 ---
 .../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 -
 .../tajo/storage/fragment/FileFragment.java     |  219 ---
 .../apache/tajo/storage/fragment/Fragment.java  |   31 -
 .../storage/fragment/FragmentConvertor.java     |  123 --
 .../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 ------
 .../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  | 1739 -----------------
 .../SchemaAwareCompressionInputStream.java      |   43 -
 .../SchemaAwareCompressionOutputStream.java     |   44 -
 .../tajo/storage/trevni/TrevniAppender.java     |  201 --
 .../tajo/storage/trevni/TrevniScanner.java      |  193 --
 .../apache/tajo/storage/v2/CSVFileScanner.java  |  386 ----
 .../apache/tajo/storage/v2/DiskDeviceInfo.java  |   62 -
 .../tajo/storage/v2/DiskFileScanScheduler.java  |  205 --
 .../org/apache/tajo/storage/v2/DiskInfo.java    |   75 -
 .../apache/tajo/storage/v2/DiskMountInfo.java   |  101 -
 .../org/apache/tajo/storage/v2/DiskUtil.java    |  199 --
 .../apache/tajo/storage/v2/FileScanRunner.java  |   70 -
 .../apache/tajo/storage/v2/FileScannerV2.java   |  203 --
 .../java/org/apache/tajo/storage/v2/RCFile.java | 1823 ------------------
 .../apache/tajo/storage/v2/RCFileScanner.java   |  297 ---
 .../apache/tajo/storage/v2/ScanScheduler.java   |  189 --
 .../tajo/storage/v2/ScheduledInputStream.java   |  513 -----
 .../tajo/storage/v2/StorageManagerV2.java       |  140 --
 .../src/main/proto/IndexProtos.proto            |   29 -
 .../src/main/resources/storage-default.xml      |  149 --
 .../tajo/storage/TestCompressionStorages.java   |  233 ---
 .../org/apache/tajo/storage/TestFrameTuple.java |   84 -
 .../org/apache/tajo/storage/TestLazyTuple.java  |  258 ---
 .../apache/tajo/storage/TestMergeScanner.java   |  179 --
 .../apache/tajo/storage/TestStorageManager.java |   93 -
 .../org/apache/tajo/storage/TestStorages.java   |  375 ----
 .../tajo/storage/TestTupleComparator.java       |   77 -
 .../org/apache/tajo/storage/TestVTuple.java     |  160 --
 .../apache/tajo/storage/index/TestBSTIndex.java |  948 ---------
 .../index/TestSingleCSVFileBSTIndex.java        |  248 ---
 .../tajo/storage/v2/TestCSVCompression.java     |  213 --
 .../apache/tajo/storage/v2/TestCSVScanner.java  |  168 --
 .../apache/tajo/storage/v2/TestStorages.java    |  242 ---
 .../src/test/resources/storage-default.xml      |  149 --
 tajo-dist/pom.xml                               |    6 +
 tajo-jdbc/pom.xml                               |    2 +-
 tajo-project/pom.xml                            |    2 +-
 tajo-storage/pom.xml                            |  383 ++++
 .../tajo/storage/AbstractStorageManager.java    |  690 +++++++
 .../java/org/apache/tajo/storage/Appender.java  |   39 +
 .../storage/BinarySerializerDeserializer.java   |  257 +++
 .../java/org/apache/tajo/storage/CSVFile.java   |  531 +++++
 .../tajo/storage/CompressedSplitLineReader.java |  182 ++
 .../org/apache/tajo/storage/DataLocation.java   |   45 +
 .../org/apache/tajo/storage/FileAppender.java   |   61 +
 .../org/apache/tajo/storage/FileScanner.java    |   93 +
 .../org/apache/tajo/storage/FrameTuple.java     |  231 +++
 .../java/org/apache/tajo/storage/LazyTuple.java |  291 +++
 .../org/apache/tajo/storage/LineReader.java     |  559 ++++++
 .../org/apache/tajo/storage/MergeScanner.java   |  154 ++
 .../tajo/storage/NumericPathComparator.java     |   34 +
 .../java/org/apache/tajo/storage/RawFile.java   |  532 +++++
 .../java/org/apache/tajo/storage/RowFile.java   |  506 +++++
 .../org/apache/tajo/storage/RowStoreUtil.java   |  206 ++
 .../java/org/apache/tajo/storage/Scanner.java   |   94 +
 .../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 |   67 +
 .../tajo/storage/StorageManagerFactory.java     |   98 +
 .../org/apache/tajo/storage/StorageUtil.java    |   83 +
 .../apache/tajo/storage/TableStatistics.java    |  117 ++
 .../storage/TextSerializerDeserializer.java     |  209 ++
 .../java/org/apache/tajo/storage/Tuple.java     |   82 +
 .../apache/tajo/storage/TupleComparator.java    |  159 ++
 .../org/apache/tajo/storage/TupleRange.java     |  103 +
 .../java/org/apache/tajo/storage/VTuple.java    |  226 +++
 .../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 +
 .../tajo/storage/fragment/FileFragment.java     |  219 +++
 .../apache/tajo/storage/fragment/Fragment.java  |   31 +
 .../storage/fragment/FragmentConvertor.java     |  123 ++
 .../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 ++++++
 .../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  | 1739 +++++++++++++++++
 .../SchemaAwareCompressionInputStream.java      |   43 +
 .../SchemaAwareCompressionOutputStream.java     |   44 +
 .../tajo/storage/trevni/TrevniAppender.java     |  201 ++
 .../tajo/storage/trevni/TrevniScanner.java      |  193 ++
 .../apache/tajo/storage/v2/CSVFileScanner.java  |  386 ++++
 .../apache/tajo/storage/v2/DiskDeviceInfo.java  |   62 +
 .../tajo/storage/v2/DiskFileScanScheduler.java  |  205 ++
 .../org/apache/tajo/storage/v2/DiskInfo.java    |   75 +
 .../apache/tajo/storage/v2/DiskMountInfo.java   |  101 +
 .../org/apache/tajo/storage/v2/DiskUtil.java    |  199 ++
 .../apache/tajo/storage/v2/FileScanRunner.java  |   70 +
 .../apache/tajo/storage/v2/FileScannerV2.java   |  203 ++
 .../java/org/apache/tajo/storage/v2/RCFile.java | 1823 ++++++++++++++++++
 .../apache/tajo/storage/v2/RCFileScanner.java   |  297 +++
 .../apache/tajo/storage/v2/ScanScheduler.java   |  189 ++
 .../tajo/storage/v2/ScheduledInputStream.java   |  513 +++++
 .../tajo/storage/v2/StorageManagerV2.java       |  140 ++
 tajo-storage/src/main/proto/IndexProtos.proto   |   29 +
 .../src/main/resources/storage-default.xml      |  149 ++
 .../tajo/storage/TestCompressionStorages.java   |  233 +++
 .../org/apache/tajo/storage/TestFrameTuple.java |   84 +
 .../org/apache/tajo/storage/TestLazyTuple.java  |  258 +++
 .../apache/tajo/storage/TestMergeScanner.java   |  179 ++
 .../apache/tajo/storage/TestStorageManager.java |   93 +
 .../org/apache/tajo/storage/TestStorages.java   |  375 ++++
 .../tajo/storage/TestTupleComparator.java       |   77 +
 .../org/apache/tajo/storage/TestVTuple.java     |  160 ++
 .../apache/tajo/storage/index/TestBSTIndex.java |  948 +++++++++
 .../index/TestSingleCSVFileBSTIndex.java        |  248 +++
 .../tajo/storage/v2/TestCSVCompression.java     |  213 ++
 .../apache/tajo/storage/v2/TestCSVScanner.java  |  168 ++
 .../apache/tajo/storage/v2/TestStorages.java    |  242 +++
 .../src/test/resources/storage-default.xml      |  149 ++
 183 files changed, 19287 insertions(+), 19198 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e3f2bff..9699782 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -386,6 +386,8 @@ Release 0.8.0 - unreleased
 
   TASKS
 
+    TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)
+
     TAJO-536: Fix warnings in tajo-core-storage. (jinho)
 
     TAJO-545: MySQLStore Documentation. (jaehwa)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d70f0e4..569e9be 100644
--- a/pom.xml
+++ b/pom.xml
@@ -87,6 +87,7 @@
     <module>tajo-client</module>
     <module>tajo-jdbc</module>
     <module>tajo-dist</module>
+    <module>tajo-storage</module>
   </modules>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-client/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-client/pom.xml b/tajo-client/pom.xml
index 9b4cd5e..c6fbb27 100644
--- a/tajo-client/pom.xml
+++ b/tajo-client/pom.xml
@@ -201,7 +201,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-core-storage</artifactId>
+      <artifactId>tajo-storage</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml
index 09978c3..e131d0e 100644
--- a/tajo-core/pom.xml
+++ b/tajo-core/pom.xml
@@ -34,7 +34,6 @@
 
   <modules>
     <module>tajo-core-backend</module>
-	  <module>tajo-core-storage</module>
 	  <module>tajo-core-pullserver</module>
   </modules>
 
@@ -161,7 +160,6 @@
                       run rm -rf ${project.artifactId}-${project.version}
                       run mkdir ${project.artifactId}-${project.version}
                       run cd ${project.artifactId}-${project.version}
-                      run cp -r ${basedir}/${project.artifactId}-storage/target/${project.artifactId}-storage-${project.version}*.jar .
                       run cp -r ${basedir}/${project.artifactId}-pullserver/target/${project.artifactId}-pullserver-${project.version}*.jar .
                       run cp -r ${basedir}/${project.artifactId}-backend/target/${project.artifactId}-backend-${project.version}*.jar .
                       run cp -r ${basedir}/${project.artifactId}-backend/target/lib .

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-backend/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/pom.xml b/tajo-core/tajo-core-backend/pom.xml
index abc217e..fce9925 100644
--- a/tajo-core/tajo-core-backend/pom.xml
+++ b/tajo-core/tajo-core-backend/pom.xml
@@ -213,7 +213,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-core-storage</artifactId>
+      <artifactId>tajo-storage</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-pullserver/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/pom.xml b/tajo-core/tajo-core-pullserver/pom.xml
index 8c6d4fe..0bdfed2 100644
--- a/tajo-core/tajo-core-pullserver/pom.xml
+++ b/tajo-core/tajo-core-pullserver/pom.xml
@@ -44,7 +44,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-core-storage</artifactId>
+      <artifactId>tajo-storage</artifactId>
       <scope>provided</scope>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/pom.xml b/tajo-core/tajo-core-storage/pom.xml
deleted file mode 100644
index dcbde44..0000000
--- a/tajo-core/tajo-core-storage/pom.xml
+++ /dev/null
@@ -1,301 +0,0 @@
-<?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">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>tajo-project</artifactId>
-    <groupId>org.apache.tajo</groupId>
-    <version>0.8.0-SNAPSHOT</version>
-    <relativePath>../../tajo-project</relativePath>
-  </parent>
-  <artifactId>tajo-core-storage</artifactId>
-  <packaging>jar</packaging>
-  <name>Tajo Core 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.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <version>2.4</version>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>create-protobuf-generated-sources-directory</id>
-            <phase>initialize</phase>
-            <configuration>
-              <target>
-                <mkdir dir="target/generated-sources/proto" />
-              </target>
-            </configuration>
-            <goals>
-              <goal>run</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <version>1.2</version>
-        <executions>
-          <execution>
-            <id>generate-sources</id>
-            <phase>generate-sources</phase>
-            <configuration>
-              <executable>protoc</executable>
-              <arguments>
-                <argument>-Isrc/main/proto/</argument>
-                <argument>--proto_path=../../tajo-common/src/main/proto</argument>
-                <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
-                <argument>--java_out=target/generated-sources/proto</argument>
-                <argument>src/main/proto/IndexProtos.proto</argument>
-              </arguments>
-            </configuration>
-            <goals>
-              <goal>exec</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>build-helper-maven-plugin</artifactId>
-        <version>1.5</version>
-        <executions>
-          <execution>
-            <id>add-source</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>add-source</goal>
-            </goals>
-            <configuration>
-              <sources>
-                <source>target/generated-sources/proto</source>
-              </sources>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-pmd-plugin</artifactId>
-        <version>2.7.1</version>
-      </plugin>
-    </plugins>
-  </build>
-
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-catalog-common</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>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>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <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>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.code.gson</groupId>
-      <artifactId>gson</artifactId>
-      <type>jar</type>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>commons-lang</groupId>
-      <artifactId>commons-lang</artifactId>
-      <version>2.6</version>
-    </dependency>
-  </dependencies>
-
-  <profiles>
-    <profile>
-      <id>docs</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-javadoc-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- build javadoc jars per jar for publishing to maven -->
-                <id>module-javadocs</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>jar</goal>
-                </goals>
-                <configuration>
-                  <destDir>${project.build.directory}</destDir>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <profile>
-      <id>src</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-source-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- builds source jars and attaches them to the project for publishing -->
-                <id>tajo-java-sources</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>jar-no-fork</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
-  <reporting>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-project-info-reports-plugin</artifactId>
-        <version>2.4</version>
-        <configuration>
-          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
-        </configuration>
-      </plugin>
-    </plugins>
-  </reporting>
-
-</project>
-

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
deleted file mode 100644
index 91a535e..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/AbstractStorageManager.java
+++ /dev/null
@@ -1,690 +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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.net.util.Base64;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.fragment.FragmentConvertor;
-import org.apache.tajo.util.Bytes;
-import org.apache.tajo.util.FileUtil;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-
-public abstract class AbstractStorageManager {
-  private final Log LOG = LogFactory.getLog(AbstractStorageManager.class);
-
-  protected final TajoConf conf;
-  protected final FileSystem fs;
-  protected final Path tableBaseDir;
-  protected final boolean blocksMetadataEnabled;
-
-  /**
-   * Cache of scanner handlers for each storage type.
-   */
-  protected static final Map<String, Class<? extends Scanner>> SCANNER_HANDLER_CACHE
-      = new ConcurrentHashMap<String, Class<? extends Scanner>>();
-
-  /**
-   * Cache of appender handlers for each storage type.
-   */
-  protected static final Map<String, Class<? extends FileAppender>> APPENDER_HANDLER_CACHE
-      = new ConcurrentHashMap<String, Class<? extends FileAppender>>();
-
-  /**
-   * Cache of constructors for each class. Pins the classes so they
-   * can't be garbage collected until ReflectionUtils can be collected.
-   */
-  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
-      new ConcurrentHashMap<Class<?>, Constructor<?>>();
-
-  public abstract Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException;
-
-  public abstract Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException;
-
-  protected AbstractStorageManager(TajoConf conf) throws IOException {
-    this.conf = conf;
-    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);
-    FileFragment fragment = new FileFragment(path.getName(), path, 0, status.getLen());
-    return getScanner(meta, schema, fragment);
-  }
-
-  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment) throws IOException {
-    return getScanner(meta, schema, FragmentConvertor.convert(conf, meta.getStoreType(), fragment), schema);
-  }
-
-  public Scanner getScanner(TableMeta meta, Schema schema, FragmentProto fragment, Schema target) throws IOException {
-    return getScanner(meta, schema, FragmentConvertor.convert(conf, meta.getStoreType(), fragment), target);
-  }
-
-  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment) throws IOException {
-    return getScanner(meta, schema, fragment, schema);
-  }
-
-  public FileSystem getFileSystem() {
-    return this.fs;
-  }
-
-  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);
-  }
-
-  public Appender getAppender(TableMeta meta, Schema schema, Path path)
-      throws IOException {
-    Appender appender;
-
-    Class<? extends FileAppender> appenderClass;
-
-    String handlerName = meta.getStoreType().name().toLowerCase();
-    appenderClass = APPENDER_HANDLER_CACHE.get(handlerName);
-    if (appenderClass == null) {
-      appenderClass = conf.getClass(
-          String.format("tajo.storage.appender-handler.%s.class",
-              meta.getStoreType().name().toLowerCase()), null,
-          FileAppender.class);
-      APPENDER_HANDLER_CACHE.put(handlerName, appenderClass);
-    }
-
-    if (appenderClass == null) {
-      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
-    }
-
-    appender = newAppenderInstance(appenderClass, conf, meta, schema, path);
-
-    return appender;
-  }
-
-
-  public TableMeta getTableMeta(Path tablePath) throws IOException {
-    TableMeta meta;
-
-    FileSystem fs = tablePath.getFileSystem(conf);
-    Path tableMetaPath = new Path(tablePath, ".meta");
-    if (!fs.exists(tableMetaPath)) {
-      throw new FileNotFoundException(".meta file not found in " + tablePath.toString());
-    }
-
-    FSDataInputStream tableMetaIn = fs.open(tableMetaPath);
-
-    CatalogProtos.TableProto tableProto = (CatalogProtos.TableProto) FileUtil.loadProto(tableMetaIn,
-        CatalogProtos.TableProto.getDefaultInstance());
-    meta = new TableMeta(tableProto);
-
-    return meta;
-  }
-
-  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
-  /////////////////////////////////////////////////////////////////////////////
-
-  private static final PathFilter hiddenFileFilter = new PathFilter() {
-    public boolean accept(Path p) {
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
-
-  /**
-   * 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 path) throws IOException {
-    List<FileStatus> result = new ArrayList<FileStatus>();
-    Path[] dirs = new Path[]{path};
-    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;
-  }
-
-  /**
-   * Get the lower bound on split size imposed by the format.
-   *
-   * @return the number of bytes of the minimal split for this format
-   */
-  protected long getFormatMinSplitSize() {
-    return 1;
-  }
-
-  /**
-   * 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 filename the file name to check
-   * @return is this file isSplittable?
-   */
-  protected boolean isSplittable(TableMeta meta, Schema schema, Path filename) throws IOException {
-    Scanner scanner = getFileScanner(meta, schema, filename);
-    return scanner.isSplittable();
-  }
-
-  @Deprecated
-  protected long computeSplitSize(long blockSize, long minSize,
-                                  long maxSize) {
-    return Math.max(minSize, Math.min(maxSize, blockSize));
-  }
-
-  @Deprecated
-  private static final double SPLIT_SLOP = 1.1;   // 10% slop
-
-  @Deprecated
-  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, TableMeta meta, Path file, long start, long length) {
-    return new FileFragment(fragmentId, file, start, length);
-  }
-
-  protected FileFragment makeSplit(String fragmentId, TableMeta meta, Path file, BlockLocation blockLocation,
-                               int[] diskIds) throws IOException {
-    return new FileFragment(fragmentId, file, blockLocation, diskIds);
-  }
-
-  // for Non Splittable. eg, compressed gzip TextFile
-  protected FileFragment makeNonSplit(String fragmentId, TableMeta meta, 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 maximum split size.
-   *
-   * @return the maximum number of bytes a split can include
-   */
-  @Deprecated
-  public static long getMaxSplitSize() {
-    // TODO - to be configurable
-    return 536870912L;
-  }
-
-  /**
-   * Get the minimum split size
-   *
-   * @return the minimum number of bytes that can be in a split
-   */
-  @Deprecated
-  public static long getMinSplitSize() {
-    // TODO - to be configurable
-    return 67108864L;
-  }
-
-  /**
-   * 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].isValid()) {
-        String volumeIdString = volumeIds[i].toString();
-        byte[] volumeIdBytes = Base64.decodeBase64(volumeIdString);
-
-        if (volumeIdBytes.length == 4) {
-          diskId = Bytes.toInt(volumeIdBytes);
-        } else if (volumeIdBytes.length == 1) {
-          diskId = (int) volumeIdBytes[0];  // support hadoop-2.0.2
-        }
-      }
-      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<FileFragment> getSplits(String tableName, TableMeta meta, Schema schema, Path inputPath) throws IOException {
-    // generate splits'
-
-    List<FileFragment> splits = new ArrayList<FileFragment>();
-    FileSystem fs = inputPath.getFileSystem(conf);
-    List<FileStatus> files;
-    if (fs.isFile(inputPath)) {
-      files = Lists.newArrayList(fs.getFileStatus(inputPath));
-    } else {
-      files = listStatus(inputPath);
-    }
-    for (FileStatus file : files) {
-      Path path = file.getPath();
-      long length = file.getLen();
-      if (length > 0) {
-        BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
-        boolean splittable = isSplittable(meta, schema, path);
-        if (blocksMetadataEnabled && fs instanceof DistributedFileSystem) {
-          // supported disk volume
-          BlockStorageLocation[] blockStorageLocations = ((DistributedFileSystem) fs)
-              .getFileBlockStorageLocations(Arrays.asList(blkLocations));
-          if (splittable) {
-            for (BlockStorageLocation blockStorageLocation : blockStorageLocations) {
-              splits.add(makeSplit(tableName, meta, path, blockStorageLocation, getDiskIds(blockStorageLocation
-                  .getVolumeIds())));
-            }
-          } else { // Non splittable
-            long blockSize = blockStorageLocations[0].getLength();
-            if (blockSize >= length) {
-              for (BlockStorageLocation blockStorageLocation : blockStorageLocations) {
-                splits.add(makeSplit(tableName, meta, path, blockStorageLocation, getDiskIds(blockStorageLocation
-                    .getVolumeIds())));
-              }
-            } else {
-              splits.add(makeNonSplit(tableName, meta, path, 0, length, blockStorageLocations));
-            }
-          }
-
-        } else {
-          if (splittable) {
-            for (BlockLocation blockLocation : blkLocations) {
-              splits.add(makeSplit(tableName, meta, path, blockLocation, null));
-            }
-          } else { // Non splittable
-            splits.add(makeNonSplit(tableName, meta, path, 0, length, blkLocations));
-          }
-        }
-      } else {
-        //for zero length files
-        splits.add(makeSplit(tableName, meta, path, 0, length));
-      }
-    }
-
-    LOG.info("Total # of splits: " + splits.size());
-    return splits;
-  }
-
-  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();
-    }
-  }
-
-  private static final Class<?>[] DEFAULT_SCANNER_PARAMS = {
-      Configuration.class,
-      Schema.class,
-      TableMeta.class,
-      FileFragment.class
-  };
-
-  private static final Class<?>[] DEFAULT_APPENDER_PARAMS = {
-      Configuration.class,
-      Schema.class,
-      TableMeta.class,
-      Path.class
-  };
-
-  /**
-   * create a scanner instance.
-   */
-  public static <T> T newScannerInstance(Class<T> theClass, Configuration conf, Schema schema, TableMeta meta,
-                                         Fragment fragment) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(DEFAULT_SCANNER_PARAMS);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(new Object[]{conf, schema, meta, fragment});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    return result;
-  }
-
-  /**
-   * create a scanner instance.
-   */
-  public static <T> T newAppenderInstance(Class<T> theClass, Configuration conf, TableMeta meta, Schema schema,
-                                          Path path) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(DEFAULT_APPENDER_PARAMS);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(new Object[]{conf, schema, meta, path});
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Appender.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Appender.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Appender.java
deleted file mode 100644
index ed6ea34..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/Appender.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.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;
-  
-  void close() throws IOException;
-
-  void enableStats();
-  
-  TableStats getStats();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
deleted file mode 100644
index ed034be..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java
+++ /dev/null
@@ -1,257 +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;
-
-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 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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
deleted file mode 100644
index 5d05d6f..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ /dev/null
@@ -1,531 +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.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-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.FileFragment;
-import org.apache.tajo.storage.rcfile.NonSyncByteArrayOutputStream;
-import org.apache.tajo.util.Bytes;
-
-import java.io.*;
-import java.util.ArrayList;
-import java.util.Arrays;
-
-public class CSVFile {
-
-  public static final String DELIMITER = "csvfile.delimiter";
-  public static final String NULL = "csvfile.null";     //read only
-  public static final String SERDE = "csvfile.serde";
-  public static final String DELIMITER_DEFAULT = "|";
-  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 NonSyncByteArrayOutputStream os = new NonSyncByteArrayOutputStream(BUFFER_SIZE);
-    private SerializerDeserializer serde;
-
-    public CSVAppender(Configuration conf, final Schema schema, final TableMeta meta, final Path path) throws IOException {
-      super(conf, schema, meta, path);
-      this.fs = path.getFileSystem(conf);
-      this.meta = meta;
-      this.schema = schema;
-      this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getOption(DELIMITER, DELIMITER_DEFAULT)).charAt(0);
-      this.columnNum = schema.getColumnNum();
-      String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getOption(NULL));
-      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());
-      }
-
-      String codecName = this.meta.getOption(TableMeta.COMPRESSION_CODEC);
-      if(!StringUtils.isEmpty(codecName)){
-        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 {
-        String serdeClass = this.meta.getOption(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 (enabledStats) {
-          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 FileFragment fragment)
-        throws IOException {
-      super(conf, schema, meta, fragment);
-      factory = new CompressionCodecFactory(conf);
-      codec = factory.getCodec(fragment.getPath());
-      if (codec == null || codec instanceof SplittableCompressionCodec) {
-        splittable = true;
-      }
-
-      //Delimiter
-      String delim  = meta.getOption(DELIMITER, DELIMITER_DEFAULT);
-      this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0);
-
-      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(NULL));
-      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 = new ArrayList<Long>();
-    private ArrayList<Integer> rowLengthList = new ArrayList<Integer>();
-    private ArrayList<Integer> startOffsets = new ArrayList<Integer>();
-    private NonSyncByteArrayOutputStream buffer = new NonSyncByteArrayOutputStream(DEFAULT_PAGE_SIZE);
-    private SerializerDeserializer serde;
-
-    @Override
-    public void init() throws IOException {
-
-      // 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.getEndKey();
-
-      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.getColumnIdByName(targets[i].getColumnName());
-      }
-
-      try {
-        String serdeClass = this.meta.getOption(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) {
-        startOffset += reader.readLine(new Text(), 0, maxBytesToConsume(startOffset));
-        pos = startOffset;
-      }
-      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;
-        }
-      }
-    }
-
-    @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 = Bytes.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 {
-        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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CompressedSplitLineReader.java
deleted file mode 100644
index 4f58e68..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/DataLocation.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/DataLocation.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/DataLocation.java
deleted file mode 100644
index 8841a31..0000000
--- a/tajo-core/tajo-core-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 +
-        '}';
-  }
-}


[05/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..a6b8781
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java
@@ -0,0 +1,1739 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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;
+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.util.Bytes;
+
+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 {
+
+    // 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();
+
+
+    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) {
+          Bytes.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);
+        }
+        addIndex++;
+      }
+
+      if (skipTotal != 0) {
+        Bytes.skipFully(in, skipTotal);
+      }
+    }
+
+    public void clearColumnBuffer() throws IOException {
+      decompressBuffer.reset();
+    }
+
+    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; // 4M
+    // 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;
+    SerializerDeserializer serde;
+
+    // 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 Schema schema, final TableMeta meta, final Path path) throws IOException {
+      super(conf, schema, meta, path);
+
+      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.getColumnNum();
+    }
+
+    public void init() throws IOException {
+      fs = path.getFileSystem(conf);
+
+      if (!fs.exists(path.getParent())) {
+        throw new FileNotFoundException(path.toString());
+      }
+
+      String codecClassname = this.meta.getOption(TableMeta.COMPRESSION_CODEC);
+      if (!StringUtils.isEmpty(codecClassname)) {
+        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(NULL));
+      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(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(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.getColumnNum();
+
+      for (int i = 0; i < size; i++) {
+        Datum datum = tuple.get(i);
+        int length = columnBuffers[i].append(schema.getColumn(i), datum);
+        columnBufferSize += length;
+        if (enabledStats) {
+          stats.analyzeField(i, datum);
+        }
+      }
+
+      if (size < columnNumber) {
+        for (int i = size; i < columnNumber; i++) {
+          columnBuffers[i].append(schema.getColumn(i), NullDatum.get());
+          if (enabledStats) {
+            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(new BufferedOutputStream(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);
+        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, deflateOut, deflateFilter);
+        }
+      } 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(new BufferedOutputStream(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) {
+        // Close the underlying stream if we own it...
+        out.flush();
+        out.close();
+        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 final byte[] sync = new byte[SYNC_HASH_SIZE];
+    private final byte[] syncCheck = new byte[SYNC_HASH_SIZE];
+    private boolean syncSeen;
+    private long lastSeenSyncPos = 0;
+
+    private long headerEnd;
+    private long start, end;
+    private 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;
+
+
+    //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 FileFragment fragment) throws IOException {
+      super(conf, schema, meta, fragment);
+
+      rowId = new LongWritable();
+      conf.setInt("io.file.buffer.size", 4096); //TODO remove
+
+
+      startOffset = fragment.getStartKey();
+      endOffset = startOffset + fragment.getEndKey();
+      more = startOffset < endOffset;
+      start = 0;
+    }
+
+    @Override
+    public void init() throws IOException {
+      String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(NULL));
+      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.getColumnIdByName(targets[i].getColumnName());
+      }
+      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(SERDE));
+
+      try {
+        String serdeClass;
+        if(text != null && !text.toString().isEmpty()){
+          serdeClass = text.toString();
+        } else{
+          serdeClass = this.meta.getOption(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();
+    }
+
+    /**
+     * 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);
+          /* 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();
+      if (sync != null && length == SYNC_ESCAPE) { // process
+        // a
+        // sync entry
+        lastSeenSyncPos = in.getPos() - 4; // minus SYNC_ESCAPE's length
+        in.readFully(syncCheck); // read syncCheck
+        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
+      } 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();
+      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);
+      }
+
+      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;
+    }
+
+    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.getColumnNum());
+      getCurrentRow(tuple);
+      return tuple;
+    }
+
+    /**
+     * 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 {
+      IOUtils.closeStream(in);
+      currentValue.close();
+      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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..60f1b06
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..c0ce8b3
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.java b/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.java
new file mode 100644
index 0000000..3209469
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniAppender.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.trevni;
+
+import org.apache.commons.io.IOUtils;
+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.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.common.TajoDataTypes.Type;
+import org.apache.tajo.storage.FileAppender;
+import org.apache.tajo.storage.TableStatistics;
+import org.apache.tajo.storage.Tuple;
+import org.apache.trevni.ColumnFileMetaData;
+import org.apache.trevni.ColumnFileWriter;
+import org.apache.trevni.ColumnMetaData;
+import org.apache.trevni.ValueType;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class TrevniAppender extends FileAppender {
+  private FileSystem fs;
+  private ColumnFileWriter writer;
+  private FSDataOutputStream fos;
+
+  private TableStatistics stats = null;
+  private boolean flushed = false;
+
+  public TrevniAppender(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
+    super(conf, schema, meta, path);
+  }
+
+  public void init() throws IOException {
+    fs = path.getFileSystem(conf);
+
+    if (!fs.exists(path.getParent())) {
+      throw new FileNotFoundException(path.toString());
+    }
+
+    fos = fs.create(path);
+
+    ColumnMetaData [] trevniMetas =
+        new ColumnMetaData[schema.getColumnNum()];
+    int i = 0;
+    for (Column column : schema.getColumns()) {
+      trevniMetas[i++] = new ColumnMetaData(column.getColumnName(),
+          getType(column.getDataType().getType()));
+    }
+
+    writer = new ColumnFileWriter(createFileMeta(), trevniMetas);
+
+    if (enabledStats) {
+      this.stats = new TableStatistics(this.schema);
+    }
+
+    super.init();
+  }
+
+  private ColumnFileMetaData createFileMeta() {
+    return new ColumnFileMetaData()
+        .setCodec("null")
+        .setChecksum("null");
+  }
+
+  private static ValueType getType(Type type) {
+    switch (type) {
+      case BOOLEAN:
+        return ValueType.INT;
+      case BIT:
+        return ValueType.INT;
+      case CHAR:
+        return ValueType.STRING;
+      case INT2:
+        return ValueType.INT;
+      case INT4:
+        return ValueType.INT;
+      case INT8:
+        return ValueType.LONG;
+      case FLOAT4:
+        return ValueType.FLOAT;
+      case FLOAT8:
+        return ValueType.DOUBLE;
+      case TEXT:
+        return ValueType.STRING;
+      case BLOB:
+        return ValueType.BYTES;
+      case INET4:
+        return ValueType.BYTES;
+      case INET6:
+        return ValueType.BYTES;
+      case PROTOBUF:
+        return ValueType.BYTES;
+      case NULL_TYPE:
+        return ValueType.NULL;
+      default:
+        return null;
+    }
+  }
+
+  @Override
+  public long getOffset() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void addTuple(Tuple t) throws IOException {
+    Column col;
+    writer.startRow();
+    for (int i = 0; i < schema.getColumnNum(); i++) {
+      if (enabledStats) {
+        stats.analyzeField(i, t.get(i));
+      }
+
+      if (!t.isNull(i)) {
+        col = schema.getColumn(i);
+        switch (col.getDataType().getType()) {
+          case NULL_TYPE:
+            break;
+          case BOOLEAN:
+          case BIT:
+          case INT2:
+          case INT4:
+            writer.writeValue(t.get(i).asInt4(), i);
+            break;
+          case INT8:
+            writer.writeValue(t.get(i).asInt8(), i);
+            break;
+          case FLOAT4:
+            writer.writeValue(t.get(i).asFloat4(), i);
+            break;
+          case FLOAT8:
+            writer.writeValue(t.get(i).asFloat8(), i);
+            break;
+          case CHAR:
+          case TEXT:
+            writer.writeValue(t.get(i).asChars(), i);
+            break;
+          case PROTOBUF:
+          case BLOB:
+          case INET4:
+          case INET6:
+            writer.writeValue(t.get(i).asByteArray(), i);
+
+          default:
+            break;
+        }
+      }
+    }
+    writer.endRow();
+
+    // Statistical section
+    if (enabledStats) {
+      stats.incrementRow();
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (!flushed) {
+      writer.writeTo(fos);
+      fos.flush();
+      flushed = true;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    flush();
+    IOUtils.closeQuietly(fos);
+  }
+
+  @Override
+  public TableStats getStats() {
+    if (enabledStats) {
+      return stats.getTableStat();
+    } else {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java
new file mode 100644
index 0000000..2c2037f
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/trevni/TrevniScanner.java
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.trevni;
+
+import com.google.protobuf.Message;
+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.datum.BlobDatum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.ProtobufDatumFactory;
+import org.apache.tajo.storage.FileScanner;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.trevni.ColumnFileReader;
+import org.apache.trevni.ColumnValues;
+import org.apache.trevni.avro.HadoopInput;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.tajo.common.TajoDataTypes.DataType;
+
+public class TrevniScanner extends FileScanner {
+  private ColumnFileReader reader;
+  private int [] projectionMap;
+  private ColumnValues [] columns;
+
+  public TrevniScanner(Configuration conf, Schema schema, TableMeta meta, FileFragment fragment) throws IOException {
+    super(conf, schema, meta, fragment);
+    reader = new ColumnFileReader(new HadoopInput(fragment.getPath(), conf));
+  }
+
+  @Override
+  public void init() throws IOException {
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+
+    prepareProjection(targets);
+
+    columns = new ColumnValues[projectionMap.length];
+
+    for (int i = 0; i < projectionMap.length; i++) {
+      columns[i] = reader.getValues(projectionMap[i]);
+    }
+
+    super.init();
+  }
+
+  private void prepareProjection(Column [] targets) {
+    projectionMap = new int[targets.length];
+    int tid;
+    for (int i = 0; i < targets.length; i++) {
+      tid = schema.getColumnIdByName(targets[i].getColumnName());
+      projectionMap[i] = tid;
+    }
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple tuple = new VTuple(schema.getColumnNum());
+
+    if (!columns[0].hasNext()) {
+      return null;
+    }
+
+    int tid; // column id of the original input schema
+    for (int i = 0; i < projectionMap.length; i++) {
+      tid = projectionMap[i];
+      columns[i].startRow();
+      DataType dataType = schema.getColumn(tid).getDataType();
+      switch (dataType.getType()) {
+        case BOOLEAN:
+          tuple.put(tid,
+              DatumFactory.createBool(((Integer)columns[i].nextValue()).byteValue()));
+          break;
+        case BIT:
+          tuple.put(tid,
+              DatumFactory.createBit(((Integer) columns[i].nextValue()).byteValue()));
+          break;
+        case CHAR:
+          String str = (String) columns[i].nextValue();
+          tuple.put(tid,
+              DatumFactory.createChar(str));
+          break;
+
+        case INT2:
+          tuple.put(tid,
+              DatumFactory.createInt2(((Integer) columns[i].nextValue()).shortValue()));
+          break;
+        case INT4:
+          tuple.put(tid,
+              DatumFactory.createInt4((Integer) columns[i].nextValue()));
+          break;
+
+        case INT8:
+          tuple.put(tid,
+              DatumFactory.createInt8((Long) columns[i].nextValue()));
+          break;
+
+        case FLOAT4:
+          tuple.put(tid,
+              DatumFactory.createFloat4((Float) columns[i].nextValue()));
+          break;
+
+        case FLOAT8:
+          tuple.put(tid,
+              DatumFactory.createFloat8((Double) columns[i].nextValue()));
+          break;
+
+        case INET4:
+          tuple.put(tid,
+              DatumFactory.createInet4(((ByteBuffer) columns[i].nextValue()).array()));
+          break;
+
+        case TEXT:
+          tuple.put(tid,
+              DatumFactory.createText((String) columns[i].nextValue()));
+          break;
+
+        case PROTOBUF: {
+          ProtobufDatumFactory factory = ProtobufDatumFactory.get(dataType.getCode());
+          Message.Builder builder = factory.newBuilder();
+          builder.mergeFrom(((ByteBuffer)columns[i].nextValue()).array());
+          tuple.put(tid, factory.createDatum(builder));
+          break;
+        }
+
+        case BLOB:
+          tuple.put(tid,
+              new BlobDatum(((ByteBuffer) columns[i].nextValue())));
+          break;
+
+        case NULL_TYPE:
+          tuple.put(tid, NullDatum.get());
+          break;
+
+        default:
+          throw new IOException("Unsupport data type");
+      }
+    }
+
+    return tuple;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    for (int i = 0; i < projectionMap.length; i++) {
+      columns[i] = reader.getValues(projectionMap[i]);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+
+  @Override
+  public boolean isSelectable() {
+    return false;
+  }
+
+  @Override
+  public boolean isSplittable(){
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.java
new file mode 100644
index 0000000..b93672b
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/CSVFileScanner.java
@@ -0,0 +1,386 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import org.apache.commons.lang.ArrayUtils;
+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.Seekable;
+import org.apache.hadoop.io.compress.*;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.LazyTuple;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.compress.CodecPool;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.Bytes;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+public class CSVFileScanner extends FileScannerV2 {
+  public static final String DELIMITER = "csvfile.delimiter";
+  public static final String DELIMITER_DEFAULT = "|";
+  public static final byte LF = '\n';
+  private static final Log LOG = LogFactory.getLog(CSVFileScanner.class);
+
+  private final static int DEFAULT_BUFFER_SIZE = 256 * 1024;
+  private int bufSize;
+  private char delimiter;
+  private ScheduledInputStream sin;
+  private InputStream is; // decompressd stream
+  private CompressionCodecFactory factory;
+  private CompressionCodec codec;
+  private Decompressor decompressor;
+  private Seekable filePosition;
+  private boolean splittable = true;
+  private long startOffset, length;
+  private byte[] buf = null;
+  private byte[][] tuples = null;
+  private long[] tupleOffsets = null;
+  private int currentIdx = 0, validIdx = 0;
+  private byte[] tail = null;
+  private long pageStart = -1;
+  private long prevTailLen = -1;
+  private int[] targetColumnIndexes;
+  private boolean eof = false;
+  private boolean first = true;
+
+  private long totalReadBytesForFetch;
+  private long totalReadBytesFromDisk;
+
+  public CSVFileScanner(Configuration conf, final Schema schema, final TableMeta meta, final FileFragment fragment)
+      throws IOException {
+    super(conf, meta, schema, fragment);
+    factory = new CompressionCodecFactory(conf);
+    codec = factory.getCodec(this.fragment.getPath());
+    if (isCompress() && !(codec instanceof SplittableCompressionCodec)) {
+      splittable = false;
+    }
+  }
+
+  @Override
+  public void init() throws IOException {
+    // Buffer size, Delimiter
+    this.bufSize = DEFAULT_BUFFER_SIZE;
+    String delim  = meta.getOption(DELIMITER, DELIMITER_DEFAULT);
+    this.delimiter = delim.charAt(0);
+
+    super.init();
+  }
+
+  @Override
+  protected boolean initFirstScan(int maxBytesPerSchedule) throws IOException {
+    synchronized(this) {
+      eof = false;
+      first = true;
+      if(sin == null) {
+        FSDataInputStream fin = fs.open(fragment.getPath(), 128 * 1024);
+        sin = new ScheduledInputStream(fragment.getPath(), fin,
+            fragment.getStartKey(), fragment.getEndKey(), fs.getLength(fragment.getPath()));
+        startOffset = fragment.getStartKey();
+        length = fragment.getEndKey();
+
+        if (startOffset > 0) {
+          startOffset--; // prev line feed
+        }
+      }
+    }
+    return true;
+  }
+
+  private boolean scanFirst() throws IOException {
+    if (codec != null) {
+      decompressor = CodecPool.getDecompressor(codec);
+      if (codec instanceof SplittableCompressionCodec) {
+        SplitCompressionInputStream cIn = ((SplittableCompressionCodec) codec).createInputStream(
+            sin, decompressor, startOffset, startOffset + length,
+            SplittableCompressionCodec.READ_MODE.BYBLOCK);
+
+        startOffset = cIn.getAdjustedStart();
+        length = cIn.getAdjustedEnd() - startOffset;
+        filePosition = cIn;
+        is = cIn;
+      } else {
+        is = new DataInputStream(codec.createInputStream(sin, decompressor));
+      }
+    } else {
+      sin.seek(startOffset);
+      filePosition = sin;
+      is = sin;
+    }
+
+    tuples = new byte[0][];
+    if (targets == null) {
+      targets = schema.toArray();
+    }
+
+    targetColumnIndexes = new int[targets.length];
+    for (int i = 0; i < targets.length; i++) {
+      targetColumnIndexes[i] = schema.getColumnIdByName(targets[i].getColumnName());
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("CSVScanner open:" + fragment.getPath() + "," + startOffset + "," + length +
+          "," + fs.getFileStatus(fragment.getPath()).getLen());
+    }
+
+    if (startOffset != 0) {
+      int rbyte;
+      while ((rbyte = is.read()) != LF) {
+        if(rbyte == -1) break;
+      }
+    }
+
+    if (fragmentable() < 1) {
+      close();
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public boolean isStopScanScheduling() {
+    if(sin != null && sin.isEndOfStream()) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  private long fragmentable() throws IOException {
+    return startOffset + length - getFilePosition();
+  }
+
+  @Override
+  protected long getFilePosition() throws IOException {
+    long retVal;
+    if (filePosition != null) {
+      retVal = filePosition.getPos();
+    } else {
+      retVal = sin.getPos();
+    }
+    return retVal;
+  }
+
+  @Override
+  public boolean isFetchProcessing() {
+    if(sin != null &&
+        (sin.getAvaliableSize() >= 64 * 1024 * 1024)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  private void page() throws IOException {
+    // Index initialization
+    currentIdx = 0;
+
+    // Buffer size set
+    if (isSplittable() && fragmentable() < DEFAULT_BUFFER_SIZE) {
+      bufSize = (int) fragmentable();
+    }
+
+    if (this.tail == null || this.tail.length == 0) {
+      this.pageStart = getFilePosition();
+      this.prevTailLen = 0;
+    } else {
+      this.pageStart = getFilePosition() - this.tail.length;
+      this.prevTailLen = this.tail.length;
+    }
+
+    // Read
+    int rbyte;
+    buf = new byte[bufSize];
+    rbyte = is.read(buf);
+
+    if (rbyte < 0) {
+      eof = true; // EOF
+      return;
+    }
+
+    if (prevTailLen == 0) {
+      tail = new byte[0];
+      tuples = Bytes.splitPreserveAllTokens(buf, rbyte, (char) LF);
+    } else {
+      byte[] lastRow = ArrayUtils.addAll(tail, buf);
+      tuples = Bytes.splitPreserveAllTokens(lastRow, rbyte + tail.length, (char) LF);
+      tail = null;
+    }
+
+    // Check tail
+    if ((char) buf[rbyte - 1] != LF) {
+      if ((fragmentable() < 1 || rbyte != bufSize)) {
+        int lineFeedPos = 0;
+        byte[] temp = new byte[DEFAULT_BUFFER_SIZE];
+
+        // find line feed
+        while ((temp[lineFeedPos] = (byte)is.read()) != (byte)LF) {
+          if(temp[lineFeedPos] < 0) {
+            break;
+          }
+          lineFeedPos++;
+        }
+
+        tuples[tuples.length - 1] = ArrayUtils.addAll(tuples[tuples.length - 1],
+            ArrayUtils.subarray(temp, 0, lineFeedPos));
+        validIdx = tuples.length;
+      } else {
+        tail = tuples[tuples.length - 1];
+        validIdx = tuples.length - 1;
+      }
+    } else {
+      tail = new byte[0];
+      validIdx = tuples.length - 1;
+    }
+
+    if(!isCompress()) makeTupleOffset();
+  }
+
+  private void makeTupleOffset() {
+    long curTupleOffset = 0;
+    this.tupleOffsets = new long[this.validIdx];
+    for (int i = 0; i < this.validIdx; i++) {
+      this.tupleOffsets[i] = curTupleOffset + this.pageStart;
+      curTupleOffset += this.tuples[i].length + 1;//tuple byte +  1byte line feed
+    }
+  }
+
+  protected Tuple nextTuple() throws IOException {
+    if(first) {
+      boolean more = scanFirst();
+      first = false;
+      if(!more) {
+        return null;
+      }
+    }
+    try {
+      if (currentIdx == validIdx) {
+        if (isSplittable() && fragmentable() < 1) {
+          close();
+          return null;
+        } else {
+          page();
+        }
+
+        if(eof){
+          close();
+          return null;
+        }
+      }
+
+      long offset = -1;
+      if(!isCompress()){
+        offset = this.tupleOffsets[currentIdx];
+      }
+
+      byte[][] cells = Bytes.splitPreserveAllTokens(tuples[currentIdx++], delimiter, targetColumnIndexes);
+      return new LazyTuple(schema, cells, offset);
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+    }
+    return null;
+  }
+
+  private boolean isCompress() {
+    return codec != null;
+  }
+
+  @Override
+  public void scannerReset() {
+    if(sin != null) {
+      try {
+        filePosition.seek(0);
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+    if(sin != null) {
+      try {
+        sin.seek(0);
+        sin.reset();
+      } catch (IOException e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(closed.get()) {
+      return;
+    }
+    if(sin != null) {
+      totalReadBytesForFetch = sin.getTotalReadBytesForFetch();
+      totalReadBytesFromDisk = sin.getTotalReadBytesFromDisk();
+    }
+    try {
+      if(is != null) {
+        is.close();
+      }
+      is = null;
+      sin = null;
+    } finally {
+      if (decompressor != null) {
+        CodecPool.returnDecompressor(decompressor);
+        decompressor = null;
+      }
+      tuples = null;
+      super.close();
+    }
+  }
+
+  @Override
+  protected boolean scanNext(int length) throws IOException {
+    synchronized(this) {
+      if(isClosed()) {
+        return false;
+      }
+      return sin.readNext(length);
+    }
+  }
+
+  @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
+  protected long[] reportReadBytes() {
+    return new long[]{totalReadBytesForFetch, totalReadBytesFromDisk};
+  }
+}


[11/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java
deleted file mode 100644
index 12b984e..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java
+++ /dev/null
@@ -1,513 +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.v2;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Seekable;
-
-import java.io.*;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-public class ScheduledInputStream extends InputStream implements Seekable, Closeable, DataInput {
-  private static final Log LOG = LogFactory.getLog(ScheduledInputStream.class);
-
-	private FSDataInputStream originStream;
-
-  private int currentScanIndex;
-
-  private Queue<ScanData> dataQueue = new LinkedList<ScanData>();
-
-  private ScanData currentScanData;
-
-  private AtomicBoolean closed = new AtomicBoolean(false);
-
-  private boolean eof = false;
-
-  private long pos;
-
-  private AtomicInteger avaliableSize = new AtomicInteger(0);
-
-  private long fileLen;
-
-  private long startOffset;
-
-  private long length;
-
-  private long endOffset;
-
-  private boolean endOfStream = false;
-
-  private Path file;
-
-  private byte readLongBuffer[] = new byte[8];
-
-  private AtomicLong totalReadBytesForFetch = new AtomicLong(0);
-
-  private AtomicLong totalReadBytesFromDisk = new AtomicLong(0);
-
-	public ScheduledInputStream(Path file, FSDataInputStream originStream,
-                              long startOffset, long length, long fileLen) throws IOException {
-		this.originStream = originStream;
-		this.startOffset = startOffset;
-		this.length = length;
-		this.endOffset = startOffset + length;
-		this.fileLen = fileLen;
-    this.file = file;
-		this.pos = this.originStream.getPos();
-
-    LOG.info("Open:" + toString());
-	}
-
-	public int getAvaliableSize() {
-		return avaliableSize.get();
-	}
-
-  public String toString() {
-    return file.getName() + ":" + startOffset + ":" + length;
-  }
-	public boolean readNext(int length) throws IOException {
-		return readNext(length, false);
-	}
-	
-	public boolean readNext(int length, boolean ignoreEOS) throws IOException {
-    synchronized(dataQueue) {
-      if(closed.get() || (!ignoreEOS && endOfStream)) {
-        return false;
-      }
-      int bufLength = ignoreEOS ? length : (int)Math.min(length,  endOffset - originStream.getPos());
-      bufLength = (int)Math.min(bufLength, fileLen - originStream.getPos());
-      if(bufLength == 0) {
-        return false;
-      }
-			byte[] buf = new byte[bufLength];
-
-      try {
-        originStream.readFully(buf);
-      } catch (EOFException e) {
-        LOG.error(e.getMessage(), e);
-        throw e;
-      } catch (Exception e) {
-        throw new IOException(e.getMessage(), e);
-      }
-
-      if(originStream.getPos() == fileLen) {
-        LOG.info("EOF:" + toString());
-        eof = true;
-      }
-      if(!ignoreEOS && originStream.getPos() >= endOffset) {
-        LOG.info("EndOfStream:" + toString());
-        endOfStream = true;
-      }
-
-      if(currentScanData == null) {
-        currentScanData = new ScanData(buf, bufLength);
-        currentScanIndex = 0;
-      } else {
-        dataQueue.offer(new ScanData(buf, bufLength));
-      }
-
-      avaliableSize.addAndGet(bufLength);
-
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Add DataQueue: queue=" + dataQueue.size() +
-          ", avaliable Size=" + avaliableSize.get() + ", pos=" + getPos() +
-          ", originPos=" + originStream.getPos() + ",endOfStream=" + endOfStream +
-          ", bufLength=" + bufLength + ",ignoreEOS=" + ignoreEOS);
-      }
-
-      totalReadBytesFromDisk.addAndGet(bufLength);
-      dataQueue.notifyAll();
-    }
-    return !eof;
-	}
-	
-	static class ScanData {
-		byte[] data;
-		int length;
-		public ScanData(byte[] buf, int length) {
-			this.data = buf;
-			this.length = length;
-		}
-		
-		@Override
-		public String toString() {
-			return "length=" + length;
-		}
-	}
-
-	@Override
-	public void seek(long pos) throws IOException {
-		synchronized(dataQueue) {
-			dataQueue.clear();
-			currentScanData = null;
-			currentScanIndex = 0;
-			avaliableSize.set(0);
-      originStream.seek(pos);
-      this.pos = pos;
-    }
-	}
-
-	@Override
-	public long getPos() throws IOException {
-		return this.pos;
-	}
-
-	public long getOriginStreamPos() {
-		try {
-			return this.originStream.getPos();
-		} catch (IOException e) {
-			e.printStackTrace();
-			return 0;
-		}
-	}
-	
-	@Override
-	public boolean seekToNewSource(long targetPos) throws IOException {
-		synchronized(dataQueue) {
-			dataQueue.clear();
-			currentScanData = null;
-			currentScanIndex = 0;
-			avaliableSize.set(0);
-      boolean result = originStream.seekToNewSource(targetPos);
-
-      this.pos = originStream.getPos();
-      return result;
-		}
-	}
-
-	@Override
-	public int read() throws IOException {
-		if(noMoreData()) {
-			return -1;
-		}
-		if(currentScanData == null || currentScanIndex >= currentScanData.length) {
-			synchronized(dataQueue) {
-				if(dataQueue.isEmpty()) {
-					if(endOfStream) {
-						readNext(64 * 1024, true);
-					} else {
-						try {
-							dataQueue.wait();
-							if(eof && dataQueue.isEmpty() && currentScanIndex > 0) {
-								//no more data
-								return -1;
-							}
-						} catch (InterruptedException e) {
-						}
-					}
-				}
-				if(!dataQueue.isEmpty() && currentScanIndex > 0) {
-					currentScanData = dataQueue.poll();
-					currentScanIndex = 0;
-				}
-			}
-		} 
-		
-		this.pos++;
-		avaliableSize.decrementAndGet();
-    totalReadBytesForFetch.incrementAndGet();
-
-		return currentScanData.data[currentScanIndex++] & 0xff;
-	}
-	
-	private boolean noMoreData() {
-		return closed.get();
-	}
-	
-	public int read(byte b[], int off, int len) throws IOException {
-		if(noMoreData()) {
-			return -1;
-		}
-		if (b == null) {
-		    throw new NullPointerException();
-		} else if (off < 0 || len < 0 || len > b.length - off) {
-		    throw new IndexOutOfBoundsException();
-		} else if (len == 0) {
-		    return 0;
-		}
-		if(currentScanData == null) {
-			synchronized(dataQueue) {
-				if(dataQueue.isEmpty()) {
-					if(endOfStream) {
-						readNext(64 * 1024, true);
-					} else {
-						try {
-							dataQueue.wait();
-							if(noMoreData()) {
-								return -1;
-							}
-						} catch (InterruptedException e) {
-						}
-					}
-				}
-				if(!dataQueue.isEmpty() && currentScanIndex > 0) {
-					currentScanData = dataQueue.poll();
-					currentScanIndex = 0;
-				}
-			}
-		} 
-		
-		int numRemainBytes = currentScanData.length - currentScanIndex;
-		if(numRemainBytes > len) {
-			System.arraycopy(currentScanData.data, currentScanIndex, b, off, len);
-			currentScanIndex += len;
-			avaliableSize.addAndGet(0 - len);
-			pos += len;
-
-      totalReadBytesForFetch.addAndGet(len);
-			return len;
-		} else {
-			int offset = off;
-			int length = 0;
-			int numCopyBytes = numRemainBytes;
-			while(true) {
-				synchronized(dataQueue) {
-					if(numCopyBytes == 0 && eof && dataQueue.isEmpty()) {
-						return -1;
-					}
-				}
-				System.arraycopy(currentScanData.data, currentScanIndex, b, offset, numCopyBytes);
-				currentScanIndex += numCopyBytes;
-				offset += numCopyBytes;
-				length += numCopyBytes;
-				if(length >= len) {
-					break;
-				}
-				synchronized(dataQueue) {
-					if(dataQueue.isEmpty()) {
-						if(eof) {
-							break;
-						}
-						if(endOfStream) {
-							readNext(64 * 1024, true);
-						} else {
-							try {
-								dataQueue.wait();
-							} catch (InterruptedException e) {
-							}
-						}
-					}
-					if(eof && dataQueue.isEmpty()) {
-						break;
-					}
-					if(!dataQueue.isEmpty() && currentScanIndex > 0) {
-						currentScanData = dataQueue.poll();
-						currentScanIndex = 0;
-					}
-					if(currentScanData == null) {
-						break;
-					}
-				}
-        if(currentScanData.length >= (len - length)) {
-          numCopyBytes = (len - length);
-        } else {
-          numCopyBytes = currentScanData.length;
-        }
-			}  //end of while
-			this.pos += length;
-			avaliableSize.addAndGet(0 - length);
-
-      totalReadBytesForFetch.addAndGet(length);
-			return length;
-		}
-	}
-
-  public long getTotalReadBytesForFetch() {
-    return totalReadBytesForFetch.get();
-  }
-
-  public long getTotalReadBytesFromDisk() {
-    return totalReadBytesFromDisk.get();
-  }
-
-	@Override
-	public void close() throws IOException {
-    LOG.info("Close:" + toString());
-		synchronized(dataQueue) {
-			if(closed.get()) {
-				return;
-			}
-			closed.set(true);
-			originStream.close();
-			dataQueue.clear();
-			currentScanIndex = 0;
-			super.close();
-		}
-	}
-
-	@Override
-	public void readFully(byte[] b) throws IOException {
-		readFully(b, 0, b.length);
-	}
-
-	@Override
-	public void readFully(byte[] b, int off, int len) throws IOException {
-		if (len < 0) {
-		    throw new IndexOutOfBoundsException();
-		}
-		int n = 0;
-		while (n < len) {
-		    int count = read(b, off + n, len - n);
-		    if (count < 0) {
-		    	throw new EOFException();
-		    }
-		    n += count;
-		}
-	}
-
-	@Override
-	public int skipBytes(int bytes) throws IOException {
-		int skipTotal = 0;
-		int currentPos = 0;
-
-		while ((skipTotal<bytes) && ((currentPos = (int)skip(bytes-skipTotal)) > 0)) {
-      skipTotal += currentPos;
-		}
-
-		return skipTotal;
-	}
-
-	@Override
-	public boolean readBoolean() throws IOException {
-		int val = read();
-		if (val < 0) {
-		    throw new EOFException();
-    }
-		return (val != 0);
-	}
-
-	@Override
-	public byte readByte() throws IOException {
-		int val = read();
-		if (val < 0) {
-		    throw new EOFException();
-    }
-		return (byte)(val);
-	}
-
-	@Override
-	public int readUnsignedByte() throws IOException {
-		int val = read();
-		if (val < 0) {
-		    throw new EOFException();
-    }
-		return val;
-	}
-
-	@Override
-	public short readShort() throws IOException {
-    int val1 = read();
-    int val2 = read();
-    if ((val1 | val2) < 0) {
-        throw new EOFException();
-    }
-    return (short)((val1 << 8) + (val2 << 0));
-	}
-
-	@Override
-	public int readUnsignedShort() throws IOException {
-    int val1 = read();
-    int val2 = read();
-    if ((val1 | val2) < 0) {
-        throw new EOFException();
-    }
-    return (val1 << 8) + (val2 << 0);
-	}
-
-	@Override
-	public char readChar() throws IOException {
-    int val1 = read();
-    int val2 = read();
-    if ((val1 | val2) < 0) {
-        throw new EOFException();
-    }
-    return (char)((val1 << 8) + (val2 << 0));
-	}
-
-	@Override
-	public int readInt() throws IOException {
-    int val1 = read();
-    int val2 = read();
-    int val3 = read();
-    int val4 = read();
-    if ((val1 | val2 | val3 | val4) < 0) {
-        throw new EOFException();
-    }
-    return ((val1 << 24) + (val2 << 16) + (val3 << 8) + (val4 << 0));
-	}
-
-	@Override
-	public long readLong() throws IOException {
-    readFully(readLongBuffer, 0, 8);
-    return  (((long) readLongBuffer[0] << 56) +
-            ((long)(readLongBuffer[1] & 255) << 48) +
-		        ((long)(readLongBuffer[2] & 255) << 40) +
-            ((long)(readLongBuffer[3] & 255) << 32) +
-            ((long)(readLongBuffer[4] & 255) << 24) +
-            ((readLongBuffer[5] & 255) << 16) +
-            ((readLongBuffer[6] & 255) <<  8) +
-            ((readLongBuffer[7] & 255) <<  0));
-	}
-
-	@Override
-	public float readFloat() throws IOException {
-		return Float.intBitsToFloat(readInt());
-	}
-
-	@Override
-	public double readDouble() throws IOException {
-		return Double.longBitsToDouble(readLong());
-	}
-
-	@Override
-	public String readLine() throws IOException {
-		throw new IOException("Unsupported operation: readLine");
-	}
-
-	@Override
-	public String readUTF() throws IOException {
-		throw new IOException("Unsupported operation: readUTF");
-	}
-
-	public boolean isEOF() {
-		return eof;
-	}
-
-	public boolean isEndOfStream() {
-		return endOfStream;
-	}
-
-  public void reset() {
-    synchronized(dataQueue) {
-      endOfStream = false;
-      eof = false;
-      closed.set(false);
-      dataQueue.clear();
-      currentScanIndex = 0;
-      currentScanData = null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java
deleted file mode 100644
index cffff00..0000000
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java
+++ /dev/null
@@ -1,140 +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.v2;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.storage.AbstractStorageManager;
-import org.apache.tajo.storage.Scanner;
-import org.apache.tajo.storage.fragment.Fragment;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.Queue;
-
-public final class StorageManagerV2 extends AbstractStorageManager {
-  private final Log LOG = LogFactory.getLog(StorageManagerV2.class);
-
-	private Queue<FileScannerV2> scanQueue = new LinkedList<FileScannerV2>();
-	
-	private Object scanQueueLock = new Object();
-	
-	private Object scanDataLock = new Object();
-	
-	private ScanScheduler scanScheduler;
-	
-	private StorgaeManagerContext context;
-
-  public StorageManagerV2(TajoConf conf) throws IOException {
-    super(conf);
-		context = new StorgaeManagerContext();
-		scanScheduler = new ScanScheduler(context);
-		scanScheduler.start();
-    LOG.info("StorageManager v2 started...");
-	}
-
-  @Override
-  public Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException {
-    Class<? extends Scanner> scannerClass;
-
-    String handlerName = storeType.name().toLowerCase();
-    String handlerNameKey = handlerName + "_v2";
-
-    scannerClass = SCANNER_HANDLER_CACHE.get(handlerNameKey);
-    if (scannerClass == null) {
-      scannerClass = conf.getClass(String.format("tajo.storage.scanner-handler.v2.%s.class",
-          storeType.name().toLowerCase()), null, Scanner.class);
-      SCANNER_HANDLER_CACHE.put(handlerNameKey, scannerClass);
-    }
-
-    return scannerClass;
-  }
-
-  @Override
-  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
-    Scanner scanner;
-
-    Class<? extends Scanner> scannerClass = getScannerClass(meta.getStoreType());
-    if (scannerClass == null) {
-      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
-    }
-
-    scanner = newScannerInstance(scannerClass, conf, schema, meta, fragment);
-    if (scanner.isProjectable()) {
-      scanner.setTarget(target.toArray());
-    }
-
-    if(scanner instanceof FileScannerV2) {
-      ((FileScannerV2)scanner).setStorageManagerContext(context);
-    }
-    return scanner;
-  }
-
-	public void requestFileScan(FileScannerV2 fileScanner) {
-		synchronized(scanQueueLock) {
-			scanQueue.offer(fileScanner);
-			
-			scanQueueLock.notifyAll();
-		}
-	}
-
-	public StorgaeManagerContext getContext() {
-		return context;
-	}
-
-  public class StorgaeManagerContext {
-		public Object getScanQueueLock() {
-			return scanQueueLock;
-		}
-
-		public Object getScanDataLock() {
-			return scanDataLock;
-		}
-		
-		public Queue<FileScannerV2> getScanQueue() {
-			return scanQueue;
-		}
-
-		public int getMaxReadBytesPerScheduleSlot() {
-			return conf.getIntVar(TajoConf.ConfVars.STORAGE_MANAGER_DISK_SCHEDULER_MAX_READ_BYTES_PER_SLOT);
-		}
-
-    public void requestFileScan(FileScannerV2 fileScanner) {
-      StorageManagerV2.this.requestFileScan(fileScanner);
-    }
-
-    public TajoConf getConf() {
-      return conf;
-    }
-
-    public void incrementReadBytes(int diskId, long[] readBytes) {
-      scanScheduler.incrementReadBytes(diskId, readBytes);
-    }
-  }
-
-	public void stop() {
-		if(scanScheduler != null) {
-			scanScheduler.stopScheduler();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/proto/IndexProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/proto/IndexProtos.proto b/tajo-core/tajo-core-storage/src/main/proto/IndexProtos.proto
deleted file mode 100644
index bcb0cbe..0000000
--- a/tajo-core/tajo-core-storage/src/main/proto/IndexProtos.proto
+++ /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.
- */
-
-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 {
-  repeated TupleComparatorSpecProto compSpecs = 1;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/main/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/resources/storage-default.xml b/tajo-core/tajo-core-storage/src/main/resources/storage-default.xml
deleted file mode 100644
index 5bf4453..0000000
--- a/tajo-core/tajo-core-storage/src/main/resources/storage-default.xml
+++ /dev/null
@@ -1,149 +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.v2</name>
-    <value>false</value>
-  </property>
-
-  <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>csv,raw,rcfile,row,trevni</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>
-
-  <!--- 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.v2.csv.class</name>
-    <value>org.apache.tajo.storage.v2.CSVFileScanner</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.v2.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.v2.rcfile.class</name>
-    <value>org.apache.tajo.storage.v2.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.v2.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.v2.trevni.class</name>
-    <value>org.apache.tajo.storage.trevni.TrevniScanner</value>
-  </property>
-
-  <!--- Appender Handler -->
-  <property>
-    <name>tajo.storage.appender-handler</name>
-    <value>csv,raw,rcfile,row,trevni</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>
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
deleted file mode 100644
index bec1556..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-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.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.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-@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}
-    });
-  }
-
-  @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 {
-      storageCompressionTest(storeType, GzipCodec.class);
-    }
-  }
-
-  @Test
-  public void testSnappyCodecCompressionData() throws IOException {
-    if (SnappyCodec.isNativeCodeLoaded()) {
-      storageCompressionTest(storeType, SnappyCodec.class);
-    }
-  }
-
-  @Test
-  public void testBzip2CodecCompressionData() throws IOException {
-    storageCompressionTest(storeType, BZip2Codec.class);
-  }
-
-  @Test
-  public void testLz4CodecCompressionData() throws IOException {
-    if(NativeCodeLoader.isNativeCodeLoaded() && Lz4Codec.isNativeCodeLoaded())
-    storageCompressionTest(storeType, Lz4Codec.class);
-  }
-
-  @Test
-  public void testSplitCompressionData() throws IOException {
-    if(StoreType.CSV != storeType) return;
-
-    Schema schema = new Schema();
-    schema.addColumn("id", Type.INT4);
-    schema.addColumn("age", Type.INT8);
-
-    TableMeta meta = CatalogUtil.newTableMeta(storeType);
-    meta.putOption("compression.codec", BZip2Codec.class.getCanonicalName());
-
-    Path tablePath = new Path(testDir, "SplitCompression");
-    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-    appender.init();
-
-    String extention = "";
-    if (appender instanceof CSVFile.CSVAppender) {
-      extention = ((CSVFile.CSVAppender) appender).getExtension();
-    }
-
-    int tupleNum = 100000;
-    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());
-    tablePath = tablePath.suffix(extention);
-
-    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("SplitCompression", tablePath, 0, randomNum);
-    tablets[1] = new FileFragment("SplitCompression", tablePath, randomNum, (fileLen - randomNum));
-
-    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-    assertTrue(scanner.isSplittable());
-    scanner.init();
-    int tupleCnt = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      tupleCnt++;
-    }
-    scanner.close();
-
-    scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
-    assertTrue(scanner.isSplittable());
-    scanner.init();
-    while ((tuple = scanner.next()) != null) {
-      tupleCnt++;
-    }
-
-    scanner.close();
-    assertEquals(tupleNum, tupleCnt);
-  }
-
-  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("rcfile.serde", TextSerializerDeserializer.class.getName());
-
-    String fileName = "Compression_" + codec.getSimpleName();
-    Path tablePath = new Path(testDir, fileName);
-    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-
-    appender.init();
-
-    String extension = "";
-    if (appender instanceof CSVFile.CSVAppender) {
-      extension = ((CSVFile.CSVAppender) 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 = StorageManagerFactory.getStorageManager(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();
-    int tupleCnt = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      tupleCnt++;
-    }
-    scanner.close();
-    assertEquals(tupleNum, tupleCnt);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestFrameTuple.java
deleted file mode 100644
index 387fed5..0000000
--- a/tajo-core/tajo-core-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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestLazyTuple.java
deleted file mode 100644
index e1430e1..0000000
--- a/tajo-core/tajo-core-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.Bytes;
-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 = Bytes.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.getColumnNum();
-
-    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.getColumnNum();
-    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.getString(0).toString(), "str");
-    assertEquals(t1.getInt(1).asInt4(), 2);
-    assertTrue(t1.getFloat(11).asFloat4() == 0.76f);
-  }
-
-  @Test
-  public void testEquals() {
-    int colNum = schema.getColumnNum();
-    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.getColumnNum();
-    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.getColumnNum();
-    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.getColumnNum()][], -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 = Bytes.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.getColumnNum();
-    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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
deleted file mode 100644
index f2a66d9..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
+++ /dev/null
@@ -1,179 +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.catalog.Options;
-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;
-  AbstractStorageManager sm;
-  private static String TEST_PATH = "target/test-data/TestMergeScanner";
-  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.TREVNI},
-        // 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", "trevni");
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-    sm = StorageManagerFactory.getStorageManager(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);
-    
-    Options options = new Options();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-
-    Path table1Path = new Path(testDir, storeType + "_1.data");
-    Appender appender1 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(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);
-    FileFragment[] fragment = new FileFragment[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.<FileFragment>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 TREVNI:
-      case CSV:
-        return true;
-      default:
-        return false;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
deleted file mode 100644
index 083670a..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.java
+++ /dev/null
@@ -1,93 +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.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.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.util.CommonTestingUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestStorageManager {
-	private TajoConf conf;
-	private static String TEST_PATH = "target/test-data/TestStorageManager";
-	AbstractStorageManager 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 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, path);
-    appender.init();
-		for(Tuple t : tuples) {
-		  appender.addTuple(t);
-		}
-		appender.close();
-
-		Scanner scanner = StorageManagerFactory.getStorageManager(conf).getFileScanner(meta, schema, path);
-    scanner.init();
-		int i=0;
-		while(scanner.next() != null) {
-			i++;
-		}
-		assertEquals(4,i);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
deleted file mode 100644
index 16b370c..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ /dev/null
@@ -1,375 +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.QueryId;
-import org.apache.tajo.TajoIdProtos;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Options;
-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.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.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 StoreType storeType;
-  private boolean splitable;
-  private boolean statsable;
-  private Path testDir;
-  private FileSystem fs;
-
-  public TestStorages(StoreType type, boolean splitable, boolean statsable) throws IOException {
-    this.storeType = type;
-    this.splitable = splitable;
-    this.statsable = statsable;
-
-    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[][] {
-        {StoreType.CSV, true, true},
-        {StoreType.RAW, false, false},
-        {StoreType.RCFILE, true, true},
-        {StoreType.TREVNI, false, true},
-    });
-  }
-		
-	@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 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-      assertTrue(scanner.isSplittable());
-      scanner.init();
-      int tupleCnt = 0;
-      while (scanner.next() != null) {
-        tupleCnt++;
-      }
-      scanner.close();
-
-      scanner = StorageManagerFactory.getStorageManager(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);
-
-    Path tablePath = new Path(testDir, "testProjection.data");
-    Appender appender = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, fragment, target);
-    scanner.init();
-    int tupleCnt = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      if (storeType == StoreType.RCFILE || storeType == StoreType.TREVNI || storeType == StoreType.CSV) {
-        assertTrue(tuple.get(0) == null);
-      }
-      assertEquals(DatumFactory.createInt8(tupleCnt + 2), tuple.getLong(1));
-      assertEquals(DatumFactory.createFloat4(tupleCnt + 3), tuple.getFloat(2));
-      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()));
-
-    Options options = new Options();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManagerFactory.getStorageManager(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 =  StorageManagerFactory.getStorageManager(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));
-      }
-    }
-  }
-
-  @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()));
-
-    Options options = new Options();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.putOption(RCFile.SERDE, TextSerializerDeserializer.class.getName());
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManagerFactory.getStorageManager(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("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);
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManagerFactory.getStorageManager(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));
-      }
-    }
-  }
-
-  @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()));
-
-    Options options = new Options();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-    meta.putOption(RCFile.SERDE, BinarySerializerDeserializer.class.getName());
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManagerFactory.getStorageManager(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("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);
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManagerFactory.getStorageManager(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));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
deleted file mode 100644
index 7092953..0000000
--- a/tajo-core/tajo-core-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.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-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 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.getColumnByFQN("col4"), true, false);
-    SortSpec sortKey2 = new SortSpec(schema.getColumnByFQN("col5"), true, false);
-
-    TupleComparator tc = new TupleComparator(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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestVTuple.java
deleted file mode 100644
index 05f47a5..0000000
--- a/tajo-core/tajo-core-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.getString(0).toString(),"str");
-		assertEquals(t1.getInt(1).asInt4(),2);
-		assertTrue(t1.getFloat(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));
-  }
-}


[02/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java
new file mode 100644
index 0000000..12b984e
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/ScheduledInputStream.java
@@ -0,0 +1,513 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Seekable;
+
+import java.io.*;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ScheduledInputStream extends InputStream implements Seekable, Closeable, DataInput {
+  private static final Log LOG = LogFactory.getLog(ScheduledInputStream.class);
+
+	private FSDataInputStream originStream;
+
+  private int currentScanIndex;
+
+  private Queue<ScanData> dataQueue = new LinkedList<ScanData>();
+
+  private ScanData currentScanData;
+
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  private boolean eof = false;
+
+  private long pos;
+
+  private AtomicInteger avaliableSize = new AtomicInteger(0);
+
+  private long fileLen;
+
+  private long startOffset;
+
+  private long length;
+
+  private long endOffset;
+
+  private boolean endOfStream = false;
+
+  private Path file;
+
+  private byte readLongBuffer[] = new byte[8];
+
+  private AtomicLong totalReadBytesForFetch = new AtomicLong(0);
+
+  private AtomicLong totalReadBytesFromDisk = new AtomicLong(0);
+
+	public ScheduledInputStream(Path file, FSDataInputStream originStream,
+                              long startOffset, long length, long fileLen) throws IOException {
+		this.originStream = originStream;
+		this.startOffset = startOffset;
+		this.length = length;
+		this.endOffset = startOffset + length;
+		this.fileLen = fileLen;
+    this.file = file;
+		this.pos = this.originStream.getPos();
+
+    LOG.info("Open:" + toString());
+	}
+
+	public int getAvaliableSize() {
+		return avaliableSize.get();
+	}
+
+  public String toString() {
+    return file.getName() + ":" + startOffset + ":" + length;
+  }
+	public boolean readNext(int length) throws IOException {
+		return readNext(length, false);
+	}
+	
+	public boolean readNext(int length, boolean ignoreEOS) throws IOException {
+    synchronized(dataQueue) {
+      if(closed.get() || (!ignoreEOS && endOfStream)) {
+        return false;
+      }
+      int bufLength = ignoreEOS ? length : (int)Math.min(length,  endOffset - originStream.getPos());
+      bufLength = (int)Math.min(bufLength, fileLen - originStream.getPos());
+      if(bufLength == 0) {
+        return false;
+      }
+			byte[] buf = new byte[bufLength];
+
+      try {
+        originStream.readFully(buf);
+      } catch (EOFException e) {
+        LOG.error(e.getMessage(), e);
+        throw e;
+      } catch (Exception e) {
+        throw new IOException(e.getMessage(), e);
+      }
+
+      if(originStream.getPos() == fileLen) {
+        LOG.info("EOF:" + toString());
+        eof = true;
+      }
+      if(!ignoreEOS && originStream.getPos() >= endOffset) {
+        LOG.info("EndOfStream:" + toString());
+        endOfStream = true;
+      }
+
+      if(currentScanData == null) {
+        currentScanData = new ScanData(buf, bufLength);
+        currentScanIndex = 0;
+      } else {
+        dataQueue.offer(new ScanData(buf, bufLength));
+      }
+
+      avaliableSize.addAndGet(bufLength);
+
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Add DataQueue: queue=" + dataQueue.size() +
+          ", avaliable Size=" + avaliableSize.get() + ", pos=" + getPos() +
+          ", originPos=" + originStream.getPos() + ",endOfStream=" + endOfStream +
+          ", bufLength=" + bufLength + ",ignoreEOS=" + ignoreEOS);
+      }
+
+      totalReadBytesFromDisk.addAndGet(bufLength);
+      dataQueue.notifyAll();
+    }
+    return !eof;
+	}
+	
+	static class ScanData {
+		byte[] data;
+		int length;
+		public ScanData(byte[] buf, int length) {
+			this.data = buf;
+			this.length = length;
+		}
+		
+		@Override
+		public String toString() {
+			return "length=" + length;
+		}
+	}
+
+	@Override
+	public void seek(long pos) throws IOException {
+		synchronized(dataQueue) {
+			dataQueue.clear();
+			currentScanData = null;
+			currentScanIndex = 0;
+			avaliableSize.set(0);
+      originStream.seek(pos);
+      this.pos = pos;
+    }
+	}
+
+	@Override
+	public long getPos() throws IOException {
+		return this.pos;
+	}
+
+	public long getOriginStreamPos() {
+		try {
+			return this.originStream.getPos();
+		} catch (IOException e) {
+			e.printStackTrace();
+			return 0;
+		}
+	}
+	
+	@Override
+	public boolean seekToNewSource(long targetPos) throws IOException {
+		synchronized(dataQueue) {
+			dataQueue.clear();
+			currentScanData = null;
+			currentScanIndex = 0;
+			avaliableSize.set(0);
+      boolean result = originStream.seekToNewSource(targetPos);
+
+      this.pos = originStream.getPos();
+      return result;
+		}
+	}
+
+	@Override
+	public int read() throws IOException {
+		if(noMoreData()) {
+			return -1;
+		}
+		if(currentScanData == null || currentScanIndex >= currentScanData.length) {
+			synchronized(dataQueue) {
+				if(dataQueue.isEmpty()) {
+					if(endOfStream) {
+						readNext(64 * 1024, true);
+					} else {
+						try {
+							dataQueue.wait();
+							if(eof && dataQueue.isEmpty() && currentScanIndex > 0) {
+								//no more data
+								return -1;
+							}
+						} catch (InterruptedException e) {
+						}
+					}
+				}
+				if(!dataQueue.isEmpty() && currentScanIndex > 0) {
+					currentScanData = dataQueue.poll();
+					currentScanIndex = 0;
+				}
+			}
+		} 
+		
+		this.pos++;
+		avaliableSize.decrementAndGet();
+    totalReadBytesForFetch.incrementAndGet();
+
+		return currentScanData.data[currentScanIndex++] & 0xff;
+	}
+	
+	private boolean noMoreData() {
+		return closed.get();
+	}
+	
+	public int read(byte b[], int off, int len) throws IOException {
+		if(noMoreData()) {
+			return -1;
+		}
+		if (b == null) {
+		    throw new NullPointerException();
+		} else if (off < 0 || len < 0 || len > b.length - off) {
+		    throw new IndexOutOfBoundsException();
+		} else if (len == 0) {
+		    return 0;
+		}
+		if(currentScanData == null) {
+			synchronized(dataQueue) {
+				if(dataQueue.isEmpty()) {
+					if(endOfStream) {
+						readNext(64 * 1024, true);
+					} else {
+						try {
+							dataQueue.wait();
+							if(noMoreData()) {
+								return -1;
+							}
+						} catch (InterruptedException e) {
+						}
+					}
+				}
+				if(!dataQueue.isEmpty() && currentScanIndex > 0) {
+					currentScanData = dataQueue.poll();
+					currentScanIndex = 0;
+				}
+			}
+		} 
+		
+		int numRemainBytes = currentScanData.length - currentScanIndex;
+		if(numRemainBytes > len) {
+			System.arraycopy(currentScanData.data, currentScanIndex, b, off, len);
+			currentScanIndex += len;
+			avaliableSize.addAndGet(0 - len);
+			pos += len;
+
+      totalReadBytesForFetch.addAndGet(len);
+			return len;
+		} else {
+			int offset = off;
+			int length = 0;
+			int numCopyBytes = numRemainBytes;
+			while(true) {
+				synchronized(dataQueue) {
+					if(numCopyBytes == 0 && eof && dataQueue.isEmpty()) {
+						return -1;
+					}
+				}
+				System.arraycopy(currentScanData.data, currentScanIndex, b, offset, numCopyBytes);
+				currentScanIndex += numCopyBytes;
+				offset += numCopyBytes;
+				length += numCopyBytes;
+				if(length >= len) {
+					break;
+				}
+				synchronized(dataQueue) {
+					if(dataQueue.isEmpty()) {
+						if(eof) {
+							break;
+						}
+						if(endOfStream) {
+							readNext(64 * 1024, true);
+						} else {
+							try {
+								dataQueue.wait();
+							} catch (InterruptedException e) {
+							}
+						}
+					}
+					if(eof && dataQueue.isEmpty()) {
+						break;
+					}
+					if(!dataQueue.isEmpty() && currentScanIndex > 0) {
+						currentScanData = dataQueue.poll();
+						currentScanIndex = 0;
+					}
+					if(currentScanData == null) {
+						break;
+					}
+				}
+        if(currentScanData.length >= (len - length)) {
+          numCopyBytes = (len - length);
+        } else {
+          numCopyBytes = currentScanData.length;
+        }
+			}  //end of while
+			this.pos += length;
+			avaliableSize.addAndGet(0 - length);
+
+      totalReadBytesForFetch.addAndGet(length);
+			return length;
+		}
+	}
+
+  public long getTotalReadBytesForFetch() {
+    return totalReadBytesForFetch.get();
+  }
+
+  public long getTotalReadBytesFromDisk() {
+    return totalReadBytesFromDisk.get();
+  }
+
+	@Override
+	public void close() throws IOException {
+    LOG.info("Close:" + toString());
+		synchronized(dataQueue) {
+			if(closed.get()) {
+				return;
+			}
+			closed.set(true);
+			originStream.close();
+			dataQueue.clear();
+			currentScanIndex = 0;
+			super.close();
+		}
+	}
+
+	@Override
+	public void readFully(byte[] b) throws IOException {
+		readFully(b, 0, b.length);
+	}
+
+	@Override
+	public void readFully(byte[] b, int off, int len) throws IOException {
+		if (len < 0) {
+		    throw new IndexOutOfBoundsException();
+		}
+		int n = 0;
+		while (n < len) {
+		    int count = read(b, off + n, len - n);
+		    if (count < 0) {
+		    	throw new EOFException();
+		    }
+		    n += count;
+		}
+	}
+
+	@Override
+	public int skipBytes(int bytes) throws IOException {
+		int skipTotal = 0;
+		int currentPos = 0;
+
+		while ((skipTotal<bytes) && ((currentPos = (int)skip(bytes-skipTotal)) > 0)) {
+      skipTotal += currentPos;
+		}
+
+		return skipTotal;
+	}
+
+	@Override
+	public boolean readBoolean() throws IOException {
+		int val = read();
+		if (val < 0) {
+		    throw new EOFException();
+    }
+		return (val != 0);
+	}
+
+	@Override
+	public byte readByte() throws IOException {
+		int val = read();
+		if (val < 0) {
+		    throw new EOFException();
+    }
+		return (byte)(val);
+	}
+
+	@Override
+	public int readUnsignedByte() throws IOException {
+		int val = read();
+		if (val < 0) {
+		    throw new EOFException();
+    }
+		return val;
+	}
+
+	@Override
+	public short readShort() throws IOException {
+    int val1 = read();
+    int val2 = read();
+    if ((val1 | val2) < 0) {
+        throw new EOFException();
+    }
+    return (short)((val1 << 8) + (val2 << 0));
+	}
+
+	@Override
+	public int readUnsignedShort() throws IOException {
+    int val1 = read();
+    int val2 = read();
+    if ((val1 | val2) < 0) {
+        throw new EOFException();
+    }
+    return (val1 << 8) + (val2 << 0);
+	}
+
+	@Override
+	public char readChar() throws IOException {
+    int val1 = read();
+    int val2 = read();
+    if ((val1 | val2) < 0) {
+        throw new EOFException();
+    }
+    return (char)((val1 << 8) + (val2 << 0));
+	}
+
+	@Override
+	public int readInt() throws IOException {
+    int val1 = read();
+    int val2 = read();
+    int val3 = read();
+    int val4 = read();
+    if ((val1 | val2 | val3 | val4) < 0) {
+        throw new EOFException();
+    }
+    return ((val1 << 24) + (val2 << 16) + (val3 << 8) + (val4 << 0));
+	}
+
+	@Override
+	public long readLong() throws IOException {
+    readFully(readLongBuffer, 0, 8);
+    return  (((long) readLongBuffer[0] << 56) +
+            ((long)(readLongBuffer[1] & 255) << 48) +
+		        ((long)(readLongBuffer[2] & 255) << 40) +
+            ((long)(readLongBuffer[3] & 255) << 32) +
+            ((long)(readLongBuffer[4] & 255) << 24) +
+            ((readLongBuffer[5] & 255) << 16) +
+            ((readLongBuffer[6] & 255) <<  8) +
+            ((readLongBuffer[7] & 255) <<  0));
+	}
+
+	@Override
+	public float readFloat() throws IOException {
+		return Float.intBitsToFloat(readInt());
+	}
+
+	@Override
+	public double readDouble() throws IOException {
+		return Double.longBitsToDouble(readLong());
+	}
+
+	@Override
+	public String readLine() throws IOException {
+		throw new IOException("Unsupported operation: readLine");
+	}
+
+	@Override
+	public String readUTF() throws IOException {
+		throw new IOException("Unsupported operation: readUTF");
+	}
+
+	public boolean isEOF() {
+		return eof;
+	}
+
+	public boolean isEndOfStream() {
+		return endOfStream;
+	}
+
+  public void reset() {
+    synchronized(dataQueue) {
+      endOfStream = false;
+      eof = false;
+      closed.set(false);
+      dataQueue.clear();
+      currentScanIndex = 0;
+      currentScanData = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java
new file mode 100644
index 0000000..cffff00
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/v2/StorageManagerV2.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.v2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.AbstractStorageManager;
+import org.apache.tajo.storage.Scanner;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.Queue;
+
+public final class StorageManagerV2 extends AbstractStorageManager {
+  private final Log LOG = LogFactory.getLog(StorageManagerV2.class);
+
+	private Queue<FileScannerV2> scanQueue = new LinkedList<FileScannerV2>();
+	
+	private Object scanQueueLock = new Object();
+	
+	private Object scanDataLock = new Object();
+	
+	private ScanScheduler scanScheduler;
+	
+	private StorgaeManagerContext context;
+
+  public StorageManagerV2(TajoConf conf) throws IOException {
+    super(conf);
+		context = new StorgaeManagerContext();
+		scanScheduler = new ScanScheduler(context);
+		scanScheduler.start();
+    LOG.info("StorageManager v2 started...");
+	}
+
+  @Override
+  public Class<? extends Scanner> getScannerClass(CatalogProtos.StoreType storeType) throws IOException {
+    Class<? extends Scanner> scannerClass;
+
+    String handlerName = storeType.name().toLowerCase();
+    String handlerNameKey = handlerName + "_v2";
+
+    scannerClass = SCANNER_HANDLER_CACHE.get(handlerNameKey);
+    if (scannerClass == null) {
+      scannerClass = conf.getClass(String.format("tajo.storage.scanner-handler.v2.%s.class",
+          storeType.name().toLowerCase()), null, Scanner.class);
+      SCANNER_HANDLER_CACHE.put(handlerNameKey, scannerClass);
+    }
+
+    return scannerClass;
+  }
+
+  @Override
+  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
+    Scanner scanner;
+
+    Class<? extends Scanner> scannerClass = getScannerClass(meta.getStoreType());
+    if (scannerClass == null) {
+      throw new IOException("Unknown Storage Type: " + meta.getStoreType());
+    }
+
+    scanner = newScannerInstance(scannerClass, conf, schema, meta, fragment);
+    if (scanner.isProjectable()) {
+      scanner.setTarget(target.toArray());
+    }
+
+    if(scanner instanceof FileScannerV2) {
+      ((FileScannerV2)scanner).setStorageManagerContext(context);
+    }
+    return scanner;
+  }
+
+	public void requestFileScan(FileScannerV2 fileScanner) {
+		synchronized(scanQueueLock) {
+			scanQueue.offer(fileScanner);
+			
+			scanQueueLock.notifyAll();
+		}
+	}
+
+	public StorgaeManagerContext getContext() {
+		return context;
+	}
+
+  public class StorgaeManagerContext {
+		public Object getScanQueueLock() {
+			return scanQueueLock;
+		}
+
+		public Object getScanDataLock() {
+			return scanDataLock;
+		}
+		
+		public Queue<FileScannerV2> getScanQueue() {
+			return scanQueue;
+		}
+
+		public int getMaxReadBytesPerScheduleSlot() {
+			return conf.getIntVar(TajoConf.ConfVars.STORAGE_MANAGER_DISK_SCHEDULER_MAX_READ_BYTES_PER_SLOT);
+		}
+
+    public void requestFileScan(FileScannerV2 fileScanner) {
+      StorageManagerV2.this.requestFileScan(fileScanner);
+    }
+
+    public TajoConf getConf() {
+      return conf;
+    }
+
+    public void incrementReadBytes(int diskId, long[] readBytes) {
+      scanScheduler.incrementReadBytes(diskId, readBytes);
+    }
+  }
+
+	public void stop() {
+		if(scanScheduler != null) {
+			scanScheduler.stopScheduler();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..bcb0cbe
--- /dev/null
+++ b/tajo-storage/src/main/proto/IndexProtos.proto
@@ -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.
+ */
+
+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 {
+  repeated TupleComparatorSpecProto compSpecs = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..5bf4453
--- /dev/null
+++ b/tajo-storage/src/main/resources/storage-default.xml
@@ -0,0 +1,149 @@
+<?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.v2</name>
+    <value>false</value>
+  </property>
+
+  <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>csv,raw,rcfile,row,trevni</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>
+
+  <!--- 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.v2.csv.class</name>
+    <value>org.apache.tajo.storage.v2.CSVFileScanner</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.v2.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.v2.rcfile.class</name>
+    <value>org.apache.tajo.storage.v2.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.v2.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.v2.trevni.class</name>
+    <value>org.apache.tajo.storage.trevni.TrevniScanner</value>
+  </property>
+
+  <!--- Appender Handler -->
+  <property>
+    <name>tajo.storage.appender-handler</name>
+    <value>csv,raw,rcfile,row,trevni</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>
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..bec1556
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
@@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.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.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@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}
+    });
+  }
+
+  @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 {
+      storageCompressionTest(storeType, GzipCodec.class);
+    }
+  }
+
+  @Test
+  public void testSnappyCodecCompressionData() throws IOException {
+    if (SnappyCodec.isNativeCodeLoaded()) {
+      storageCompressionTest(storeType, SnappyCodec.class);
+    }
+  }
+
+  @Test
+  public void testBzip2CodecCompressionData() throws IOException {
+    storageCompressionTest(storeType, BZip2Codec.class);
+  }
+
+  @Test
+  public void testLz4CodecCompressionData() throws IOException {
+    if(NativeCodeLoader.isNativeCodeLoaded() && Lz4Codec.isNativeCodeLoaded())
+    storageCompressionTest(storeType, Lz4Codec.class);
+  }
+
+  @Test
+  public void testSplitCompressionData() throws IOException {
+    if(StoreType.CSV != storeType) return;
+
+    Schema schema = new Schema();
+    schema.addColumn("id", Type.INT4);
+    schema.addColumn("age", Type.INT8);
+
+    TableMeta meta = CatalogUtil.newTableMeta(storeType);
+    meta.putOption("compression.codec", BZip2Codec.class.getCanonicalName());
+
+    Path tablePath = new Path(testDir, "SplitCompression");
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    appender.enableStats();
+    appender.init();
+
+    String extention = "";
+    if (appender instanceof CSVFile.CSVAppender) {
+      extention = ((CSVFile.CSVAppender) appender).getExtension();
+    }
+
+    int tupleNum = 100000;
+    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());
+    tablePath = tablePath.suffix(extention);
+
+    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("SplitCompression", tablePath, 0, randomNum);
+    tablets[1] = new FileFragment("SplitCompression", tablePath, randomNum, (fileLen - randomNum));
+
+    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+    assertTrue(scanner.isSplittable());
+    scanner.init();
+    int tupleCnt = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      tupleCnt++;
+    }
+    scanner.close();
+
+    scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
+    assertTrue(scanner.isSplittable());
+    scanner.init();
+    while ((tuple = scanner.next()) != null) {
+      tupleCnt++;
+    }
+
+    scanner.close();
+    assertEquals(tupleNum, tupleCnt);
+  }
+
+  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("rcfile.serde", TextSerializerDeserializer.class.getName());
+
+    String fileName = "Compression_" + codec.getSimpleName();
+    Path tablePath = new Path(testDir, fileName);
+    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
+    appender.enableStats();
+
+    appender.init();
+
+    String extension = "";
+    if (appender instanceof CSVFile.CSVAppender) {
+      extension = ((CSVFile.CSVAppender) 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 = StorageManagerFactory.getStorageManager(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();
+    int tupleCnt = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      tupleCnt++;
+    }
+    scanner.close();
+    assertEquals(tupleNum, tupleCnt);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..387fed5
--- /dev/null
+++ b/tajo-storage/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.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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..e1430e1
--- /dev/null
+++ b/tajo-storage/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.Bytes;
+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 = Bytes.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.getColumnNum();
+
+    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.getColumnNum();
+    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.getString(0).toString(), "str");
+    assertEquals(t1.getInt(1).asInt4(), 2);
+    assertTrue(t1.getFloat(11).asFloat4() == 0.76f);
+  }
+
+  @Test
+  public void testEquals() {
+    int colNum = schema.getColumnNum();
+    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.getColumnNum();
+    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.getColumnNum();
+    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.getColumnNum()][], -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 = Bytes.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.getColumnNum();
+    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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..f2a66d9
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestMergeScanner.java
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Options;
+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;
+  AbstractStorageManager sm;
+  private static String TEST_PATH = "target/test-data/TestMergeScanner";
+  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.TREVNI},
+        // 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", "trevni");
+    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
+    fs = testDir.getFileSystem(conf);
+    sm = StorageManagerFactory.getStorageManager(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);
+    
+    Options options = new Options();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+
+    Path table1Path = new Path(testDir, storeType + "_1.data");
+    Appender appender1 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(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);
+    FileFragment[] fragment = new FileFragment[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.<FileFragment>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 TREVNI:
+      case CSV:
+        return true;
+      default:
+        return false;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..083670a
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorageManager.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;
+
+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.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.util.CommonTestingUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStorageManager {
+	private TajoConf conf;
+	private static String TEST_PATH = "target/test-data/TestStorageManager";
+	AbstractStorageManager 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 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, path);
+    appender.init();
+		for(Tuple t : tuples) {
+		  appender.addTuple(t);
+		}
+		appender.close();
+
+		Scanner scanner = StorageManagerFactory.getStorageManager(conf).getFileScanner(meta, schema, path);
+    scanner.init();
+		int i=0;
+		while(scanner.next() != null) {
+			i++;
+		}
+		assertEquals(4,i);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..16b370c
--- /dev/null
+++ b/tajo-storage/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -0,0 +1,375 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.QueryId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Options;
+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.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.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 StoreType storeType;
+  private boolean splitable;
+  private boolean statsable;
+  private Path testDir;
+  private FileSystem fs;
+
+  public TestStorages(StoreType type, boolean splitable, boolean statsable) throws IOException {
+    this.storeType = type;
+    this.splitable = splitable;
+    this.statsable = statsable;
+
+    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[][] {
+        {StoreType.CSV, true, true},
+        {StoreType.RAW, false, false},
+        {StoreType.RCFILE, true, true},
+        {StoreType.TREVNI, false, true},
+    });
+  }
+		
+	@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 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
+      assertTrue(scanner.isSplittable());
+      scanner.init();
+      int tupleCnt = 0;
+      while (scanner.next() != null) {
+        tupleCnt++;
+      }
+      scanner.close();
+
+      scanner = StorageManagerFactory.getStorageManager(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);
+
+    Path tablePath = new Path(testDir, "testProjection.data");
+    Appender appender = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, fragment, target);
+    scanner.init();
+    int tupleCnt = 0;
+    Tuple tuple;
+    while ((tuple = scanner.next()) != null) {
+      if (storeType == StoreType.RCFILE || storeType == StoreType.TREVNI || storeType == StoreType.CSV) {
+        assertTrue(tuple.get(0) == null);
+      }
+      assertEquals(DatumFactory.createInt8(tupleCnt + 2), tuple.getLong(1));
+      assertEquals(DatumFactory.createFloat4(tupleCnt + 3), tuple.getFloat(2));
+      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()));
+
+    Options options = new Options();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    Appender appender = StorageManagerFactory.getStorageManager(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 =  StorageManagerFactory.getStorageManager(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));
+      }
+    }
+  }
+
+  @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()));
+
+    Options options = new Options();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.putOption(RCFile.SERDE, TextSerializerDeserializer.class.getName());
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    Appender appender = StorageManagerFactory.getStorageManager(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("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);
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  StorageManagerFactory.getStorageManager(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));
+      }
+    }
+  }
+
+  @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()));
+
+    Options options = new Options();
+    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
+    meta.putOption(RCFile.SERDE, BinarySerializerDeserializer.class.getName());
+
+    Path tablePath = new Path(testDir, "testVariousTypes.data");
+    Appender appender = StorageManagerFactory.getStorageManager(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("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);
+    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
+    Scanner scanner =  StorageManagerFactory.getStorageManager(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));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..7092953
--- /dev/null
+++ b/tajo-storage/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.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+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 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.getColumnByFQN("col4"), true, false);
+    SortSpec sortKey2 = new SortSpec(schema.getColumnByFQN("col5"), true, false);
+
+    TupleComparator tc = new TupleComparator(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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..05f47a5
--- /dev/null
+++ b/tajo-storage/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.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.getString(0).toString(),"str");
+		assertEquals(t1.getInt(1).asInt4(),2);
+		assertTrue(t1.getFloat(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));
+  }
+}


[10/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
deleted file mode 100644
index 34c362c..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
+++ /dev/null
@@ -1,948 +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.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-@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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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.getColumnNum());
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + storeType + ".idx"), keySchema, comp);
-    reader.open();
-    scanner = StorageManagerFactory.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) StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema,
-        tablePath);
-    appender.init();
-
-    SortSpec[] sortKeys = new SortSpec[2];
-    sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    TupleComparator comp = new TupleComparator(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.getColumnNum());
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testBuildIndexWithAppender_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    SeekableScanner scanner = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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.getColumnNum());
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindMinValue_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    scanner = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), false, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), false, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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.getColumnNum());
-
-    BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValueDescOrder_" + storeType + ".idx"),
-        keySchema, comp);
-    reader.open();
-    scanner = StorageManagerFactory.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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), false, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), false, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    TupleComparator comp = new TupleComparator(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 = StorageManagerFactory.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 = StorageManagerFactory.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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
deleted file mode 100644
index 6d5d90f..0000000
--- a/tajo-core/tajo-core-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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("long"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("long", Type.INT8));
-    keySchema.addColumn(new Column("double", Type.FLOAT8));
-
-    TupleComparator comp = new TupleComparator(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.getColumnNum());
-    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 = StorageManagerFactory.getStorageManager(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.getColumnByFQN("int"), true, false);
-    sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
-
-    Schema keySchema = new Schema();
-    keySchema.addColumn(new Column("int", Type.INT4));
-    keySchema.addColumn(new Column("long", Type.INT8));
-
-    TupleComparator comp = new TupleComparator(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/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java
deleted file mode 100644
index 7e95b8b..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVCompression.java
+++ /dev/null
@@ -1,213 +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.v2;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.*;
-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;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes;
-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.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.assertEquals;
-
-@RunWith(Parameterized.class)
-public class TestCSVCompression {
-  private TajoConf conf;
-  private static String TEST_PATH = "target/test-data/v2/TestCSVCompression";
-
-  private CatalogProtos.StoreType storeType;
-  private Path testDir;
-  private FileSystem fs;
-
-  public TestCSVCompression(CatalogProtos.StoreType type) throws IOException {
-    this.storeType = type;
-    conf = new TajoConf();
-    conf.setBoolVar(TajoConf.ConfVars.STORAGE_MANAGER_VERSION_2, true);
-
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> generateParameters() {
-    return Arrays.asList(new Object[][]{
-        {CatalogProtos.StoreType.CSV}
-    });
-  }
-
-  @Test
-  public void testDeflateCodecCompressionData() throws IOException {
-    storageCompressionTest(storeType, DeflateCodec.class);
-  }
-
-  @Test
-  public void testGzipCodecCompressionData() throws IOException {
-    storageCompressionTest(storeType, GzipCodec.class);
-  }
-
-  @Test
-  public void testSnappyCodecCompressionData() throws IOException {
-    if (SnappyCodec.isNativeCodeLoaded()) {
-      storageCompressionTest(storeType, SnappyCodec.class);
-    }
-  }
-
-  @Test
-  public void testBzip2CodecCompressionData() throws IOException {
-    storageCompressionTest(storeType, BZip2Codec.class);
-  }
-
-  @Test
-  public void testLz4CodecCompressionData() throws IOException {
-    if(NativeCodeLoader.isNativeCodeLoaded() && Lz4Codec.isNativeCodeLoaded())
-      storageCompressionTest(storeType, Lz4Codec.class);
-  }
-
-  // TODO - See https://issues.apache.org/jira/browse/HADOOP-9622
-  //@Test
-  public void testSplitCompressionData() throws IOException {
-
-    Schema schema = new Schema();
-    schema.addColumn("id", TajoDataTypes.Type.INT4);
-    schema.addColumn("age", TajoDataTypes.Type.INT8);
-
-    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV);
-    meta.putOption("compression.codec", BZip2Codec.class.getCanonicalName());
-
-    Path tablePath = new Path(testDir, "SplitCompression");
-    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-
-    appender.init();
-
-    String extention = "";
-    if (appender instanceof CSVFile.CSVAppender) {
-      extention = ((CSVFile.CSVAppender) appender).getExtension();
-    }
-
-    int tupleNum = 100000;
-    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());
-    tablePath = tablePath.suffix(extention);
-
-    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("SplitCompression", tablePath, 0, randomNum);
-    tablets[1] = new FileFragment("SplitCompression", tablePath, randomNum, (fileLen - randomNum));
-
-    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-    scanner.init();
-    int tupleCnt = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      tupleCnt++;
-    }
-    scanner.close();
-
-    scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
-    scanner.init();
-    while ((tuple = scanner.next()) != null) {
-      tupleCnt++;
-    }
-
-    scanner.close();
-    assertEquals(tupleNum, tupleCnt);
-  }
-
-  private void storageCompressionTest(CatalogProtos.StoreType storeType, Class<? extends CompressionCodec> codec)
-      throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("id", TajoDataTypes.Type.INT4);
-    schema.addColumn("age", TajoDataTypes.Type.INT8);
-
-    TableMeta meta = CatalogUtil.newTableMeta(storeType);
-    meta.putOption("compression.codec", codec.getCanonicalName());
-
-    String fileName = "Compression_" + codec.getSimpleName();
-    Path tablePath = new Path(testDir, fileName);
-    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.enableStats();
-
-    appender.init();
-
-    String extension = "";
-    if (appender instanceof CSVFile.CSVAppender) {
-      extension = ((CSVFile.CSVAppender) appender).getExtension();
-    }
-
-    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());
-    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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-    scanner.init();
-    int tupleCnt = 0;
-    while (scanner.next() != null) {
-      tupleCnt++;
-    }
-    scanner.close();
-    assertEquals(tupleCnt, tupleNum);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java
deleted file mode 100644
index c356548..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestCSVScanner.java
+++ /dev/null
@@ -1,168 +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.v2;
-
-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;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.util.CommonTestingUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-import static org.junit.Assert.assertEquals;
-
-public class TestCSVScanner {
-  private TajoConf conf;
-  private static String TEST_PATH = "target/test-data/v2/TestCSVScanner";
-  AbstractStorageManager sm = null;
-  private Path testDir;
-  private FileSystem fs;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = new TajoConf();
-    conf.setBoolVar(ConfVars.STORAGE_MANAGER_VERSION_2, true);
-    testDir = CommonTestingUtil.getTestDir(TEST_PATH);
-    fs = testDir.getFileSystem(conf);
-    sm = StorageManagerFactory.getStorageManager(conf, testDir);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
-  @Test
-  public final void testGetScannerAndAppender() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("id", TajoDataTypes.Type.INT4);
-    schema.addColumn("age", TajoDataTypes.Type.INT4);
-    schema.addColumn("name", TajoDataTypes.Type.TEXT);
-
-    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.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 = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, path);
-    appender.init();
-    for(Tuple t : tuples) {
-      appender.addTuple(t);
-    }
-    appender.close();
-
-    Scanner scanner = StorageManagerFactory.getStorageManager(conf).getFileScanner(meta, schema, path);
-    scanner.init();
-    int i=0;
-    Tuple tuple = null;
-    while( (tuple = scanner.next()) != null) {
-      i++;
-    }
-    assertEquals(4,i);
-  }
-
-  @Test
-  public final void testPartitionFile() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("key", TajoDataTypes.Type.TEXT);
-    schema.addColumn("age", TajoDataTypes.Type.INT4);
-    schema.addColumn("name", TajoDataTypes.Type.TEXT);
-
-    TableMeta meta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.CSV);
-
-
-    Path path = StorageUtil.concatPath(testDir, "testPartitionFile", "table.csv");
-    fs.mkdirs(path.getParent());
-    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, path);
-    appender.init();
-
-    String keyValue = "";
-    for(int i = 0; i < 100; i++) {
-      keyValue += "0123456789";
-    }
-    keyValue = "key_" + keyValue + "_";
-
-    String nameValue = "";
-    for(int i = 0; i < 100; i++) {
-      nameValue += "0123456789";
-    }
-    nameValue = "name_" + nameValue + "_";
-
-    int numTuples = 100000;
-    for(int i = 0; i < numTuples; i++) {
-      Tuple tuple = new VTuple(3);
-      tuple.put(new Datum[] {
-          DatumFactory.createText(keyValue + i),
-          DatumFactory.createInt4(i + 32),
-          DatumFactory.createText(nameValue + i)});
-      appender.addTuple(tuple);
-    }
-    appender.close();
-
-    long fileLength = fs.getLength(path);
-    long totalTupleCount = 0;
-
-    int scanCount = 0;
-    Tuple startTuple = null;
-    Tuple lastTuple = null;
-    while(true) {
-      long startOffset = (64 * 1024 * 1024) * scanCount;
-      long length = Math.min(64 * 1024 * 1024, fileLength - startOffset);
-
-      FileFragment fragment = new FileFragment("Test", path, startOffset, length, null);
-      Scanner scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, fragment, schema);
-      scanner.init();
-      Tuple tuple = null;
-      while( (tuple = scanner.next()) != null) {
-        if(startTuple == null) {
-          startTuple = tuple;
-        }
-        lastTuple = tuple;
-        totalTupleCount++;
-      }
-      scanCount++;
-      if(length < 64 * 1024 * 1024) {
-        break;
-      }
-    }
-    assertEquals(numTuples, totalTupleCount);
-    assertEquals(keyValue + 0, startTuple.get(0).toString());
-    assertEquals(keyValue + (numTuples - 1), lastTuple.get(0).toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java
deleted file mode 100644
index a3d3133..0000000
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/v2/TestStorages.java
+++ /dev/null
@@ -1,242 +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.v2;
-
-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.Options;
-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.storage.*;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.rcfile.RCFile;
-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.apache.tajo.conf.TajoConf.ConfVars;
-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/v2/TestStorages";
-
-  private StoreType storeType;
-  private boolean splitable;
-  private boolean statsable;
-  private Path testDir;
-  private FileSystem fs;
-
-  public TestStorages(StoreType type, boolean splitable, boolean statsable) throws IOException {
-    this.storeType = type;
-    this.splitable = splitable;
-    this.statsable = statsable;
-
-    conf = new TajoConf();
-    conf.setBoolVar(ConfVars.STORAGE_MANAGER_VERSION_2, true);
-
-    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[][]{
-        {StoreType.CSV, true, true},
-        {StoreType.RCFILE, true, true},
-        {StoreType.TREVNI, false, true},
-        {StoreType.RAW, false, 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 = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[0], schema);
-      scanner.init();
-      int tupleCnt = 0;
-      while (scanner.next() != null) {
-        tupleCnt++;
-      }
-      scanner.close();
-
-      scanner = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, tablets[1], schema);
-      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);
-
-    Path tablePath = new Path(testDir, "testProjection.data");
-    Appender appender = StorageManagerFactory.getStorageManager(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 = StorageManagerFactory.getStorageManager(conf).getScanner(meta, schema, fragment, target);
-    scanner.init();
-    int tupleCnt = 0;
-    Tuple tuple;
-    while ((tuple = scanner.next()) != null) {
-      if (storeType == StoreType.RCFILE || storeType == StoreType.TREVNI || storeType == StoreType.CSV) {
-        assertTrue(tuple.get(0) == null || tuple.get(0) instanceof NullDatum);
-      }
-      assertEquals(DatumFactory.createInt8(tupleCnt + 2), tuple.getLong(1));
-      assertEquals(DatumFactory.createFloat4(tupleCnt + 3), tuple.getFloat(2));
-      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);
-
-    Options options = new Options();
-    TableMeta meta = CatalogUtil.newTableMeta(storeType, options);
-
-    Path tablePath = new Path(testDir, "testVariousTypes.data");
-    Appender appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, schema, tablePath);
-    appender.init();
-
-    Tuple tuple = new VTuple(12);
-    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()
-    });
-    appender.addTuple(tuple);
-    appender.flush();
-    appender.close();
-
-    FileStatus status = fs.getFileStatus(tablePath);
-    FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen());
-    Scanner scanner =  StorageManagerFactory.getStorageManager(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));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-core/tajo-core-storage/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/resources/storage-default.xml b/tajo-core/tajo-core-storage/src/test/resources/storage-default.xml
deleted file mode 100644
index 5bf4453..0000000
--- a/tajo-core/tajo-core-storage/src/test/resources/storage-default.xml
+++ /dev/null
@@ -1,149 +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.v2</name>
-    <value>false</value>
-  </property>
-
-  <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>csv,raw,rcfile,row,trevni</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>
-
-  <!--- 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.v2.csv.class</name>
-    <value>org.apache.tajo.storage.v2.CSVFileScanner</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.v2.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.v2.rcfile.class</name>
-    <value>org.apache.tajo.storage.v2.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.v2.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.v2.trevni.class</name>
-    <value>org.apache.tajo.storage.trevni.TrevniScanner</value>
-  </property>
-
-  <!--- Appender Handler -->
-  <property>
-    <name>tajo.storage.appender-handler</name>
-    <value>csv,raw,rcfile,row,trevni</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>
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index 0712d01..534cf22 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -54,6 +54,11 @@
       <artifactId>tajo-core-backend</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-storage</artifactId>
+      <scope>provided</scope>
+    </dependency>
   </dependencies>
 
   <profiles>
@@ -105,6 +110,7 @@
                       run cp -r $ROOT/tajo-client/target/tajo-client-${project.version}/* .
                       run cp -r $ROOT/tajo-catalog/target/tajo-catalog-${project.version}/* .
                       run cp -r $ROOT/tajo-core/target/tajo-core-${project.version}/* .
+                      run cp -r $ROOT/tajo-storage/target/tajo-storage-${project.version}/* .
                       run cp -r ${project.basedir}/src/main/bin .
                       run cp -r ${project.basedir}/src/main/conf .
                       run rm -rf lib/tajo-*-${project.version}.jar

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-jdbc/pom.xml b/tajo-jdbc/pom.xml
index b1ee8e6..d691af7 100644
--- a/tajo-jdbc/pom.xml
+++ b/tajo-jdbc/pom.xml
@@ -93,7 +93,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>
-      <artifactId>tajo-core-storage</artifactId>
+      <artifactId>tajo-storage</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tajo</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index ee000d8..59628a9 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -615,7 +615,7 @@
       </dependency>
       <dependency>
         <groupId>org.apache.tajo</groupId>
-        <artifactId>tajo-core-storage</artifactId>
+        <artifactId>tajo-storage</artifactId>
         <version>${tajo.version}</version>
       </dependency>
       <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/pom.xml b/tajo-storage/pom.xml
new file mode 100644
index 0000000..5521ddc
--- /dev/null
+++ b/tajo-storage/pom.xml
@@ -0,0 +1,383 @@
+<?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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>tajo-project</artifactId>
+    <groupId>org.apache.tajo</groupId>
+    <version>0.8.0-SNAPSHOT</version>
+    <relativePath>../tajo-project</relativePath>
+  </parent>
+
+  <artifactId>tajo-storage</artifactId>
+  <version>0.8.0-SNAPSHOT</version>
+  <packaging>jar</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>
+  </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.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.4</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-protobuf-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <configuration>
+              <target>
+                <mkdir dir="target/generated-sources/proto" />
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2</version>
+        <executions>
+          <execution>
+            <id>generate-sources</id>
+            <phase>generate-sources</phase>
+            <configuration>
+              <executable>protoc</executable>
+              <arguments>
+                <argument>-Isrc/main/proto/</argument>
+                <argument>--proto_path=../tajo-common/src/main/proto</argument>
+                <argument>--proto_path=../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
+                <argument>--java_out=target/generated-sources/proto</argument>
+                <argument>src/main/proto/IndexProtos.proto</argument>
+              </arguments>
+            </configuration>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.5</version>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>target/generated-sources/proto</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-pmd-plugin</artifactId>
+        <version>2.7.1</version>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tajo</groupId>
+      <artifactId>tajo-catalog-common</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.tajo</groupId>
+          <artifactId>tajo-rpc</artifactId>
+        </exclusion>
+      </exclusions>
+    </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>
+        <exclusion>
+          <artifactId>avro-mapred</artifactId>
+          <groupId>org.apache.avro</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <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>
+      <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>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+      <type>jar</type>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <version>2.6</version>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>src</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-source-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- builds source jars and attaches them to the project for publishing -->
+                <id>tajo-java-sources</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar-no-fork</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+        <property>
+          <name>tar|rpm|deb</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                    <echo file="${project.build.directory}/dist-layout-stitching.sh">
+                      run() {
+                      echo "\$ ${@}"
+                      "${@}"
+                      res=$?
+                      if [ $res != 0 ]; then
+                      echo
+                      echo "Failed!"
+                      echo
+                      exit $res
+                      fi
+                      }
+
+                      ROOT=`cd ${basedir}/..;pwd`
+                      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 .
+                      echo
+                      echo "Tajo Storage dist layout available at: ${project.build.directory}/${project.artifactId}-${project.version}"
+                      echo
+                    </echo>
+                    <exec executable="sh" dir="${project.build.directory}" failonerror="true">
+                      <arg line="./dist-layout-stitching.sh"/>
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+        <version>2.4</version>
+        <configuration>
+          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+    </plugins>
+  </reporting>
+
+</project>
+


[08/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..66c610a
--- /dev/null
+++ b/tajo-storage/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 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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..e4439f3
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/MergeScanner.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;
+
+import com.google.common.collect.Lists;
+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.conf.TajoConf;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+public class MergeScanner implements Scanner {
+  private Configuration conf;
+  private TableMeta meta;
+  private Schema schema;
+  private List<FileFragment> fragments;
+  private Iterator<FileFragment> iterator;
+  private FileFragment currentFragment;
+  private Scanner currentScanner;
+  private Tuple tuple;
+  private boolean projectable = false;
+  private boolean selectable = false;
+  private Schema target;
+
+  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, Collection<FileFragment> rawFragmentList)
+      throws IOException {
+    this(conf, schema, meta, rawFragmentList, schema);
+  }
+
+  public MergeScanner(Configuration conf, Schema schema, TableMeta meta, Collection<FileFragment> rawFragmentList,
+                      Schema target)
+      throws IOException {
+    this.conf = conf;
+    this.schema = schema;
+    this.meta = meta;
+    this.fragments = Lists.newArrayList();
+    for (Fragment f : rawFragmentList) {
+      fragments.add((FileFragment) f);
+    }
+    Collections.sort(fragments);
+
+    this.target = target;
+    this.reset();
+    if (currentScanner != null) {
+      this.projectable = currentScanner.isProjectable();
+      this.selectable = currentScanner.isSelectable();
+    }
+  }
+
+  @Override
+  public void init() throws IOException {
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    if (currentScanner != null)
+      tuple = currentScanner.next();
+
+    if (tuple != null) {
+      return tuple;
+    } else {
+      if (currentScanner != null) {
+        currentScanner.close();
+      }
+      currentScanner = getNextScanner();
+      if (currentScanner != null) {
+        tuple = currentScanner.next();
+      }
+    }
+    return tuple;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    this.iterator = fragments.iterator();
+    this.currentScanner = getNextScanner();
+  }
+
+  private Scanner getNextScanner() throws IOException {
+    if (iterator.hasNext()) {
+      currentFragment = iterator.next();
+      currentScanner = StorageManagerFactory.getStorageManager((TajoConf)conf).getScanner(meta, schema,
+          currentFragment, target);
+      currentScanner.init();
+      return currentScanner;
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(currentScanner != null) {
+      currentScanner.close();
+    }
+    iterator = null;
+    if(fragments != null) {
+      fragments.clear();
+    }
+  }
+
+  @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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..94d13ee
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..db511dc
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/RawFile.java
@@ -0,0 +1,532 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.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.datum.TimestampDatum;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.util.BitArray;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Arrays;
+
+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 Path path;
+
+    private ByteBuffer buffer;
+    private Tuple tuple;
+
+    private int headerSize = 0;
+    private BitArray nullFlags;
+    private static final int RECORD_SIZE = 4;
+    private boolean eof = false;
+    private long fileSize;
+    private FileInputStream fis;
+
+    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
+      super(conf, schema, meta, null);
+      this.path = path;
+      init();
+    }
+
+    @SuppressWarnings("unused")
+    public RawFileScanner(Configuration conf, Schema schema, TableMeta meta, FileFragment fragment) throws IOException {
+      this(conf, schema, meta, fragment.getPath());
+    }
+
+    public void init() throws IOException {
+      //Preconditions.checkArgument(FileUtil.isLocalPath(path));
+      // TODO - to make it unified one.
+      URI uri = path.toUri();
+      fis = new FileInputStream(new File(uri));
+      channel = fis.getChannel();
+      fileSize = channel.size();
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RawFileScanner open:" + path + "," + channel.position() + ", size :" + channel.size());
+      }
+
+      buffer = ByteBuffer.allocateDirect(128 * 1024);
+
+      columnTypes = new DataType[schema.getColumnNum()];
+      for (int i = 0; i < schema.getColumnNum(); i++) {
+        columnTypes[i] = schema.getColumn(i).getDataType();
+      }
+
+      tuple = new VTuple(columnTypes.length);
+
+      // initial read
+      channel.read(buffer);
+      buffer.flip();
+
+      nullFlags = new BitArray(schema.getColumnNum());
+      headerSize = RECORD_SIZE + 2 + nullFlags.bytesLength();
+
+      super.init();
+    }
+
+    @Override
+    public long getNextOffset() throws IOException {
+      return channel.position() - buffer.remaining();
+    }
+
+    @Override
+    public void seek(long offset) throws IOException {
+      long currentPos = channel.position();
+      if(currentPos < offset &&  offset < currentPos + buffer.limit()){
+        buffer.position((int)(offset - currentPos));
+      } else {
+        buffer.clear();
+        channel.position(offset);
+        channel.read(buffer);
+        buffer.flip();
+        eof = false;
+      }
+    }
+
+    private boolean fillBuffer() throws IOException {
+      buffer.compact();
+      if (channel.read(buffer) == -1) {
+        eof = true;
+        return false;
+      } else {
+        buffer.flip();
+        return true;
+      }
+    }
+
+    @Override
+    public Tuple next() throws IOException {
+      if(eof) return null;
+
+      if (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);
+      //buffer.position(recordOffset + headerSize);
+      if (buffer.remaining() < (recordSize - headerSize)) {
+        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 = buffer.getInt();
+            byte[] buf = new byte[columnTypes[i].getLength()];
+            buffer.get(buf);
+            byte[] charBuf = Arrays.copyOf(buf, realLen);
+            tuple.put(i, DatumFactory.createChar(charBuf));
+            break;
+
+          case INT2 :
+            tuple.put(i, DatumFactory.createInt2(buffer.getShort()));
+            break;
+
+          case INT4 :
+            tuple.put(i, DatumFactory.createInt4(buffer.getInt()));
+            break;
+
+          case INT8 :
+            tuple.put(i, DatumFactory.createInt8(buffer.getLong()));
+            break;
+
+          case FLOAT4 :
+            tuple.put(i, DatumFactory.createFloat4(buffer.getFloat()));
+            break;
+
+          case FLOAT8 :
+            tuple.put(i, DatumFactory.createFloat8(buffer.getDouble()));
+            break;
+
+          case TEXT :
+            // TODO - shoud use CharsetEncoder / CharsetDecoder
+            //byte [] rawBytes = getColumnBytes();
+            int strSize2 = buffer.getInt();
+            byte [] strBytes2 = new byte[strSize2];
+            buffer.get(strBytes2);
+            tuple.put(i, DatumFactory.createText(new String(strBytes2)));
+            break;
+
+          case TIMESTAMP:
+            tuple.put(i, DatumFactory.createTimeStampFromMillis(buffer.getLong()));
+            break;
+
+          case BLOB : {
+            //byte [] rawBytes = getColumnBytes();
+            int byteSize = buffer.getInt();
+            byte [] rawBytes = new byte[byteSize];
+            buffer.get(rawBytes);
+            tuple.put(i, DatumFactory.createBlob(rawBytes));
+            break;
+          }
+
+          case PROTOBUF: {
+            //byte [] rawBytes = getColumnBytes();
+            int byteSize = buffer.getInt();
+            byte [] rawBytes = new byte[byteSize];
+            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 NULL_TYPE:
+            tuple.put(i, NullDatum.get());
+            break;
+
+          default:
+        }
+      }
+
+      if(!buffer.hasRemaining() && channel.position() == fileSize){
+        eof = true;
+      }
+      return tuple;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      // clear the buffer
+      buffer.clear();
+      // reload initial buffer
+      channel.position(0);
+      channel.read(buffer);
+      buffer.flip();
+      eof = false;
+    }
+
+    @Override
+    public void close() throws IOException {
+      buffer.clear();
+      channel.close();
+      fis.close();
+    }
+
+    @Override
+    public boolean isProjectable() {
+      return false;
+    }
+
+    @Override
+    public boolean isSelectable() {
+      return false;
+    }
+
+    @Override
+    public boolean isSplittable(){
+      return false;
+    }
+  }
+
+  public static class RawFileAppender extends FileAppender {
+    private FileChannel channel;
+    private RandomAccessFile randomAccessFile;
+    private DataType[] columnTypes;
+
+    private ByteBuffer buffer;
+    private BitArray nullFlags;
+    private int headerSize = 0;
+    private static final int RECORD_SIZE = 4;
+    private long pos;
+
+    private TableStatistics stats;
+
+    public RawFileAppender(Configuration conf, Schema schema, TableMeta meta, Path path) throws IOException {
+      super(conf, schema, meta, path);
+    }
+
+    public void init() throws IOException {
+      // TODO - RawFile only works on Local File System.
+      //Preconditions.checkArgument(FileUtil.isLocalPath(path));
+      File file = new File(path.toUri());
+      randomAccessFile = new RandomAccessFile(file, "rw");
+      channel = randomAccessFile.getChannel();
+      pos = 0;
+
+      columnTypes = new DataType[schema.getColumnNum()];
+      for (int i = 0; i < schema.getColumnNum(); i++) {
+        columnTypes[i] = schema.getColumn(i).getDataType();
+      }
+
+      buffer = ByteBuffer.allocateDirect(64 * 1024);
+
+      // comput the number of bytes, representing the null flags
+
+      nullFlags = new BitArray(schema.getColumnNum());
+      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.limit(buffer.position());
+      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;
+      }
+    }
+
+    @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.getColumnNum(); 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.get(i).asByte());
+            break;
+
+          case CHAR :
+            byte[] src = t.getChar(i).asByteArray();
+            byte[] dst = Arrays.copyOf(src, columnTypes[i].getLength());
+            buffer.putInt(src.length);
+            buffer.put(dst);
+            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 TEXT:
+            byte [] strBytes2 = t.get(i).asByteArray();
+            if (flushBufferAndReplace(recordOffset, strBytes2.length + 4)) {
+              recordOffset = 0;
+            }
+            buffer.putInt(strBytes2.length);
+            buffer.put(strBytes2);
+            break;
+
+          case TIMESTAMP:
+            buffer.putLong(((TimestampDatum)t.get(i)).getMillis());
+            break;
+
+          case BLOB : {
+            byte [] rawBytes = t.get(i).asByteArray();
+            if (flushBufferAndReplace(recordOffset, rawBytes.length + 4)) {
+              recordOffset = 0;
+            }
+            buffer.putInt(rawBytes.length);
+            buffer.put(rawBytes);
+            break;
+          }
+
+          case PROTOBUF: {
+            // TODO - to be fixed
+//            byte [] lengthByte = new byte[4];
+//            byte [] byteArray = t.get(i).asByteArray();
+//            CodedOutputStream outputStream = CodedOutputStream.newInstance(lengthByte);
+//            outputStream.writeUInt32NoTag(byteArray.length);
+//            outputStream.flush();
+//            int legnthByteLength = CodedOutputStream.computeInt32SizeNoTag(byteArray.length);
+//            if (flushBufferAndReplace(recordOffset, byteArray.length + legnthByteLength)) {
+//              recordOffset = 0;
+//            }
+//            buffer.put(lengthByte, 0, legnthByteLength);
+            byte [] rawBytes = t.get(i).asByteArray();
+            if (flushBufferAndReplace(recordOffset, rawBytes.length + 4)) {
+              recordOffset = 0;
+            }
+            buffer.putInt(rawBytes.length);
+            buffer.put(rawBytes);
+            break;
+          }
+
+          case INET4 :
+            buffer.put(t.get(i).asByteArray());
+            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 {
+      flushBuffer();
+    }
+
+    @Override
+    public void close() throws IOException {
+      flush();
+      if (enabledStats) {
+        stats.setNumBytes(getOffset());
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RawFileAppender written: " + getOffset() + " bytes, path: " + path);
+      }
+      channel.close();
+      randomAccessFile.close();
+    }
+
+    @Override
+    public TableStats getStats() {
+      if (enabledStats) {
+        return stats.getTableStat();
+      } else {
+        return null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..1e89f31
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/RowFile.java
@@ -0,0 +1,506 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FileFragment;
+import org.apache.tajo.util.BitArray;
+import org.apache.tajo.util.Bytes;
+
+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 FileFragment fragment)
+        throws IOException {
+      super(conf, schema, meta, fragment);
+
+      SYNC_INTERVAL =
+          conf.getInt(ConfVars.RAWFILE_SYNC_INTERVAL.varname,
+              SYNC_SIZE * 100);
+
+      nullFlags = new BitArray(schema.getColumnNum());
+      tupleHeaderSize = nullFlags.bytesLength() + (2 * Short.SIZE / 8);
+      this.start = fragment.getStartKey();
+      this.end = this.start + fragment.getEndKey();
+    }
+
+    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.getColumnNum());
+      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();
+      Bytes.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.getColumnNum());
+
+      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.getColumnNum(); 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 len = buffer.getShort();
+//              byte[] buf = new byte[len];
+//              buffer.get(buf, 0, len);
+//              datum = DatumFactory.createText(buf);
+//              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 Schema schema, final TableMeta meta, final Path path)
+        throws IOException {
+      super(conf, schema, meta, path);
+    }
+
+    public void init() throws IOException {
+      SYNC_INTERVAL = conf.getInt(ConfVars.RAWFILE_SYNC_INTERVAL.varname, 100);
+
+      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.getColumnNum());
+
+      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.getColumnNum(); 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.getBoolean(i).asByte());
+              break;
+            case BIT:
+              buffer.put(t.getByte(i).asByte());
+              break;
+            case CHAR:
+              byte[] src = t.getChar(i).asByteArray();
+              byte[] dst = Arrays.copyOf(src, col.getDataType().getLength());
+              buffer.putInt(src.length);
+              buffer.put(dst);
+              break;
+            case TEXT:
+              byte [] strbytes = t.getText(i).asByteArray();
+              buffer.putShort((short)strbytes.length);
+              buffer.put(strbytes, 0, strbytes.length);
+              break;
+            case INT2:
+              buffer.putShort(t.getShort(i).asInt2());
+              break;
+            case INT4:
+              buffer.putInt(t.getInt(i).asInt4());
+              break;
+            case INT8:
+              buffer.putLong(t.getLong(i).asInt8());
+              break;
+            case FLOAT4:
+              buffer.putFloat(t.getFloat(i).asFloat4());
+              break;
+            case FLOAT8:
+              buffer.putDouble(t.getDouble(i).asFloat8());
+              break;
+            case BLOB:
+              byte [] bytes = t.getBytes(i).asByteArray();
+              buffer.putShort((short)bytes.length);
+              buffer.put(bytes);
+              break;
+            case INET4:
+              buffer.put(t.getIPv4Bytes(i));
+              break;
+            case INET6:
+              buffer.put(t.getIPv6Bytes(i));
+              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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..9f32028
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/RowStoreUtil.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 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.TimestampDatum;
+import org.apache.tajo.util.Bytes;
+
+import java.nio.ByteBuffer;
+
+public class RowStoreUtil {
+  public static int[] getTargetIds(Schema inSchema, Schema outSchema) {
+    int[] targetIds = new int[outSchema.getColumnNum()];
+    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 class RowStoreDecoder {
+
+    public static Tuple toTuple(Schema schema, byte [] bytes) {
+      ByteBuffer bb = ByteBuffer.wrap(bytes);
+      Tuple tuple = new VTuple(schema.getColumnNum());
+      Column col;
+      TajoDataTypes.DataType type;
+      for (int i =0; i < schema.getColumnNum(); i++) {
+        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();
+            if(b == 0) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            } else {
+              tuple.put(i, DatumFactory.createBit(b));
+            }
+            break;
+
+          case CHAR:
+            byte c = bb.get();
+            if(c == 0) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            } else {
+              tuple.put(i, DatumFactory.createChar(c));
+            }
+            break;
+
+          case INT2:
+            short s = bb.getShort();
+            if(s < Short.MIN_VALUE + 1) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            }else {
+              tuple.put(i, DatumFactory.createInt2(s));
+            }
+            break;
+
+          case INT4:
+          case DATE:
+            int i_ = bb.getInt();
+            if ( i_ < Integer.MIN_VALUE + 1) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            } else {
+              tuple.put(i, DatumFactory.createFromInt4(type, i_));
+            }
+            break;
+
+          case INT8:
+          case TIME:
+          case TIMESTAMP:
+            long l = bb.getLong();
+            if ( l < Long.MIN_VALUE + 1) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            }else {
+              tuple.put(i, DatumFactory.createFromInt8(type, l));
+            }
+            break;
+
+          case FLOAT4:
+            float f = bb.getFloat();
+            if (Float.isNaN(f)) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            }else {
+              tuple.put(i, DatumFactory.createFloat4(f));
+            }
+            break;
+
+          case FLOAT8:
+            double d = bb.getDouble();
+            if(Double.isNaN(d)) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            }else {
+              tuple.put(i, DatumFactory.createFloat8(d));
+            }
+            break;
+
+          case TEXT:
+            byte [] _string = new byte[bb.getInt()];
+            bb.get(_string);
+            String str = new String(_string);
+            if(str.compareTo("NULL") == 0) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            }else {
+            tuple.put(i, DatumFactory.createText(str));
+            }
+            break;
+
+          case BLOB:
+            byte [] _bytes = new byte[bb.getInt()];
+            bb.get(_bytes);
+            if(Bytes.compareTo(bytes, Bytes.toBytes("NULL")) == 0) {
+              tuple.put(i, DatumFactory.createNullDatum());
+            } else {
+              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
+        }
+      }
+      return tuple;
+    }
+  }
+
+  public static class RowStoreEncoder {
+
+    public static byte [] toBytes(Schema schema, Tuple tuple) {
+      int size = StorageUtil.getRowByteSize(schema);
+      ByteBuffer bb = ByteBuffer.allocate(size);
+      Column col;
+      for (int i = 0; i < schema.getColumnNum(); i++) {
+        col = schema.getColumn(i);
+        switch (col.getDataType().getType()) {
+          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 TIMESTAMP: bb.putLong(((TimestampDatum)tuple.get(i)).getMillis()); break;
+          case BLOB:
+            byte [] bytes = tuple.get(i).asByteArray();
+            bb.putInt(bytes.length);
+            bb.put(bytes);
+            break;
+          case INET4:
+            byte [] ipBytes = tuple.getIPv4Bytes(i);
+            bb.put(ipBytes);
+            break;
+          case INET6: bb.put(tuple.getIPv6Bytes(i)); break;
+          default:
+        }
+      }
+
+      bb.flip();
+      byte [] buf = new byte [bb.limit()];
+      bb.get(buf);
+      return buf;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..6dca3f2
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/Scanner.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..894e7ee
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..333f205
--- /dev/null
+++ b/tajo-storage/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;
+
+
+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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..3579674
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..cc85c1d
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/Storage.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;
+
+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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..1b852d4
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManager.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.fragment.Fragment;
+
+import java.io.IOException;
+
+/**
+ * StorageManager
+ */
+public class StorageManager extends AbstractStorageManager {
+
+  protected StorageManager(TajoConf conf) throws IOException {
+    super(conf);
+  }
+
+  @Override
+  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;
+  }
+
+  @Override
+  public Scanner getScanner(TableMeta meta, Schema schema, Fragment fragment, Schema target) throws IOException {
+    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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java
new file mode 100644
index 0000000..85bb861
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageManagerFactory.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.v2.StorageManagerV2;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Map;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+
+public class StorageManagerFactory {
+  private static final Map<String, AbstractStorageManager> storageManagers = Maps.newHashMap();
+
+  public static AbstractStorageManager getStorageManager(TajoConf conf) throws IOException {
+    return getStorageManager(conf, null);
+  }
+
+  public static synchronized AbstractStorageManager getStorageManager (
+      TajoConf conf, Path warehouseDir) throws IOException {
+    return getStorageManager(conf, warehouseDir, conf.getBoolVar(ConfVars.STORAGE_MANAGER_VERSION_2));
+  }
+
+  private static synchronized AbstractStorageManager getStorageManager (
+      TajoConf conf, Path warehouseDir, boolean v2) throws IOException {
+
+    URI uri;
+    TajoConf localConf = new TajoConf(conf);
+    if (warehouseDir != null) {
+      localConf.setVar(ConfVars.WAREHOUSE_DIR, warehouseDir.toUri().toString());
+    }
+
+    uri = TajoConf.getWarehouseDir(localConf).toUri();
+
+    String key = "file".equals(uri.getScheme()) ? "file" : uri.toString();
+
+    if(v2) {
+      key += "_v2";
+    }
+
+    if(storageManagers.containsKey(key)) {
+      AbstractStorageManager sm = storageManagers.get(key);
+      return sm;
+    } else {
+      AbstractStorageManager storageManager;
+
+      if(v2) {
+        storageManager = new StorageManagerV2(localConf);
+      } else {
+        storageManager = new StorageManager(localConf);
+      }
+
+      storageManagers.put(key, storageManager);
+
+      return storageManager;
+    }
+  }
+
+  public static synchronized SeekableScanner getSeekableScanner(
+      TajoConf conf, TableMeta meta, Schema schema, FileFragment fragment, Schema target) throws IOException {
+    return (SeekableScanner)getStorageManager(conf, null, false).getScanner(meta, schema, fragment, target);
+  }
+
+  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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..9627a5d
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/StorageUtil.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.util.FileUtil;
+
+import java.io.IOException;
+
+public class StorageUtil {
+  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;
+    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());
+  }
+}


[07/18] TAJO-520: Move tajo-core-storage to tajo-storage. (jinho)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..41c9d61
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/TableStatistics.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;
+
+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 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.getColumnNum());
+    maxValues = new VTuple(schema.getColumnNum());
+
+    numNulls = new long[schema.getColumnNum()];
+    comparable = new boolean[schema.getColumnNum()];
+
+    DataType type;
+    for (int i = 0; i < schema.getColumnNum(); 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.getColumnNum(); i++) {
+      columnStats = new ColumnStats(schema.getColumn(i));
+      columnStats.setNumNulls(numNulls[i]);
+      columnStats.setMinValue(minValues.get(i));
+      columnStats.setMaxValue(maxValues.get(i));
+      stat.addColumnStat(columnStats);
+    }
+
+    stat.setNumRows(this.numRows);
+    stat.setNumBytes(this.numBytes);
+
+    return stat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..07ea79b
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.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 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.datum.*;
+import org.apache.tajo.datum.protobuf.ProtobufJsonFormat;
+import org.apache.tajo.util.Bytes;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+//Compatibility with Apache Hive
+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 TIME:
+      case TIMESTAMP:
+        bytes = datum.asTextBytes();
+        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 INT2:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createInt2(new String(bytes, offset, length));
+        break;
+      case INT4:
+        datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get()
+            : DatumFactory.createInt4(new String(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(new String(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 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/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/Tuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/Tuple.java b/tajo-storage/src/main/java/org/apache/tajo/storage/Tuple.java
new file mode 100644
index 0000000..ba35988
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/Tuple.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.net.InetAddress;
+
+public interface Tuple extends Cloneable {
+  
+	public int size();
+	
+	public boolean contains(int fieldid);
+
+  public boolean isNull(int fieldid);
+	
+	public void clear();
+	
+	public void put(int fieldId, Datum value);
+
+  public void put(int fieldId, Datum [] values);
+
+  public void put(int fieldId, Tuple tuple);
+	
+	public void put(Datum [] values);
+	
+	public Datum get(int fieldId);
+	
+	public void setOffset(long offset);
+	
+	public long getOffset();
+
+	public BooleanDatum getBoolean(int fieldId);
+	
+	public BitDatum getByte(int fieldId);
+
+  public CharDatum getChar(int fieldId);
+	
+	public BlobDatum getBytes(int fieldId);
+	
+	public Int2Datum getShort(int fieldId);
+	
+	public Int4Datum getInt(int fieldId);
+	
+	public Int8Datum getLong(int fieldId);
+	
+	public Float4Datum getFloat(int fieldId);
+	
+	public Float8Datum getDouble(int fieldId);
+	
+	public Inet4Datum getIPv4(int fieldId);
+	
+	public byte [] getIPv4Bytes(int fieldId);
+	
+	public InetAddress getIPv6(int fieldId);
+	
+	public byte [] getIPv6Bytes(int fieldId);
+	
+	public TextDatum getString(int fieldId);
+
+  public TextDatum getText(int fieldId);
+
+  public Tuple clone() throws CloneNotSupportedException;
+
+  public Datum[] getValues();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..69c1c04
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/TupleComparator.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 org.apache.tajo.datum.NullDatum;
+
+import java.util.Comparator;
+
+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 TupleComparator implements Comparator<Tuple>, ProtoObject<TupleComparatorProto> {
+  private final int[] sortKeyIds;
+  private final boolean[] asc;
+  @SuppressWarnings("unused")
+  private final boolean[] nullFirsts;  
+
+  private Datum left;
+  private Datum right;
+  private int compVal;
+
+  public TupleComparator(Schema schema, SortSpec[] sortKeys) {
+    Preconditions.checkArgument(sortKeys.length > 0, 
+        "At least one sort key must be specified.");
+
+    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++) {
+      this.sortKeyIds[i] = schema.getColumnId(sortKeys[i].getSortKey().getQualifiedName());
+          
+      this.asc[i] = sortKeys[i].isAscending();
+      this.nullFirsts[i]= sortKeys[i].isNullFirst();
+    }
+  }
+
+  public TupleComparator(TupleComparatorProto proto) {
+    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 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 instanceof NullDatum || right instanceof NullDatum) {
+        if (!left.equals(right)) {
+          if (left instanceof NullDatum) {
+            compVal = 1;
+          } else if (right instanceof NullDatum) {
+            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 TupleComparator) {
+      TupleComparator other = (TupleComparator) 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();
+    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();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..7d0f674
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/TupleRange.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 com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.SortSpec;
+
+import java.util.Comparator;
+
+public class TupleRange implements Comparable<TupleRange> {
+  private final Schema schema;
+  private final Tuple start;
+  private final Tuple end;
+  private final TupleComparator comp;
+
+  public TupleRange(final Schema schema, final Tuple start, final Tuple end) {
+    this.comp = new TupleComparator(schema, schemaToSortSpecs(schema));
+    // if there is only one value, start == end
+    Preconditions.checkArgument(comp.compare(start, end) <= 0, ("start=" + start) + ", end=" + end);
+    this.schema = schema;
+    this.start = start;
+    this.end = end;
+  }
+
+  public static SortSpec[] schemaToSortSpecs(Schema schema) {
+    SortSpec[] specs = new SortSpec[schema.getColumnNum()];
+
+    for (int i = 0; i < schema.getColumnNum(); i++) {
+      specs[i] = new SortSpec(schema.getColumn(i), true, false);
+    }
+
+    return specs;
+  }
+
+  public final Schema getSchema() {
+    return this.schema;
+  }
+
+  public final Tuple getStart() {
+    return this.start;
+  }
+
+  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 -(left.compareTo(right));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/tajo-storage/src/main/java/org/apache/tajo/storage/VTuple.java
----------------------------------------------------------------------
diff --git a/tajo-storage/src/main/java/org/apache/tajo/storage/VTuple.java b/tajo-storage/src/main/java/org/apache/tajo/storage/VTuple.java
new file mode 100644
index 0000000..878c05e
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/VTuple.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.gson.annotations.Expose;
+import org.apache.tajo.datum.*;
+import org.apache.tajo.datum.exception.InvalidCastException;
+
+import java.net.InetAddress;
+import java.util.Arrays;
+
+public class VTuple implements Tuple, Cloneable {
+	@Expose public Datum [] values;
+	@Expose private long offset;
+	
+	public VTuple(int size) {
+		values = new Datum[size];
+	}
+
+  public VTuple(Tuple tuple) {
+    this.values = new Datum[tuple.size()];
+    System.arraycopy(((VTuple)tuple).values, 0, values, 0, tuple.size());
+    this.offset = ((VTuple)tuple).offset;
+  }
+
+  public VTuple(Datum [] datum) {
+    this(datum.length);
+    put(datum);
+  }
+
+	@Override
+	public int size() {	
+		return values.length;
+	}
+	
+	public boolean contains(int fieldId) {
+		return values[fieldId] != null;
+	}
+
+  @Override
+  public boolean isNull(int fieldid) {
+    return values[fieldid] instanceof NullDatum;
+  }
+
+  @Override
+  public void clear() {   
+    for (int i=0; i < values.length; i++) {
+      values[i] = null;
+    }
+  }
+	
+	//////////////////////////////////////////////////////
+	// Setter
+	//////////////////////////////////////////////////////	
+	public void put(int fieldId, Datum value) {
+		values[fieldId] = value;
+	}
+
+  @Override
+  public void put(int fieldId, Datum[] values) {
+    for (int i = fieldId, j = 0; j < values.length; i++, j++) {
+      values[i] = values[j];
+    }
+  }
+
+  @Override
+  public void put(int fieldId, Tuple tuple) {
+    for (int i = fieldId, j = 0; j < tuple.size(); i++, j++) {
+      values[i] = tuple.get(j);
+    }
+  }
+
+  public void put(Datum [] values) {
+    System.arraycopy(values, 0, this.values, 0, size());
+	}
+	
+	//////////////////////////////////////////////////////
+	// Getter
+	//////////////////////////////////////////////////////
+	public Datum get(int fieldId) {
+		return this.values[fieldId];
+	}
+	
+	public void setOffset(long offset) {
+	  this.offset = offset;
+	}
+	
+	public long getOffset() {
+	  return this.offset;
+	}
+	
+	@Override
+	public BooleanDatum getBoolean(int fieldId) {
+		return (BooleanDatum) values[fieldId];
+	}
+
+	public BitDatum getByte(int fieldId) {
+		return (BitDatum) values[fieldId];
+	}
+
+  public CharDatum getChar(int fieldId) {
+    return (CharDatum) values[fieldId];
+  }
+
+	public BlobDatum getBytes(int fieldId) {
+		return (BlobDatum) values[fieldId];
+	}
+
+	public Int2Datum getShort(int fieldId) {
+		return (Int2Datum) values[fieldId];
+	}
+
+	public Int4Datum getInt(int fieldId) {
+		return (Int4Datum) values[fieldId];
+	}
+
+	public Int8Datum getLong(int fieldId) {
+		return (Int8Datum) values[fieldId];
+	}
+
+	public Float4Datum getFloat(int fieldId) {
+		return (Float4Datum) values[fieldId];
+	}
+
+	public Float8Datum getDouble(int fieldId) {
+		return (Float8Datum) values[fieldId];
+	}
+
+	public Inet4Datum getIPv4(int fieldId) {
+		return (Inet4Datum) values[fieldId];
+	}
+
+	public byte[] getIPv4Bytes(int fieldId) {
+		return values[fieldId].asByteArray();
+	}
+
+	public InetAddress getIPv6(int fieldId) {
+		throw new InvalidCastException("IPv6 is unsupported yet");
+	}
+
+	public byte[] getIPv6Bytes(int fieldId) {
+	  throw new InvalidCastException("IPv6 is unsupported yet");
+	}
+
+	public TextDatum getString(int fieldId) {
+		return (TextDatum) values[fieldId];
+	}
+
+  @Override
+  public TextDatum getText(int fieldId) {
+    return (TextDatum) values[fieldId];
+  }
+
+  @Override
+  public Tuple clone() throws CloneNotSupportedException {
+    VTuple tuple = (VTuple) super.clone();
+
+    tuple.values = new Datum[size()];
+    System.arraycopy(values, 0, tuple.values, 0, size()); //shallow copy
+    return tuple;
+  }
+
+  public String toString() {
+		boolean first = true;
+		StringBuilder str = new StringBuilder();
+		str.append("(");
+		for(int i=0; i < values.length; i++) {			
+			if(values[i] != null) {
+				if(first) {
+					first = false;
+				} else {
+					str.append(", ");
+				}
+				str.append(i)
+				.append("=>")
+				.append(values[i]);
+			}
+		}
+		str.append(")");
+		return str.toString();
+	}
+	
+	@Override
+	public int hashCode() {
+	  int hashCode = 37;
+	  for (int i=0; i < values.length; i++) {
+	    if(values[i] != null) {
+        hashCode ^= (values[i].hashCode() * 41);
+	    } else {
+	      hashCode = hashCode ^ (i + 17);
+	    }
+	  }
+	  
+	  return hashCode;
+	}
+
+  @Override
+  public Datum[] getValues() {
+    return values;
+  }
+
+  @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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..ad19101
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..baeda8c
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..bb035a8
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..a67d1f7
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..d18b5a0
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..8b197d6
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..ea8bf9f
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FileFragment.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.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.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 private Long startOffset; // required
+  @Expose private 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, int[] diskIds)
+      throws IOException {
+    this.set(tableName, uri, blockLocation.getOffset(), blockLocation.getLength(),
+        blockLocation.getHosts(), diskIds);
+  }
+
+  // Non splittable
+  public FileFragment(String tableName, Path uri, long start, long length, String[] hosts) {
+    this.set(tableName, uri, start, length, null, null);
+    this.hosts = hosts;
+  }
+
+  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 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;
+  }
+
+  public Long getEndKey() {
+    return this.length;
+  }
+
+  /**
+   * 
+   * 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.getEndKey(), this.getEndKey())) {
+        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\": "
+        + getEndKey() + "}" ;
+  }
+
+  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.setContents(builder.buildPartial().toByteString());
+    return fragmentBuilder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..3f9c160
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/Fragment.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..3bfe96f
--- /dev/null
+++ b/tajo-storage/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+
+@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, StoreType storeType)
+      throws IOException {
+    String handlerName = storeType.name().toLowerCase();
+    Class<? extends Fragment> fragmentClass = CACHED_FRAGMENT_CLASSES.get(handlerName);
+    if (fragmentClass == null) {
+      fragmentClass = conf.getClass(
+          String.format("tajo.storage.fragment.%s.class", storeType.name().toLowerCase()), null, Fragment.class);
+      CACHED_FRAGMENT_CLASSES.put(handlerName, fragmentClass);
+    }
+
+    if (fragmentClass == null) {
+      throw new IOException("No such a fragment for " + storeType.name());
+    }
+
+    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, StoreType storeType, FragmentProto fragment)
+      throws IOException {
+    Class<T> fragmentClass = (Class<T>) getFragmentClass(conf, storeType);
+    if (fragmentClass == null) {
+      throw new IOException("No such a fragment class for " + storeType.name());
+    }
+    return convert(fragmentClass, fragment);
+  }
+
+  public static <T extends Fragment> List<T> convert(Class<T> clazz, FragmentProto...fragments)
+      throws IOException {
+    List<T> list = Lists.newArrayList();
+    for (FragmentProto proto : fragments) {
+      list.add(convert(clazz, proto));
+    }
+    return list;
+  }
+
+  public static <T extends Fragment> List<T> convert(Configuration conf, StoreType storeType,
+                                                           FragmentProto...fragments) throws IOException {
+    List<T> list = Lists.newArrayList();
+    for (FragmentProto proto : fragments) {
+      list.add((T) convert(conf, storeType, proto));
+    }
+    return list;
+  }
+
+  public static List<FragmentProto> toFragmentProtoList(Fragment... fragments) {
+    List<FragmentProto> list = Lists.newArrayList();
+    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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..74be7ff
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..7baf7aa
--- /dev/null
+++ b/tajo-storage/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 IOException 
+   */
+  public long find(Tuple key) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..04738f8
--- /dev/null
+++ b/tajo-storage/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/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..688bbc7
--- /dev/null
+++ b/tajo-storage/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 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;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bbf9b7bf/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
new file mode 100644
index 0000000..bc8fe96
--- /dev/null
+++ b/tajo-storage/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.RowStoreUtil;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.index.IndexMethod;
+import org.apache.tajo.storage.index.IndexWriter;
+import org.apache.tajo.storage.index.OrderIndexReader;
+import org.apache.tajo.util.Bytes;
+
+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 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);
+    }
+
+   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 = RowStoreUtil.RowStoreEncoder.toBytes(keySchema,
+            firstKey);
+        out.writeInt(minBytes.length);
+        out.write(minBytes);
+        byte [] maxBytes = RowStoreUtil.RowStoreEncoder.toBytes(keySchema,
+            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 = RowStoreUtil.RowStoreEncoder.toBytes(this.keySchema, 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 = RowStoreUtil.RowStoreEncoder.toBytes(keySchema, 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();
+
+    /**
+     *
+     * @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;
+    }
+
+    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];
+      Bytes.readFully(indexIn, schemaBytes, 0, schemaByteSize);
+
+      SchemaProto.Builder builder = SchemaProto.newBuilder();
+      builder.mergeFrom(schemaBytes);
+      SchemaProto proto = builder.build();
+      this.keySchema = new Schema(proto);
+
+      // comparator
+      int compByteSize = indexIn.readInt();
+      byte [] compBytes = new byte[compByteSize];
+      Bytes.readFully(indexIn, compBytes, 0, compByteSize);
+
+      TupleComparatorProto.Builder compProto = TupleComparatorProto.newBuilder();
+      compProto.mergeFrom(compBytes);
+      this.comparator = new TupleComparator(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()];
+        Bytes.readFully(indexIn, minBytes, 0, minBytes.length);
+        this.firstKey = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, minBytes);
+
+        byte [] maxBytes = new byte[indexIn.readInt()];
+        Bytes.readFully(indexIn, maxBytes, 0, maxBytes.length);
+        this.lastKey = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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;
+        switch (this.level) {
+          case ONE_LEVEL_INDEX:
+            pos = oneLevBS(key);
+            break;
+          case TWO_LEVEL_INDEX:
+            pos = twoLevBS(key, this.loadNum + 1);
+            break;
+        }
+
+        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()];
+          Bytes.readFully(in, buf, 0, buf.length);
+          dataSubIndex[i] = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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()];
+          Bytes.readFully(in, buf, 0, buf.length);
+          dataSubIndex[i] = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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()];
+        Bytes.readFully(in, buf, 0, buf.length);
+        keyTuple = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, 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;
+    }
+  }
+}